Repository: cassandra
Updated Branches:
  refs/heads/trunk 613a8b43d -> 9359e1e97


Fix column filter creation for wildcard queries

Patch by Zhao Yang; reviewed by Alex Petrov for CASSANDRA-13650

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

Branch: refs/heads/trunk
Commit: 9359e1e977361774daf27e80112774210e55baa4
Parents: 613a8b4
Author: Zhao Yang <jasonstack@jasonstack-mac.local>
Authored: Sat Jul 1 11:12:41 2017 +0800
Committer: Alex Petrov <oleksandr.pet...@gmail.com>
Committed: Thu Jul 6 15:28:33 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/filter/ColumnFilter.java       |  28 ++--
 .../cassandra/db/filter/ColumnFilterTest.java   | 137 ++++++++++++++++++-
 3 files changed, 144 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/9359e1e9/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index aa98554..52bb6d2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 4.0
+ * Fix column filter creation for wildcard queries (CASSANDRA-13650)
  * Add 'nodetool getbatchlogreplaythrottle' and 'nodetool 
setbatchlogreplaythrottle' (CASSANDRA-13614)
  * fix race condition in PendingRepairManager (CASSANDRA-13659)
  * Allow noop incremental repair state transitions (CASSANDRA-13658)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9359e1e9/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java 
b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
index 58c4cec..1d7d1c8 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnFilter.java
@@ -69,11 +69,11 @@ public class ColumnFilter
 
     // True if _fetched_ includes all regular columns (and any static in 
_queried_), in which case metadata must not be
     // null. If false, then _fetched_ == _queried_ and we only store _queried_.
-    public final boolean fetchAllRegulars;
+    final boolean fetchAllRegulars;
 
-    private final RegularAndStaticColumns fetched;
-    private final RegularAndStaticColumns queried; // can be null if 
fetchAllRegulars, to represent a wildcard query (all
-                                                   // static and regular 
columns are both _fetched_ and _queried_).
+    final RegularAndStaticColumns fetched;
+    final RegularAndStaticColumns queried; // can be null if fetchAllRegulars, 
to represent a wildcard query (all
+                                           // static and regular columns are 
both _fetched_ and _queried_).
     private final SortedSetMultimap<ColumnIdentifier, ColumnSubselection> 
subSelections; // can be null
 
     private ColumnFilter(boolean fetchAllRegulars,
@@ -88,23 +88,17 @@ public class ColumnFilter
         if (fetchAllRegulars)
         {
             RegularAndStaticColumns all = metadata.regularAndStaticColumns();
-            if (queried == null)
-            {
-                this.fetched = this.queried = all;
-            }
-            else
-            {
-                this.fetched = all.statics.isEmpty()
-                               ? all
-                               : new RegularAndStaticColumns(queried.statics, 
all.regulars);
-                this.queried = queried;
-            }
+
+            this.fetched = (all.statics.isEmpty() || queried == null)
+                           ? all
+                           : new RegularAndStaticColumns(queried.statics, 
all.regulars);
         }
         else
         {
-            this.fetched = this.queried = queried;
+            this.fetched = queried;
         }
 
+        this.queried = queried;
         this.subSelections = subSelections;
     }
 
@@ -170,7 +164,7 @@ public class ColumnFilter
      */
     public RegularAndStaticColumns queriedColumns()
     {
-        return queried;
+        return queried == null ? fetched : queried;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/9359e1e9/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java 
b/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java
index fa08950..15bfb9c 100644
--- a/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java
+++ b/test/unit/org/apache/cassandra/db/filter/ColumnFilterTest.java
@@ -18,10 +18,15 @@
 
 package org.apache.cassandra.db.filter;
 
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
-import junit.framework.Assert;
+import org.apache.cassandra.db.RegularAndStaticColumns;
 import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.SetType;
+import org.apache.cassandra.db.rows.CellPath;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.io.util.DataInputBuffer;
 import org.apache.cassandra.io.util.DataInputPlus;
@@ -30,13 +35,16 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.junit.Test;
+
+import junit.framework.Assert;
 
 public class ColumnFilterTest
 {
     final static ColumnFilter.Serializer serializer = new 
ColumnFilter.Serializer();
 
     @Test
-    public void columnFilterSerialisationRoundTrip() throws Exception
+    public void testColumnFilterSerialisationRoundTrip() throws Exception
     {
         TableMetadata metadata = TableMetadata.builder("ks", "table")
                                               
.partitioner(Murmur3Partitioner.instance)
@@ -54,16 +62,135 @@ public class ColumnFilterTest
         columnFilter = ColumnFilter.all(metadata);
         testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_30), metadata, MessagingService.VERSION_30);
         testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_3014), metadata, MessagingService.VERSION_3014);
