You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2017/09/12 21:18:39 UTC

[1/6] cassandra git commit: INSERT statement fails when Tuple type is used as clustering column with default DESC order

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 a586f6c88 -> a08a816a6
  refs/heads/cassandra-3.11 c05d98a30 -> cb2a1c8f4
  refs/heads/trunk 7d4d1a325 -> c6cd82462


INSERT statement fails when Tuple type is used as clustering column with default DESC order

patch by Stavros Kontopoulos, reviewed by jasobrown for CASSANDRA-13717


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

Branch: refs/heads/cassandra-3.0
Commit: a08a816a6a3497046ba75a38d76d5095347dfe95
Parents: a586f6c
Author: Stavros Kontopoulos <st...@lightbend.com>
Authored: Thu Aug 10 04:23:26 2017 +0300
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Sep 12 14:10:34 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/cql3/Tuples.java  | 24 +++++++++++++++-----
 .../cql3/validation/entities/TupleTypeTest.java | 14 ++++++++++++
 3 files changed, 33 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6053117..3d3903e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.15
+ * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
  * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
  * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
  * Range deletes in a CAS batch are ignored (CASSANDRA-13655)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index ee08efe..c7564d3 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -47,7 +47,7 @@ public class Tuples
         return new ColumnSpecification(column.ksName,
                                        column.cfName,
                                        new ColumnIdentifier(String.format("%s[%d]", column.name, component), true),
-                                       ((TupleType)column.type).type(component));
+                                       (getTupleType(column.type)).type(component));
     }
 
     /**
@@ -77,7 +77,7 @@ public class Tuples
 
                 values.add(value);
             }
-            DelayedValue value = new DelayedValue((TupleType)receiver.type, values);
+            DelayedValue value = new DelayedValue(getTupleType(receiver.type), values);
             return allTerminal ? value.bind(QueryOptions.DEFAULT) : value;
         }
 
@@ -104,10 +104,10 @@ public class Tuples
 
         private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            if (!(receiver.type instanceof TupleType))
+            if (!checkIfTupleType(receiver.type))
                 throw new InvalidRequestException(String.format("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type()));
 
-            TupleType tt = (TupleType)receiver.type;
+            TupleType tt = getTupleType(receiver.type);
             for (int i = 0; i < elements.size(); i++)
             {
                 if (i >= tt.size())
@@ -256,7 +256,7 @@ public class Tuples
                 List<?> l = type.getSerializer().deserializeForNativeProtocol(value, options.getProtocolVersion());
 
                 assert type.getElementsType() instanceof TupleType;
-                TupleType tupleType = (TupleType) type.getElementsType();
+                TupleType tupleType = Tuples.getTupleType(type.getElementsType());
 
                 // type.split(bytes)
                 List<List<ByteBuffer>> elements = new ArrayList<>(l.size());
@@ -375,7 +375,7 @@ public class Tuples
             ByteBuffer value = options.getValues().get(bindIndex);
             if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
                 throw new InvalidRequestException(String.format("Invalid unset value for tuple %s", receiver.name));
-            return value == null ? null : Value.fromSerialized(value, (TupleType)receiver.type);
+            return value == null ? null : Value.fromSerialized(value, getTupleType(receiver.type));
         }
     }
 
@@ -412,4 +412,16 @@ public class Tuples
         sb.append(')');
         return sb.toString();
     }
+
+    public static boolean checkIfTupleType(AbstractType<?> tuple)
+    {
+        return (tuple instanceof TupleType) ||
+               (tuple instanceof ReversedType && ((ReversedType) tuple).baseType instanceof TupleType);
+
+    }
+
+    public static TupleType getTupleType(AbstractType<?> tuple)
+    {
+        return (tuple instanceof ReversedType ? ((TupleType) ((ReversedType) tuple).baseType) : (TupleType)tuple);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
index cbe4a15..bace751 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -17,6 +17,10 @@
  */
 package org.apache.cassandra.cql3.validation.entities;
 
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Locale;
+
 import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
@@ -211,4 +215,14 @@ public class TupleTypeTest extends CQLTester
         assertInvalidMessage("Not enough bytes to read 0th component",
                              "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, Long.MAX_VALUE);
     }
