You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "jacek-lewandowski (via GitHub)" <gi...@apache.org> on 2023/02/10 14:30:03 UTC

[GitHub] [cassandra] jacek-lewandowski opened a new pull request, #2150: Fix null value handling for static columns

jacek-lewandowski opened a new pull request, #2150:
URL: https://github.com/apache/cassandra/pull/2150

   Thanks for sending a pull request! Here are some tips if you're new here:
    
    * Ensure you have added or run the [appropriate tests](https://cassandra.apache.org/_/development/testing.html) for your PR.
    * Be sure to keep the PR description updated to reflect all changes.
    * Write your PR title to summarize what this PR proposes.
    * If possible, provide a concise example to reproduce the issue for a faster review.
    * Read our [contributor guidelines](https://cassandra.apache.org/_/development/index.html)
    * If you're making a documentation change, see our [guide to documentation contribution](https://cassandra.apache.org/_/development/documentation.html)
    
   Commit messages should follow the following format:
   
   ```
   <One sentence description, usually Jira title or CHANGES.txt summary>
   
   <Optional lengthier description (context on patch)>
   
   patch by <Authors>; reviewed by <Reviewers> for CASSANDRA-#####
   
   Co-authored-by: Name1 <email1>
   Co-authored-by: Name2 <email2>
   
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/issues/CASSANDRA-18241)
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103045010


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -181,29 +191,49 @@ private SimpleQueryResult executeWithRetry0(int count, Cluster cluster, String c
         {
             return execute(cluster, check, boundValues);
         }
-        catch (Throwable t)
+        catch (RuntimeException ex)
         {
-            if (AssertionUtils.rootCauseIs(Preempted.class).matches(t))
+            if (count <= 10 && AssertionUtils.rootCauseIs(Preempted.class).matches(ex))
             {
-                logger.warn("[Retry attempt={}] Preempted failure for {}", count, check);
+                logger.warn("[Retry attempt={}] Preempted failure for\n{}", count, check);
                 return executeWithRetry0(count + 1, cluster, check, boundValues);
             }
 
-            throw t;
+            throw ex;
         }
     }
 
     protected SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
     {
+        check = normalizeStatement(check);
+
         // is this method safe?
-        cluster.get(1).runOnInstance(() -> {
-            TransactionStatement stmt = AccordTestUtils.parse(check);
-            if (!isIdempotent(stmt))
-                throw new AssertionError("Unable to retry txn that is not idempotent: cql=" + check);
-        });
+
+        if (!isIdempotent(cluster, check))
+            throw new AssertionError("Unable to retry txn that is not idempotent: cql=\n" + check);
+
         return executeWithRetry0(0, cluster, check, boundValues);
     }
 
+    private boolean isIdempotent(Cluster cluster, String cql)
+    {
+        return cluster.get(1).callOnInstance(() -> {
+            String normalized = normalizeStatement(cql);
+            TransactionStatement stmt = AccordTestUtils.parse(normalizeStatement(normalizeStatement(normalized)));

Review Comment:
   Are we normalizing the same statement 3 times here? Confused...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103120196


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -181,29 +191,49 @@ private SimpleQueryResult executeWithRetry0(int count, Cluster cluster, String c
         {
             return execute(cluster, check, boundValues);
         }
-        catch (Throwable t)
+        catch (RuntimeException ex)
         {
-            if (AssertionUtils.rootCauseIs(Preempted.class).matches(t))
+            if (count <= 10 && AssertionUtils.rootCauseIs(Preempted.class).matches(ex))

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103119719


##########
test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java:
##########
@@ -63,7 +63,14 @@ public Coordinator(Instance instance)
     @Override
     public SimpleQueryResult executeWithResult(String query, ConsistencyLevel consistencyLevel, Object... boundValues)
     {
-        return instance().sync(() -> unsafeExecuteInternal(query, consistencyLevel, boundValues)).call();

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103151724


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordCQLTest.java:
##########
@@ -266,6 +269,111 @@ private void testScalarIsNull(String tableDDL) throws Exception {
              });
     }
 
+    @Test
+    public void testQueryStaticColumn() throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, s int static, v int, primary key (k, c))",
+             cluster ->
+             {
+                 // select partition key, clustering key and static column, restrict on partition and clustering
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0");
+
+                 // select partition key, clustering key and static column, restrict on partition and limit to 1 row
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1");
+
+                 // select just static column, restrict on partition and clustering
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT s, v FROM " + currentTable + " WHERE k = ? AND c = 0);\n" +
+                                       "SELECT row0.s, row0.v;\n",
+
+                                       "SELECT s, v FROM " + currentTable + " WHERE k = ? AND c = 0");
+
+                 // select just static column, restrict on partition and limit to 1 row

Review Comment:
   This selects s and v, not just s?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] dcapwell commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "dcapwell (via GitHub)" <gi...@apache.org>.
dcapwell commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103010031


##########
test/distributed/org/apache/cassandra/distributed/impl/Coordinator.java:
##########
@@ -63,7 +63,14 @@ public Coordinator(Instance instance)
     @Override
     public SimpleQueryResult executeWithResult(String query, ConsistencyLevel consistencyLevel, Object... boundValues)
     {
-        return instance().sync(() -> unsafeExecuteInternal(query, consistencyLevel, boundValues)).call();

Review Comment:
   in this case I think we should revert, but think we should change this behavior on trunk... I strongly regret making the decision to not "wrap" the internal exceptions... its so much harder to debug =(



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103043582


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -181,29 +191,49 @@ private SimpleQueryResult executeWithRetry0(int count, Cluster cluster, String c
         {
             return execute(cluster, check, boundValues);
         }
-        catch (Throwable t)
+        catch (RuntimeException ex)
         {
-            if (AssertionUtils.rootCauseIs(Preempted.class).matches(t))
+            if (count <= 10 && AssertionUtils.rootCauseIs(Preempted.class).matches(ex))
             {
-                logger.warn("[Retry attempt={}] Preempted failure for {}", count, check);
+                logger.warn("[Retry attempt={}] Preempted failure for\n{}", count, check);
                 return executeWithRetry0(count + 1, cluster, check, boundValues);
             }
 
-            throw t;
+            throw ex;
         }
     }
 
     protected SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
     {
+        check = normalizeStatement(check);
+
         // is this method safe?
-        cluster.get(1).runOnInstance(() -> {
-            TransactionStatement stmt = AccordTestUtils.parse(check);
-            if (!isIdempotent(stmt))
-                throw new AssertionError("Unable to retry txn that is not idempotent: cql=" + check);
-        });
+
+        if (!isIdempotent(cluster, check))
+            throw new AssertionError("Unable to retry txn that is not idempotent: cql=\n" + check);
+
         return executeWithRetry0(0, cluster, check, boundValues);
     }
 
+    private boolean isIdempotent(Cluster cluster, String cql)
+    {
+        return cluster.get(1).callOnInstance(() -> {
+            String normalized = normalizeStatement(cql);
+            TransactionStatement stmt = AccordTestUtils.parse(normalizeStatement(normalizeStatement(normalized)));
+            return isIdempotent(stmt);
+        });
+    }
+
+    private static String normalizeStatement(String statement)

Review Comment:
   ```suggestion
       private static String wrapInTxn(String statement)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103167986


##########
src/java/org/apache/cassandra/service/accord/txn/TxnNamedRead.java:
##########
@@ -130,9 +130,13 @@ public Future<Data> read(boolean isForWriteTxn, SafeCommandStore safeStore, Time
                  UnfilteredPartitionIterator partition = read.executeLocally(controller);
                  PartitionIterator iterator = UnfilteredPartitionIterators.filter(partition, read.nowInSec()))
             {
-                FilteredPartition filtered = FilteredPartition.create(PartitionIterators.getOnlyElement(iterator, read));
                 TxnData result = new TxnData();
-                result.put(name, filtered);
+                if (iterator.hasNext())
+                {
+                    FilteredPartition filtered = FilteredPartition.create(PartitionIterators.getOnlyElement(iterator, read));

Review Comment:
   ```suggestion
                       FilteredPartition filtered = FilteredPartition.create(iterator.next());
   ```
   We should be able to simplify ^
   
   The only other notable thing that `getOnlyElement()` seems to do is close the iterator, but `iterator` is already in a try-with-resources block above, so that shouldn't be an issue.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103152508


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordCQLTest.java:
##########
@@ -266,6 +269,111 @@ private void testScalarIsNull(String tableDDL) throws Exception {
              });
     }
 
+    @Test
+    public void testQueryStaticColumn() throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, s int static, v int, primary key (k, c))",
+             cluster ->
+             {
+                 // select partition key, clustering key and static column, restrict on partition and clustering
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0");
+
+                 // select partition key, clustering key and static column, restrict on partition and limit to 1 row
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1");
+
+                 // select just static column, restrict on partition and clustering

Review Comment:
   This selects s and v, not just s?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103042795


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -181,29 +191,49 @@ private SimpleQueryResult executeWithRetry0(int count, Cluster cluster, String c
         {
             return execute(cluster, check, boundValues);
         }
-        catch (Throwable t)
+        catch (RuntimeException ex)
         {
-            if (AssertionUtils.rootCauseIs(Preempted.class).matches(t))
+            if (count <= 10 && AssertionUtils.rootCauseIs(Preempted.class).matches(ex))

Review Comment:
   nit: Pull up a constant `MAX_RETRIES`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103120538


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -181,29 +191,49 @@ private SimpleQueryResult executeWithRetry0(int count, Cluster cluster, String c
         {
             return execute(cluster, check, boundValues);
         }
-        catch (Throwable t)
+        catch (RuntimeException ex)
         {
-            if (AssertionUtils.rootCauseIs(Preempted.class).matches(t))
+            if (count <= 10 && AssertionUtils.rootCauseIs(Preempted.class).matches(ex))
             {
-                logger.warn("[Retry attempt={}] Preempted failure for {}", count, check);
+                logger.warn("[Retry attempt={}] Preempted failure for\n{}", count, check);
                 return executeWithRetry0(count + 1, cluster, check, boundValues);
             }
 
-            throw t;
+            throw ex;
         }
     }
 
     protected SimpleQueryResult executeWithRetry(Cluster cluster, String check, Object... boundValues)
     {
+        check = normalizeStatement(check);
+
         // is this method safe?
-        cluster.get(1).runOnInstance(() -> {
-            TransactionStatement stmt = AccordTestUtils.parse(check);
-            if (!isIdempotent(stmt))
-                throw new AssertionError("Unable to retry txn that is not idempotent: cql=" + check);
-        });
+
+        if (!isIdempotent(cluster, check))
+            throw new AssertionError("Unable to retry txn that is not idempotent: cql=\n" + check);
+
         return executeWithRetry0(0, cluster, check, boundValues);
     }
 
+    private boolean isIdempotent(Cluster cluster, String cql)
+    {
+        return cluster.get(1).callOnInstance(() -> {
+            String normalized = normalizeStatement(cql);
+            TransactionStatement stmt = AccordTestUtils.parse(normalizeStatement(normalizeStatement(normalized)));

Review Comment:
   🤦🏻‍♂️ 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] jacek-lewandowski merged pull request #2150: Fix null value handling for static columns

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski merged PR #2150:
URL: https://github.com/apache/cassandra/pull/2150


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] jacek-lewandowski commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "jacek-lewandowski (via GitHub)" <gi...@apache.org>.
jacek-lewandowski commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103222273


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordCQLTest.java:
##########
@@ -266,6 +269,111 @@ private void testScalarIsNull(String tableDDL) throws Exception {
              });
     }
 
+    @Test
+    public void testQueryStaticColumn() throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, s int static, v int, primary key (k, c))",
+             cluster ->
+             {
+                 // select partition key, clustering key and static column, restrict on partition and clustering
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0");
+
+                 // select partition key, clustering key and static column, restrict on partition and limit to 1 row
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1");
+
+                 // select just static column, restrict on partition and clustering

Review Comment:
   ```org.apache.cassandra.exceptions.InvalidRequestException: Cannot restrict clustering columns when selecting only static columns```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103043113


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordTestBase.java:
##########
@@ -140,8 +142,16 @@ private static Cluster createCluster() throws IOException
                            .start());
     }
 
-    private static SimpleQueryResult execute(Cluster cluster, String check, Object... boundValues)
+    protected static SimpleQueryResult executeTxn(Cluster cluster, String check, Object... boundValues)

Review Comment:
   ```suggestion
       protected static SimpleQueryResult executeAsTxn(Cluster cluster, String check, Object... boundValues)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [cassandra] maedhroz commented on a diff in pull request #2150: Fix null value handling for static columns

Posted by "maedhroz (via GitHub)" <gi...@apache.org>.
maedhroz commented on code in PR #2150:
URL: https://github.com/apache/cassandra/pull/2150#discussion_r1103245659


##########
test/distributed/org/apache/cassandra/distributed/test/accord/AccordCQLTest.java:
##########
@@ -266,6 +269,111 @@ private void testScalarIsNull(String tableDDL) throws Exception {
              });
     }
 
+    @Test
+    public void testQueryStaticColumn() throws Exception
+    {
+        test("CREATE TABLE " + currentTable + " (k int, c int, s int static, v int, primary key (k, c))",
+             cluster ->
+             {
+                 // select partition key, clustering key and static column, restrict on partition and clustering
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? AND c = 0");
+
+                 // select partition key, clustering key and static column, restrict on partition and limit to 1 row
+                 testQueryStaticColumn(cluster,
+                                       "LET row0 = (SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1);\n" +
+                                       "SELECT row0.k, row0.c, row0.s, row0.v;\n",
+
+                                       "SELECT k, c, s, v FROM " + currentTable + " WHERE k = ? LIMIT 1");
+
+                 // select just static column, restrict on partition and clustering

Review Comment:
   Ah, yeah, in that case, makes sense, although I'd just update the comment?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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