You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2015/06/25 11:09:39 UTC

[3/6] cassandra git commit: Don't advance reader if column name not found

Don't advance reader if column name not found

Patch by Carl Yeksigian; reviewed by Sam Tunnicliffe for CASSANDRA-9540


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

Branch: refs/heads/trunk
Commit: d6c37bdd18b2632ea9093f2422a0cfa723e36b96
Parents: f797bfa
Author: Carl Yeksigian <ca...@apache.org>
Authored: Wed Jun 24 11:27:07 2015 -0400
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Jun 25 09:50:22 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../db/columniterator/SSTableNamesIterator.java |  3 --
 .../cassandra/cql3/LargeCompactValueTest.java   | 50 ++++++++++++++++++++
 3 files changed, 51 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d6c37bdd/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b3c76ed..e8ac3e7 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.7
+ * 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)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d6c37bdd/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
index 221f499..eb5eed9 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
@@ -235,10 +235,7 @@ public class SSTableNamesIterator extends AbstractIterator<OnDiskAtom> implement
                     result.add(deserializer.readNext());
                 }
                 else
-                {
-                    deserializer.skipNext();
                     nextToFetch = toFetch.hasNext() ? toFetch.next() : null;
-                }
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d6c37bdd/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java b/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.java
new file mode 100644
index 0000000..fe91650
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/LargeCompactValueTest.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.cql3;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.io.util.RandomAccessReader;
+
+public class LargeCompactValueTest extends CQLTester
+{
+    @Before
+    public void before()
+    {
+        createTable("CREATE TABLE %s (key TEXT, column TEXT, value BLOB, PRIMARY KEY (key, column)) WITH COMPACT STORAGE");
+    }
+
+    @Test
+    public void testInsertAndQuery() throws Throwable
+    {
+        ByteBuffer largeBytes = ByteBuffer.wrap(new byte[100000]);
+        execute("INSERT INTO %s (key, column, value) VALUES (?, ?, ?)", "test", "a", largeBytes);
+        ByteBuffer smallBytes = ByteBuffer.wrap(new byte[10]);
+        execute("INSERT INTO %s (key, column, value) VALUES (?, ?, ?)", "test", "c", smallBytes);
+
+        flush();
+
+        assertRows(execute("SELECT column FROM %s WHERE key = ? AND column IN (?, ?, ?)", "test", "c", "a", "b"),
+                   row("a"),
+                   row("c"));
+    }
+}