+
+    @Test
+    public void testReversedTypeTuple() throws Throwable
+    {
+        // CASSANDRA-13717
+        createTable("CREATE TABLE %s (id int, tdemo frozen<tuple<timestamp, varchar>>, primary key (id, tdemo)) with clustering order by (tdemo desc)");
+        execute("INSERT INTO %s (id, tdemo) VALUES (1, ('2017-02-03 03:05+0000','Europe'))");
+        DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mmX", Locale.ENGLISH);
+        assertRows(execute("SELECT tdemo FROM %s"), row(tuple( df.parse("2017-02-03 03:05+0000"), "Europe")));
+    }
 }


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


[6/6] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by ja...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: c6cd8246280acde5e2244d8960b2d5c17353424f
Parents: 7d4d1a3 cb2a1c8
Author: Jason Brown <ja...@gmail.com>
Authored: Tue Sep 12 14:14:54 2017 -0700
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Sep 12 14:18:00 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/cql3/Tuples.java  | 28 ++++++++++++++------
 .../cql3/validation/entities/TupleTypeTest.java | 17 +++++++++++-
 3 files changed, 37 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c6cd8246/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Tuples.java
index bae756a,01f3466..317e192
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@@ -68,12 -65,7 +68,12 @@@ public class Tuple
  
          public Term prepare(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
          {
 -            validateAssignableTo(keyspace, receiver);
 +            // The parser cannot differentiate between a tuple with one element and a term between parenthesis.
 +            // By consequence, we need to wait until we know the target type to determine which one it is.
-             if (elements.size() == 1 && !(receiver.type instanceof TupleType))
++            if (elements.size() == 1 && !checkIfTupleType(receiver.type))
 +                return elements.get(0).prepare(keyspace, receiver);
 +
 +            validateTupleAssignableTo(receiver, elements);
  
              List<Term> values = new ArrayList<>(elements.size());
              boolean allTerminal = true;
@@@ -110,14 -102,38 +110,14 @@@
              return allTerminal ? value.bind(QueryOptions.DEFAULT) : value;
          }
  
 -        private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
 -        {
 -            if (!checkIfTupleType(receiver.type))
 -                throw new InvalidRequestException(String.format("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type()));
 -
 -            TupleType tt = getTupleType(receiver.type);
 -            for (int i = 0; i < elements.size(); i++)
 -            {
 -                if (i >= tt.size())
 -                {
 -                    throw new InvalidRequestException(String.format("Invalid tuple literal for %s: too many elements. Type %s expects %d but got %d",
 -                            receiver.name, tt.asCQL3Type(), tt.size(), elements.size()));
 -                }
 -
 -                Term.Raw value = elements.get(i);
 -                ColumnSpecification spec = componentSpecOf(receiver, i);
 -                if (!value.testAssignment(keyspace, spec).isAssignable())
 -                    throw new InvalidRequestException(String.format("Invalid tuple literal for %s: component %d is not of type %s", receiver.name, i, spec.type.asCQL3Type()));
 -            }
 -        }
 -
          public AssignmentTestable.TestResult testAssignment(String keyspace, ColumnSpecification receiver)
          {
 -            try
 -            {
 -                validateAssignableTo(keyspace, receiver);
 -                return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
 -            }
 -            catch (InvalidRequestException e)
 -            {
 -                return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
 -            }
 +            // The parser cannot differentiate between a tuple with one element and a term between parenthesis.
 +            // By consequence, we need to wait until we know the target type to determine which one it is.
-             if (elements.size() == 1 && !(receiver.type instanceof TupleType))
++            if (elements.size() == 1 && !checkIfTupleType(receiver.type))
 +                return elements.get(0).testAssignment(keyspace, receiver);
 +
 +            return testTupleAssignment(receiver, elements);
          }
  
          @Override
@@@ -420,100 -436,29 +420,112 @@@
          }
      }
  
 -    public static String tupleToString(List<?> items)
 +    /**
 +     * Create a <code>String</code> representation of the tuple containing the specified elements.
 +     *
 +     * @param elements the tuple elements
 +     * @return a <code>String</code> representation of the tuple
 +     */
 +    public static String tupleToString(List<?> elements)
 +    {
 +        return tupleToString(elements, Object::toString);
 +    }
 +
 +    /**
 +     * Create a <code>String</code> representation of the tuple from the specified items associated to
 +     * the tuples elements.
 +     *
 +     * @param items items associated to the tuple elements
 +     * @param mapper the mapper used to map the items to the <code>String</code> representation of the tuple elements
 +     * @return a <code>String</code> representation of the tuple
 +     */
 +    public static <T> String tupleToString(Iterable<T> items, java.util.function.Function<T, String> mapper)
 +    {
 +        return StreamSupport.stream(items.spliterator(), false)
 +                            .map(e -> mapper.apply(e))
 +                            .collect(Collectors.joining(", ", "(", ")"));
 +    }
 +
 +    /**
 +     * Returns the exact TupleType from the items if it can be known.
 +     *
 +     * @param items the items mapped to the tuple elements
 +     * @param mapper the mapper used to retrieve the element types from the  items
 +     * @return the exact TupleType from the items if it can be known or <code>null</code>
 +     */
 +    public static <T> AbstractType<?> getExactTupleTypeIfKnown(List<T> items,
 +                                                               java.util.function.Function<T, AbstractType<?>> mapper)
 +    {
 +        List<AbstractType<?>> types = new ArrayList<>(items.size());
 +        for (T item : items)
 +        {
 +            AbstractType<?> type = mapper.apply(item);
 +            if (type == null)
 +                return null;
 +            types.add(type);
 +        }
 +        return new TupleType(types);
 +    }
 +
 +    /**
 +     * Checks if the tuple with the specified elements can be assigned to the specified column.
 +     *
 +     * @param receiver the receiving column
 +     * @param elements the tuple elements
 +     * @throws InvalidRequestException if the tuple cannot be assigned to the specified column.
 +     */
 +    public static void validateTupleAssignableTo(ColumnSpecification receiver,
 +                                                 List<? extends AssignmentTestable> elements)
      {
-         if (!(receiver.type instanceof TupleType))
++        if (!checkIfTupleType(receiver.type))
 +            throw invalidRequest("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type());
  
-         TupleType tt = (TupleType)receiver.type;
 -        StringBuilder sb = new StringBuilder("(");
 -        for (int i = 0; i < items.size(); i++)
++        TupleType tt = getTupleType(receiver.type);
 +        for (int i = 0; i < elements.size(); i++)
 +        {
 +            if (i >= tt.size())
 +            {
 +                throw invalidRequest("Invalid tuple literal for %s: too many elements. Type %s expects %d but got %d",
 +                                     receiver.name, tt.asCQL3Type(), tt.size(), elements.size());
 +            }
 +
 +            AssignmentTestable value = elements.get(i);
 +            ColumnSpecification spec = componentSpecOf(receiver, i);
 +            if (!value.testAssignment(receiver.ksName, spec).isAssignable())
 +                throw invalidRequest("Invalid tuple literal for %s: component %d is not of type %s",
 +                                     receiver.name, i, spec.type.asCQL3Type());
 +        }
 +    }
 +
 +    /**
 +     * Tests that the tuple with the specified elements can be assigned to the specified column.
 +     *
 +     * @param receiver the receiving column
 +     * @param elements the tuple elements
 +     */
 +    public static AssignmentTestable.TestResult testTupleAssignment(ColumnSpecification receiver,
 +                                                                    List<? extends AssignmentTestable> elements)
 +    {
 +        try
 +        {
 +            validateTupleAssignableTo(receiver, elements);
 +            return AssignmentTestable.TestResult.WEAKLY_ASSIGNABLE;
 +        }
 +        catch (InvalidRequestException e)
          {
 -            sb.append(items.get(i));
 -            if (i < items.size() - 1)
 -                sb.append(", ");
 +            return AssignmentTestable.TestResult.NOT_ASSIGNABLE;
          }
 -        sb.append(')');
 -        return sb.toString();
      }
+ 
+     public static boolean checkIfTupleType(AbstractType<?> tuple)
+     {
+         return (tuple instanceof TupleType) ||
+                (tuple instanceof ReversedType && ((ReversedType) tuple).baseType instanceof TupleType);
+ 
+     }
+ 
+     public static TupleType getTupleType(AbstractType<?> tuple)
+     {
+         return (tuple instanceof ReversedType ? ((TupleType) ((ReversedType) tuple).baseType) : (TupleType)tuple);
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c6cd8246/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
index 4dc8d18,bace751..28430cb
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@@ -213,10 -217,12 +217,21 @@@ public class TupleTypeTest extends CQLT
      }
  
      @Test
 +    public void testTupleModification() throws Throwable
 +    {
 +        createTable("CREATE TABLE %s(pk int PRIMARY KEY, value tuple<int, int>)");
 +        assertInvalidMessage("Invalid operation (value = value + (1, 1)) for tuple column value",
 +                             "UPDATE %s SET value += (1, 1) WHERE k=0;");
 +    }
- }
++
++    @Test
+     public void testReversedTypeTuple() throws Throwable
+     {
+         // CASSANDRA-13717
+         createTable("CREATE TABLE %s (id int, tdemo frozen<tuple<timestamp, varchar>>, primary key (id, tdemo)) with clustering order by (tdemo desc)");
+         execute("INSERT INTO %s (id, tdemo) VALUES (1, ('2017-02-03 03:05+0000','Europe'))");
+         DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mmX", Locale.ENGLISH);
+         assertRows(execute("SELECT tdemo FROM %s"), row(tuple( df.parse("2017-02-03 03:05+0000"), "Europe")));
+     }
+ }
++


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


