You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/11/12 19:23:23 UTC

[1/5] git commit: Fix json2sstable handling of range tombstones

Updated Branches:
  refs/heads/trunk 9a59d250a -> bfd0144a1


Fix json2sstable handling of range tombstones

patch by Lyuben Todorov; reviewed by Aleksey Yeschenko for
CASSANDRA-6316


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

Branch: refs/heads/trunk
Commit: 159744f244125af84f4e819f45d2e2ebef2b369a
Parents: 512ea5a
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Tue Nov 12 20:09:02 2013 +0300
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Nov 12 20:09:02 2013 +0300

----------------------------------------------------------------------
 CHANGES.txt                                        |  1 +
 .../org/apache/cassandra/tools/SSTableImport.java  | 17 +++++++++++++++--
 2 files changed, 16 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/159744f2/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bc0d65b..e0a2320 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -14,6 +14,7 @@
  * Fix potential socket leak in connectionpool creation (CASSANDRA-6308)
  * Allow LOCAL_ONE/LOCAL_QUORUM to work with SimpleStrategy (CASSANDRA-6238)
  * cqlsh: handle 'null' as session duration (CASSANDRA-6317)
+ * Fix json2sstable handling of range tombstones (CASSANDRA-6316)
 
 
 1.2.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/159744f2/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 68d4397..80d1aad 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -22,6 +22,7 @@ import static org.apache.cassandra.utils.ByteBufferUtil.hexToBytes;
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import java.util.SortedMap;
@@ -50,6 +51,7 @@ import org.apache.cassandra.db.SuperColumn;
 import org.apache.cassandra.db.filter.QueryPath;
 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.MarshalException;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.sstable.SSTableWriter;
@@ -155,8 +157,19 @@ public class SSTableImport
                     }
                 }
 
-                value = isDeleted() ? ByteBufferUtil.hexToBytes((String) fields.get(1))
-                                    : stringAsType((String) fields.get(1), meta.getValueValidator(meta.getColumnDefinitionFromColumnName(name)));
+                if (isDeleted())
+                {
+                    value = ByteBufferUtil.hexToBytes((String) fields.get(1));
+                }
+                else if (isRangeTombstone())
+                {
+                    AbstractType<?> type = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{meta.getColumnDefinitionComparator(0)}));
+                    value = type.fromString((String)fields.get(1));
+                }
+                else
+                {
+                    value = stringAsType((String) fields.get(1), meta.getValueValidator(meta.getColumnDefinitionFromColumnName(name)));
+                }
             }
         }
 


[2/5] git commit: Fixing missing one row during reverse query on compact tables

Posted by sl...@apache.org.
Fixing missing one row during reverse query on compact tables

patch by slebresne; reviewed by iamaleksey for CASSANDRA-6330


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

Branch: refs/heads/trunk
Commit: 3cb5854e9271ffc4e338841a49dd0b11d18e8c4f
Parents: 159744f
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Nov 12 19:09:46 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Nov 12 19:09:46 2013 +0100

----------------------------------------------------------------------
 CHANGES.txt                                               |  1 +
 .../apache/cassandra/cql3/statements/SelectStatement.java | 10 +++++++---
 2 files changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/3cb5854e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e0a2320..7abf5d8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,7 @@
  * Allow LOCAL_ONE/LOCAL_QUORUM to work with SimpleStrategy (CASSANDRA-6238)
  * cqlsh: handle 'null' as session duration (CASSANDRA-6317)
  * Fix json2sstable handling of range tombstones (CASSANDRA-6316)
