You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2014/11/04 09:48:36 UTC

git commit: Fix default timestamp in QueryOptions

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 a8072ae3b -> e95e569c2


Fix default timestamp in QueryOptions

patch by beobal; reviewed by slebresne for CASSANDRA-8246


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

Branch: refs/heads/cassandra-2.1
Commit: e95e569c248f5f08702ca06a338f5f92975d3077
Parents: a8072ae
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Tue Nov 4 09:47:28 2014 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Nov 4 09:48:24 2014 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/cql3/QueryOptions.java |  2 +-
 .../cassandra/cql3/NonNativeTimestampTest.java  | 86 ++++++++++++++++++++
 3 files changed, 88 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e95e569c/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index b1a8439..8dd2613 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.2
+ * Fix default timestamp in QueryOptions (CASSANDRA-8246)
  * Set socket timeout when reading remote version (CASSANDRA-8188)
  * Refactor how we track live size (CASSANDRA-7852)
  * Make sure unfinished compaction files are removed (CASSANDRA-8124)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e95e569c/src/java/org/apache/cassandra/cql3/QueryOptions.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryOptions.java b/src/java/org/apache/cassandra/cql3/QueryOptions.java
index 5431a42..b2569e7 100644
--- a/src/java/org/apache/cassandra/cql3/QueryOptions.java
+++ b/src/java/org/apache/cassandra/cql3/QueryOptions.java
@@ -242,7 +242,7 @@ public abstract class QueryOptions
     // Options that are likely to not be present in most queries
     static class SpecificOptions
     {
-        private static final SpecificOptions DEFAULT = new SpecificOptions(-1, null, null, -1L);
+        private static final SpecificOptions DEFAULT = new SpecificOptions(-1, null, null, Long.MIN_VALUE);
 
         private final int pageSize;
         private final PagingState state;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e95e569c/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java b/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
new file mode 100644
index 0000000..8e3fbed
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/NonNativeTimestampTest.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.cql3;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+import java.nio.charset.CharacterCodingException;
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.service.EmbeddedCassandraService;
+import org.apache.cassandra.service.QueryState;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class NonNativeTimestampTest extends SchemaLoader
+{
+    @BeforeClass
+    public static void setup() throws Exception
+    {
+        Schema.instance.clear();
+        EmbeddedCassandraService cassandra = new EmbeddedCassandraService();
+        cassandra.start();
+    }
+
+    @Test
+    public void setServerTimestampForNonCqlNativeStatements() throws RequestValidationException, RequestExecutionException, CharacterCodingException, UnsupportedEncodingException
+    {
+        String createKsCQL = "CREATE KEYSPACE non_native_ts_test" +
+                             " WITH REPLICATION = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 };";
+        String createTableCQL = "CREATE TABLE non_native_ts_test.table_0 (k int PRIMARY KEY, v int)";
+        String insertCQL = "INSERT INTO non_native_ts_test.table_0 (k, v) values (1, ?)";
+        String selectCQL = "SELECT v, writetime(v) AS wt FROM non_native_ts_test.table_0 WHERE k = 1";
+
+        QueryProcessor.instance.process(createKsCQL,
+                                        QueryState.forInternalCalls(),
+                                        QueryOptions.forInternalCalls(Collections.<ByteBuffer>emptyList()));
+        QueryProcessor.instance.process(createTableCQL,
+                                        QueryState.forInternalCalls(),
+                                        QueryOptions.forInternalCalls(Collections.<ByteBuffer>emptyList()));
+        QueryProcessor.instance.process(insertCQL,
+                                        QueryState.forInternalCalls(),
+                                        QueryOptions.forInternalCalls(ConsistencyLevel.ONE,
+                                                                      Arrays.asList(ByteBufferUtil.bytes(2))));
+        UntypedResultSet.Row row = QueryProcessor.instance.executeInternal(selectCQL).one();
+        assertEquals(2, row.getInt("v"));
+        long timestamp1 = row.getLong("wt");
+        assertFalse(timestamp1 == -1l);
+
+        // per CASSANDRA-8246 the two updates will have the same (incorrect)
+        // timestamp, so reconcilliation is by value and the "older" update wins
+        QueryProcessor.instance.process(insertCQL,
+                                        QueryState.forInternalCalls(),
+                                        QueryOptions.forInternalCalls(ConsistencyLevel.ONE,
+                                                                      Arrays.asList(ByteBufferUtil.bytes(1))));
+        row = QueryProcessor.executeInternal(selectCQL).one();
+        assertEquals(1, row.getInt("v"));
+        assertTrue(row.getLong("wt") > timestamp1);
+    }
+}