[4/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by ja...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: cb2a1c8f4209ffe9aea8e40e7f0e45dc70613645
Parents: c05d98a a08a816
Author: Jason Brown <ja...@gmail.com>
Authored: Tue Sep 12 14:11:06 2017 -0700
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Sep 12 14:14:28 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/cql3/Tuples.java  | 24 +++++++++++++++-----
 .../cql3/validation/entities/TupleTypeTest.java | 14 ++++++++++++
 3 files changed, 33 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb2a1c8f/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2c48ab2,3d3903e..ec9d126
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,14 -1,5 +1,15 @@@
 -3.0.15
 +3.11.1
 + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418)
 + * BTree.Builder memory leak (CASSANDRA-13754)
 + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798)
 + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938)
 + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744)
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
+  * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
   * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
   * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
   * Range deletes in a CAS batch are ignored (CASSANDRA-13655)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb2a1c8f/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------


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


[2/6] cassandra git commit: INSERT statement fails when Tuple type is used as clustering column with default DESC order

Posted by ja...@apache.org.
INSERT statement fails when Tuple type is used as clustering column with default DESC order

patch by Stavros Kontopoulos, reviewed by jasobrown for CASSANDRA-13717


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

Branch: refs/heads/cassandra-3.11
Commit: a08a816a6a3497046ba75a38d76d5095347dfe95
Parents: a586f6c
Author: Stavros Kontopoulos <st...@lightbend.com>
Authored: Thu Aug 10 04:23:26 2017 +0300
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Sep 12 14:10:34 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/cql3/Tuples.java  | 24 +++++++++++++++-----
 .../cql3/validation/entities/TupleTypeTest.java | 14 ++++++++++++
 3 files changed, 33 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6053117..3d3903e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.15
+ * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
  * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
  * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
  * Range deletes in a CAS batch are ignored (CASSANDRA-13655)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index ee08efe..c7564d3 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -47,7 +47,7 @@ public class Tuples
         return new ColumnSpecification(column.ksName,
                                        column.cfName,
                                        new ColumnIdentifier(String.format("%s[%d]", column.name, component), true),
-                                       ((TupleType)column.type).type(component));
+                                       (getTupleType(column.type)).type(component));
     }
 
     /**
@@ -77,7 +77,7 @@ public class Tuples
 
                 values.add(value);
             }
-            DelayedValue value = new DelayedValue((TupleType)receiver.type, values);
+            DelayedValue value = new DelayedValue(getTupleType(receiver.type), values);
             return allTerminal ? value.bind(QueryOptions.DEFAULT) : value;
         }
 
@@ -104,10 +104,10 @@ public class Tuples
 
         private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            if (!(receiver.type instanceof TupleType))
+            if (!checkIfTupleType(receiver.type))
                 throw new InvalidRequestException(String.format("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type()));
 
-            TupleType tt = (TupleType)receiver.type;
+            TupleType tt = getTupleType(receiver.type);
             for (int i = 0; i < elements.size(); i++)
             {
                 if (i >= tt.size())
@@ -256,7 +256,7 @@ public class Tuples
                 List<?> l = type.getSerializer().deserializeForNativeProtocol(value, options.getProtocolVersion());
 
                 assert type.getElementsType() instanceof TupleType;
-                TupleType tupleType = (TupleType) type.getElementsType();
+                TupleType tupleType = Tuples.getTupleType(type.getElementsType());
 
                 // type.split(bytes)
                 List<List<ByteBuffer>> elements = new ArrayList<>(l.size());
@@ -375,7 +375,7 @@ public class Tuples
             ByteBuffer value = options.getValues().get(bindIndex);
             if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
                 throw new InvalidRequestException(String.format("Invalid unset value for tuple %s", receiver.name));
-            return value == null ? null : Value.fromSerialized(value, (TupleType)receiver.type);
+            return value == null ? null : Value.fromSerialized(value, getTupleType(receiver.type));
         }
     }
 
@@ -412,4 +412,16 @@ public class Tuples
         sb.append(')');
         return sb.toString();
     }
+
+    public static boolean checkIfTupleType(AbstractType<?> tuple)
+    {
+        return (tuple instanceof TupleType) ||
+               (tuple instanceof ReversedType && ((ReversedType) tuple).baseType instanceof TupleType);
+
+    }
+
+    public static TupleType getTupleType(AbstractType<?> tuple)
+    {
+        return (tuple instanceof ReversedType ? ((TupleType) ((ReversedType) tuple).baseType) : (TupleType)tuple);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
index cbe4a15..bace751 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -17,6 +17,10 @@
  */
 package org.apache.cassandra.cql3.validation.entities;
 
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Locale;
+
 import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