-        testRoundTrip(ColumnFilter.all(metadata), metadata, 
MessagingService.VERSION_40);
+        testRoundTrip(columnFilter, metadata, MessagingService.VERSION_40);
+
+        
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1)),
 metadata, MessagingService.VERSION_30);
+        
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1)),
 metadata, MessagingService.VERSION_3014);
+        
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1)),
 metadata, MessagingService.VERSION_40);
+
+        columnFilter = ColumnFilter.selection(metadata, 
metadata.regularAndStaticColumns().without(v1));
+        testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_30), metadata, MessagingService.VERSION_30);
+        testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_3014), metadata, MessagingService.VERSION_3014);
+        testRoundTrip(columnFilter, metadata, MessagingService.VERSION_40);
+
+        // Table with static column
+        metadata = TableMetadata.builder("ks", "table")
+                                .partitioner(Murmur3Partitioner.instance)
+                                .addPartitionKeyColumn("pk", 
Int32Type.instance)
+                                .addClusteringColumn("ck", Int32Type.instance)
+                                .addStaticColumn("s1", Int32Type.instance)
+                                .addRegularColumn("v1", Int32Type.instance)
+                                .addRegularColumn("v2", Int32Type.instance)
+                                .addRegularColumn("v3", Int32Type.instance)
+                                .build();
+
+        v1 = metadata.getColumn(ByteBufferUtil.bytes("v1"));
+        ColumnMetadata s1 = metadata.getColumn(ByteBufferUtil.bytes("s1"));
+
+        columnFilter = ColumnFilter.all(metadata);
+        testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_30), metadata, MessagingService.VERSION_30);
+        testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_3014), metadata, MessagingService.VERSION_3014);
+        testRoundTrip(columnFilter, metadata, MessagingService.VERSION_40);
 
         
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1)),
 metadata, MessagingService.VERSION_30);
         
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1)),
 metadata, MessagingService.VERSION_3014);
         
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1)),
 metadata, MessagingService.VERSION_40);
 
+        
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1).without(s1)),
 metadata, MessagingService.VERSION_30);
+        
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1).without(s1)),
 metadata, MessagingService.VERSION_3014);
+        
testRoundTrip(ColumnFilter.selection(metadata.regularAndStaticColumns().without(v1).without(s1)),
 metadata, MessagingService.VERSION_40);
+
         columnFilter = ColumnFilter.selection(metadata, 
metadata.regularAndStaticColumns().without(v1));
         testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_30), metadata, MessagingService.VERSION_30);
         testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_3014), metadata, MessagingService.VERSION_3014);
