Merge branch 'cassandra-3.0' into trunk
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/12f5ca36 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/12f5ca36 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/12f5ca36 Branch: refs/heads/trunk Commit: 12f5ca36ffab227a1531a554b00cf83d898f9f28 Parents: 5692c59 21d8a7d Author: Tyler Hobbs <tylerlho...@gmail.com> Authored: Tue Sep 27 12:04:35 2016 -0500 Committer: Tyler Hobbs <tylerlho...@gmail.com> Committed: Tue Sep 27 12:04:35 2016 -0500 ---------------------------------------------------------------------- CHANGES.txt | 2 ++ .../cassandra/db/SinglePartitionReadCommand.java | 16 ++++++++-------- 2 files changed, 10 insertions(+), 8 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/12f5ca36/CHANGES.txt ---------------------------------------------------------------------- diff --cc CHANGES.txt index b6a687d,4280abd..75e7d2a --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,74 -1,6 +1,76 @@@ -3.0.10 +3.10 + * Tune compaction thread count via nodetool (CASSANDRA-12248) + * Add +=/-= shortcut syntax for update queries (CASSANDRA-12232) + * Include repair session IDs in repair start message (CASSANDRA-12532) + * Add a blocking task to Index, run before joining the ring (CASSANDRA-12039) + * Fix NPE when using CQLSSTableWriter (CASSANDRA-12667) + * Support optional backpressure strategies at the coordinator (CASSANDRA-9318) + * Make randompartitioner work with new vnode allocation (CASSANDRA-12647) + * Fix cassandra-stress graphing (CASSANDRA-12237) + * Allow filtering on partition key columns for queries without secondary indexes (CASSANDRA-11031) + * Fix Cassandra Stress reporting thread model and precision (CASSANDRA-12585) + * Add JMH benchmarks.jar (CASSANDRA-12586) + * Add row offset support to SASI (CASSANDRA-11990) + * Cleanup uses of AlterTableStatementColumn (CASSANDRA-12567) + * Add keep-alive to streaming (CASSANDRA-11841) + * Tracing payload is passed through newSession(..) (CASSANDRA-11706) + * avoid deleting non existing sstable files and improve related log messages (CASSANDRA-12261) + * json/yaml output format for nodetool compactionhistory (CASSANDRA-12486) + * Retry all internode messages once after a connection is + closed and reopened (CASSANDRA-12192) + * Add support to rebuild from targeted replica (CASSANDRA-9875) + * Add sequence distribution type to cassandra stress (CASSANDRA-12490) + * "SELECT * FROM foo LIMIT ;" does not error out (CASSANDRA-12154) + * Define executeLocally() at the ReadQuery Level (CASSANDRA-12474) + * Extend read/write failure messages with a map of replica addresses + to error codes in the v5 native protocol (CASSANDRA-12311) + * Fix rebuild of SASI indexes with existing index files (CASSANDRA-12374) + * Let DatabaseDescriptor not implicitly startup services (CASSANDRA-9054, 12550) + * Fix clustering indexes in presence of static columns in SASI (CASSANDRA-12378) + * Fix queries on columns with reversed type on SASI indexes (CASSANDRA-12223) + * Added slow query log (CASSANDRA-12403) + * Count full coordinated request against timeout (CASSANDRA-12256) + * Allow TTL with null value on insert and update (CASSANDRA-12216) + * Make decommission operation resumable (CASSANDRA-12008) + * Add support to one-way targeted repair (CASSANDRA-9876) + * Remove clientutil jar (CASSANDRA-11635) + * Fix compaction throughput throttle (CASSANDRA-12366) + * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358) + * Cassandra stress should dump all setting on startup (CASSANDRA-11914) + * Make it possible to compact a given token range (CASSANDRA-10643) + * Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179) + * Collect metrics on queries by consistency level (CASSANDRA-7384) + * Add support for GROUP BY to SELECT statement (CASSANDRA-10707) + * Deprecate memtable_cleanup_threshold and update default for memtable_flush_writers (CASSANDRA-12228) + * Upgrade to OHC 0.4.4 (CASSANDRA-12133) + * Add version command to cassandra-stress (CASSANDRA-12258) + * Create compaction-stress tool (CASSANDRA-11844) + * Garbage-collecting compaction operation and schema option (CASSANDRA-7019) + * Add beta protocol flag for v5 native protocol (CASSANDRA-12142) + * Support filtering on non-PRIMARY KEY columns in the CREATE + MATERIALIZED VIEW statement's WHERE clause (CASSANDRA-10368) + * Unify STDOUT and SYSTEMLOG logback format (CASSANDRA-12004) + * COPY FROM should raise error for non-existing input files (CASSANDRA-12174) + * Faster write path (CASSANDRA-12269) + * Option to leave omitted columns in INSERT JSON unset (CASSANDRA-11424) + * Support json/yaml output in nodetool tpstats (CASSANDRA-12035) + * Expose metrics for successful/failed authentication attempts (CASSANDRA-10635) + * Prepend snapshot name with "truncated" or "dropped" when a snapshot + is taken before truncating or dropping a table (CASSANDRA-12178) + * Optimize RestrictionSet (CASSANDRA-12153) + * cqlsh does not automatically downgrade CQL version (CASSANDRA-12150) + * Omit (de)serialization of state variable in UDAs (CASSANDRA-9613) + * Create a system table to expose prepared statements (CASSANDRA-8831) + * Reuse DataOutputBuffer from ColumnIndex (CASSANDRA-11970) + * Remove DatabaseDescriptor dependency from SegmentedFile (CASSANDRA-11580) + * Add supplied username to authentication error messages (CASSANDRA-12076) + * Remove pre-startup check for open JMX port (CASSANDRA-12074) + * Remove compaction Severity from DynamicEndpointSnitch (CASSANDRA-11738) + * Restore resumable hints delivery (CASSANDRA-11960) +Merged from 3.0: + * Fix potentially incomplete non-frozen UDT values when querying with the + full primary key specified (CASSANDRA-12605) + * Make sure repaired tombstones are dropped when only_purge_repaired_tombstones is enabled (CASSANDRA-12703) * Skip writing MV mutations to commitlog on mutation.applyUnsafe() (CASSANDRA-11670) * Establish consistent distinction between non-existing partition and NULL value for LWTs on static columns (CASSANDRA-12060) * Extend ColumnIdentifier.internedInstances key to include the type that generated the byte buffer (CASSANDRA-12516) http://git-wip-us.apache.org/repos/asf/cassandra/blob/12f5ca36/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java ---------------------------------------------------------------------- diff --cc src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java index 67e0d45,23b02f3..52df6a4 --- a/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java +++ b/src/java/org/apache/cassandra/db/SinglePartitionReadCommand.java @@@ -527,12 -511,12 +527,12 @@@ public class SinglePartitionReadComman * 2) If we have a name filter (so we query specific rows), we can make a bet: that all column for all queried row * will have data in the most recent sstable(s), thus saving us from reading older ones. This does imply we * have a way to guarantee we have all the data for what is queried, which is only possible for name queries - * and if we have neither collections nor counters (indeed, for a collection, we can't guarantee an older sstable - * won't have some elements that weren't in the most recent sstables, and counters are intrinsically a collection - * of shards so have the same problem). + * and if we have neither non-frozen collections/UDTs nor counters (indeed, for a non-frozen collection or UDT, + * we can't guarantee an older sstable won't have some elements that weren't in the most recent sstables, + * and counters are intrinsically a collection of shards and so have the same problem). */ - if (clusteringIndexFilter() instanceof ClusteringIndexNamesFilter && queryNeitherCountersNorCollections()) + if (clusteringIndexFilter() instanceof ClusteringIndexNamesFilter && !queriesMulticellType()) - return queryMemtableAndSSTablesInTimestampOrder(cfs, copyOnHeap, (ClusteringIndexNamesFilter)clusteringIndexFilter()); + return queryMemtableAndSSTablesInTimestampOrder(cfs, (ClusteringIndexNamesFilter)clusteringIndexFilter()); Tracing.trace("Acquiring sstable references"); ColumnFamilyStore.ViewFragment view = cfs.select(View.select(SSTableSet.LIVE, partitionKey())); @@@ -659,53 -662,7 +659,53 @@@ return clusteringIndexFilter().shouldInclude(sstable); } + private UnfilteredRowIteratorWithLowerBound makeIterator(ColumnFamilyStore cfs, final SSTableReader sstable, boolean applyThriftTransformation) + { + return StorageHook.instance.makeRowIteratorWithLowerBound(cfs, + partitionKey(), + sstable, + clusteringIndexFilter(), + columnFilter(), + isForThrift(), + nowInSec(), + applyThriftTransformation); + + } + + /** + * Return a wrapped iterator that when closed will update the sstables iterated and READ sample metrics. + * Note that we cannot use the Transformations framework because they greedily get the static row, which + * would cause all iterators to be initialized and hence all sstables to be accessed. + */ + private UnfilteredRowIterator withSSTablesIterated(List<UnfilteredRowIterator> iterators, + TableMetrics metrics) + { + @SuppressWarnings("resource") // Closed through the closing of the result of the caller method. + UnfilteredRowIterator merged = UnfilteredRowIterators.merge(iterators, nowInSec()); + + if (!merged.isEmpty()) + { + DecoratedKey key = merged.partitionKey(); + metrics.samplers.get(TableMetrics.Sampler.READS).addSample(key.getKey(), key.hashCode(), 1); + } + + class UpdateSstablesIterated extends Transformation + { + public void onPartitionClose() + { + int sstablesIterated = (int)iterators.stream() + .filter(it -> it instanceof LazilyInitializedUnfilteredRowIterator) + .filter(it -> ((LazilyInitializedUnfilteredRowIterator)it).initialized()) + .count(); + + metrics.updateSSTableIterated(sstablesIterated); + Tracing.trace("Merged data from memtables and {} sstables", sstablesIterated); + } + }; + return Transformation.apply(merged, new UpdateSstablesIterated()); + } + - private boolean queryNeitherCountersNorCollections() + private boolean queriesMulticellType() { for (ColumnDefinition column : columnFilter().fetchedColumns()) {