@@ -211,4 +215,14 @@ public class TupleTypeTest extends CQLTester
         assertInvalidMessage("Not enough bytes to read 0th component",
                              "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, Long.MAX_VALUE);
     }
+
+    @Test
+    public void testReversedTypeTuple() throws Throwable
+    {
+        // CASSANDRA-13717
+        createTable("CREATE TABLE %s (id int, tdemo frozen<tuple<timestamp, varchar>>, primary key (id, tdemo)) with clustering order by (tdemo desc)");
+        execute("INSERT INTO %s (id, tdemo) VALUES (1, ('2017-02-03 03:05+0000','Europe'))");
+        DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mmX", Locale.ENGLISH);
+        assertRows(execute("SELECT tdemo FROM %s"), row(tuple( df.parse("2017-02-03 03:05+0000"), "Europe")));
+    }
 }


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


[3/6] cassandra git commit: INSERT statement fails when Tuple type is used as clustering column with default DESC order

Posted by ja...@apache.org.
INSERT statement fails when Tuple type is used as clustering column with default DESC order

patch by Stavros Kontopoulos, reviewed by jasobrown for CASSANDRA-13717


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

Branch: refs/heads/trunk
Commit: a08a816a6a3497046ba75a38d76d5095347dfe95
Parents: a586f6c
Author: Stavros Kontopoulos <st...@lightbend.com>
Authored: Thu Aug 10 04:23:26 2017 +0300
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Sep 12 14:10:34 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/cql3/Tuples.java  | 24 +++++++++++++++-----
 .../cql3/validation/entities/TupleTypeTest.java | 14 ++++++++++++
 3 files changed, 33 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 6053117..3d3903e 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.15
+ * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
  * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
  * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
  * Range deletes in a CAS batch are ignored (CASSANDRA-13655)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Tuples.java b/src/java/org/apache/cassandra/cql3/Tuples.java