-        testRoundTrip(ColumnFilter.selection(metadata, 
metadata.regularAndStaticColumns().without(v1)), metadata, 
MessagingService.VERSION_40);
+        testRoundTrip(columnFilter, metadata, MessagingService.VERSION_40);
+
+        columnFilter = ColumnFilter.selection(metadata, 
metadata.regularAndStaticColumns().without(v1).without(s1));
+        testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_30), metadata, MessagingService.VERSION_30);
+        testRoundTrip(columnFilter, 
ColumnFilter.Serializer.maybeUpdateForBackwardCompatility(columnFilter, 
MessagingService.VERSION_3014), metadata, MessagingService.VERSION_3014);
+        testRoundTrip(columnFilter, metadata, MessagingService.VERSION_40);
+    }
+
+    @Test
+    public void testColumnFilterConstruction()
+    {
+        // all regular column
+        TableMetadata metadata = TableMetadata.builder("ks", "table")
+                                              
.partitioner(Murmur3Partitioner.instance)
+                                              .addPartitionKeyColumn("pk", 
Int32Type.instance)
+                                              .addClusteringColumn("ck", 
Int32Type.instance)
+                                              .addRegularColumn("v1", 
Int32Type.instance)
+                                              .addRegularColumn("v2", 
Int32Type.instance)
+                                              .addRegularColumn("v3", 
Int32Type.instance)
+                                              .build();
+        ColumnFilter columnFilter = ColumnFilter.all(metadata);
+        assertTrue(columnFilter.fetchAllRegulars);
+        assertEquals(metadata.regularAndStaticColumns(), columnFilter.fetched);
+        assertNull(columnFilter.queried);
+        assertEquals("*", columnFilter.toString());
+
+        RegularAndStaticColumns queried = RegularAndStaticColumns.builder()
+                                                                 
.add(metadata.getColumn(ByteBufferUtil.bytes("v1"))).build();
+        columnFilter = ColumnFilter.selection(queried);
+        assertFalse(columnFilter.fetchAllRegulars);
+        assertEquals(queried, columnFilter.fetched);
+        assertEquals(queried, columnFilter.queried);
+        assertEquals("v1", columnFilter.toString());
+
+        // with static column
+        metadata = TableMetadata.builder("ks", "table")
+                                .partitioner(Murmur3Partitioner.instance)
+                                .addPartitionKeyColumn("pk", 
Int32Type.instance)
+                                .addClusteringColumn("ck", Int32Type.instance)
+                                .addStaticColumn("sc1", Int32Type.instance)
+                                .addStaticColumn("sc2", Int32Type.instance)
+                                .addRegularColumn("v1", Int32Type.instance)
+                                .build();
+
+        columnFilter = ColumnFilter.all(metadata);
+        assertTrue(columnFilter.fetchAllRegulars);
+        assertEquals(metadata.regularAndStaticColumns(), columnFilter.fetched);
+        assertNull(columnFilter.queried);
+        assertEquals("*", columnFilter.toString());
+
+        queried = RegularAndStaticColumns.builder()
+                                         
.add(metadata.getColumn(ByteBufferUtil.bytes("v1"))).build();
+        columnFilter = ColumnFilter.selection(metadata, queried);
+        assertEquals("v1", columnFilter.toString());
+
+        // only static
+        metadata = TableMetadata.builder("ks", "table")
+                                .partitioner(Murmur3Partitioner.instance)
+                                .addPartitionKeyColumn("pk", 
Int32Type.instance)
+                                .addClusteringColumn("ck", Int32Type.instance)
+                                .addStaticColumn("sc", Int32Type.instance)
+                                .build();
+
+        columnFilter = ColumnFilter.all(metadata);
+        assertTrue(columnFilter.fetchAllRegulars);
+        assertEquals(metadata.regularAndStaticColumns(), columnFilter.fetched);
+        assertNull(columnFilter.queried);
+        assertEquals("*", columnFilter.toString());
+
+        // with collection type
+        metadata = TableMetadata.builder("ks", "table")
+                                .partitioner(Murmur3Partitioner.instance)
+                                .addPartitionKeyColumn("pk", 
Int32Type.instance)
+                                .addClusteringColumn("ck", Int32Type.instance)
+                                .addRegularColumn("v1", Int32Type.instance)
+                                .addRegularColumn("set", 
SetType.getInstance(Int32Type.instance, true))
+                                .build();
+
+        columnFilter = ColumnFilter.all(metadata);
+        assertTrue(columnFilter.fetchAllRegulars);
+        assertEquals(metadata.regularAndStaticColumns(), columnFilter.fetched);
+        assertNull(columnFilter.queried);
+        assertEquals("*", columnFilter.toString());
+
+        columnFilter = 
ColumnFilter.selectionBuilder().add(metadata.getColumn(ByteBufferUtil.bytes("v1")))
+                                   
.select(metadata.getColumn(ByteBufferUtil.bytes("set")), 
CellPath.create(ByteBufferUtil.bytes(1)))
+                                   .build();
+        assertEquals("set[1], v1", columnFilter.toString());
     }
 
     static void testRoundTrip(ColumnFilter columnFilter, TableMetadata 
metadata, int version) throws Exception


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to