You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2016/08/16 13:53:11 UTC

[11/12] cassandra git commit: Merge branch cassandra-3.0 into cassandra-3.9

Merge branch cassandra-3.0 into cassandra-3.9


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

Branch: refs/heads/trunk
Commit: cb97649f51bb01285e813daae65cc0287e33b6c2
Parents: de2c449 bb56193
Author: Benjamin Lerer <b....@gmail.com>
Authored: Tue Aug 16 15:39:11 2016 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Tue Aug 16 15:41:34 2016 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |   9 +
 .../apache/cassandra/db/BufferClustering.java   |   4 -
 .../columniterator/AbstractSSTableIterator.java |  24 +-
 .../db/columniterator/SSTableIterator.java      |  17 +
 .../columniterator/SSTableReversedIterator.java |  19 +-
 .../cassandra/db/compaction/Scrubber.java       | 169 +++-
 .../cassandra/db/marshal/ReversedType.java      |  10 -
 .../db/partitions/AbstractBTreePartition.java   |   7 +-
 .../db/partitions/ImmutableBTreePartition.java  |  34 +-
 .../db/rows/UnfilteredRowIterators.java         |  28 +
 .../validation/entities/SecondaryIndexTest.java | 131 ++++
 .../cql3/validation/operations/DeleteTest.java  | 113 +++
 .../operations/SelectOrderByTest.java           | 765 ++++++++++---------
 .../cql3/validation/operations/SelectTest.java  | 355 ++++++++-
 .../cassandra/db/marshal/ReversedTypeTest.java  |   4 +-
 16 files changed, 1268 insertions(+), 422 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 249077d,0bd3920..2edff8e
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,35 -13,18 +13,44 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -3.0.9
 -=====
++3.9
++===
+ 
+ Upgrading
+ ---------
+    - The ReversedType behaviour has been corrected for clustering columns of
+      BYTES type containing empty value. Scrub should be run on the existing
+      SSTables containing a descending clustering column of BYTES type to correct
+      their ordering. See CASSANDRA-12127 for more details.
  
 -3.0.8
 -=====
 +3.8
 +===
 +
 +New features
 +------------
 +   - Shared pool threads are now named according to the stage they are executing
 +     tasks for. Thread names mentioned in traced queries change accordingly.
 +   - A new option has been added to cassandra-stress "-rate fixed={number}/s"
 +     that forces a scheduled rate of operations/sec over time. Using this, stress can
 +     accurately account for coordinated ommission from the stress process.
 +   - The cassandra-stress "-rate limit=" option has been renamed to "-rate throttle="
 +   - hdr histograms have been added to stress runs, it's output can be saved to disk using:
 +     "-log hdrfile=" option. This histogram includes response/service/wait times when used with the
 +     fixed or throttle rate options.  The histogram file can be plotted on
 +     http://hdrhistogram.github.io/HdrHistogram/plotFiles.html
 +   - TimeWindowCompactionStrategy has been added. This has proven to be a better approach
 +     to time series compaction and new tables should use this instead of DTCS. See
 +     CASSANDRA-9666 for details.
 +   - Change-Data-Capture is now available. See cassandra.yaml and for cdc-specific flags and
 +     a brief explanation of on-disk locations for archived data in CommitLog form. This can
 +     be enabled via ALTER TABLE ... WITH cdc=true.
 +     Upon flush, CommitLogSegments containing data for CDC-enabled tables are moved to
 +     the data/cdc_raw directory until removed by the user and writes to CDC-enabled tables
 +     will be rejected with a WriteTimeoutException once cdc_total_space_in_mb is reached
 +     between unflushed CommitLogSegments and cdc_raw.
 +     NOTE: CDC is disabled by default in the .yaml file. Do not enable CDC on a mixed-version
 +     cluster as it will lead to exceptions which can interrupt traffic. Once all nodes
 +     have been upgraded to 3.8 it is safe to enable this feature and restart the cluster.
  
  Upgrading
  ---------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/src/java/org/apache/cassandra/db/BufferClustering.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/BufferClustering.java