index ee08efe..c7564d3 100644
--- a/src/java/org/apache/cassandra/cql3/Tuples.java
+++ b/src/java/org/apache/cassandra/cql3/Tuples.java
@@ -47,7 +47,7 @@ public class Tuples
         return new ColumnSpecification(column.ksName,
                                        column.cfName,
                                        new ColumnIdentifier(String.format("%s[%d]", column.name, component), true),
-                                       ((TupleType)column.type).type(component));
+                                       (getTupleType(column.type)).type(component));
     }
 
     /**
@@ -77,7 +77,7 @@ public class Tuples
 
                 values.add(value);
             }
-            DelayedValue value = new DelayedValue((TupleType)receiver.type, values);
+            DelayedValue value = new DelayedValue(getTupleType(receiver.type), values);
             return allTerminal ? value.bind(QueryOptions.DEFAULT) : value;
         }
 
@@ -104,10 +104,10 @@ public class Tuples
 
         private void validateAssignableTo(String keyspace, ColumnSpecification receiver) throws InvalidRequestException
         {
-            if (!(receiver.type instanceof TupleType))
+            if (!checkIfTupleType(receiver.type))
                 throw new InvalidRequestException(String.format("Invalid tuple type literal for %s of type %s", receiver.name, receiver.type.asCQL3Type()));
 
-            TupleType tt = (TupleType)receiver.type;
+            TupleType tt = getTupleType(receiver.type);
             for (int i = 0; i < elements.size(); i++)
             {
                 if (i >= tt.size())
@@ -256,7 +256,7 @@ public class Tuples
                 List<?> l = type.getSerializer().deserializeForNativeProtocol(value, options.getProtocolVersion());
 
                 assert type.getElementsType() instanceof TupleType;
-                TupleType tupleType = (TupleType) type.getElementsType();
+                TupleType tupleType = Tuples.getTupleType(type.getElementsType());
 
                 // type.split(bytes)
                 List<List<ByteBuffer>> elements = new ArrayList<>(l.size());
@@ -375,7 +375,7 @@ public class Tuples
             ByteBuffer value = options.getValues().get(bindIndex);
             if (value == ByteBufferUtil.UNSET_BYTE_BUFFER)
                 throw new InvalidRequestException(String.format("Invalid unset value for tuple %s", receiver.name));
-            return value == null ? null : Value.fromSerialized(value, (TupleType)receiver.type);
+            return value == null ? null : Value.fromSerialized(value, getTupleType(receiver.type));
         }
     }
 
@@ -412,4 +412,16 @@ public class Tuples
         sb.append(')');
         return sb.toString();
     }
+
+    public static boolean checkIfTupleType(AbstractType<?> tuple)
+    {
+        return (tuple instanceof TupleType) ||
+               (tuple instanceof ReversedType && ((ReversedType) tuple).baseType instanceof TupleType);
+
+    }
+
+    public static TupleType getTupleType(AbstractType<?> tuple)
+    {
+        return (tuple instanceof ReversedType ? ((TupleType) ((ReversedType) tuple).baseType) : (TupleType)tuple);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a08a816a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
index cbe4a15..bace751 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
@@ -17,6 +17,10 @@
  */
 package org.apache.cassandra.cql3.validation.entities;
 
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Locale;
+
 import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
@@ -211,4 +215,14 @@ public class TupleTypeTest extends CQLTester
         assertInvalidMessage("Not enough bytes to read 0th component",
                              "INSERT INTO %s (pk, t) VALUES (?, ?)", 1, Long.MAX_VALUE);
     }
+
+    @Test
+    public void testReversedTypeTuple() throws Throwable
+    {
+        // CASSANDRA-13717
+        createTable("CREATE TABLE %s (id int, tdemo frozen<tuple<timestamp, varchar>>, primary key (id, tdemo)) with clustering order by (tdemo desc)");
+        execute("INSERT INTO %s (id, tdemo) VALUES (1, ('2017-02-03 03:05+0000','Europe'))");
+        DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mmX", Locale.ENGLISH);
+        assertRows(execute("SELECT tdemo FROM %s"), row(tuple( df.parse("2017-02-03 03:05+0000"), "Europe")));
+    }
 }


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


[5/6] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by ja...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/cassandra-3.11
Commit: cb2a1c8f4209ffe9aea8e40e7f0e45dc70613645
Parents: c05d98a a08a816
Author: Jason Brown <ja...@gmail.com>
Authored: Tue Sep 12 14:11:06 2017 -0700
Committer: Jason Brown <ja...@gmail.com>
Committed: Tue Sep 12 14:14:28 2017 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 src/java/org/apache/cassandra/cql3/Tuples.java  | 24 +++++++++++++++-----
 .../cql3/validation/entities/TupleTypeTest.java | 14 ++++++++++++
 3 files changed, 33 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb2a1c8f/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 2c48ab2,3d3903e..ec9d126
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,14 -1,5 +1,15 @@@
 -3.0.15
 +3.11.1
 + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418)
 + * BTree.Builder memory leak (CASSANDRA-13754)
 + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798)
 + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938)
 + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744)
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
+  * INSERT statement fails when Tuple type is used as clustering column with default DESC order (CASSANDRA-13717)
   * Fix pending view mutations handling and cleanup batchlog when there are local and remote paired mutations (CASSANDRA-13069)
   * Improve config validation and documentation on overflow and NPE (CASSANDRA-13622)
   * Range deletes in a CAS batch are ignored (CASSANDRA-13655)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb2a1c8f/src/java/org/apache/cassandra/cql3/Tuples.java
----------------------------------------------------------------------


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