You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/04/14 13:45:03 UTC

[GitHub] [cassandra] adelapena opened a new pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

adelapena opened a new pull request #967:
URL: https://github.com/apache/cassandra/pull/967


   


-- 
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.

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] adelapena commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r618356292



##########
File path: src/java/org/apache/cassandra/db/ColumnFamilyStore.java
##########
@@ -2290,14 +2290,15 @@ public void run()
             indexManager.truncateAllIndexesBlocking(truncatedAt);
             viewManager.truncateBlocking(replayAfter, truncatedAt);
 
-                SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
-                logger.trace("cleaning out row cache");
-                invalidateCaches();
-
-            }
+            SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
+            logger.trace("cleaning out row cache");
+            invalidateCaches();
         };
 
         runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true);
+
+        viewManager.build();

Review comment:
       > Can we also test the viewManager after truncate?
   
   I don't understand this, the new `testTruncateWhileBuilding` waits for this second view build and verifies that it marks the view as built in the `system.built_views` table and that the MV is still empty. What else should we test? Maybe trying to write some data to the truncated view?
   
   > I am a bit confused actually...did you mean build or start? From your comment it sounded more as you meant to call viewManager.start?
   
   I don't see a `viewManager.start` method, do you mean renaming the method `TableViews#build()` to `TableViews#start()`? By the way, it's a bit confusing that `ColumnFamilyStore#viewManager` is a `TableViews` object, and not an actual `ViewManager`, I guess this is done so for the sake of symmetry with `ColumnFamilyStore#indexManager`.




-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r617093122



##########
File path: src/java/org/apache/cassandra/db/ColumnFamilyStore.java
##########
@@ -2290,14 +2290,15 @@ public void run()
             indexManager.truncateAllIndexesBlocking(truncatedAt);
             viewManager.truncateBlocking(replayAfter, truncatedAt);
 
-                SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
-                logger.trace("cleaning out row cache");
-                invalidateCaches();
-
-            }
+            SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
+            logger.trace("cleaning out row cache");
+            invalidateCaches();
         };
 
         runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true);
+
+        viewManager.build();

Review comment:
       that is something I was thinking about last night and now it clicks... good catch
   Can we also test the viewManager after truncate?

##########
File path: src/java/org/apache/cassandra/db/ColumnFamilyStore.java
##########
@@ -2290,14 +2290,15 @@ public void run()
             indexManager.truncateAllIndexesBlocking(truncatedAt);
             viewManager.truncateBlocking(replayAfter, truncatedAt);
 
-                SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
-                logger.trace("cleaning out row cache");
-                invalidateCaches();
-
-            }
+            SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
+            logger.trace("cleaning out row cache");
+            invalidateCaches();
         };
 
         runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true);
+
+        viewManager.build();

Review comment:
       I am a bit confused actually...did you mean build or start? From your comment it sounded more as you meant to call viewManager.start?




-- 
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.

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] adelapena commented on pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on pull request #967:
URL: https://github.com/apache/cassandra/pull/967#issuecomment-822650536


   CI:
   * [CircleCI j8](https://app.circleci.com/pipelines/github/adelapena/cassandra/278/workflows/ebd918cf-6259-4726-9b54-5f2a6e43b2e1)
   * [CircleCI j11](https://app.circleci.com/pipelines/github/adelapena/cassandra/278/workflows/bdbf9cb9-8be7-4527-8a55-fac9c5bb42c8)
   * [Jenkins](https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/684/pipeline/)
   * [Multiplexer](https://jenkins-dse.build.dsinternal.org/view/Parameterized/job/parameterized-testall/810/)


-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615975528



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       I didn't mean to improve the tests which I agree we should do in post 4.0, absolutely




-- 
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.

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] adelapena commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r616010985



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       I have opted for changing the debug message to a simple `Stopping current view builder due to schema change or truncate`. Perhaps we could pass an argument to the method about the specific cause, although being a debug message I guess that the specific cause will be also logged around, wdyt?




-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615976613



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       I had to say "improve" instead of "revise", my bad, sorry for the confusion. 




-- 
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.

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] adelapena commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615969298



##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1469,4 +1460,32 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
                    row("\"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL"),
                    row("token(\"theKey\") > token(1) AND \"theClustering_1\" = 1 AND \"theClustering_2\" > 2 AND \"theValue\" IS NOT NULL"));
     }
