You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2015/05/01 18:20:38 UTC

[1/3] cassandra git commit: Fix index selection during rebuild with certain table layouts

Repository: cassandra
Updated Branches:
  refs/heads/trunk 184f5b3ad -> c799a98f0


Fix index selection during rebuild with certain table layouts

patch by Sam Tunnicliffe; reviewed by Jeremiah Jordan for CASSANDRA-9281


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

Branch: refs/heads/trunk
Commit: 1e35fa4b5a9afb4f37147a7a932a7892f2f450c1
Parents: f0c7a6f
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Fri May 1 10:17:39 2015 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri May 1 18:55:08 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                               |  3 +++
 .../apache/cassandra/db/index/SecondaryIndexManager.java  |  2 +-
 .../org/apache/cassandra/db/ColumnFamilyStoreTest.java    | 10 +++++++---
 .../cassandra/db/index/PerRowSecondaryIndexTest.java      |  9 +--------
 4 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e35fa4b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index dfdad51..2dabbf9 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.15:
+ * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281)
  * Fix partition-level-delete-only workload accounting (CASSANDRA-9194)
  * Allow scrub to handle corrupted compressed chunks (CASSANDRA-9140)
  * Fix assertion error when resetlocalschema is run during repair (CASSANDRA-9249)
@@ -57,9 +58,11 @@
  * Fix MT mismatch between empty and GC-able data (CASSANDRA-8979)
  * Fix incorrect validation when snapshotting single table (CASSANDRA-8056)
 
+
 2.0.14
  * Bind JMX to localhost unless explicitly configured otherwise (CASSANDRA-9085)
 