+ * Fix missing one row in reverse query (CASSANDRA-6330)
 
 
 1.2.11

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3cb5854e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 44a1e64..c1c88ba 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -376,9 +376,13 @@ public class SelectStatement implements CQLStatement
 
     private int getLimit()
     {
-        // Internally, we don't support exclusive bounds for slices. Instead,
-        // we query one more element if necessary and exclude
-        return sliceRestriction != null && !sliceRestriction.isInclusive(Bound.START) && parameters.limit != Integer.MAX_VALUE
+        // Internally, we don't support exclusive bounds for COMPACT slices. Instead, when we know we have an exlcusive
+        // slice on a COMPACT table, we query one more element (to make sure we don't return less results than asked post-exclusion)
+        // and exclude the post-query. Note that while we might excluse both the START and END bound, there is no reason to
+        // ask for limit + 2 since if we exlude both bound from the result it means we can't have missed non-fetched results.
+        return (sliceRestriction != null
+                && parameters.limit != Integer.MAX_VALUE
+                && (!sliceRestriction.isInclusive(Bound.START) || !sliceRestriction.isInclusive(Bound.END)))
              ? parameters.limit + 1
              : parameters.limit;
     }


[3/5] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0

Posted by sl...@apache.org.
Merge branch 'cassandra-1.2' into cassandra-2.0

Conflicts:
	src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
	src/java/org/apache/cassandra/tools/SSTableImport.java


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

Branch: refs/heads/trunk
Commit: c6af75a19b3d19fdb3aa6040b655793b58b0db97
Parents: 2cc4670 3cb5854
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Nov 12 19:13:00 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Nov 12 19:13:00 2013 +0100

----------------------------------------------------------------------
 CHANGES.txt                                       |  2 ++
 .../cql3/statements/SelectStatement.java          |  2 +-
 .../org/apache/cassandra/tools/SSTableImport.java | 18 +++++++++++++++---
 3 files changed, 18 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c6af75a1/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index cc5fb9e,7abf5d8..424a9f4
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -34,42 -14,11 +34,44 @@@ Merged from 1.2
   * Fix potential socket leak in connectionpool creation (CASSANDRA-6308)
   * Allow LOCAL_ONE/LOCAL_QUORUM to work with SimpleStrategy (CASSANDRA-6238)
   * cqlsh: handle 'null' as session duration (CASSANDRA-6317)