+
+    /**
+     * Tests that truncating a table stops the ongoing builds of its materialized views,
+     * so they don't write into the MV data that has been truncated in the base table.
+     *
+     * See CASSANDRA-16567 for further details.
+     */
+    @Test
+    @BMRule(name = "Delay materialized view mutations",
+    targetClass = "StorageProxy",
+    targetMethod = "mutateMV",
+    action = "Thread.sleep(4000);")
+    public void testTruncateWhileBuilding() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
+        execute("USE " + keyspace());
+        executeNet("USE " + keyspace());
+        updateView("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);

Review comment:
       Good catch, we don't need `updateView` at all, replaced by a regular `execute`.




-- 
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.

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] adelapena commented on pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on pull request #967:
URL: https://github.com/apache/cassandra/pull/967#issuecomment-823190585


   I have realized that, since we are stopping the build at the beginning of truncate, the view never gets marked as built in the system tables. To fix this I'm just running a new view build at the end of truncate. We could also directly mark the views as built instead of running a builder assuming that the view is going to be empty. However it feels safer to just reuse the regular MV lifecycle, in case new sstables appear in the meantime.


-- 
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.

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] adelapena commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615976421



##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1469,4 +1460,32 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
                    row("\"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL"),
                    row("token(\"theKey\") > token(1) AND \"theClustering_1\" = 1 AND \"theClustering_2\" > 2 AND \"theValue\" IS NOT NULL"));
     }
+
+    /**
+     * Tests that truncating a table stops the ongoing builds of its materialized views,
+     * so they don't write into the MV data that has been truncated in the base table.
+     *
+     * See CASSANDRA-16567 for further details.
+     */
+    @Test
+    @BMRule(name = "Delay materialized view mutations",
+    targetClass = "StorageProxy",
+    targetMethod = "mutateMV",
+    action = "Thread.sleep(4000);")
+    public void testTruncateWhileBuilding() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
+        execute("USE " + keyspace());
+        executeNet("USE " + keyspace());
+        updateView("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
+        createView("mv",
+                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+                   "WHERE k IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL " +
+                   "PRIMARY KEY (v, c, k)");
+        Thread.sleep(2000); // wait for the creation of MV build tasks

Review comment:
       I have replaced the sleeps by a `CountDownLatch` manipulated by a couple of Byteman rules. One rule blocks the view builder tasks on the latch, and the other rule unblocks them when the truncation starts. That seems enough to consistently reproduce the bug when we don't stop the build on `truncateBlocking`. I'll give it a few runs in the multiplexer just in case. 




-- 
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.

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] adelapena commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615982046



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       Ah, I understand, good catch, I'll change it.




-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r618364926



##########
File path: src/java/org/apache/cassandra/db/ColumnFamilyStore.java
##########
@@ -2290,14 +2290,15 @@ public void run()
             indexManager.truncateAllIndexesBlocking(truncatedAt);
             viewManager.truncateBlocking(replayAfter, truncatedAt);
 
-                SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
-                logger.trace("cleaning out row cache");
-                invalidateCaches();
-
-            }
+            SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
+            logger.trace("cleaning out row cache");
+            invalidateCaches();
         };
 
         runWithCompactionsDisabled(Executors.callable(truncateRunnable), true, true);
+
+        viewManager.build();

Review comment:
       _I don't see a viewManager.start method, do you mean renaming the method TableViews#build() to TableViews#start()? By the way, it's a bit confusing that ColumnFamilyStore#viewManager is a TableViews object, and not an actual ViewManager, I guess this is done so for the sake of symmetry with ColumnFamilyStore#indexManager._
   
   Yes, my bad It seems I switched again to the ViewManager class at some point. I. was thinking of changing the ViewManager name to reduce the confusion but not sure whether this will be appreciated now. 
   I agree with all your points and double-checked the test. 
   I think we are ready to commit this one, thank you!




-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615048451



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       Should we revise the debug log messages? WDYT?
   https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/view/View.java#L223
   

##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -50,11 +51,12 @@
 import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.FBUtilities;
+import org.jboss.byteman.contrib.bmunit.BMRule;
+import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
 
-
+@RunWith(BMUnitRunner.class)
 public class ViewTest extends CQLTester
 {
-    ProtocolVersion protocolVersion = ProtocolVersion.V4;

Review comment:
       Good catch, thanks for fixing it and making it impossible to be missed again.

##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1469,4 +1460,32 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
                    row("\"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL"),
                    row("token(\"theKey\") > token(1) AND \"theClustering_1\" = 1 AND \"theClustering_2\" > 2 AND \"theValue\" IS NOT NULL"));
     }
