[ https://issues.apache.org/jira/browse/CASSANDRA-17587?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17551451#comment-17551451 ]
Ekaterina Dimitrova commented on CASSANDRA-17587: ------------------------------------------------- I was bisecting this one and it points to CEP-11, the Memtable API - CASSANDRA-17034. [~barnie], [~maedhroz], [~adelapena], anyone of you having circles to take a look? We hit it fairly rarely, but it seems legit bug. I ran it in a loop like 500 times with a few older commits to be sure. At least once we hit it when we apply CASSANDRA-17034. 1000 [successful runs|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/1706/workflows/049f7235-337a-4569-ae34-7f26cec46c33] with the last commit before CASSANDRA-17034. [Failing already|https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/1705/workflows/40525702-9518-4e16-a2fd-82537fcf6fa8/jobs/11931] when we apply the patch. > Flaky compaction_test.py::TestCompaction:: test_user_defined_compaction > ----------------------------------------------------------------------- > > Key: CASSANDRA-17587 > URL: https://issues.apache.org/jira/browse/CASSANDRA-17587 > Project: Cassandra > Issue Type: Bug > Components: CI > Reporter: Ekaterina Dimitrova > Priority: Normal > Fix For: 4.1-beta, 4.1.x > > > As proved this test is flaky on trunk compaction_test.py::TestCompaction:: > test_user_defined_compaction: > [https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/1575/workflows/f9409df6-825f-4996-976a-0182549d87dc/jobs/10405/tests] > I haven't tested other branches, probably good to be tested by whoever > assigns this ticket. > {code:java} > test teardown failure Unexpected error found in node logs (see stdout for > full details). Errors: [ERROR [Native-Transport-Requests-53] 2022-04-29 > 16:54:33,003 StorageProxy.java:1631 - Failed to apply mutation locally : > java.lang.RuntimeException: null for ks: keyspace1, table: standard1 at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1442) > at > org.apache.cassandra.db.CassandraTableWriteHandler.write(CassandraTableWriteHandler.java:40) > at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:653) at > org.apache.cassandra.db.Keyspace.apply(Keyspace.java:506) at > org.apache.cassandra.db.Mutation.apply(Mutation.java:228) at > org.apache.cassandra.db.Mutation.apply(Mutation.java:248) at > org.apache.cassandra.service.StorageProxy$4.runMayThrow(StorageProxy.java:1625) > at > org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2537) > at > org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:124) > at > org.apache.cassandra.concurrent.SEPExecutor.maybeExecuteImmediately(SEPExecutor.java:214) > at > org.apache.cassandra.concurrent.Stage.maybeExecuteImmediately(Stage.java:127) > at > org.apache.cassandra.service.StorageProxy.performLocally(StorageProxy.java:1619) > at > org.apache.cassandra.service.StorageProxy.sendToHintedReplicas(StorageProxy.java:1526) > at > org.apache.cassandra.service.StorageProxy.lambda$static$0(StorageProxy.java:212) > at > org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:1358) > at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:867) > at > org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:1129) > at > org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:516) > at > org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:491) > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:258) > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:825) > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:803) > at > org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:167) > at org.apache.cassandra.transport.Message$Request.execute(Message.java:254) > at > org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:122) > at > org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:141) > at > org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:168) > at > org.apache.cassandra.transport.Dispatcher.lambda$dispatch$0(Dispatcher.java:82) > at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:81) at > org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:47) at > org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:57) at > org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:120) at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.base/java.lang.Thread.run(Thread.java:834) Caused by: > java.lang.NullPointerException: null at > org.apache.cassandra.db.memtable.AbstractMemtableWithCommitlog.accepts(AbstractMemtableWithCommitlog.java:101) > at > org.apache.cassandra.db.lifecycle.Tracker.getMemtableFor(Tracker.java:329) at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1421) > ... 33 common frames omitted, ERROR [Native-Transport-Requests-53] 2022-04-29 > 16:54:33,003 StorageProxy.java:1631 - Failed to apply mutation locally : > java.lang.RuntimeException: null for ks: keyspace1, table: standard1 at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1442) > at > org.apache.cassandra.db.CassandraTableWriteHandler.write(CassandraTableWriteHandler.java:40) > at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:653) at > org.apache.cassandra.db.Keyspace.apply(Keyspace.java:506) at > org.apache.cassandra.db.Mutation.apply(Mutation.java:228) at > org.apache.cassandra.db.Mutation.apply(Mutation.java:248) at > org.apache.cassandra.service.StorageProxy$4.runMayThrow(StorageProxy.java:1625) > at > org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2537) > at > org.apache.cassandra.concurrent.ExecutionFailure$1.run(ExecutionFailure.java:124) > at > org.apache.cassandra.concurrent.SEPExecutor.maybeExecuteImmediately(SEPExecutor.java:214) > at > org.apache.cassandra.concurrent.Stage.maybeExecuteImmediately(Stage.java:127) > at > org.apache.cassandra.service.StorageProxy.performLocally(StorageProxy.java:1619) > at > org.apache.cassandra.service.StorageProxy.sendToHintedReplicas(StorageProxy.java:1526) > at > org.apache.cassandra.service.StorageProxy.lambda$static$0(StorageProxy.java:212) > at > org.apache.cassandra.service.StorageProxy.performWrite(StorageProxy.java:1358) > at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:867) > at > org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageProxy.java:1129) > at > org.apache.cassandra.cql3.statements.ModificationStatement.executeWithoutCondition(ModificationStatement.java:516) > at > org.apache.cassandra.cql3.statements.ModificationStatement.execute(ModificationStatement.java:491) > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:258) > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:825) > at > org.apache.cassandra.cql3.QueryProcessor.processPrepared(QueryProcessor.java:803) > at > org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:167) > at org.apache.cassandra.transport.Message$Request.execute(Message.java:254) > at > org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:122) > at > org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:141) > at > org.apache.cassandra.transport.Dispatcher.processRequest(Dispatcher.java:168) > at > org.apache.cassandra.transport.Dispatcher.lambda$dispatch$0(Dispatcher.java:82) > at org.apache.cassandra.concurrent.FutureTask$1.call(FutureTask.java:81) at > org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:47) at > org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:57) at > org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:120) at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.base/java.lang.Thread.run(Thread.java:834) Caused by: > java.lang.NullPointerException: null at > org.apache.cassandra.db.memtable.AbstractMemtableWithCommitlog.accepts(AbstractMemtableWithCommitlog.java:101) > at > org.apache.cassandra.db.lifecycle.Tracker.getMemtableFor(Tracker.java:329) at > org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1421) > ... 33 common frames omitted]{code} -- This message was sent by Atlassian Jira (v8.20.7#820007) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org