+  * Fix json2sstable handling of range tombstones (CASSANDRA-6316)
+  * Fix missing one row in reverse query (CASSANDRA-6330)
  
  
 -1.2.11
 +2.0.2
 + * Update FailureDetector to use nanontime (CASSANDRA-4925)
 + * Fix FileCacheService regressions (CASSANDRA-6149)
 + * Never return WriteTimeout for CL.ANY (CASSANDRA-6032)
 + * Fix race conditions in bulk loader (CASSANDRA-6129)
 + * Add configurable metrics reporting (CASSANDRA-4430)
 + * drop queries exceeding a configurable number of tombstones (CASSANDRA-6117)
 + * Track and persist sstable read activity (CASSANDRA-5515)
 + * Fixes for speculative retry (CASSANDRA-5932, CASSANDRA-6194)
 + * Improve memory usage of metadata min/max column names (CASSANDRA-6077)
 + * Fix thrift validation refusing row markers on CQL3 tables (CASSANDRA-6081)
 + * Fix insertion of collections with CAS (CASSANDRA-6069)
 + * Correctly send metadata on SELECT COUNT (CASSANDRA-6080)
 + * Track clients' remote addresses in ClientState (CASSANDRA-6070)
 + * Create snapshot dir if it does not exist when migrating
 +   leveled manifest (CASSANDRA-6093)
 + * make sequential nodetool repair the default (CASSANDRA-5950)
 + * Add more hooks for compaction strategy implementations (CASSANDRA-6111)
 + * Fix potential NPE on composite 2ndary indexes (CASSANDRA-6098)
 + * Delete can potentially be skipped in batch (CASSANDRA-6115)
 + * Allow alter keyspace on system_traces (CASSANDRA-6016)
 + * Disallow empty column names in cql (CASSANDRA-6136)
 + * Use Java7 file-handling APIs and fix file moving on Windows (CASSANDRA-5383)
 + * Save compaction history to system keyspace (CASSANDRA-5078)
 + * Fix NPE if StorageService.getOperationMode() is executed before full startup (CASSANDRA-6166)
 + * CQL3: support pre-epoch longs for TimestampType (CASSANDRA-6212)
 + * Add reloadtriggers command to nodetool (CASSANDRA-4949)
 + * cqlsh: ignore empty 'value alias' in DESCRIBE (CASSANDRA-6139)
 + * Fix sstable loader (CASSANDRA-6205)
 + * Reject bootstrapping if the node already exists in gossip (CASSANDRA-5571)
 + * Fix NPE while loading paxos state (CASSANDRA-6211)
 + * cqlsh: add SHOW SESSION <tracing-session> command (CASSANDRA-6228)
 +Merged from 1.2:
 + * (Hadoop) Require CFRR batchSize to be at least 2 (CASSANDRA-6114)
   * Add a warning for small LCS sstable size (CASSANDRA-6191)
   * Add ability to list specific KS/CF combinations in nodetool cfstats (CASSANDRA-4191)
   * Mark CF clean if a mutation raced the drop and got it marked dirty (CASSANDRA-5946)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c6af75a1/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 14400d5,c1c88ba..4b9a334
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@@ -405,35 -374,17 +405,35 @@@ public class SelectStatement implement
          }
      }
  
 -    private int getLimit()
 +    private int getLimit(List<ByteBuffer> variables) throws InvalidRequestException
      {
 -        // Internally, we don't support exclusive bounds for COMPACT slices. Instead, when we know we have an exlcusive
 -        // slice on a COMPACT table, we query one more element (to make sure we don't return less results than asked post-exclusion)
 -        // and exclude the post-query. Note that while we might excluse both the START and END bound, there is no reason to
 -        // ask for limit + 2 since if we exlude both bound from the result it means we can't have missed non-fetched results.
 -        return (sliceRestriction != null
 -                && parameters.limit != Integer.MAX_VALUE
 -                && (!sliceRestriction.isInclusive(Bound.START) || !sliceRestriction.isInclusive(Bound.END)))
 -             ? parameters.limit + 1
 -             : parameters.limit;
 +        int l = Integer.MAX_VALUE;
 +        if (limit != null)
 +        {
 +            ByteBuffer b = limit.bindAndGet(variables);
 +            if (b == null)
 +                throw new InvalidRequestException("Invalid null value of limit");
 +
 +            try
 +            {
 +                Int32Type.instance.validate(b);
 +                l = Int32Type.instance.compose(b);
 +            }
 +            catch (MarshalException e)
 +            {
 +                throw new InvalidRequestException("Invalid limit value");
 +            }
 +        }
 +
 +        if (l <= 0)
 +            throw new InvalidRequestException("LIMIT must be strictly positive");
 +
 +        // Internally, we don't support exclusive bounds for slices. Instead,
 +        // we query one more element if necessary and exclude
-         if (sliceRestriction != null && !sliceRestriction.isInclusive(Bound.START) && l != Integer.MAX_VALUE)
++        if (sliceRestriction != null && (!sliceRestriction.isInclusive(Bound.START) || !sliceRestriction.isInclusive(Bound.END)) && l != Integer.MAX_VALUE)
 +            l += 1;
 +
 +        return l;
      }
  
      private Collection<ByteBuffer> getKeys(final List<ByteBuffer> variables) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c6af75a1/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableImport.java
index 79584c5,80d1aad..0e8172b
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@@ -26,9 -27,7 +27,8 @@@ import java.util.List
  import java.util.Map;
  import java.util.SortedMap;
  import java.util.TreeMap;
 +import java.util.concurrent.TimeUnit;
  
- import org.apache.cassandra.serializers.MarshalException;
  import org.apache.commons.cli.CommandLine;
  import org.apache.commons.cli.CommandLineParser;
  import org.apache.commons.cli.Option;


[4/5] git commit: Fix missing results on query with IN clauses

Posted by sl...@apache.org.
Fix missing results on query with IN clauses

patch by slebresne; reviewed by iamaleksey for CASSANDRA-6327


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

Branch: refs/heads/trunk
Commit: acea7160ed8eb1779b9830821ea5efe89040769c
Parents: c6af75a
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Nov 12 19:14:28 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Nov 12 19:14:28 2013 +0100