+
+    /**
+     * Tests that truncating a table stops the ongoing builds of its materialized views,
+     * so they don't write into the MV data that has been truncated in the base table.
+     *
+     * See CASSANDRA-16567 for further details.
+     */
+    @Test
+    @BMRule(name = "Delay materialized view mutations",
+    targetClass = "StorageProxy",
+    targetMethod = "mutateMV",
+    action = "Thread.sleep(4000);")
+    public void testTruncateWhileBuilding() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
+        execute("USE " + keyspace());
+        executeNet("USE " + keyspace());
+        updateView("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);

Review comment:
       updateView before createView?
   

##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1469,4 +1460,32 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
                    row("\"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL"),
                    row("token(\"theKey\") > token(1) AND \"theClustering_1\" = 1 AND \"theClustering_2\" > 2 AND \"theValue\" IS NOT NULL"));
     }
+
+    /**
+     * Tests that truncating a table stops the ongoing builds of its materialized views,
+     * so they don't write into the MV data that has been truncated in the base table.
+     *
+     * See CASSANDRA-16567 for further details.
+     */
+    @Test
+    @BMRule(name = "Delay materialized view mutations",
+    targetClass = "StorageProxy",
+    targetMethod = "mutateMV",
+    action = "Thread.sleep(4000);")
+    public void testTruncateWhileBuilding() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
+        execute("USE " + keyspace());
+        executeNet("USE " + keyspace());
+        updateView("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
+        createView("mv",
+                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+                   "WHERE k IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL " +
+                   "PRIMARY KEY (v, c, k)");
+        Thread.sleep(2000); // wait for the creation of MV build tasks

Review comment:
       I hope these sleeps survive our CIs, I have bad feeling when we rely on timing but I guess it should be fine, you also pushed it on the multiplexer :-) 

##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1469,4 +1460,32 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
                    row("\"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL"),
                    row("token(\"theKey\") > token(1) AND \"theClustering_1\" = 1 AND \"theClustering_2\" > 2 AND \"theValue\" IS NOT NULL"));
     }
+
+    /**
+     * Tests that truncating a table stops the ongoing builds of its materialized views,
+     * so they don't write into the MV data that has been truncated in the base table.
+     *
+     * See CASSANDRA-16567 for further details.
+     */
+    @Test
+    @BMRule(name = "Delay materialized view mutations",
+    targetClass = "StorageProxy",
+    targetMethod = "mutateMV",
+    action = "Thread.sleep(4000);")

Review comment:
       I guess 4000 came based on testing

##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1469,4 +1460,32 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
                    row("\"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL"),
                    row("token(\"theKey\") > token(1) AND \"theClustering_1\" = 1 AND \"theClustering_2\" > 2 AND \"theValue\" IS NOT NULL"));
     }
+
+    /**
+     * Tests that truncating a table stops the ongoing builds of its materialized views,
+     * so they don't write into the MV data that has been truncated in the base table.
+     *
+     * See CASSANDRA-16567 for further details.
+     */
+    @Test
+    @BMRule(name = "Delay materialized view mutations",
+    targetClass = "StorageProxy",
+    targetMethod = "mutateMV",
+    action = "Thread.sleep(4000);")
+    public void testTruncateWhileBuilding() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
+        execute("USE " + keyspace());
+        executeNet("USE " + keyspace());
+        updateView("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
+        createView("mv",
+                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+                   "WHERE k IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL " +
+                   "PRIMARY KEY (v, c, k)");
+        Thread.sleep(2000); // wait for the creation of MV build tasks

Review comment:
       I would also probably add one more `assertRows(execute("SELECT * FROM mv WHERE v = ? and c = ?", 0, 0));` before `TRUNCATE`




-- 
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.

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] adelapena closed pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena closed pull request #967:
URL: https://github.com/apache/cassandra/pull/967


   