+
 2.0.13:
  * Add offline tool to relevel sstables (CASSANDRA-8301)
  * Preserve stream ID for more protocol errors (CASSANDRA-8848)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e35fa4b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index 1db7de6..d5e88d0 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -589,7 +589,7 @@ public class SecondaryIndexManager
         {
             for (ColumnDefinition column : baseCfs.metadata.allColumns())
             {
-                if (candidate.indexes(column.name))
+                if (candidate.getColumnDefs().contains(column))
                 {
                     filtered.add(candidate.getIndexName());
                     break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e35fa4b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 8f4a18f..001f838 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -2177,14 +2177,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRebuildSecondaryIndex() throws IOException
     {
+        ByteBuffer indexedColumnName = ByteBufferUtil.bytes("indexed");
         RowMutation rm;
 
         rm = new RowMutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("indexed"), ByteBufferUtil.bytes("foo"), 1);
+        rm.add("Indexed1", indexedColumnName, ByteBufferUtil.bytes("foo"), 1);
         rm.apply();
         assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
 
-        Keyspace.open("PerRowSecondaryIndex").getColumnFamilyStore("Indexed1").forceBlockingFlush();
+        ColumnFamilyStore cfs = Keyspace.open("PerRowSecondaryIndex").getColumnFamilyStore("Indexed1");
+        cfs.forceBlockingFlush();
 
         PerRowSecondaryIndexTest.TestIndex.reset();
 
@@ -2193,7 +2195,9 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         PerRowSecondaryIndexTest.TestIndex.reset();
 
-        PerRowSecondaryIndexTest.TestIndex.ACTIVE = false;
+        ColumnDefinition indexedColumnDef = cfs.metadata.getColumnDefinition(indexedColumnName);
+        cfs.indexManager.getIndexForColumn(indexedColumnName).getColumnDefs().remove(indexedColumnDef);
+
         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
         assertNull(PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1e35fa4b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
index 6a6956c..81173b2 100644
--- a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
@@ -27,6 +27,7 @@ import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.QueryFilter;
@@ -115,24 +116,16 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
 
     public static class TestIndex extends PerRowSecondaryIndex
     {
-        public static volatile boolean ACTIVE = true;
         public static ColumnFamily LAST_INDEXED_ROW;
         public static ByteBuffer LAST_INDEXED_KEY;
 
         public static void reset()
         {
-            ACTIVE = true;
             LAST_INDEXED_KEY = null;
             LAST_INDEXED_ROW = null;
         }
 
         @Override
-        public boolean indexes(ByteBuffer name)
-        {
-            return ACTIVE;
-        }
-
-        @Override
         public void index(ByteBuffer rowKey, ColumnFamily cf)
         {
             QueryFilter filter = QueryFilter.getIdentityFilter(DatabaseDescriptor.getPartitioner().decorateKey(rowKey),


[2/3] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1

Posted by al...@apache.org.
Merge branch 'cassandra-2.0' into cassandra-2.1


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

Branch: refs/heads/trunk
Commit: 33c59132f730599859407a8460f1433e36f83fdf
Parents: 5fcf8df 1e35fa4
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri May 1 19:01:00 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri May 1 19:01:00 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                               |  5 +++++
 .../org/apache/cassandra/db/index/SecondaryIndex.java     | 10 ----------
 .../apache/cassandra/db/index/SecondaryIndexManager.java  |  2 +-
 .../org/apache/cassandra/db/ColumnFamilyStoreTest.java    | 10 +++++++---
 .../cassandra/db/index/PerRowSecondaryIndexTest.java      |  8 --------
 5 files changed, 13 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/33c59132/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 55b474b,2dabbf9..b0e4fc6
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,101 -1,13 +1,103 @@@
 -2.0.15:
 +2.1.6
 + * Reduce runWithCompactionsDisabled poll interval to 1ms (CASSANDRA-9271)
 + * Fix PITR commitlog replay (CASSANDRA-9195)
 + * GCInspector logs very different times (CASSANDRA-9124)
 + * Fix deleting from an empty list (CASSANDRA-9198)
 + * Update tuple and collection types that use a user-defined type when that UDT
 +   is modified (CASSANDRA-9148, CASSANDRA-9192)
 +Merged from 2.0:
+  * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281)
   * Fix partition-level-delete-only workload accounting (CASSANDRA-9194)
   * Allow scrub to handle corrupted compressed chunks (CASSANDRA-9140)
   * Fix assertion error when resetlocalschema is run during repair (CASSANDRA-9249)
   * Disable single sstable tombstone compactions for DTCS by default (CASSANDRA-9234)
 - * Do more agressive ttl expiration checks to be able to
 -   drop more sstables (CASSANDRA-8243)
   * IncomingTcpConnection thread is not named (CASSANDRA-9262)
   * Close incoming connections when MessagingService is stopped (CASSANDRA-9238)
 +
++
 +2.1.5
 + * Re-add deprecated cold_reads_to_omit param for backwards compat (CASSANDRA-9203)
 + * Make anticompaction visible in compactionstats (CASSANDRA-9098)
 + * Improve nodetool getendpoints documentation about the partition
 +   key parameter (CASSANDRA-6458)
 + * Don't check other keyspaces for schema changes when an user-defined
 +   type is altered (CASSANDRA-9187)
 + * 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)
 + * Fix NullPointerException when deleting or setting an element by index on
 +   a null list collection (CASSANDRA-9077)
 + * Buffer bloom filter serialization (CASSANDRA-9066)
 + * Fix anti-compaction target bloom filter size (CASSANDRA-9060)
 + * Make FROZEN and TUPLE unreserved keywords in CQL (CASSANDRA-9047)
 + * Prevent AssertionError from SizeEstimatesRecorder (CASSANDRA-9034)
 + * Avoid overwriting index summaries for sstables with an older format that
 +   does not support downsampling; rebuild summaries on startup when this
 +   is detected (CASSANDRA-8993)
 + * Fix potential data loss in CompressedSequentialWriter (CASSANDRA-8949)
 + * Make PasswordAuthenticator number of hashing rounds configurable (CASSANDRA-8085)
 + * Fix AssertionError when binding nested collections in DELETE (CASSANDRA-8900)
 + * Check for overlap with non-early sstables in LCS (CASSANDRA-8739)
 + * Only calculate max purgable timestamp if we have to (CASSANDRA-8914)
 + * (cqlsh) Greatly improve performance of COPY FROM (CASSANDRA-8225)
 + * IndexSummary effectiveIndexInterval is now a guideline, not a rule (CASSANDRA-8993)
 + * Use correct bounds for page cache eviction of compressed files (CASSANDRA-8746)
 + * SSTableScanner enforces its bounds (CASSANDRA-8946)
 + * Cleanup cell equality (CASSANDRA-8947)
 + * Introduce intra-cluster message coalescing (CASSANDRA-8692)
 + * DatabaseDescriptor throws NPE when rpc_interface is used (CASSANDRA-8839)
 + * 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)
 + * 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)
 + * Make EstimatedHistogram#percentile() use ceil instead of floor (CASSANDRA-8883)
 + * Fix top partitions reporting wrong cardinality (CASSANDRA-8834)
 + * Fix rare NPE in KeyCacheSerializer (CASSANDRA-8067)
 + * Pick sstables for validation as late as possible inc repairs (CASSANDRA-8366)
 + * Fix commitlog getPendingTasks to not increment (CASSANDRA-8856)
 + * Fix parallelism adjustment in range and secondary index queries
 +   when the first fetch does not satisfy the limit (CASSANDRA-8856)
 + * Check if the filtered sstables is non-empty in STCS (CASSANDRA-8843)
 + * Upgrade java-driver used for cassandra-stress (CASSANDRA-8842)
 + * Fix CommitLog.forceRecycleAllSegments() memory access error (CASSANDRA-8812)
 + * Improve assertions in Memory (CASSANDRA-8792)
 + * Fix SSTableRewriter cleanup (CASSANDRA-8802)
 + * Introduce SafeMemory for CompressionMetadata.Writer (CASSANDRA-8758)
 + * 'nodetool info' prints exception against older node (CASSANDRA-8796)
 + * Ensure SSTableReader.last corresponds exactly with the file end (CASSANDRA-8750)
 + * Make SSTableWriter.openEarly more robust and obvious (CASSANDRA-8747)
 + * Enforce SSTableReader.first/last (CASSANDRA-8744)
 + * Cleanup SegmentedFile API (CASSANDRA-8749)
 + * Avoid overlap with early compaction replacement (CASSANDRA-8683)
 + * Safer Resource Management++ (CASSANDRA-8707)
 + * Write partition size estimates into a system table (CASSANDRA-7688)
 + * cqlsh: Fix keys() and full() collection indexes in DESCRIBE output
 +   (CASSANDRA-8154)
 + * Show progress of streaming in nodetool netstats (CASSANDRA-8886)
 + * IndexSummaryBuilder utilises offheap memory, and shares data between
 +   each IndexSummary opened from it (CASSANDRA-8757)
 + * markCompacting only succeeds if the exact SSTableReader instances being 
 +   marked are in the live set (CASSANDRA-8689)
 + * cassandra-stress support for varint (CASSANDRA-8882)
 + * Fix Adler32 digest for compressed sstables (CASSANDRA-8778)
 + * Add nodetool statushandoff/statusbackup (CASSANDRA-8912)
 + * Use stdout for progress and stats in sstableloader (CASSANDRA-8982)
 + * Correctly identify 2i datadir from older versions (CASSANDRA-9116)
 +Merged from 2.0:
 + * Ignore gossip SYNs after shutdown (CASSANDRA-9238)
   * Avoid overflow when calculating max sstable size in LCS (CASSANDRA-9235)
   * Make sstable blacklisting work with compression (CASSANDRA-9138)
   * Do not attempt to rebuild indexes if no index accepts any column (CASSANDRA-9196)
@@@ -150,107 -74,10 +152,109 @@@
     table with ASC ordering and paging (CASSANDRA-8767)
   * AssertionError: "Memory was freed" when running cleanup (CASSANDRA-8716)
   * Make it possible to set max_sstable_age to fractional days (CASSANDRA-8406)
 - * Fix memory leak in SSTableSimple*Writer and SSTableReader.validate()
 -   (CASSANDRA-8748)
   * Fix some multi-column relations with indexes on some clustering
     columns (CASSANDRA-8275)
 + * Fix memory leak in SSTableSimple*Writer and SSTableReader.validate()
 +   (CASSANDRA-8748)
 + * Throw OOM if allocating memory fails to return a valid pointer (CASSANDRA-8726)
 + * Fix SSTableSimpleUnsortedWriter ConcurrentModificationException (CASSANDRA-8619)
 + * 'nodetool info' prints exception against older node (CASSANDRA-8796)
 + * Ensure SSTableSimpleUnsortedWriter.close() terminates if
 +   disk writer has crashed (CASSANDRA-8807)
 +
++
 +2.1.4
 + * Bind JMX to localhost unless explicitly configured otherwise (CASSANDRA-9085)
 +
++
 +2.1.3
 + * Fix HSHA/offheap_objects corruption (CASSANDRA-8719)
 + * Upgrade libthrift to 0.9.2 (CASSANDRA-8685)
 + * Don't use the shared ref in sstableloader (CASSANDRA-8704)
 + * Purge internal prepared statements if related tables or
 +   keyspaces are dropped (CASSANDRA-8693)
 + * (cqlsh) Handle unicode BOM at start of files (CASSANDRA-8638)
 + * Stop compactions before exiting offline tools (CASSANDRA-8623)
 + * Update tools/stress/README.txt to match current behaviour (CASSANDRA-7933)
 + * Fix schema from Thrift conversion with empty metadata (CASSANDRA-8695)
 + * Safer Resource Management (CASSANDRA-7705)
 + * Make sure we compact highly overlapping cold sstables with
 +   STCS (CASSANDRA-8635)
 + * rpc_interface and listen_interface generate NPE on startup when specified
 +   interface doesn't exist (CASSANDRA-8677)
 + * Fix ArrayIndexOutOfBoundsException in nodetool cfhistograms (CASSANDRA-8514)
 + * Switch from yammer metrics for nodetool cf/proxy histograms (CASSANDRA-8662)
 + * Make sure we don't add tmplink files to the compaction
 +   strategy (CASSANDRA-8580)
 + * (cqlsh) Handle maps with blob keys (CASSANDRA-8372)
 + * (cqlsh) Handle DynamicCompositeType schemas correctly (CASSANDRA-8563)
 + * Duplicate rows returned when in clause has repeated values (CASSANDRA-6706)
 + * Add tooling to detect hot partitions (CASSANDRA-7974)
 + * Fix cassandra-stress user-mode truncation of partition generation (CASSANDRA-8608)
 + * Only stream from unrepaired sstables during inc repair (CASSANDRA-8267)
 + * Don't allow starting multiple inc repairs on the same sstables (CASSANDRA-8316)
 + * Invalidate prepared BATCH statements when related tables
 +   or keyspaces are dropped (CASSANDRA-8652)
 + * Fix missing results in secondary index queries on collections
 +   with ALLOW FILTERING (CASSANDRA-8421)
 + * Expose EstimatedHistogram metrics for range slices (CASSANDRA-8627)
 + * (cqlsh) Escape clqshrc passwords properly (CASSANDRA-8618)
 + * Fix NPE when passing wrong argument in ALTER TABLE statement (CASSANDRA-8355)
 + * Pig: Refactor and deprecate CqlStorage (CASSANDRA-8599)
 + * Don't reuse the same cleanup strategy for all sstables (CASSANDRA-8537)
 + * Fix case-sensitivity of index name on CREATE and DROP INDEX
 +   statements (CASSANDRA-8365)
 + * Better detection/logging for corruption in compressed sstables (CASSANDRA-8192)
 + * Use the correct repairedAt value when closing writer (CASSANDRA-8570)
 + * (cqlsh) Handle a schema mismatch being detected on startup (CASSANDRA-8512)
 + * Properly calculate expected write size during compaction (CASSANDRA-8532)
 + * Invalidate affected prepared statements when a table's columns
 +   are altered (CASSANDRA-7910)
 + * Stress - user defined writes should populate sequentally (CASSANDRA-8524)
 + * Fix regression in SSTableRewriter causing some rows to become unreadable 
 +   during compaction (CASSANDRA-8429)
 + * Run major compactions for repaired/unrepaired in parallel (CASSANDRA-8510)
 + * (cqlsh) Fix compression options in DESCRIBE TABLE output when compression
 +   is disabled (CASSANDRA-8288)
 + * (cqlsh) Fix DESCRIBE output after keyspaces are altered (CASSANDRA-7623)
 + * Make sure we set lastCompactedKey correctly (CASSANDRA-8463)
 + * (cqlsh) Fix output of CONSISTENCY command (CASSANDRA-8507)
 + * (cqlsh) Fixed the handling of LIST statements (CASSANDRA-8370)
 + * Make sstablescrub check leveled manifest again (CASSANDRA-8432)
 + * Check first/last keys in sstable when giving out positions (CASSANDRA-8458)
 + * Disable mmap on Windows (CASSANDRA-6993)
 + * Add missing ConsistencyLevels to cassandra-stress (CASSANDRA-8253)
 + * Add auth support to cassandra-stress (CASSANDRA-7985)
 + * Fix ArrayIndexOutOfBoundsException when generating error message
 +   for some CQL syntax errors (CASSANDRA-8455)
 + * Scale memtable slab allocation logarithmically (CASSANDRA-7882)
 + * cassandra-stress simultaneous inserts over same seed (CASSANDRA-7964)
 + * Reduce cassandra-stress sampling memory requirements (CASSANDRA-7926)
 + * Ensure memtable flush cannot expire commit log entries from its future (CASSANDRA-8383)
 + * Make read "defrag" async to reclaim memtables (CASSANDRA-8459)
 + * Remove tmplink files for offline compactions (CASSANDRA-8321)
 + * Reduce maxHintsInProgress (CASSANDRA-8415)
 + * 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
 +   partition (CASSANDRA-8203)
 + * Do more aggressive entire-sstable TTL expiry checks (CASSANDRA-8243)
 + * Add more log info if readMeter is null (CASSANDRA-8238)
 + * add check of the system wall clock time at startup (CASSANDRA-8305)
 + * Support for frozen collections (CASSANDRA-7859)
 + * Fix overflow on histogram computation (CASSANDRA-8028)
 + * Have paxos reuse the timestamp generation of normal queries (CASSANDRA-7801)
 + * Fix incremental repair not remove parent session on remote (CASSANDRA-8291)
 + * Improve JBOD disk utilization (CASSANDRA-7386)
 + * Log failed host when preparing incremental repair (CASSANDRA-8228)
 + * Force config client mode in CQLSSTableWriter (CASSANDRA-8281)
 + * Fix sstableupgrade throws exception (CASSANDRA-8688)
 + * Fix hang when repairing empty keyspace (CASSANDRA-8694)
 +Merged from 2.0:
   * Fix IllegalArgumentException in dynamic snitch (CASSANDRA-8448)
   * Add support for UPDATE ... IF EXISTS (CASSANDRA-8610)
   * Fix reversal of list prepends (CASSANDRA-8733)
@@@ -310,36 -144,7 +314,37 @@@
   * Avoid overlap in L1 when L0 contains many nonoverlapping
     sstables (CASSANDRA-8211)
   * Improve PropertyFileSnitch logging (CASSANDRA-8183)
 - * Abort liveRatio calculation if the memtable is flushed (CASSANDRA-8164)
 + * Add DC-aware sequential repair (CASSANDRA-8193)
 + * Use live sstables in snapshot repair if possible (CASSANDRA-8312)
 + * Fix hints serialized size calculation (CASSANDRA-8587)
 +
++
 +2.1.2
 + * (cqlsh) parse_for_table_meta errors out on queries with undefined
 +   grammars (CASSANDRA-8262)
 + * (cqlsh) Fix SELECT ... TOKEN() function broken in C* 2.1.1 (CASSANDRA-8258)
 + * Fix Cassandra crash when running on JDK8 update 40 (CASSANDRA-8209)
 + * Optimize partitioner tokens (CASSANDRA-8230)
 + * Improve compaction of repaired/unrepaired sstables (CASSANDRA-8004)
 + * Make cache serializers pluggable (CASSANDRA-8096)
 + * Fix issues with CONTAINS (KEY) queries on secondary indexes
 +   (CASSANDRA-8147)
 + * Fix read-rate tracking of sstables for some queries (CASSANDRA-8239)
 + * Fix default timestamp in QueryOptions (CASSANDRA-8246)
 + * Set socket timeout when reading remote version (CASSANDRA-8188)
 + * Refactor how we track live size (CASSANDRA-7852)
 + * Make sure unfinished compaction files are removed (CASSANDRA-8124)
 + * Fix shutdown when run as Windows service (CASSANDRA-8136)
 + * Fix DESCRIBE TABLE with custom indexes (CASSANDRA-8031)
 + * Fix race in RecoveryManagerTest (CASSANDRA-8176)
 + * Avoid IllegalArgumentException while sorting sstables in
 +   IndexSummaryManager (CASSANDRA-8182)
 + * Shutdown JVM on file descriptor exhaustion (CASSANDRA-7579)
 + * Add 'die' policy for commit log and disk failure (CASSANDRA-7927)
 + * Fix installing as service on Windows (CASSANDRA-8115)
 + * Fix CREATE TABLE for CQL2 (CASSANDRA-8144)
 + * Avoid boxing in ColumnStats min/max trackers (CASSANDRA-8109)
 +Merged from 2.0:
   * Correctly handle non-text column names in cql3 (CASSANDRA-8178)
   * Fix deletion for indexes on primary key columns (CASSANDRA-8206)
   * Add 'nodetool statusgossip' (CASSANDRA-8125)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33c59132/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/index/SecondaryIndex.java
index 5a6aecc,a83f5dd..3081016
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@@ -304,21 -290,22 +304,11 @@@ public abstract class SecondaryInde
      }
  
      /**
 -     * Returns true if the provided column name is indexed by this secondary index.
 -     *
 -     * The default implement checks whether the name is one the columnDef name,
 -     * but this should be overriden but subclass if needed.
 +     * Returns true if the provided cell name is indexed by this secondary index.
       */
 -    public boolean indexes(ByteBuffer name)
 -    {
 -        for (ColumnDefinition columnDef : columnDefs)
 -        {
 -            if (baseCfs.getComparator().compare(columnDef.name, name) == 0)
 -                return true;
 -        }
 -        return false;
 -    }
 +    public abstract boolean indexes(CellName name);
  
      /**
-      * Returns true if the defined column is indexed by this secondary index.
-      * @param column definition of the column to check
-      * @return whether the supplied column is indexed or not
-      */
-     public boolean indexes(ColumnDefinition column)
-     {
-         return columnDefs.contains(column);
-     }
- 
-     /**
       * This is the primary way to create a secondary index instance for a CF column.
       * It will validate the index_options before initializing.
       *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33c59132/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33c59132/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index a095dba,001f838..84e7d20
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@@ -2168,10 -2177,11 +2168,11 @@@ public class ColumnFamilyStoreTest exte
      @Test
      public void testRebuildSecondaryIndex() throws IOException
      {
 -        ByteBuffer indexedColumnName = ByteBufferUtil.bytes("indexed");
 -        RowMutation rm;
++        CellName indexedCellName = cellname("indexed");
 +        Mutation rm;
  
 -        rm = new RowMutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
 -        rm.add("Indexed1", indexedColumnName, ByteBufferUtil.bytes("foo"), 1);
 +        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
-         rm.add("Indexed1", cellname("indexed"), ByteBufferUtil.bytes("foo"), 1);
++        rm.add("Indexed1", indexedCellName, ByteBufferUtil.bytes("foo"), 1);
          rm.apply();
          assertTrue(Arrays.equals("k1".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
  
@@@ -2184,7 -2195,9 +2186,9 @@@
  
          PerRowSecondaryIndexTest.TestIndex.reset();
  
-         PerRowSecondaryIndexTest.TestIndex.ACTIVE = false;
 -        ColumnDefinition indexedColumnDef = cfs.metadata.getColumnDefinition(indexedColumnName);
 -        cfs.indexManager.getIndexForColumn(indexedColumnName).getColumnDefs().remove(indexedColumnDef);
++        ColumnDefinition indexedColumnDef = cfs.metadata.getColumnDefinition(indexedCellName);
++        cfs.indexManager.getIndexForColumn(indexedColumnDef.name.bytes).getColumnDefs().remove(indexedColumnDef);
+ 
          ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
          assertNull(PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY);
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33c59132/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
----------------------------------------------------------------------


[3/3] cassandra git commit: Merge branch 'cassandra-2.1' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: c799a98f0a454010cb3a63c874c404d1d2c79048
Parents: 184f5b3 33c5913
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Fri May 1 19:21:04 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Fri May 1 19:21:04 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  5 +++
 .../cassandra/db/index/SecondaryIndex.java      | 10 ------
 .../db/index/SecondaryIndexManager.java         |  2 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 37 ++++++++++++++++++++
 .../db/index/PerRowSecondaryIndexTest.java      |  8 -----
 5 files changed, 43 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c799a98f/CHANGES.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c799a98f/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c799a98f/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c799a98f/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 0c221ac,84e7d20..349edbe
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@@ -43,12 -30,8 +43,14 @@@ import java.util.concurrent.TimeUnit
  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.BeforeClass;
  import org.junit.Test;
  import org.junit.runner.RunWith;
  
@@@ -105,23 -70,9 +107,24 @@@ import static org.junit.Assert.assertSa
  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_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
      {
@@@ -132,36 -83,6 +135,40 @@@
          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.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()
@@@ -2243,4 -2164,34 +2250,34 @@@
          });
          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()));
+ 
+         ColumnFamilyStore cfs = Keyspace.open("PerRowSecondaryIndex").getColumnFamilyStore("Indexed1");
+         cfs.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();
+ 
+         ColumnDefinition indexedColumnDef = cfs.metadata.getColumnDefinition(indexedCellName);
+         cfs.indexManager.getIndexForColumn(indexedColumnDef.name.bytes).getColumnDefs().remove(indexedColumnDef);
+ 
+         ColumnFamilyStore.rebuildSecondaryIndex("PerRowSecondaryIndex", "Indexed1", PerRowSecondaryIndexTest.TestIndex.class.getSimpleName());
+         assertNull(PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY);
+ 
+         PerRowSecondaryIndexTest.TestIndex.reset();
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c799a98f/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
----------------------------------------------------------------------