----------------------------------------------------------------------
 CHANGES.txt                                                 | 1 +
 src/java/org/apache/cassandra/db/marshal/CompositeType.java | 6 +++---
 2 files changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/acea7160/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 424a9f4..69b22a3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -18,6 +18,7 @@
  * Correctly throw error when frame too large (CASSANDRA-5981)
  * Fix serialization bug in PagedRange with 2ndary indexes (CASSANDRA-6299)
  * Fix CQL3 table validation in Thrift (CASSANDRA-6140)
+ * Fix bug missing results with IN clauses (CASSANDRA-6327)
 Merged from 1.2:
  * add non-jamm path for cached statements (CASSANDRA-6293)
  * (Hadoop) Require CFRR batchSize to be at least 2 (CASSANDRA-6114)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/acea7160/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CompositeType.java b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
index 3110049..c9e2ab9 100644
--- a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CompositeType.java
@@ -229,11 +229,11 @@ public class CompositeType extends AbstractCompositeType
                 AbstractType<?> t = types.get(i);
                 ByteBuffer s = i < start.length ? start[i] : ByteBufferUtil.EMPTY_BYTE_BUFFER;
                 ByteBuffer f = i < finish.length ? finish[i] : ByteBufferUtil.EMPTY_BYTE_BUFFER;
-                if (!t.intersects(minColumnNames.get(i), maxColumnNames.get(i), s, f))
-                    return false;
+                if (t.intersects(minColumnNames.get(i), maxColumnNames.get(i), s, f))
+                    return true;
             }
         }
-        return true;
+        return false;
     }
 
     private static class StaticParsedComparator implements ParsedComparator


[5/5] git commit: Merge branch 'cassandra-2.0' into trunk

Posted by sl...@apache.org.
Merge branch 'cassandra-2.0' into trunk

Conflicts:
	src/java/org/apache/cassandra/tools/SSTableImport.java


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

Branch: refs/heads/trunk
Commit: bfd0144a189c4326c987d91cc20e66e47236dc1a
Parents: 9a59d25 acea716
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Nov 12 19:23:12 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Nov 12 19:23:12 2013 +0100

----------------------------------------------------------------------
 CHANGES.txt                                        |  3 +++
 .../cassandra/cql3/statements/SelectStatement.java |  2 +-
 .../apache/cassandra/db/marshal/CompositeType.java |  6 +++---
 .../org/apache/cassandra/tools/SSTableImport.java  | 17 ++++++++++++++---
 4 files changed, 21 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfd0144a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfd0144a/src/java/org/apache/cassandra/db/marshal/CompositeType.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/bfd0144a/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableImport.java
index 2e9c4ed,0e8172b..3dcb851
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@@ -44,8 -44,9 +44,9 @@@ import org.apache.cassandra.config.Sche
  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.MarshalException;
  import org.apache.cassandra.dht.IPartitioner;
  import org.apache.cassandra.io.sstable.SSTableWriter;
++import org.apache.cassandra.serializers.MarshalException;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.codehaus.jackson.JsonFactory;
  import org.codehaus.jackson.JsonParser;
@@@ -151,8 -152,19 +152,18 @@@ public class SSTableImpor
                      }
                  }
  
-                 value = isDeleted() ? ByteBufferUtil.hexToBytes((String) fields.get(1))
-                                     : stringAsType((String) fields.get(1), meta.getValueValidator(meta.getColumnDefinitionFromCellName(name)));
+                 if (isDeleted())
+                 {
+                     value = ByteBufferUtil.hexToBytes((String) fields.get(1));
+                 }
+                 else if (isRangeTombstone())
+                 {
 -                    AbstractType<?> type = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{meta.getColumnDefinitionComparator(0)}));
 -                    value = type.fromString((String)fields.get(1));
++                    value = comparator.fromString((String)fields.get(1));
+                 }
+                 else
+                 {
 -                    value = stringAsType((String) fields.get(1), meta.getValueValidator(meta.getColumnDefinitionFromColumnName(name)));
++                    value = stringAsType((String) fields.get(1), meta.getValueValidator(meta.getColumnDefinitionFromCellName(name)));
+                 }
              }
          }