index 7c6bb20,0000000..df6a473
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/BufferClustering.java
+++ b/src/java/org/apache/cassandra/db/BufferClustering.java
@@@ -1,44 -1,0 +1,40 @@@
 +/*
 + * 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.db;
 +
 +import java.nio.ByteBuffer;
 +
- import org.apache.cassandra.config.CFMetaData;
- import org.apache.cassandra.config.ColumnDefinition;
- import org.apache.cassandra.utils.memory.AbstractAllocator;
- 
 +/**
 + * The clustering column values for a row.
 + * <p>
 + * A {@code Clustering} is a {@code ClusteringPrefix} that must always be "complete", i.e. have
 + * as many values as there is clustering columns in the table it is part of. It is the clustering
 + * prefix used by rows.
 + * <p>
 + * Note however that while it's size must be equal to the table clustering size, a clustering can have
 + * {@code null} values, and this mostly for thrift backward compatibility (in practice, if a value is null,
 + * all of the following ones will be too because that's what thrift allows, but it's never assumed by the
 + * code so we could start generally allowing nulls for clustering columns if we wanted to).
 + */
 +public class BufferClustering extends AbstractBufferClusteringPrefix implements Clustering
 +{
 +    BufferClustering(ByteBuffer... values)
 +    {
 +        super(Kind.CLUSTERING, values);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index 005bb2c,d57e6bc..5959a20
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@@ -47,13 -46,8 +47,12 @@@ public abstract class AbstractSSTableIt
  
      private final boolean isForThrift;
  
 +    protected final SegmentedFile ifile;
 +
      private boolean isClosed;
  
 +    protected final Slices slices;
-     protected int slice;
 +
      @SuppressWarnings("resource") // We need this because the analysis is not able to determine that we do close
                                    // file on every path where we created it.
      protected AbstractSSTableIterator(SSTableReader sstable,
@@@ -119,9 -109,6 +118,9 @@@
                      this.reader = needsReader ? createReader(indexEntry, file, shouldCloseFile) : null;
                  }
  
 +                if (reader != null && !slices.isEmpty())
-                     reader.setForSlice(slices.get(0));
++                    reader.setForSlice(nextSlice());
 +
                  if (reader == null && file != null && shouldCloseFile)
                      file.close();
              }
@@@ -145,6 -132,6 +144,23 @@@
          }
      }
  
