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/01/26 23:45:07 UTC

[1/2] cassandra git commit: Simplify logic of ABSC#BatchRemoveIterator#commit()

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 1bb0c149e -> 33297bad0


Simplify logic of ABSC#BatchRemoveIterator#commit()

patch by Aleksey Yeschenko; reviewed by Joshua McKenzie for
CASSANDRA-8666


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

Branch: refs/heads/cassandra-2.1
Commit: b077cda83bb1d32d5e6836027cfdfe3009b266b4
Parents: 2bf63f6
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jan 27 00:20:38 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jan 27 00:20:38 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +-
 .../cassandra/db/ArrayBackedSortedColumns.java  | 53 +++++++-------------
 2 files changed, 20 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b077cda8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 0d08cce..792f8c1 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,6 +1,6 @@
 2.0.13:
  * Round up time deltas lower than 1ms in BulkLoader (CASSANDRA-8645)
- * Add batch remove iterator to ABSC (CASSANDRA-8414)
+ * Add batch remove iterator to ABSC (CASSANDRA-8414, 8666)
 
 
 2.0.12:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b077cda8/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
index 8d553be..482f04f 100644
--- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
@@ -314,43 +314,28 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
                 if (!removedAnything)
                     return;
 
-                // the lowest index both not visited and known to be not removed
-                int keepIdx = removedIndexes.nextClearBit(0);
-                // the running total of kept items
-                int resultLength = 0;
-                // start from the first not-removed cell, and shift left.
-                int removeIdx = removedIndexes.nextSetBit(keepIdx + 1);
-                while (removeIdx >= 0)
-                {
-                    int length = removeIdx - keepIdx;
-                    if (length > 0)
-                    {
-                        copy(keepIdx, resultLength, length);
-                        resultLength += length;
-                    }
-                    keepIdx = removedIndexes.nextClearBit(removeIdx + 1);
-                    if (keepIdx < 0)
-                        keepIdx = columns.size();
-                    removeIdx = removedIndexes.nextSetBit(keepIdx + 1);
-                }
-                // Copy everything after the last deleted column
-                int length = columns.size() - keepIdx;
-                if (length > 0)
+                int size = columns.size();
+                int retainedCount = 0;
+                int clearIdx, setIdx = -1;
+
+                // shift all [clearIdx, setIdx) segments to the left, skipping any removed columns
+                while (true)
                 {
-                    copy(keepIdx, resultLength, length);
-                    resultLength += length;
+                    clearIdx = removedIndexes.nextClearBit(setIdx + 1);
+                    if (clearIdx >= size)
+                        break; // nothing left to retain
+
+                    setIdx = removedIndexes.nextSetBit(clearIdx + 1);
+                    if (setIdx < 0)
+                        setIdx = size; // no removals past retainIdx - copy all remaining cells
+
+                    if (retainedCount != clearIdx)
+                        Collections.copy(columns.subList(retainedCount, retainedCount + setIdx - clearIdx),
+                                         columns.subList(clearIdx, setIdx));
+                    retainedCount += (setIdx - clearIdx);
                 }
 
-                columns.subList(resultLength, columns.size()).clear();
-            }
-
-            private void copy(int src, int dst, int len)
-            {
-                // [src, src+len) and [dst, dst+len) might overlap, but it's okay because we're going from left to right
-                assert dst <= src : "dst must not be greater than src";
-
-                if (dst < src)
-                    Collections.copy(columns.subList(dst, dst + len), columns.subList(src, src + len));
+                columns.subList(retainedCount, size).clear();
             }
 
             public boolean hasNext()


[2/2] 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

Conflicts:
	CHANGES.txt
	src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java


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

Branch: refs/heads/cassandra-2.1
Commit: 33297bad040c660c8cd5ea6378189fffb100b0ab
Parents: 1bb0c14 b077cda
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Jan 27 01:44:54 2015 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Jan 27 01:44:54 2015 +0300

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +-
 .../cassandra/db/ArrayBackedSortedColumns.java  | 54 +++++++-------------
 2 files changed, 20 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/33297bad/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 7673a3b,792f8c1..4d50496
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,77 -1,9 +1,77 @@@
 -2.0.13:
 - * Round up time deltas lower than 1ms in BulkLoader (CASSANDRA-8645)
 +2.1.3
 + * 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-6707)
 + * 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)
 +Merged from 2.0:
-  * Add batch remove iterator to ABSC (CASSANDRA-8414)
+  * Add batch remove iterator to ABSC (CASSANDRA-8414, 8666)
 -
 -
 -2.0.12:
 + * Round up time deltas lower than 1ms in BulkLoader (CASSANDRA-8645)
   * Use more efficient slice size for querying internal secondary
     index tables (CASSANDRA-8550)
   * Fix potentially returning deleted rows with range tombstone (CASSANDRA-8558)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/33297bad/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
index 64752e3,482f04f..c53832b
--- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
@@@ -110,103 -80,15 +110,87 @@@ public class ArrayBackedSortedColumns e
          return reversed;
      }
  
 -    private Comparator<ByteBuffer> internalComparator()
 +    public BatchRemoveIterator<Cell> batchRemoveIterator()
      {
 -        return reversed ? getComparator().reverseComparator : getComparator();
 +        maybeSortCells();
 +
 +        return new BatchRemoveIterator<Cell>()
 +        {
 +            private final Iterator<Cell> iter = iterator();
 +            private BitSet removedIndexes = new BitSet(size);
 +            private int idx = -1;
 +            private boolean shouldCallNext = false;
 +            private boolean isCommitted = false;
 +            private boolean removedAnything = false;
 +
 +            public void commit()
 +            {
 +                if (isCommitted)
 +                    throw new IllegalStateException();
 +                isCommitted = true;
 +
 +                if (!removedAnything)
 +                    return;
 +
-                 // the lowest index both not visited and known to be not removed
-                 int keepIdx = removedIndexes.nextClearBit(0);
-                 // the running total of kept items
-                 int resultLength = 0;
-                 // start from the first not-removed cell, and shift left.
-                 int removeIdx = removedIndexes.nextSetBit(keepIdx + 1);
-                 while (removeIdx >= 0)
-                 {
-                     int length = removeIdx - keepIdx;
-                     if (length > 0)
-                     {
-                         copy(keepIdx, resultLength, length);
-                         resultLength += length;
-                     }
-                     keepIdx = removedIndexes.nextClearBit(removeIdx + 1);
-                     if (keepIdx < 0)
-                         keepIdx = size;
-                     removeIdx = removedIndexes.nextSetBit(keepIdx + 1);
-                 }
-                 // Copy everything after the last deleted column
-                 int length = size - keepIdx;
-                 if (length > 0)
++                int retainedCount = 0;
++                int clearIdx, setIdx = -1;
++
++                // shift all [clearIdx, setIdx) segments to the left, skipping any removed columns
++                while (true)
 +                {
-                     copy(keepIdx, resultLength, length);
-                     resultLength += length;
-                 }
++                    clearIdx = removedIndexes.nextClearBit(setIdx + 1);
++                    if (clearIdx >= size)
++                        break; // nothing left to retain
 +
-                 for (int i = resultLength; i < size; i++)
-                     cells[i] = null;
++                    setIdx = removedIndexes.nextSetBit(clearIdx + 1);
++                    if (setIdx < 0)
++                        setIdx = size; // no removals past retainIdx - copy all remaining cells
 +
-                 size = sortedSize = resultLength;
-             }
++                    if (retainedCount != clearIdx)
++                        System.arraycopy(cells, clearIdx, cells, retainedCount, setIdx - clearIdx);
 +
-             private void copy(int src, int dst, int len)
-             {
-                 // [src, src+len) and [dst, dst+len) might overlap but it's okay because we're going from left to right
-                 assert dst <= src : "dst must not be greater than src";
++                    retainedCount += (setIdx - clearIdx);
++                }
++
++                for (int i = retainedCount; i < size; i++)
++                    cells[i] = null;
 +
-                 if (dst < src)
-                     System.arraycopy(cells, src, cells, dst, len);
++                size = sortedSize = retainedCount;
 +            }
 +
 +            public boolean hasNext()
 +            {
 +                return iter.hasNext();
 +            }
 +
 +            public Cell next()
 +            {
 +                idx++;
 +                shouldCallNext = false;
 +                return iter.next();
 +            }
 +
 +            public void remove()
 +            {
 +                if (shouldCallNext)
 +                    throw new IllegalStateException();
 +
 +                removedIndexes.set(reversed ? size - idx - 1 : idx);
 +                removedAnything = true;
 +                shouldCallNext = true;
 +            }
 +        };
      }
  
 -    public Column getColumn(ByteBuffer name)
 +    private Comparator<Composite> internalComparator()
      {
 -        int pos = binarySearch(name);
 -        return pos >= 0 ? columns.get(pos) : null;
 +        return reversed ? getComparator().reverseComparator() : getComparator();
 +    }
 +
 +    private void maybeSortCells()
 +    {
 +        if (!isSorted)
 +            sortCells();
      }
  
      /**