-- 
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.

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] adelapena commented on pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on pull request #967:
URL: https://github.com/apache/cassandra/pull/967#issuecomment-823193070


   CI:
   * [CircleCI j8](https://app.circleci.com/pipelines/github/adelapena/cassandra/279/workflows/7a4f495b-ed8e-4aec-91c6-c34f16d8cc2e)
   * [CircleCI j11](https://app.circleci.com/pipelines/github/adelapena/cassandra/279/workflows/29b4ae05-ac9e-4d53-82e0-3757cf6cf2f0)
   * [ci-cassandra](https://ci-cassandra.apache.org/blue/organizations/jenkins/Cassandra-devbranch/detail/Cassandra-devbranch/685/pipeline)
   * [Multiplexer](https://jenkins-dse.build.dsinternal.org/view/Parameterized/job/parameterized-testall/812/)


-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r616288202



##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1469,4 +1460,32 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
                    row("\"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL"),
                    row("token(\"theKey\") > token(1) AND \"theClustering_1\" = 1 AND \"theClustering_2\" > 2 AND \"theValue\" IS NOT NULL"));
     }
+
+    /**
+     * Tests that truncating a table stops the ongoing builds of its materialized views,
+     * so they don't write into the MV data that has been truncated in the base table.
+     *
+     * See CASSANDRA-16567 for further details.
+     */
+    @Test
+    @BMRule(name = "Delay materialized view mutations",
+    targetClass = "StorageProxy",
+    targetMethod = "mutateMV",
+    action = "Thread.sleep(4000);")
+    public void testTruncateWhileBuilding() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
+        execute("USE " + keyspace());
+        executeNet("USE " + keyspace());
+        updateView("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
+        createView("mv",
+                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+                   "WHERE k IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL " +
+                   "PRIMARY KEY (v, c, k)");
+        Thread.sleep(2000); // wait for the creation of MV build tasks

Review comment:
       Looks quite good, thanks




-- 
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.

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] adelapena commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615980232



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       As an alternative to log checks I have added a couple of assertions on the number of MV builders tasks running on the view build executor.




-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r616287328



##########
File path: test/unit/org/apache/cassandra/cql3/ViewTest.java
##########
@@ -1468,24 +1476,44 @@ public void testQuotedIdentifiersInWhereClause() throws Throwable
      * See CASSANDRA-16567 for further details.
      */
     @Test
-    @BMRule(name = "Delay materialized view mutations",
-    targetClass = "StorageProxy",
-    targetMethod = "mutateMV",
-    action = "Thread.sleep(4000);")
+    @BMRules(rules = {
+    @BMRule(name = "Block view builder tasks",
+    targetClass = "ViewBuilderTask",
+    targetMethod = "buildKey",
+    action = "com.google.common.util.concurrent.Uninterruptibles.awaitUninterruptibly" +
+             "(org.apache.cassandra.cql3.ViewTest.blockViewBuild);"),
+    @BMRule(name = "Unblock view builder tasks",
+    targetClass = "ColumnFamilyStore",
+    targetMethod = "truncateBlocking",
+    action = "org.apache.cassandra.cql3.ViewTest.blockViewBuild.countDown();")
+    })
     public void testTruncateWhileBuilding() throws Throwable
     {
         createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
         execute("USE " + keyspace());
         executeNet("USE " + keyspace());
-        updateView("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
+        execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
         createView("mv",
                    "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
                    "WHERE k IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL " +
                    "PRIMARY KEY (v, c, k)");
-        Thread.sleep(2000); // wait for the creation of MV build tasks
+
+        // check that the delayed view builder tasks are running and they haven't written anything yet
+        assertThat(runningViewBuilds()).isPositive();

Review comment:
       From the comment it seems you check only active tasks, but it contains also pending ones




-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r616115176



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       I think this should be enough (`Stopping current view builder due to schema change or truncate`). Thank you




-- 
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.

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] ekaterinadimitrova2 commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
ekaterinadimitrova2 commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615975007



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       I meant the message I pointed to will be the same even if it is for truncation now. Might be confusing for users. 




-- 
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.

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] adelapena commented on a change in pull request #967: CASSANDRA-16567 (trunk): Fix test ViewTest.testCompoundPartitionKey

Posted by GitBox <gi...@apache.org>.
adelapena commented on a change in pull request #967:
URL: https://github.com/apache/cassandra/pull/967#discussion_r615968613



##########
File path: src/java/org/apache/cassandra/db/view/TableViews.java
##########
@@ -93,6 +93,11 @@ public boolean add(View view)
         return Iterables.transform(views, view -> keyspace.getColumnFamilyStore(view.getDefinition().name()));
     }
 
+    public void stopBuild()
+    {
+        views.forEach(View::stopBuild);

Review comment:
       I think we don't have an easy way to check log messages in unit tests. We have some checks for that type of messages in [materialized_views_test.py](https://github.com/apache/cassandra-dtest/blob/trunk/materialized_views_test.py), and probably we should add some additional tests for truncate there. However, some of those dtests are skipped due to being flaky, and indeed the entire class is marked as flaky. It's my understanding that fixing those will be part of the post-4.0 MV hardening effort, wdyt?




-- 
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.

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