++    private Slice nextSlice()
++    {
++        return slices.get(nextSliceIndex());
++    }
++
++    /**
++     * Returns the index of the next slice to process.
++     * @return the index of the next slice to process
++     */
++    protected abstract int nextSliceIndex();
++
++    /**
++     * Checks if there are more slice to process.
++     * @return {@code true} if there are more slice to process, {@code false} otherwise.
++     */
++    protected abstract boolean hasMoreSlices();
++
      private static Row readStaticRow(SSTableReader sstable,
                                       FileDataInput file,
                                       SerializationHelper helper,
@@@ -233,19 -216,7 +249,19 @@@
  
      public boolean hasNext()
      {
 -        return reader != null && reader.hasNext();
 +        while (true)
 +        {
 +            if (reader == null)
 +                return false;
 +
 +            if (reader.hasNext())
 +                return true;
 +
-             if (++slice >= slices.size())
++            if (!hasMoreSlices())
 +                return false;
 +
-             slice(slices.get(slice));
++            slice(nextSlice());
 +        }
      }
  
      public Unfiltered next()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
index e4f6700,0409310..56ba8f4
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableIterator.java
@@@ -32,6 -31,11 +32,11 @@@ import org.apache.cassandra.io.util.Seg
   */
  public class SSTableIterator extends AbstractSSTableIterator
  {
 -    public SSTableIterator(SSTableReader sstable, DecoratedKey key, ColumnFilter columns, boolean isForThrift)
 -    {
 -        this(sstable, null, key, sstable.getPosition(key, SSTableReader.Operator.EQ), columns, isForThrift);
 -    }
++    /**
++     * The index of the slice being processed.
++     */
++    private int slice;
+ 
      public SSTableIterator(SSTableReader sstable,
                             FileDataInput file,
                             DecoratedKey key,
@@@ -51,6 -53,6 +56,18 @@@
               : new ForwardReader(file, shouldCloseFile);
      }
  
++    protected int nextSliceIndex()
++    {
++        int next = slice;
++        slice++;
++        return next;
++    }
++
++    protected boolean hasMoreSlices()
++    {
++        return slice < slices.size();
++    }
++
      public boolean isReverseOrder()
      {
          return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
index 903fd59,3e49a3a..b8b6f76
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableReversedIterator.java
@@@ -35,6 -34,11 +35,11 @@@ import org.apache.cassandra.utils.btree
   */
  public class SSTableReversedIterator extends AbstractSSTableIterator
  {
 -    public SSTableReversedIterator(SSTableReader sstable, DecoratedKey key, ColumnFilter columns, boolean isForThrift)
 -    {
 -        this(sstable, null, key, sstable.getPosition(key, SSTableReader.Operator.EQ), columns, isForThrift);
 -    }
++    /**
++     * The index of the slice being processed.
++     */
++    private int slice;
+ 
      public SSTableReversedIterator(SSTableReader sstable,
                                     FileDataInput file,
                                     DecoratedKey key,
@@@ -59,6 -61,6 +64,18 @@@
          return true;
      }
  
++    protected int nextSliceIndex()
++    {
++        int next = slice;
++        slice++;
++        return slices.size() - (next + 1);
++    }
++
++    protected boolean hasMoreSlices()
++    {
++        return slice < slices.size();
++    }
++
      private class ReverseReader extends Reader
      {
          protected ReusablePartitionData buffer;
@@@ -106,7 -108,7 +123,7 @@@
                  buffer = createBuffer(1);
                  // Note that we can reuse that buffer between slices (we could alternatively re-read from disk
                  // every time, but that feels more wasteful) so we want to include everything from the beginning.
--                // We can stop at the slice end however since any following slice will be before that.
++                // We can stop at the last slice end however since any following slice will be before that.
                  loadFromDisk(null, slice.end(), true);
              }
              setIterator(slice);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
index 7fb01a3,c63acc2..4a8e4f2
--- a/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
+++ b/src/java/org/apache/cassandra/db/partitions/AbstractBTreePartition.java
@@@ -270,8 -282,47 +270,13 @@@ public abstract class AbstractBTreePart
          }
      }
  
 -    public class SliceableIterator extends AbstractIterator implements SliceableUnfilteredRowIterator
 -    {
 -        private Iterator<Unfiltered> iterator;
 -
 -        protected SliceableIterator(ColumnFilter selection, boolean isReversed)
 -        {
 -            super(selection, isReversed);
 -        }
 -
 -        protected Unfiltered computeNext()
 -        {
 -            if (iterator == null)
 -                iterator = unfilteredIterator(selection, Slices.ALL, isReverseOrder);
 -            if (!iterator.hasNext())
 -                return endOfData();
 -            return iterator.next();
 -        }
 -
 -        public Iterator<Unfiltered> slice(Slice slice)
 -        {
 -            return sliceIterator(selection, slice, isReverseOrder, current, staticRow);
 -        }
 -    }
 -
 -    public SliceableUnfilteredRowIterator sliceableUnfilteredIterator(ColumnFilter columns, boolean reversed)
 -    {
 -        return new SliceableIterator(columns, reversed);
 -    }
 -
 -    protected SliceableUnfilteredRowIterator sliceableUnfilteredIterator()
 -    {
 -        return sliceableUnfilteredIterator(ColumnFilter.all(metadata), false);
 -    }
 -
      protected static Holder build(UnfilteredRowIterator iterator, int initialRowCapacity)
      {
+         return build(iterator, initialRowCapacity, true);
+     }
+ 
+     protected static Holder build(UnfilteredRowIterator iterator, int initialRowCapacity, boolean ordered)
+     {
          CFMetaData metadata = iterator.metadata();
          PartitionColumns columns = iterator.columns();
          boolean reversed = iterator.isReverseOrder();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
index d324190,43653a9..46447ec
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredRowIterators.java
@@@ -230,6 -213,60 +230,34 @@@ public abstract class UnfilteredRowIter
      }
  
      /**
+      * Returns an iterator that concatenate the specified atom with the iterator.
+      */
+     public static UnfilteredRowIterator concat(final Unfiltered first, final UnfilteredRowIterator rest)
+     {
+         return new WrappingUnfilteredRowIterator(rest)
+         {
+             private boolean hasReturnedFirst;
+ 
+             @Override
+             public boolean hasNext()
+             {
+                 return hasReturnedFirst ? super.hasNext() : true;
+             }
+ 
+             @Override
+             public Unfiltered next()
+             {
+                 if (!hasReturnedFirst)
+                 {
+                     hasReturnedFirst = true;
+                     return first;
+                 }
+                 return super.next();
+             }
+         };
+     }
+ 
 -    public static UnfilteredRowIterator cloningIterator(UnfilteredRowIterator iterator, final AbstractAllocator allocator)
 -    {
 -        class Cloner extends Transformation
 -        {
 -            private final Row.Builder builder = allocator.cloningBTreeRowBuilder();
 -
 -            public Row applyToStatic(Row row)
 -            {
 -                return Rows.copy(row, builder).build();
 -            }
 -
 -            @Override
 -            public Row applyToRow(Row row)
 -            {
 -                return Rows.copy(row, builder).build();
 -            }
 -
 -            @Override
 -            public RangeTombstoneMarker applyToMarker(RangeTombstoneMarker marker)
 -            {
 -                return marker.copy(allocator);
 -            }
 -        }
 -        return Transformation.apply(iterator, new Cloner());
 -    }
 -
+     /**
       * Validate that the data of the provided iterator is valid, that is that the values
       * it contains are valid for the type they represent, and more generally that the
       * infos stored are sensible.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/test/unit/org/apache/cassandra/cql3/validation/entities/SecondaryIndexTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
index 9b92ebb,b9ccafb..118a1e2
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/DeleteTest.java
@@@ -1107,28 -1058,142 +1112,136 @@@ public class DeleteTest extends CQLTest
      }
  
      @Test
 +    public void testDeleteAndReverseQueries() throws Throwable
 +    {
 +        // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
 +        // validates we correctly get only the non-removed rows when doing reverse queries.
 +
 +        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
 +
 +        for (int i = 0; i < 10; i++)
 +            execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
 +
 +        flush();
 +
 +        execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
 +
 +        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
 +            row(9), row(8), row(1), row(0)
 +        );
 +
 +        flush();
 +
 +        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
 +            row(9), row(8), row(1), row(0)
 +        );
 +    }
++
++    @Test
+     public void testDeleteWithEmptyRestrictionValue() throws Throwable
+     {
+         for (String options : new String[] { "", " WITH COMPACT STORAGE" })
+         {
+             createTable("CREATE TABLE %s (pk blob, c blob, v blob, PRIMARY KEY (pk, c))" + options);
+ 
+             if (StringUtils.isEmpty(options))
+             {
+                 execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
+                 execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c = textAsBlob('');");
+ 
+                 assertEmpty(execute("SELECT * FROM %s"));
+ 
+                 execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
+                 execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'));");
+ 
+                 assertEmpty(execute("SELECT * FROM %s"));
+ 
+                 execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"));
+                 execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
+                 execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
+ 
+                 execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')");
+ 
+                 assertRows(execute("SELECT * FROM %s"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1")));
+ 
+                 execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('')");
+ 
+                 assertEmpty(execute("SELECT * FROM %s"));
+             }
+             else
+             {
+                 assertInvalid("Invalid empty or null value for column c",
+                               "DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c = textAsBlob('')");
+                 assertInvalid("Invalid empty or null value for column c",
+                               "DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'))");
+             }
+ 
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
+ 
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')");
+ 
+             assertEmpty(execute("SELECT * FROM %s"));
+ 
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)", bytes("foo123"), bytes("2"), bytes("2"));
+ 
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('')");
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('')");
+ 
+             assertRows(execute("SELECT * FROM %s"),
+                        row(bytes("foo123"), bytes("1"), bytes("1")),
+                        row(bytes("foo123"), bytes("2"), bytes("2")));
+         }
+     }
+ 
+     @Test
+     public void testDeleteWithMultipleClusteringColumnsAndEmptyRestrictionValue() throws Throwable
+     {
+         for (String options : new String[] { "", " WITH COMPACT STORAGE" })
+         {
+             createTable("CREATE TABLE %s (pk blob, c1 blob, c2 blob, v blob, PRIMARY KEY (pk, c1, c2))" + options);
+ 
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("1"));
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('');");
+ 
+             assertEmpty(execute("SELECT * FROM %s"));
+ 
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("1"));
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1');");
+ 
+             assertEmpty(execute("SELECT * FROM %s"));
+ 
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("0"));
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
+ 
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('')");
+ 
+             assertRows(execute("SELECT * FROM %s"),
+                        row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("0")));
+ 
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 >= textAsBlob('')");
+ 
+             assertEmpty(execute("SELECT * FROM %s"));
+ 
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
+ 
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('')");
+ 
+             assertEmpty(execute("SELECT * FROM %s"));
+ 
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("3"));
+ 
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 <= textAsBlob('')");
+             execute("DELETE FROM %s WHERE pk = textAsBlob('foo123') AND c1 < textAsBlob('')");
+ 
+             assertRows(execute("SELECT * FROM %s"),
+                        row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                        row(bytes("foo123"), bytes("1"), bytes("2"), bytes("3")));
+         }
+     }
 -
 -    private void flush(boolean forceFlush)
 -    {
 -        if (forceFlush)
 -            flush();
 -    }
 -
 -    @Test
 -    public void testDeleteAndReverseQueries() throws Throwable
 -    {
 -        // This test insert rows in one sstable and a range tombstone covering some of those rows in another, and it
 -        // validates we correctly get only the non-removed rows when doing reverse queries.
 -
 -        createTable("CREATE TABLE %s (k text, i int, PRIMARY KEY (k, i))");
 -
 -        for (int i = 0; i < 10; i++)
 -            execute("INSERT INTO %s(k, i) values (?, ?)", "a", i);
 -
 -        flush();
 -
 -        execute("DELETE FROM %s WHERE k = ? AND i >= ? AND i <= ?", "a", 2, 7);
 -
 -        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
 -            row(9), row(8), row(1), row(0)
 -        );
 -
 -        flush();
 -
 -        assertRows(execute("SELECT i FROM %s WHERE k = ? ORDER BY i DESC", "a"),
 -            row(9), row(8), row(1), row(0)
 -        );
 -    }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
index 6e06419,bb35458..bfc0c86
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectOrderByTest.java
@@@ -105,15 -109,17 +109,17 @@@ public class SelectOrderByTest extends 
              execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 1, 1);
              execute("INSERT INTO %s (a, b, c) VALUES (?, ?, {a: ?})", 0, 2, 2);
  
-             // order by a column not in the selection
-             assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
-                     row(0), row(1), row(2));
+             beforeAndAfterFlush(() -> {
+                 // order by a column not in the selection
+                 assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b ASC", 0),
 -                        row(0), row(1), row(2));
 -    
++                           row(0), row(1), row(2));
 +
-             assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
-                     row(2), row(1), row(0));
+                 assertRows(execute("SELECT c.a FROM %s WHERE a=? ORDER BY b DESC", 0),
 -                        row(2), row(1), row(0));
 -    
++                           row(2), row(1), row(0));
 +
-             assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
-                        row(2), row(1), row(0));
+                 assertRows(execute("SELECT blobAsInt(intAsBlob(c.a)) FROM %s WHERE a=? ORDER BY b DESC", 0),
+                            row(2), row(1), row(0));
+             });
              dropTable("DROP TABLE %s");
          }
      }
@@@ -351,34 -368,36 +368,36 @@@
          execute("INSERT INTO %s (pk1, pk2, c, v) VALUES (?, ?, ?, ?)", 1, 3, 3, "C");
          execute("INSERT INTO %s (pk1, pk2, c, v) VALUES (?, ?, ?, ?)", 1, 1, 4, "D");
  
-         assertRows(execute("SELECT v, ttl(v), c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                    row("B", null, 1),
-                    row("A", null, 2),
-                    row("D", null, 4));
- 
-         assertRows(execute("SELECT v, ttl(v), c as name_1 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                    row("B", null, 1),
-                    row("A", null, 2),
-                    row("D", null, 4));
- 
-         assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                    row("B"),
-                    row("A"),
-                    row("D"));
- 
-         assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c LIMIT 2; ", 1, 1, 2),
-                    row("B"),
-                    row("A"));
- 
-         assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c LIMIT 10; ", 1, 1, 2),
-                    row("B"),
-                    row("A"),
-                    row("D"));
- 
-         assertRows(execute("SELECT v as c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
-                    row("B"),
-                    row("A"),
-                    row("D"));
+         beforeAndAfterFlush(() -> {
 -        assertRows(execute("SELECT v, ttl(v), c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
 -                   row("B", null, 1),
 -                   row("A", null, 2),
 -                   row("D", null, 4));
 -
 -        assertRows(execute("SELECT v, ttl(v), c as name_1 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
 -                   row("B", null, 1),
 -                   row("A", null, 2),
 -                   row("D", null, 4));
 -
 -        assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
 -                   row("B"),
 -                   row("A"),
 -                   row("D"));
 -
 -        assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c LIMIT 2; ", 1, 1, 2),
 -                   row("B"),
 -                   row("A"));
 -
 -        assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c LIMIT 10; ", 1, 1, 2),
 -                   row("B"),
 -                   row("A"),
 -                   row("D"));
 -
 -        assertRows(execute("SELECT v as c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
 -                   row("B"),
 -                   row("A"),
 -                   row("D"));
++            assertRows(execute("SELECT v, ttl(v), c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
++                       row("B", null, 1),
++                       row("A", null, 2),
++                       row("D", null, 4));
++
++            assertRows(execute("SELECT v, ttl(v), c as name_1 FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
++                       row("B", null, 1),
++                       row("A", null, 2),
++                       row("D", null, 4));
++
++            assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
++                       row("B"),
++                       row("A"),
++                       row("D"));
++
++            assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c LIMIT 2; ", 1, 1, 2),
++                       row("B"),
++                       row("A"));
++
++            assertRows(execute("SELECT v FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c LIMIT 10; ", 1, 1, 2),
++                       row("B"),
++                       row("A"),
++                       row("D"));
++
++            assertRows(execute("SELECT v as c FROM %s where pk1 = ? AND pk2 IN (?, ?) ORDER BY c; ", 1, 1, 2),
++                       row("B"),
++                       row("A"),
++                       row("D"));
+         });
  
          createTable("CREATE TABLE %s (pk1 int, pk2 int, c1 int, c2 int, v text, PRIMARY KEY ((pk1, pk2), c1, c2) )");
          execute("INSERT INTO %s (pk1, pk2, c1, c2, v) VALUES (?, ?, ?, ?, ?)", 1, 1, 4, 4, "A");
@@@ -534,22 -561,24 +561,24 @@@
      {
          createTable("CREATE TABLE %s (k text, c1 int, c2 int, PRIMARY KEY (k, c1, c2) ) WITH CLUSTERING ORDER BY (c1 ASC, c2 DESC)");
  
 -        beforeAndAfterFlush(() -> {
 -            for (int i = 0; i < 2; i++)
 -                for (int j = 0; j < 2; j++)
 -                    execute("INSERT INTO %s (k, c1, c2) VALUES ('foo', ?, ?)", i, j);
 +        for (int i = 0; i < 2; i++)
 +            for (int j = 0; j < 2; j++)
 +                execute("INSERT INTO %s (k, c1, c2) VALUES ('foo', ?, ?)", i, j);
  
-         assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo'"),
-                    row(0, 1), row(0, 0), row(1, 1), row(1, 0));
++        beforeAndAfterFlush(() -> {
+             assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo'"),
+                        row(0, 1), row(0, 0), row(1, 1), row(1, 0));
  
-         assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 DESC"),
-                    row(0, 1), row(0, 0), row(1, 1), row(1, 0));
+             assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 DESC"),
+                        row(0, 1), row(0, 0), row(1, 1), row(1, 0));
  
-         assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 DESC, c2 ASC"),
-                    row(1, 0), row(1, 1), row(0, 0), row(0, 1));
+             assertRows(execute("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 DESC, c2 ASC"),
+                        row(1, 0), row(1, 1), row(0, 0), row(0, 1));
  
-         assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 DESC");
-         assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 ASC");
-         assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 ASC");
+             assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 DESC");
+             assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c2 ASC");
+             assertInvalid("SELECT c1, c2 FROM %s WHERE k = 'foo' ORDER BY c1 ASC, c2 ASC");
+         });
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb97649f/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index 9a1493b,a5f153e..db0a4cd
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@@ -21,16 -21,18 +21,18 @@@ import java.nio.ByteBuffer
  import java.util.UUID;
  
  import org.junit.Test;
++import org.junit.Assert;
  
- import junit.framework.Assert;
  import org.apache.cassandra.cql3.CQLTester;
  import org.apache.cassandra.cql3.UntypedResultSet;
  import org.apache.cassandra.cql3.restrictions.StatementRestrictions;
  import org.apache.cassandra.exceptions.InvalidRequestException;
- import org.apache.cassandra.utils.ByteBufferUtil;
  
 -import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER;
 -import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
  import static org.junit.Assert.assertEquals;
  import static org.junit.Assert.assertTrue;
++import static org.apache.cassandra.utils.ByteBufferUtil.EMPTY_BYTE_BUFFER;
++import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+ 
 -import junit.framework.Assert;
  
  /**
   * Test column ranges and ordering with static column in table
@@@ -2973,22 -2571,356 +2975,367 @@@ public class SelectTest extends CQLTest
  
          assertRows(execute("SELECT * FROM %s WHERE pk = 1 AND  c1 IN(0,1,2) AND c2 = 1 AND v = 3"),
                     row(1, 1, 1, 3, 3));
 +    }
  
 -        assertInvalidMessage("Clustering column \"c2\" cannot be restricted (preceding column \"c1\" is restricted by a non-EQ relation)",
 -                             "SELECT * FROM %s WHERE pk = 1 AND  c1 > 0 AND c1 < 5 AND c2 = 1 ALLOW FILTERING;");
 +    @Test
 +    public void testIndexQueryWithCompositePartitionKey() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s (p1 int, p2 int, v int, PRIMARY KEY ((p1, p2)))");
 +        assertInvalidMessage("Partition key parts: p2 must be restricted as other parts are",
 +                             "SELECT * FROM %s WHERE p1 = 1 AND v = 3 ALLOW FILTERING");
 +        createIndex("CREATE INDEX ON %s(v)");
  
 -        assertInvalidMessage("PRIMARY KEY column \"c2\" cannot be restricted as preceding column \"c1\" is not restricted",
 -                             "SELECT * FROM %s WHERE pk = 1 AND  c2 = 1 ALLOW FILTERING;");
 +        execute("INSERT INTO %s(p1, p2, v) values (?, ?, ?)", 1, 1, 3);
 +        execute("INSERT INTO %s(p1, p2, v) values (?, ?, ?)", 1, 2, 3);
 +        execute("INSERT INTO %s(p1, p2, v) values (?, ?, ?)", 2, 1, 3);
 +
 +        assertRows(execute("SELECT * FROM %s WHERE p1 = 1 AND v = 3 ALLOW FILTERING"),
 +                   row(1, 2, 3),
 +                   row(1, 1, 3));
      }
+ 
+     @Test
+     public void testEmptyRestrictionValue() throws Throwable
+     {
+         for (String options : new String[] { "", " WITH COMPACT STORAGE" })
+         {
+             createTable("CREATE TABLE %s (pk blob, c blob, v blob, PRIMARY KEY ((pk), c))" + options);
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                     bytes("foo123"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                     bytes("foo123"), bytes("2"), bytes("2"));
+ 
+             beforeAndAfterFlush(() -> {
+ 
+                 assertInvalidMessage("Key may not be empty", "SELECT * FROM %s WHERE pk = textAsBlob('');");
+                 assertInvalidMessage("Key may not be empty", "SELECT * FROM %s WHERE pk IN (textAsBlob(''), textAsBlob('1'));");
+ 
+                 assertInvalidMessage("Key may not be empty",
+                                      "INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                                      EMPTY_BYTE_BUFFER, bytes("2"), bytes("2"));
+ 
+                 // Test clustering columns restrictions
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c = textAsBlob('');"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) = (textAsBlob(''));"));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) IN ((textAsBlob('')), (textAsBlob('1')));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('');"),
+                            row(bytes("foo123"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) > (textAsBlob(''));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('');"),
+                            row(bytes("foo123"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) >= (textAsBlob(''));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('');"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) <= (textAsBlob(''));"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('');"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) < (textAsBlob(''));"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('') AND c < textAsBlob('');"));
+             });
+ 
+             if (options.contains("COMPACT"))
+             {
+                 assertInvalidMessage("Invalid empty or null value for column c",
+                                      "INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                                      bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4"));
+             }
+             else
+             {
+                 execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                         bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4"));
+ 
+                 beforeAndAfterFlush(() -> {
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c = textAsBlob('');"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) = (textAsBlob(''));"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'));"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")),
+                                row(bytes("foo123"), bytes("1"), bytes("1")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) IN ((textAsBlob('')), (textAsBlob('1')));"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")),
+                                row(bytes("foo123"), bytes("1"), bytes("1")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('');"),
+                                row(bytes("foo123"), bytes("1"), bytes("1")),
+                                row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) > (textAsBlob(''));"),
+                                row(bytes("foo123"), bytes("1"), bytes("1")),
+                                row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('');"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")),
+                                row(bytes("foo123"), bytes("1"), bytes("1")),
+                                row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) >= (textAsBlob(''));"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")),
+                                row(bytes("foo123"), bytes("1"), bytes("1")),
+                                row(bytes("foo123"), bytes("2"), bytes("2")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('');"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) <= (textAsBlob(''));"),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")));
+ 
+                     assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('');"));
+ 
+                     assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c) < (textAsBlob(''));"));
+ 
+                     assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('') AND c < textAsBlob('');"));
+                 });
+             }
+ 
+             // Test restrictions on non-primary key value
+             assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND v = textAsBlob('') ALLOW FILTERING;"));
+ 
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                     bytes("foo123"), bytes("3"), EMPTY_BYTE_BUFFER);
+ 
+             beforeAndAfterFlush(() -> {
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND v = textAsBlob('') ALLOW FILTERING;"),
+                            row(bytes("foo123"), bytes("3"), EMPTY_BYTE_BUFFER));
+             });
+         }
+     }
+ 
+     @Test
+     public void testEmptyRestrictionValueWithMultipleClusteringColumns() throws Throwable
+     {
+         for (String options : new String[] { "", " WITH COMPACT STORAGE" })
+         {
+             createTable("CREATE TABLE %s (pk blob, c1 blob, c2 blob, v blob, PRIMARY KEY (pk, c1, c2))" + options);
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("2"));
+ 
+             beforeAndAfterFlush(() -> {
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('');"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 = textAsBlob('');"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) = (textAsBlob('1'), textAsBlob(''));"));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1');"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 IN (textAsBlob(''), textAsBlob('1'));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) IN ((textAsBlob(''), textAsBlob('1')), (textAsBlob('1'), textAsBlob('1')));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('');"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 > textAsBlob('');"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) > (textAsBlob(''), textAsBlob('1'));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 >= textAsBlob('');"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 <= textAsBlob('');"));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) <= (textAsBlob('1'), textAsBlob(''));"));
+             });
+ 
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)",
+                     bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4"));
+ 
+             beforeAndAfterFlush(() -> {
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('');"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('') AND c2 = textAsBlob('1');"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) = (textAsBlob(''), textAsBlob('1'));"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1');"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) IN ((textAsBlob(''), textAsBlob('1')), (textAsBlob('1'), textAsBlob('1')));"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) > (textAsBlob(''), textAsBlob('1'));"),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) >= (textAsBlob(''), textAsBlob('1'));"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) <= (textAsBlob(''), textAsBlob('1'));"),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) < (textAsBlob(''), textAsBlob('1'));"));
+             });
+         }
+     }
+ 
+     @Test
+     public void testEmptyRestrictionValueWithOrderBy() throws Throwable
+     {
+         for (String options : new String[] { "",
+                                              " WITH COMPACT STORAGE",
+                                              " WITH CLUSTERING ORDER BY (c DESC)",
+                                              " WITH COMPACT STORAGE AND CLUSTERING ORDER BY (c DESC)"})
+         {
+             String orderingClause = options.contains("ORDER") ? "" : "ORDER BY c DESC" ;
+ 
+             createTable("CREATE TABLE %s (pk blob, c blob, v blob, PRIMARY KEY ((pk), c))" + options);
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                     bytes("foo123"),
+                     bytes("1"),
+                     bytes("1"));
+             execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                     bytes("foo123"),
+                     bytes("2"),
+                     bytes("2"));
+ 
+             beforeAndAfterFlush(() -> {
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')" + orderingClause),
+                            row(bytes("foo123"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('')" + orderingClause),
+                            row(bytes("foo123"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1")));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('')" + orderingClause));
+ 
+                 assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('')" + orderingClause));
+ 
+             });
+ 
+             if (options.contains("COMPACT"))
+             {
+                 assertInvalidMessage("Invalid empty or null value for column c",
+                                      "INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                                      bytes("foo123"),
+                                      EMPTY_BYTE_BUFFER,
+                                      bytes("4"));
+             }
+             else
+             {
+                 execute("INSERT INTO %s (pk, c, v) VALUES (?, ?, ?)",
+                         bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4"));
+ 
+                 beforeAndAfterFlush(() -> {
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c IN (textAsBlob(''), textAsBlob('1'))" + orderingClause),
+                                row(bytes("foo123"), bytes("1"), bytes("1")),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c > textAsBlob('')" + orderingClause),
+                                row(bytes("foo123"), bytes("2"), bytes("2")),
+                                row(bytes("foo123"), bytes("1"), bytes("1")));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c >= textAsBlob('')" + orderingClause),
+                                row(bytes("foo123"), bytes("2"), bytes("2")),
+                                row(bytes("foo123"), bytes("1"), bytes("1")),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")));
+ 
+                     assertEmpty(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c < textAsBlob('')" + orderingClause));
+ 
+                     assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c <= textAsBlob('')" + orderingClause),
+                                row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("4")));
+                 });
+             }
+         }
+     }
+ 
+     @Test
+     public void testEmptyRestrictionValueWithMultipleClusteringColumnsAndOrderBy() throws Throwable
+     {
+         for (String options : new String[] { "",
+                 " WITH COMPACT STORAGE",
+                 " WITH CLUSTERING ORDER BY (c1 DESC, c2 DESC)",
+                 " WITH COMPACT STORAGE AND CLUSTERING ORDER BY (c1 DESC, c2 DESC)"})
+         {
+             String orderingClause = options.contains("ORDER") ? "" : "ORDER BY c1 DESC, c2 DESC" ;
+ 
+             createTable("CREATE TABLE %s (pk blob, c1 blob, c2 blob, v blob, PRIMARY KEY (pk, c1, c2))" + options);
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("1"), bytes("1"));
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)", bytes("foo123"), bytes("1"), bytes("2"), bytes("2"));
+ 
+             beforeAndAfterFlush(() -> {
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 > textAsBlob('')" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 > textAsBlob('')" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) > (textAsBlob(''), textAsBlob('1'))" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 = textAsBlob('1') AND c2 >= textAsBlob('')" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+             });
+ 
+             execute("INSERT INTO %s (pk, c1, c2, v) VALUES (?, ?, ?, ?)",
+                     bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4"));
+ 
+             beforeAndAfterFlush(() -> {
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND c1 IN (textAsBlob(''), textAsBlob('1')) AND c2 = textAsBlob('1')" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) IN ((textAsBlob(''), textAsBlob('1')), (textAsBlob('1'), textAsBlob('1')))" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) > (textAsBlob(''), textAsBlob('1'))" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")));
+ 
+                 assertRows(execute("SELECT * FROM %s WHERE pk = textAsBlob('foo123') AND (c1, c2) >= (textAsBlob(''), textAsBlob('1'))" + orderingClause),
+                            row(bytes("foo123"), bytes("1"), bytes("2"), bytes("2")),
+                            row(bytes("foo123"), bytes("1"), bytes("1"), bytes("1")),
+                            row(bytes("foo123"), EMPTY_BYTE_BUFFER, bytes("1"), bytes("4")));
+             });
+         }
+     }
  }