[jira] [Commented] (CASSANDRA-15967) Add support in jvm dtest to test thrift
[ https://issues.apache.org/jira/browse/CASSANDRA-15967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162534#comment-17162534 ] Alex Petrov commented on CASSANDRA-15967: - I don't have a strong preference whether we have a separate {{THRIFT}} feature, or will use the existing {{NATIVE}}. I think it's fine to use the existing one, even. Maybe we need to renamie it from NATIVE to CLIENT or something like that not to suggest we're enabling _only_ native transport. That said, CQLTester and ThriftCQLTester are separate in code. As regards out-of-tree upgrade tests, when we were talking about dtest-api and writing tests outside main repository, we did want to have some API that would let us annotate tests with Cassandra versions they'll be running against, and when they'll be skipped, just like in python dtests. Different versions sometimes have different behaviour, so this is definitely required. We don't need to have it right now though, just "eventually". > Add support in jvm dtest to test thrift > --- > > Key: CASSANDRA-15967 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15967 > Project: Cassandra > Issue Type: Improvement > Components: Test/dtest >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 2.2.x, 3.0.x, 3.11.x > > > In 2.2 and 3.x thrift is supported and can cause problems for the non thrift > code. To make sure we can test these interactions, need to add support for > thrift in jvm dtest. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15968) nodetool drain fails in jvm dtest
[ https://issues.apache.org/jira/browse/CASSANDRA-15968?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162488#comment-17162488 ] David Capwell commented on CASSANDRA-15968: --- have this fixed but will hold off sending a patch till CASSANDRA-15967 is ready, mostly because it then allows me to include the test case with it. > nodetool drain fails in jvm dtest > - > > Key: CASSANDRA-15968 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15968 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 2.2.x, 3.0.x, 3.11.x, 4.0-beta > > > Drain fails since MessagingService doesn’t expect to be shutdown twice. > {code} > Caused by: java.lang.AssertionError > at > org.apache.cassandra.net.MessagingService.shutdown(MessagingService.java:766) > at > org.apache.cassandra.distributed.impl.Instance.lambda$parallelRun$28(Instance.java:805) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > ... 3 more > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15969) jvm dtest execute APIs do not support collections
[ https://issues.apache.org/jira/browse/CASSANDRA-15969?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162487#comment-17162487 ] David Capwell commented on CASSANDRA-15969: --- I am ok if anyone wants to take this from me =) > jvm dtest execute APIs do not support collections > - > > Key: CASSANDRA-15969 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15969 > Project: Cassandra > Issue Type: Improvement > Components: Test/dtest >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 2.2.x, 3.0.x, 3.11.x, 4.0-beta > > > If you use a collection type they will be transferred to the instance and we > call org.apache.cassandra.utils.ByteBufferUtil#objectToBytes to convert to > ByteBuffers; this doesn’t support collections. If you try to work around > this by converting before sending, it will fail since that method doesn’t > support ByteBuffer as input -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15967) Add support in jvm dtest to test thrift
[ https://issues.apache.org/jira/browse/CASSANDRA-15967?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162486#comment-17162486 ] David Capwell commented on CASSANDRA-15967: --- I have this working locally by overloading Feature.NATIVE, if you select native you also get thrift. The main thing I want to make sure we are in-sync on is if this is ok or if we should add a new Feature.THRIFT (which no-op in 4.0)? This does pose a problem though, since thrift was removed all upgrade tests which use thrift can't show the upgrade to 4.0 is success... this may be a great time to also consider tests outside of the cassandra repo. > Add support in jvm dtest to test thrift > --- > > Key: CASSANDRA-15967 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15967 > Project: Cassandra > Issue Type: Improvement > Components: Test/dtest >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 2.2.x, 3.0.x, 3.11.x > > > In 2.2 and 3.x thrift is supported and can cause problems for the non thrift > code. To make sure we can test these interactions, need to add support for > thrift in jvm dtest. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15970) 3.x fails to start if commit log has range tombstones from a column which is also deleted
[ https://issues.apache.org/jira/browse/CASSANDRA-15970?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162483#comment-17162483 ] David Capwell commented on CASSANDRA-15970: --- The dtest above needs the following JIRA to run: CASSANDRA-15969, CASSANDRA-15968, CASSANDRA-15967 > 3.x fails to start if commit log has range tombstones from a column which is > also deleted > - > > Key: CASSANDRA-15970 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15970 > Project: Cassandra > Issue Type: Bug > Components: Legacy/Local Write-Read Paths, Local/Commit Log >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > Cassandra crashes with the following exception > {code} > ERROR [node1_isolatedExecutor:1] node1 2020-07-21 18:59:39,048 > JVMStabilityInspector.java:102 - Exiting due to error while processing commit > log during initialization. > org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: > Unexpected error deserializing mutation; saved to > /var/folders/cm/08cddl2s25j7fq3jdb76gh4rgn/T/mutation6239873170066752296dat. > This may be caused by replaying a mutation against a table with the same > name but incompatible schema. > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:731) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:656) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:609) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:493) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:189) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:170) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:151) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.distributed.impl.Instance.lambda$startup$7(Instance.java:535) > [dtest-3.0.21.jar:na] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_242] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > ~[na:1.8.0_242] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > ~[na:1.8.0_242] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > ~[na:1.8.0_242] > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:83) > ~[dtest-3.0.21.jar:na] > at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_242] > Caused by: java.lang.NullPointerException: null > at > org.apache.cassandra.db.ClusteringComparator.validate(ClusteringComparator.java:206) > ~[dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.partitions.PartitionUpdate.validate(PartitionUpdate.java:494) > ~[dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:629) > [dtest-3.0.21.jar:na] > ... 12 common frames omitted > {code} > If you drain in 2.2 before upgrade, you get the following > {code} > ERROR [SharedPool-Worker-1] node1 2020-07-21 22:17:25,661 > AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread > Thread[SharedPool-Worker-1,5,node1] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2537) > ~[dtest-3.0.21.jar:na] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_242] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165) > ~[dtest-3.0.21.jar:na] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [dtest-3.0.21.jar:na] > at java.lang.Thread.run(Thread.java:748) [na:1.8.0_242] > Caused by: java.lang.NullPointerException: null > at > org.apache.cassandra.db.ClusteringComparator.compare(ClusteringComparator.java:131) > ~[dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.compareNextTo(UnfilteredDeserializer.java:391) > ~[dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.handlePreSliceData(SSTableIterator.java:105) > ~[dtest-3.0.21.jar:na] > at
[jira] [Updated] (CASSANDRA-15970) 3.x fails to start if commit log has range tombstones from a column which is also deleted
[ https://issues.apache.org/jira/browse/CASSANDRA-15970?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15970: -- Bug Category: Parent values: Correctness(12982)Level 1 values: Unrecoverable Corruption / Loss(13161) Complexity: Normal Discovered By: Workload Replay Fix Version/s: 3.11.x 3.0.x Severity: Critical Status: Open (was: Triage Needed) Marked as 3.x only since this is upgrading from 2.x. I assume the "fix" will be local to 3.x and won't impact 4.0, but if wrong will need to mark as 4.0 as well. I am also ok if anyone wants to take this from me! > 3.x fails to start if commit log has range tombstones from a column which is > also deleted > - > > Key: CASSANDRA-15970 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15970 > Project: Cassandra > Issue Type: Bug > Components: Legacy/Local Write-Read Paths, Local/Commit Log >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 3.0.x, 3.11.x > > > Cassandra crashes with the following exception > {code} > ERROR [node1_isolatedExecutor:1] node1 2020-07-21 18:59:39,048 > JVMStabilityInspector.java:102 - Exiting due to error while processing commit > log during initialization. > org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: > Unexpected error deserializing mutation; saved to > /var/folders/cm/08cddl2s25j7fq3jdb76gh4rgn/T/mutation6239873170066752296dat. > This may be caused by replaying a mutation against a table with the same > name but incompatible schema. > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:731) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:656) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:609) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:493) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:189) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:170) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:151) > [dtest-3.0.21.jar:na] > at > org.apache.cassandra.distributed.impl.Instance.lambda$startup$7(Instance.java:535) > [dtest-3.0.21.jar:na] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_242] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > ~[na:1.8.0_242] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > ~[na:1.8.0_242] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > ~[na:1.8.0_242] > at > org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:83) > ~[dtest-3.0.21.jar:na] > at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_242] > Caused by: java.lang.NullPointerException: null > at > org.apache.cassandra.db.ClusteringComparator.validate(ClusteringComparator.java:206) > ~[dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.partitions.PartitionUpdate.validate(PartitionUpdate.java:494) > ~[dtest-3.0.21.jar:na] > at > org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:629) > [dtest-3.0.21.jar:na] > ... 12 common frames omitted > {code} > If you drain in 2.2 before upgrade, you get the following > {code} > ERROR [SharedPool-Worker-1] node1 2020-07-21 22:17:25,661 > AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread > Thread[SharedPool-Worker-1,5,node1] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2537) > ~[dtest-3.0.21.jar:na] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_242] > at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165) > ~[dtest-3.0.21.jar:na] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) > [dtest-3.0.21.jar:na] > at java.lang.Thread.run(Thread.java:748) [na:1.8.0_242] > Caused by: java.lang.NullPointerException: null > at > org.apache.cassandra.db.ClusteringComparator.compare(ClusteringComparator.jav
[jira] [Created] (CASSANDRA-15970) 3.x fails to start if commit log has range tombstones from a column which is also deleted
David Capwell created CASSANDRA-15970: - Summary: 3.x fails to start if commit log has range tombstones from a column which is also deleted Key: CASSANDRA-15970 URL: https://issues.apache.org/jira/browse/CASSANDRA-15970 Project: Cassandra Issue Type: Bug Components: Legacy/Local Write-Read Paths, Local/Commit Log Reporter: David Capwell Assignee: David Capwell Cassandra crashes with the following exception {code} ERROR [node1_isolatedExecutor:1] node1 2020-07-21 18:59:39,048 JVMStabilityInspector.java:102 - Exiting due to error while processing commit log during initialization. org.apache.cassandra.db.commitlog.CommitLogReplayer$CommitLogReplayException: Unexpected error deserializing mutation; saved to /var/folders/cm/08cddl2s25j7fq3jdb76gh4rgn/T/mutation6239873170066752296dat. This may be caused by replaying a mutation against a table with the same name but incompatible schema. at org.apache.cassandra.db.commitlog.CommitLogReplayer.handleReplayError(CommitLogReplayer.java:731) [dtest-3.0.21.jar:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:656) [dtest-3.0.21.jar:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replaySyncSection(CommitLogReplayer.java:609) [dtest-3.0.21.jar:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:493) [dtest-3.0.21.jar:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.recover(CommitLogReplayer.java:189) [dtest-3.0.21.jar:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:170) [dtest-3.0.21.jar:na] at org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:151) [dtest-3.0.21.jar:na] at org.apache.cassandra.distributed.impl.Instance.lambda$startup$7(Instance.java:535) [dtest-3.0.21.jar:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_242] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_242] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) ~[na:1.8.0_242] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) ~[na:1.8.0_242] at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:83) ~[dtest-3.0.21.jar:na] at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_242] Caused by: java.lang.NullPointerException: null at org.apache.cassandra.db.ClusteringComparator.validate(ClusteringComparator.java:206) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.partitions.PartitionUpdate.validate(PartitionUpdate.java:494) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.commitlog.CommitLogReplayer.replayMutation(CommitLogReplayer.java:629) [dtest-3.0.21.jar:na] ... 12 common frames omitted {code} If you drain in 2.2 before upgrade, you get the following {code} ERROR [SharedPool-Worker-1] node1 2020-07-21 22:17:25,661 AbstractLocalAwareExecutorService.java:169 - Uncaught exception on thread Thread[SharedPool-Worker-1,5,node1] java.lang.RuntimeException: java.lang.NullPointerException at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2537) ~[dtest-3.0.21.jar:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_242] at org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:165) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:109) [dtest-3.0.21.jar:na] at java.lang.Thread.run(Thread.java:748) [na:1.8.0_242] Caused by: java.lang.NullPointerException: null at org.apache.cassandra.db.ClusteringComparator.compare(ClusteringComparator.java:131) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.UnfilteredDeserializer$OldFormatDeserializer.compareNextTo(UnfilteredDeserializer.java:391) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.handlePreSliceData(SSTableIterator.java:105) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.columniterator.SSTableIterator$ForwardReader.hasNextInternal(SSTableIterator.java:164) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.columniterator.AbstractSSTableIterator$Reader.hasNext(AbstractSSTableIterator.java:336) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.filter.ClusteringIndexNamesFilter$1.hasNext(ClusteringIndexNamesFilter.java:157) ~[dtest-3.0.21.jar:na] at org.apache.cassandra.db.rows.UnfilteredRowIterator.isEmpty(UnfilteredRowIterator.java:70) ~[dtest-3.0.21.jar:na] at o
[jira] [Updated] (CASSANDRA-15968) nodetool drain fails in jvm dtest
[ https://issues.apache.org/jira/browse/CASSANDRA-15968?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15968: -- Bug Category: Parent values: Correctness(12982)Level 1 values: Test Failure(12990) Complexity: Low Hanging Fruit Discovered By: Unit Test Fix Version/s: 4.0-beta 3.11.x 3.0.x 2.2.x Severity: Normal Status: Open (was: Triage Needed) > nodetool drain fails in jvm dtest > - > > Key: CASSANDRA-15968 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15968 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 2.2.x, 3.0.x, 3.11.x, 4.0-beta > > > Drain fails since MessagingService doesn’t expect to be shutdown twice. > {code} > Caused by: java.lang.AssertionError > at > org.apache.cassandra.net.MessagingService.shutdown(MessagingService.java:766) > at > org.apache.cassandra.distributed.impl.Instance.lambda$parallelRun$28(Instance.java:805) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > ... 3 more > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15969) jvm dtest execute APIs do not support collections
[ https://issues.apache.org/jira/browse/CASSANDRA-15969?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15969: -- Change Category: Quality Assurance Complexity: Low Hanging Fruit Fix Version/s: 4.0-beta 3.11.x 3.0.x 2.2.x Status: Open (was: Triage Needed) > jvm dtest execute APIs do not support collections > - > > Key: CASSANDRA-15969 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15969 > Project: Cassandra > Issue Type: Improvement > Components: Test/dtest >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 2.2.x, 3.0.x, 3.11.x, 4.0-beta > > > If you use a collection type they will be transferred to the instance and we > call org.apache.cassandra.utils.ByteBufferUtil#objectToBytes to convert to > ByteBuffers; this doesn’t support collections. If you try to work around > this by converting before sending, it will fail since that method doesn’t > support ByteBuffer as input -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-15969) jvm dtest execute APIs do not support collections
David Capwell created CASSANDRA-15969: - Summary: jvm dtest execute APIs do not support collections Key: CASSANDRA-15969 URL: https://issues.apache.org/jira/browse/CASSANDRA-15969 Project: Cassandra Issue Type: Improvement Components: Test/dtest Reporter: David Capwell Assignee: David Capwell If you use a collection type they will be transferred to the instance and we call org.apache.cassandra.utils.ByteBufferUtil#objectToBytes to convert to ByteBuffers; this doesn’t support collections. If you try to work around this by converting before sending, it will fail since that method doesn’t support ByteBuffer as input -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-15967) Add support in jvm dtest to test thrift
David Capwell created CASSANDRA-15967: - Summary: Add support in jvm dtest to test thrift Key: CASSANDRA-15967 URL: https://issues.apache.org/jira/browse/CASSANDRA-15967 Project: Cassandra Issue Type: Improvement Components: Test/dtest Reporter: David Capwell Assignee: David Capwell In 2.2 and 3.x thrift is supported and can cause problems for the non thrift code. To make sure we can test these interactions, need to add support for thrift in jvm dtest. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-15968) nodetool drain fails in jvm dtest
David Capwell created CASSANDRA-15968: - Summary: nodetool drain fails in jvm dtest Key: CASSANDRA-15968 URL: https://issues.apache.org/jira/browse/CASSANDRA-15968 Project: Cassandra Issue Type: Bug Components: Test/dtest Reporter: David Capwell Assignee: David Capwell Drain fails since MessagingService doesn’t expect to be shutdown twice. {code} Caused by: java.lang.AssertionError at org.apache.cassandra.net.MessagingService.shutdown(MessagingService.java:766) at org.apache.cassandra.distributed.impl.Instance.lambda$parallelRun$28(Instance.java:805) at java.util.concurrent.FutureTask.run(FutureTask.java:266) ... 3 more {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15967) Add support in jvm dtest to test thrift
[ https://issues.apache.org/jira/browse/CASSANDRA-15967?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15967: -- Change Category: Quality Assurance Complexity: Low Hanging Fruit Fix Version/s: 3.11.x 3.0.x 2.2.x Status: Open (was: Triage Needed) > Add support in jvm dtest to test thrift > --- > > Key: CASSANDRA-15967 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15967 > Project: Cassandra > Issue Type: Improvement > Components: Test/dtest >Reporter: David Capwell >Assignee: David Capwell >Priority: Normal > Fix For: 2.2.x, 3.0.x, 3.11.x > > > In 2.2 and 3.x thrift is supported and can cause problems for the non thrift > code. To make sure we can test these interactions, need to add support for > thrift in jvm dtest. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure
[ https://issues.apache.org/jira/browse/CASSANDRA-15861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162384#comment-17162384 ] Blake Eggleston commented on CASSANDRA-15861: - I don't think introducing a lock here is going to work. The incremental repair concern is an issue, but I'm more concerned about this freezing up compaction. If an outgoing stream is taking a really long time for some reason, any compaction task trying to mutate the level of that sstable will be blocking a compaction thread waiting to acquire that lock. There typically aren't a lot of compaction executors (2-8 if auto configured, probably no more than 4 in most cases), so a few of these would noticably constrain compaction throughput, or stop it entirely. > Mutating sstable component may race with entire-sstable-streaming(ZCS) > causing checksum validation failure > -- > > Key: CASSANDRA-15861 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15861 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Repair, Consistency/Streaming, > Local/Compaction >Reporter: ZhaoYang >Assignee: ZhaoYang >Priority: Normal > Fix For: 4.0-beta > > > Flaky dtest: [test_dead_sync_initiator - > repair_tests.repair_test.TestRepair|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sync_initiator/] > {code:java|title=stacktrace} > Unexpected error found in node logs (see stdout for full details). Errors: > [ERROR [Stream-Deserializer-127.0.0.1:7000-570871f3] 2020-06-03 04:05:19,081 > CassandraEntireSSTableStreamReader.java:145 - [Stream > 6f1c3360-a54f-11ea-a808-2f23710fdc90] Error while reading sstable from stream > for table = keyspace1.standard1 > org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeValidateChecksum(MetadataSerializer.java:219) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:198) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:129) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.mutate(MetadataSerializer.java:226) > at > org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:140) > at > org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:78) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:49) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:36) > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:49) > at > org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:181) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.io.IOException: Checksums do not match for > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > {code} > > In the above test, it executes "nodetool repair" on node1 and kills node2 > during repair. At the end, node3 reports checksum validation failure on > sstable transferred from node1. > {code:java|title=what happened} > 1. When repair started on node1, it performs anti-compaction which modifies > sstable's repairAt to 0 and pending repair id to session-id. > 2. Then node1 creates {{ComponentManifest}} which contains file lengths to be > transferred to node3. > 3. Before node1 actually sends the files to node3, node2 is killed and node1 > starts to broadcast repair-failure-message to all participants in > {{CoordinatorSession#fail}} > 4. Node1 receives its own repair-failure-message and fails its local repair > sessions at {{LocalSessions#failSession}} which triggers async background > compaction. > 5. Node1's background compaction will mutate sstable's repairAt to 0 and > pending repair id to null via > {{PendingRepairManager#getNextRepairFinishedTask}}, as there is no more > in-progress repair. > 6. Node1 actually sends the sstable to node3 where the sstable's STATS > component size is different from the origina
[jira] [Comment Edited] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162342#comment-17162342 ] Benedict Elliott Smith edited comment on CASSANDRA-15234 at 7/21/20, 9:39 PM: -- bq. it would be great to start this dialog on user@ I'd personally prefer to hash out here for a week or so, to get a good proposal to take to the user list, or a couple of competing proposals. I find public engagements works best when there's a good case to challenge/consider. It's a difficult balancing act getting any given approach right, and there are multiple approaches. I would love to see another approach taken more to its conclusion for comparison. I've made some further changes, and to make it clearer created a [yaml|https://github.com/belliottsmith/cassandra/blob/5f80d1c0d38873b7a27dc137656d8b81f8e6bbd7/conf/cassandra_nocomment.yaml] with comments mostly stripped. In this version, there are basic settings for network, disk etc all grouped together, followed by operator tuneables mostly under {{limits}} within which we now have {{throughput}}, {{concurrency}}, {{capacity}}. This leads to settings for some features being kept separate (most notably for caching), but helps the operator understand what they have to play with for controlling resource consumption. It's still incomplete, but 90%+ done, and thoughts would be most welcome. bq. My main concern is if this makes it more confusing for users; they may find some docs which say the old name, some using the nested name, and others using the flat name. This is a possibility, and I'm not wed to the idea - but I think the balance of benefit to risk is probably pretty good, particularly since the names are fairly consistent (and we can have a blurb at the start to explain the dual system), so I doubt it should lead to too much confusion if we opt for it. was (Author: benedict): bq. it would be great to start this dialog on user@ I'd personally prefer to hash out here for a week or so, to get a good proposal to take to the user list, or a couple of competing proposals. I find public engagements works best when there's a good case to challenge/consider. It's a difficult balancing act getting any given approach right, and there are multiple approaches. I would love to see another approach taken more to its conclusion for comparison. I've made some further changes, and to make it clearer created a [yaml|https://github.com/belliottsmith/cassandra/blob/acac38be9f528e380974423b86fad5e895e3/conf/cassandra_nocomment.yaml] with comments mostly stripped. In this version, there are basic settings for network, disk etc all grouped together, followed by operator tuneables mostly under {{limits}} within which we now have {{throughput}}, {{concurrency}}, {{capacity}}. This leads to settings for some features being kept separate (most notably for caching), but helps the operator understand what they have to play with for controlling resource consumption. It's still incomplete, but 90%+ done, and thoughts would be most welcome. bq. My main concern is if this makes it more confusing for users; they may find some docs which say the old name, some using the nested name, and others using the flat name. This is a possibility, and I'm not wed to the idea - but I think the balance of benefit to risk is probably pretty good, particularly since the names are fairly consistent (and we can have a blurb at the start to explain the dual system), so I doubt it should lead to too much confusion if we opt for it. > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s, TiB/s}}. > Perhaps for avoiding ambiguity we could not accept bauds {{bs, Mbps}} or > powers of 1000 such as {{KB/s}}, given these are regularly used for e
[jira] [Comment Edited] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure
[ https://issues.apache.org/jira/browse/CASSANDRA-15861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162356#comment-17162356 ] Caleb Rackliffe edited comment on CASSANDRA-15861 at 7/21/20, 9:28 PM: --- bq. During entire-sstable streaming, CassandraOutgoingFile will execute the streaming code within the sstable read-lock. So multiple streamings on the same sstable can start at the same time. I think it's fine to block stats-mutation/index-summary redistribution until streaming completion. I'm still looking at the patch itself, but the thing I'm most curious about at a design level is whether holding the SSTable read lock until we write the entirety of the SSTable to the network is safe in the face of a concurrent incremental repair. In other words, how likely is it that an incremental repair already in flight would have to wait a significant period of time to complete with an entire SSTable in the middle of streaming? We've already talked about the possibility of just buffering the stats component, but if we think that makes things too hard to reason around, what if we instead broke the streaming into mutable vs. immutable components and wrote the mutable components first (then released the read lock)? That might at least reduce the window, and repair couldn't block while the largest components (like the data file, and perhaps index components in the future) are streaming. WDYT? CC [~bdeggleston] was (Author: maedhroz): bq. During entire-sstable streaming, CassandraOutgoingFile will execute the streaming code within the sstable read-lock. So multiple streamings on the same sstable can start at the same time. I think it's fine to block stats-mutation/index-summary redistribution until streaming completion. I'm still looking at the patch itself, but the thing I'm most curious about at a design level is whether holding the SSTable read lock until we write the entirety of the SSTable to the network is safe in the face of a concurrent incremental repair. In other words, how likely is it that an incremental repair already in flight would have to wait a significant period of time to complete with an entire SSTable in the middle of streaming? We've already talked about the possibility of just buffering the stats component, but if we think that makes things too hard to reason around, what if we instead broke the streaming into mutable vs. immutable components and wrote the mutable components first? That might at least reduce the window, and repair couldn't block while the largest components (like the data file, and perhaps index components in the future) are streaming. WDYT? CC [~bdeggleston] > Mutating sstable component may race with entire-sstable-streaming(ZCS) > causing checksum validation failure > -- > > Key: CASSANDRA-15861 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15861 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Repair, Consistency/Streaming, > Local/Compaction >Reporter: ZhaoYang >Assignee: ZhaoYang >Priority: Normal > Fix For: 4.0-beta > > > Flaky dtest: [test_dead_sync_initiator - > repair_tests.repair_test.TestRepair|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sync_initiator/] > {code:java|title=stacktrace} > Unexpected error found in node logs (see stdout for full details). Errors: > [ERROR [Stream-Deserializer-127.0.0.1:7000-570871f3] 2020-06-03 04:05:19,081 > CassandraEntireSSTableStreamReader.java:145 - [Stream > 6f1c3360-a54f-11ea-a808-2f23710fdc90] Error while reading sstable from stream > for table = keyspace1.standard1 > org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeValidateChecksum(MetadataSerializer.java:219) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:198) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:129) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.mutate(MetadataSerializer.java:226) > at > org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:140) > at > org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:78) > at > org.apache.cassandra.streaming.messages.Incom
[jira] [Commented] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure
[ https://issues.apache.org/jira/browse/CASSANDRA-15861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162356#comment-17162356 ] Caleb Rackliffe commented on CASSANDRA-15861: - bq. During entire-sstable streaming, CassandraOutgoingFile will execute the streaming code within the sstable read-lock. So multiple streamings on the same sstable can start at the same time. I think it's fine to block stats-mutation/index-summary redistribution until streaming completion. I'm still looking at the patch itself, but the thing I'm most curious about at a design level is whether holding the SSTable read lock until we write the entirety of the SSTable to the network is safe in the face of a concurrent incremental repair. In other words, how likely is it that an incremental repair already in flight would have to wait a significant period of time to complete with an entire SSTable in the middle of streaming? We've already talked about the possibility of just buffering the stats component, but if we think that makes things too hard to reason around, what if we instead broke the streaming into mutable vs. immutable components and wrote the mutable components first? That might at least reduce the window, and repair couldn't block while the largest components (like the data file, and perhaps index components in the future) are streaming. WDYT? CC [~bdeggleston] > Mutating sstable component may race with entire-sstable-streaming(ZCS) > causing checksum validation failure > -- > > Key: CASSANDRA-15861 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15861 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Repair, Consistency/Streaming, > Local/Compaction >Reporter: ZhaoYang >Assignee: ZhaoYang >Priority: Normal > Fix For: 4.0-beta > > > Flaky dtest: [test_dead_sync_initiator - > repair_tests.repair_test.TestRepair|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sync_initiator/] > {code:java|title=stacktrace} > Unexpected error found in node logs (see stdout for full details). Errors: > [ERROR [Stream-Deserializer-127.0.0.1:7000-570871f3] 2020-06-03 04:05:19,081 > CassandraEntireSSTableStreamReader.java:145 - [Stream > 6f1c3360-a54f-11ea-a808-2f23710fdc90] Error while reading sstable from stream > for table = keyspace1.standard1 > org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeValidateChecksum(MetadataSerializer.java:219) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:198) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:129) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.mutate(MetadataSerializer.java:226) > at > org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:140) > at > org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:78) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:49) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:36) > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:49) > at > org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:181) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.io.IOException: Checksums do not match for > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > {code} > > In the above test, it executes "nodetool repair" on node1 and kills node2 > during repair. At the end, node3 reports checksum validation failure on > sstable transferred from node1. > {code:java|title=what happened} > 1. When repair started on node1, it performs anti-compaction which modifies > sstable's repairAt to 0 and pending repair id to session-id. > 2. Then node1 creates {{ComponentManifest}} which contains file lengths to be > transferred to node3. > 3. Before node1 actually sends the file
[jira] [Commented] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162346#comment-17162346 ] Lorina Poland commented on CASSANDRA-15234: --- I will be happy to write clear information about this topic in the docs after implementation, to explain old/new. Lorina Lorina Poland e. lor...@datastax.com w. www.datastax.com On Tue, Jul 21, 2020 at 2:09 PM Benedict Elliott Smith (Jira) < > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s, TiB/s}}. > Perhaps for avoiding ambiguity we could not accept bauds {{bs, Mbps}} or > powers of 1000 such as {{KB/s}}, given these are regularly used for either > their old or new definition e.g. {{KiB/s}}, or we could support them and > simply log the value in bytes/s. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162342#comment-17162342 ] Benedict Elliott Smith edited comment on CASSANDRA-15234 at 7/21/20, 9:08 PM: -- bq. it would be great to start this dialog on user@ I'd personally prefer to hash out here for a week or so, to get a good proposal to take to the user list, or a couple of competing proposals. I find public engagements works best when there's a good case to challenge/consider. It's a difficult balancing act getting any given approach right, and there are multiple approaches. I would love to see another approach taken more to its conclusion for comparison. I've made some further changes, and to make it clearer created a [yaml|https://github.com/belliottsmith/cassandra/blob/acac38be9f528e380974423b86fad5e895e3/conf/cassandra_nocomment.yaml] with comments mostly stripped. In this version, there are basic settings for network, disk etc all grouped together, followed by operator tuneables mostly under {{limits}} within which we now have {{throughput}}, {{concurrency}}, {{capacity}}. This leads to settings for some features being kept separate (most notably for caching), but helps the operator understand what they have to play with for controlling resource consumption. It's still incomplete, but 90%+ done, and thoughts would be most welcome. bq. My main concern is if this makes it more confusing for users; they may find some docs which say the old name, some using the nested name, and others using the flat name. This is a possibility, and I'm not wed to the idea - but I think the balance of benefit to risk is probably pretty good, particularly since the names are fairly consistent (and we can have a blurb at the start to explain the dual system), so I doubt it should lead to too much confusion if we opt for it. was (Author: benedict): bq. it would be great to start this dialog on user@ I'd personally prefer to hash out here for a week or so, to get a good proposal to take to the user list, or a couple of competing proposals. I find public engagements works best when there's a good case to challenge/consider. It's a difficult balancing act getting any given approach right, and there are multiple approaches. I would love to see another approach taken more to its conclusion for comparison. I've made some further changes, and to make it clearer created a [yaml|https://github.com/belliottsmith/cassandra/blob/acac38be9f528e380974423b86fad5e895e3/conf/cassandra_nocomment.yaml] with comments mostly stripped. In this version, there are basic settings for network, disk etc all grouped together, followed by operator tuneables mostly under {{limits}} within which we now have {{throughput}}, {{concurrency}}, {{capacity}}. This leads to settings for some features being kept separate (most notably for caching), but helps the operator understand what they have to play with for controlling resource consumption. It's still incomplete, but 90%+ done, and thoughts would be most welcome. > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s, TiB/s}}. > Perhaps for avoiding ambiguity we could not accept bauds {{bs, Mbps}} or > powers of 1000 such as {{KB/s}}, given these are regularly used for either > their old or new definition e.g. {{KiB/s}}, or we could support them and > simply log the value in bytes/s. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162342#comment-17162342 ] Benedict Elliott Smith commented on CASSANDRA-15234: bq. it would be great to start this dialog on user@ I'd personally prefer to hash out here for a week or so, to get a good proposal to take to the user list, or a couple of competing proposals. I find public engagements works best when there's a good case to challenge/consider. It's a difficult balancing act getting any given approach right, and there are multiple approaches. I would love to see another approach taken more to its conclusion for comparison. I've made some further changes, and to make it clearer created a [yaml|https://github.com/belliottsmith/cassandra/blob/acac38be9f528e380974423b86fad5e895e3/conf/cassandra_nocomment.yaml] with comments mostly stripped. In this version, there are basic settings for network, disk etc all grouped together, followed by operator tuneables mostly under {{limits}} within which we now have {{throughput}}, {{concurrency}}, {{capacity}}. This leads to settings for some features being kept separate (most notably for caching), but helps the operator understand what they have to play with for controlling resource consumption. It's still incomplete, but 90%+ done, and thoughts would be most welcome. > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s, TiB/s}}. > Perhaps for avoiding ambiguity we could not accept bauds {{bs, Mbps}} or > powers of 1000 such as {{KB/s}}, given these are regularly used for either > their old or new definition e.g. {{KiB/s}}, or we could support them and > simply log the value in bytes/s. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17152248#comment-17152248 ] Caleb Rackliffe edited comment on CASSANDRA-15907 at 7/21/20, 8:33 PM: --- [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/72/workflows/4ca3d4d2-a9d3-4e08-ace2-e7bb073d54e6] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. was (Author: maedhroz): [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/71/workflows/4dfbc4d2-19c2-452d-b2f3-7b6b3fdcbe9a] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 6h 50m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method. > * {{DataResolver:111 & 228}} - Cache an instance of > {{UnaryOperator#identity()}} instead of creating one on the fly. > * {{ReplicaFilteringProtection:217}} - We may be able to scatter/gather > rather than serially querying every row that needs to be completed. This > isn't a clear win perhaps, given it targets the latency of single queries and > adds some complexity. (Certainly a decent candidate to kick even out of this > issue.) > *Documentation and Intelligibility* > * There are a few places (CHANGES.txt, tracing output in > {{ReplicaFilteringProtection}}, etc.) where we mention "replica-side > filtering protection" (which makes it seem like the coordinator doesn't > filter) rather than "replica filtering protection" (which sounds more like > what we actually do, which
[jira] [Updated] (CASSANDRA-15966) Circle CI start_ jobs should not require build
[ https://issues.apache.org/jira/browse/CASSANDRA-15966?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15966: -- Change Category: Quality Assurance Complexity: Low Hanging Fruit Priority: Low (was: Normal) Status: Open (was: Triage Needed) > Circle CI start_ jobs should not require build > -- > > Key: CASSANDRA-15966 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15966 > Project: Cassandra > Issue Type: Improvement > Components: Build >Reporter: David Capwell >Assignee: David Capwell >Priority: Low > > The Circle CI start_ jobs all depend on build, so can’t approve them until > the build is done, but their downstream also depends on build. Given this, > we can remove this dependency which will allow someone to approve the > downstream pipelines even if the build isn’t complete yet. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Created] (CASSANDRA-15966) Circle CI start_ jobs should not require build
David Capwell created CASSANDRA-15966: - Summary: Circle CI start_ jobs should not require build Key: CASSANDRA-15966 URL: https://issues.apache.org/jira/browse/CASSANDRA-15966 Project: Cassandra Issue Type: Improvement Components: Build Reporter: David Capwell Assignee: David Capwell The Circle CI start_ jobs all depend on build, so can’t approve them until the build is done, but their downstream also depends on build. Given this, we can remove this dependency which will allow someone to approve the downstream pipelines even if the build isn’t complete yet. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17152248#comment-17152248 ] Caleb Rackliffe edited comment on CASSANDRA-15907 at 7/21/20, 8:02 PM: --- [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/71/workflows/4dfbc4d2-19c2-452d-b2f3-7b6b3fdcbe9a] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. was (Author: maedhroz): [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/70/workflows/312ef7ca-823f-4d38-8fd7-f3a7fc9de15c] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 6h 50m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method. > * {{DataResolver:111 & 228}} - Cache an instance of > {{UnaryOperator#identity()}} instead of creating one on the fly. > * {{ReplicaFilteringProtection:217}} - We may be able to scatter/gather > rather than serially querying every row that needs to be completed. This > isn't a clear win perhaps, given it targets the latency of single queries and > adds some complexity. (Certainly a decent candidate to kick even out of this > issue.) > *Documentation and Intelligibility* > * There are a few places (CHANGES.txt, tracing output in > {{ReplicaFilteringProtection}}, etc.) where we mention "replica-side > filtering protection" (which makes it seem like the coordinator doesn't > filter) rather than "replica filtering protection" (which sounds more like > what we actually do, which
[jira] [Comment Edited] (CASSANDRA-15907) Operational Improvements & Hardening for Replica Filtering Protection
[ https://issues.apache.org/jira/browse/CASSANDRA-15907?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17152248#comment-17152248 ] Caleb Rackliffe edited comment on CASSANDRA-15907 at 7/21/20, 7:55 PM: --- [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/70/workflows/312ef7ca-823f-4d38-8fd7-f3a7fc9de15c] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. was (Author: maedhroz): [~jwest] I've hopefully addressed the points from [~adelapena]'s first round of review, so I think this is officially ready for a second reviewer. 3.0: [patch|https://github.com/apache/cassandra/pull/659], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/66/workflows/c3e60efe-a50e-47ff-a8a7-51de36deb17b] WIP (avoid review ATM) 3.11: [patch|https://github.com/apache/cassandra/pull/665], [CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/38/workflows/c3a3b51b-d105-49d9-91f8-2a149cf211b6] trunk: [patch|https://github.com/apache/cassandra/pull/666], [j8 CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/84e48d9e-f3dd-45ff-b70a-b69a86f6eb96] [j11 Circle CI|https://app.circleci.com/pipelines/github/maedhroz/cassandra/37/workflows/79b581ab-94a1-4920-a894-7f0f91ef466b] If we're happy with the implementation, the next step will be to do some basic stress testing. Note: Existing issues described by CASSANDRA-14595 (Thrift dtest) and CASSANDRA-15881 (SASI memtable switching) are visible in the test results so far. > Operational Improvements & Hardening for Replica Filtering Protection > - > > Key: CASSANDRA-15907 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15907 > Project: Cassandra > Issue Type: Improvement > Components: Consistency/Coordination, Feature/2i Index >Reporter: Caleb Rackliffe >Assignee: Caleb Rackliffe >Priority: Normal > Labels: 2i, memory > Fix For: 3.0.x, 3.11.x, 4.0-beta > > Time Spent: 6h 50m > Remaining Estimate: 0h > > CASSANDRA-8272 uses additional space on the heap to ensure correctness for 2i > and filtering queries at consistency levels above ONE/LOCAL_ONE. There are a > few things we should follow up on, however, to make life a bit easier for > operators and generally de-risk usage: > (Note: Line numbers are based on {{trunk}} as of > {{3cfe3c9f0dcf8ca8b25ad111800a21725bf152cb}}.) > *Minor Optimizations* > * {{ReplicaFilteringProtection:114}} - Given we size them up-front, we may be > able to use simple arrays instead of lists for {{rowsToFetch}} and > {{originalPartitions}}. Alternatively (or also), we may be able to null out > references in these two collections more aggressively. (ex. Using > {{ArrayList#set()}} instead of {{get()}} in {{queryProtectedPartitions()}}, > assuming we pass {{toFetch}} as an argument to {{querySourceOnKey()}}.) > * {{ReplicaFilteringProtection:323}} - We may be able to use > {{EncodingStats.merge()}} and remove the custom {{stats()}} method. > * {{DataResolver:111 & 228}} - Cache an instance of > {{UnaryOperator#identity()}} instead of creating one on the fly. > * {{ReplicaFilteringProtection:217}} - We may be able to scatter/gather > rather than serially querying every row that needs to be completed. This > isn't a clear win perhaps, given it targets the latency of single queries and > adds some complexity. (Certainly a decent candidate to kick even out of this > issue.) > *Documentation and Intelligibility* > * There are a few places (CHANGES.txt, tracing output in > {{ReplicaFilteringProtection}}, etc.) where we mention "replica-side > filtering protection" (which makes it seem like the coordinator doesn't > filter) rather than "replica filtering protection" (which sounds more like > what we actually do, which
[jira] [Created] (CASSANDRA-15965) Fix flaky test StreamingInboundHandlerTest channelRead_Normal
David Capwell created CASSANDRA-15965: - Summary: Fix flaky test StreamingInboundHandlerTest channelRead_Normal Key: CASSANDRA-15965 URL: https://issues.apache.org/jira/browse/CASSANDRA-15965 Project: Cassandra Issue Type: Bug Components: Test/unit Reporter: David Capwell channelRead_Normal - org.apache.cassandra.streaming.async.StreamingInboundHandlerTest {code} junit.framework.AssertionFailedError: expected:<8> but was:<0> at org.apache.cassandra.streaming.async.StreamingInboundHandlerTest.channelRead_Normal(StreamingInboundHandlerTest.java:98) {code} Failed build: https://app.circleci.com/pipelines/github/dcapwell/cassandra/298/workflows/e3296f33-2289-401c-8fc8-a7f786e3692a/jobs/1445 -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15965) Fix flaky test StreamingInboundHandlerTest channelRead_Normal
[ https://issues.apache.org/jira/browse/CASSANDRA-15965?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15965: -- Bug Category: Parent values: Correctness(12982)Level 1 values: Test Failure(12990) Complexity: Normal Discovered By: Unit Test Fix Version/s: 4.0-beta Severity: Normal Status: Open (was: Triage Needed) > Fix flaky test StreamingInboundHandlerTest channelRead_Normal > - > > Key: CASSANDRA-15965 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15965 > Project: Cassandra > Issue Type: Bug > Components: Test/unit >Reporter: David Capwell >Priority: Normal > Fix For: 4.0-beta > > > channelRead_Normal - > org.apache.cassandra.streaming.async.StreamingInboundHandlerTest > {code} > junit.framework.AssertionFailedError: expected:<8> but was:<0> > at > org.apache.cassandra.streaming.async.StreamingInboundHandlerTest.channelRead_Normal(StreamingInboundHandlerTest.java:98) > {code} > Failed build: > https://app.circleci.com/pipelines/github/dcapwell/cassandra/298/workflows/e3296f33-2289-401c-8fc8-a7f786e3692a/jobs/1445 -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15804) system_schema keyspace complain of schema mismatch during upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-15804?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Caleb Rackliffe updated CASSANDRA-15804: Fix Version/s: 4.0-beta > system_schema keyspace complain of schema mismatch during upgrade > - > > Key: CASSANDRA-15804 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15804 > Project: Cassandra > Issue Type: Bug >Reporter: Pedro Gordo >Priority: Low > Fix For: 3.11.x, 4.0-beta > > > When upgrading from 3.11.4 to 3.11.6, we got the following error: > {code:Plain Text} > ERROR [MessagingService-Incoming-/10.20.11.59] 2020-05-07 13:53:52,627 > CassandraDaemon.java:228 - Exception in thread > Thread[MessagingService-Incoming-/10.20.11.59,5,main] > java.lang.RuntimeException: Unknown column kind during deserialization > at > org.apache.cassandra.db.Columns$Serializer.deserialize(Columns.java:464) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.SerializationHeader$Serializer.deserializeForMessaging(SerializationHeader.java:419) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.deserializeHeader(UnfilteredRowIteratorSerializer.java:195) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize30(PartitionUpdate.java:851) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:839) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:425) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:434) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:675) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:658) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at org.apache.cassandra.net.MessageIn.read(MessageIn.java:123) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:192) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:180) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:94) > ~[apache-cassandra-3.11.4.jar:3.11.4] > {code} > I've noticed that system_schema.dropped_columns has a new column called > "kind". > No issues arise from this error message, and the error disappeared after > upgrading all nodes. But it still caused concerns due to the ERROR logging > level, although "nodetool describecluster" reported only one schema version. > It makes sense for the system keyspaces to not be included for the > "describecluster" schema version check, but it seems to me that these > internal schema mismatches should be ignored if the versions are different > between the nodes. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15804) system_schema keyspace complain of schema mismatch during upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-15804?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Caleb Rackliffe updated CASSANDRA-15804: Fix Version/s: 3.11.x > system_schema keyspace complain of schema mismatch during upgrade > - > > Key: CASSANDRA-15804 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15804 > Project: Cassandra > Issue Type: Bug >Reporter: Pedro Gordo >Priority: Low > Fix For: 3.11.x > > > When upgrading from 3.11.4 to 3.11.6, we got the following error: > {code:Plain Text} > ERROR [MessagingService-Incoming-/10.20.11.59] 2020-05-07 13:53:52,627 > CassandraDaemon.java:228 - Exception in thread > Thread[MessagingService-Incoming-/10.20.11.59,5,main] > java.lang.RuntimeException: Unknown column kind during deserialization > at > org.apache.cassandra.db.Columns$Serializer.deserialize(Columns.java:464) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.SerializationHeader$Serializer.deserializeForMessaging(SerializationHeader.java:419) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.rows.UnfilteredRowIteratorSerializer.deserializeHeader(UnfilteredRowIteratorSerializer.java:195) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize30(PartitionUpdate.java:851) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:839) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:425) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:434) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:675) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.service.MigrationManager$MigrationsSerializer.deserialize(MigrationManager.java:658) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at org.apache.cassandra.net.MessageIn.read(MessageIn.java:123) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:192) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:180) > ~[apache-cassandra-3.11.4.jar:3.11.4] > at > org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:94) > ~[apache-cassandra-3.11.4.jar:3.11.4] > {code} > I've noticed that system_schema.dropped_columns has a new column called > "kind". > No issues arise from this error message, and the error disappeared after > upgrading all nodes. But it still caused concerns due to the ERROR logging > level, although "nodetool describecluster" reported only one schema version. > It makes sense for the system keyspaces to not be included for the > "describecluster" schema version check, but it seems to me that these > internal schema mismatches should be ignored if the versions are different > between the nodes. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure
[ https://issues.apache.org/jira/browse/CASSANDRA-15861?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Caleb Rackliffe updated CASSANDRA-15861: Reviewers: Caleb Rackliffe, Caleb Rackliffe (was: Caleb Rackliffe) Caleb Rackliffe, Caleb Rackliffe (was: Caleb Rackliffe) Status: Review In Progress (was: Patch Available) > Mutating sstable component may race with entire-sstable-streaming(ZCS) > causing checksum validation failure > -- > > Key: CASSANDRA-15861 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15861 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Repair, Consistency/Streaming, > Local/Compaction >Reporter: ZhaoYang >Assignee: ZhaoYang >Priority: Normal > Fix For: 4.0-beta > > > Flaky dtest: [test_dead_sync_initiator - > repair_tests.repair_test.TestRepair|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sync_initiator/] > {code:java|title=stacktrace} > Unexpected error found in node logs (see stdout for full details). Errors: > [ERROR [Stream-Deserializer-127.0.0.1:7000-570871f3] 2020-06-03 04:05:19,081 > CassandraEntireSSTableStreamReader.java:145 - [Stream > 6f1c3360-a54f-11ea-a808-2f23710fdc90] Error while reading sstable from stream > for table = keyspace1.standard1 > org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeValidateChecksum(MetadataSerializer.java:219) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:198) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:129) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.mutate(MetadataSerializer.java:226) > at > org.apache.cassandra.db.streaming.CassandraEntireSSTableStreamReader.read(CassandraEntireSSTableStreamReader.java:140) > at > org.apache.cassandra.db.streaming.CassandraIncomingFile.read(CassandraIncomingFile.java:78) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:49) > at > org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:36) > at > org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:49) > at > org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:181) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.io.IOException: Checksums do not match for > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > {code} > > In the above test, it executes "nodetool repair" on node1 and kills node2 > during repair. At the end, node3 reports checksum validation failure on > sstable transferred from node1. > {code:java|title=what happened} > 1. When repair started on node1, it performs anti-compaction which modifies > sstable's repairAt to 0 and pending repair id to session-id. > 2. Then node1 creates {{ComponentManifest}} which contains file lengths to be > transferred to node3. > 3. Before node1 actually sends the files to node3, node2 is killed and node1 > starts to broadcast repair-failure-message to all participants in > {{CoordinatorSession#fail}} > 4. Node1 receives its own repair-failure-message and fails its local repair > sessions at {{LocalSessions#failSession}} which triggers async background > compaction. > 5. Node1's background compaction will mutate sstable's repairAt to 0 and > pending repair id to null via > {{PendingRepairManager#getNextRepairFinishedTask}}, as there is no more > in-progress repair. > 6. Node1 actually sends the sstable to node3 where the sstable's STATS > component size is different from the original size recorded in the manifest. > 7. At the end, node3 reports checksum validation failure when it tries to > mutate sstable level and "isTransient" attribute in > {{CassandraEntireSSTableStreamReader#read}}. > {code} > Currently, entire-sstable-streaming requires sstable components to be > immutable, because \{{ComponentManifest}} > with component sizes are sent before sending actual files. This isn't a > problem in
[jira] [Commented] (CASSANDRA-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162283#comment-17162283 ] David Capwell commented on CASSANDRA-15191: --- FYI conversation has been happening in slack: https://the-asf.slack.com/archives/CK23JSY2K/p1595280621333400 Updates: * the tests are flaky, looks like there is a race condition in the test where the flag isn't updated yet. A workaround was added to query multiple times with a 5 second sleep in hopes to make the tests stable > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Vincent White >Assignee: Stefan Miklosovic >Priority: Normal > Fix For: 3.11.x, 4.0-beta > > Attachments: log.txt > > Time Spent: 3.5h > Remaining Estimate: 0h > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > The core reason for ignoring that exception is that thrown exception in > AbstractLocalAwareExecturorService is not CorruptSSTableException but it is > RuntimeException and that exception is as its cause. Hence it is better if we > handle this in JVMStabilityInspector which can recursively examine it, hence > act accordingly. > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15861) Mutating sstable component may race with entire-sstable-streaming(ZCS) causing checksum validation failure
[ https://issues.apache.org/jira/browse/CASSANDRA-15861?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17148893#comment-17148893 ] Caleb Rackliffe edited comment on CASSANDRA-15861 at 7/21/20, 6:48 PM: --- [~jasonstack] I thought a bit more about our earlier chat (and had a quick chat w/ [~bdeggleston]), and it seems like the simplest thing might be handling the stats and index summary in slightly different ways. The STATS component is small. We could just buffer it up, use that buffered size in the manifest, and stream that buffer. It special-cases this component, but we more or less avoid having to reason about the risk of blocking compactions, a repair completing, etc. For the SUMMARY, we take advantage of the fact that possibly/infrequently delaying the redistribution task isn't a big suboptimal outcome. We have a simple lock that protects it (on {{SSTableRader}}, similar to what you've already mentioned or as a threadsafe set of readers in a central location), i.e. streaming acquires it when the manifest is created and releases it when the index summary completes streaming (where that "completion" happens in the non-SSL case isn't 100% clear to me)...and index redistribution acquires it _before_ it creates a transaction in {{getRestributionTransactions()}}, then releases it when the redistribution is complete (so we never have to block a compaction). Streaming might have to deal with a short delay if a redistribution is running, but a.) that doesn't happen that often and b.) the summary (I think) is usually not very large. ({{getRestributionTransactions()}} can ignore streaming SSTables just like it ignores compacting ones. was (Author: maedhroz): [~jasonstack] I thought a bit more about our earlier chat (and had a quick chat w/ [~bdeggleston]), and it seems like the simplest thing might be handling the stats an index summary in slightly different ways. The STATS component is small. We could just buffer it up, use that buffered size in the manifest, and stream that buffer. It special-cases this component, but we more or less avoid having to reason about the risk of blocking compactions, a repair completing, etc. For the SUMMARY, we take advantage of the fact that possibly/infrequently delaying the redistribution task isn't a big suboptimal outcome. We have a simple lock that protects it (on {{SSTableRader}}, similar to what you've already mentioned or as a threadsafe set of readers in a central location), i.e. streaming acquires it when the manifest is created and releases it when the index summary completes streaming (where that "completion" happens in the non-SSL case isn't 100% clear to me)...and index redistribution acquires it _before_ it creates a transaction in {{getRestributionTransactions()}}, then releases it when the redistribution is complete (so we never have to block a compaction). Streaming might have to deal with a short delay if a redistribution is running, but a.) that doesn't happen that often and b.) the summary (I think) is usually not very large. ({{getRestributionTransactions()}} can ignore streaming SSTables just like it ignores compacting ones. > Mutating sstable component may race with entire-sstable-streaming(ZCS) > causing checksum validation failure > -- > > Key: CASSANDRA-15861 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15861 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Repair, Consistency/Streaming, > Local/Compaction >Reporter: ZhaoYang >Assignee: ZhaoYang >Priority: Normal > Fix For: 4.0-beta > > > Flaky dtest: [test_dead_sync_initiator - > repair_tests.repair_test.TestRepair|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/143/testReport/junit/dtest.repair_tests.repair_test/TestRepair/test_dead_sync_initiator/] > {code:java|title=stacktrace} > Unexpected error found in node logs (see stdout for full details). Errors: > [ERROR [Stream-Deserializer-127.0.0.1:7000-570871f3] 2020-06-03 04:05:19,081 > CassandraEntireSSTableStreamReader.java:145 - [Stream > 6f1c3360-a54f-11ea-a808-2f23710fdc90] Error while reading sstable from stream > for table = keyspace1.standard1 > org.apache.cassandra.io.sstable.CorruptSSTableException: Corrupted: > /home/cassandra/cassandra/cassandra-dtest/tmp/dtest-te4ty0r9/test/node3/data0/keyspace1/standard1-5f5ab140a54f11eaa8082f23710fdc90/na-2-big-Statistics.db > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.maybeValidateChecksum(MetadataSerializer.java:219) > at > org.apache.cassandra.io.sstable.metadata.MetadataSerializer.deserialize(MetadataSerializer.java:198) > at > org.apac
[jira] [Commented] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162271#comment-17162271 ] Benedict Elliott Smith commented on CASSANDRA-15234: Thanks [~e.dimitrova] bq. If you decide at some point this work or part of it to be committed It was a confusing disagreement, certainly. I understood us to have agreed that - if wanted - we could commit the work that's been reviewed, excluding only the new config file (and the new config parameter name tests etc). It's up to you of course. I think there could be value in this, since it might make the config file change more manageable, but I'm personally neutral on sequencing. I'm fairly certain the main body of this work should be utilised to support whatever our new config file layout is; whether we commit it first or all together is fine by me, and I'm happy to defer to others' preferences (most importantly yours, as the author) > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s, TiB/s}}. > Perhaps for avoiding ambiguity we could not accept bauds {{bs, Mbps}} or > powers of 1000 such as {{KB/s}}, given these are regularly used for either > their old or new definition e.g. {{KiB/s}}, or we could support them and > simply log the value in bytes/s. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162209#comment-17162209 ] Ekaterina Dimitrova edited comment on CASSANDRA-15234 at 7/21/20, 5:53 PM: --- Good catch [~benedict] . Thank you. My bad, just pushed a fast commit [here|https://github.com/ekaterinadimitrova2/cassandra/commit/b4eebe080835da79d032f9314262c268b71172a8] for the three rate parameters we have. For the record, I stopped working on this patch until it is clear whether the code will be used at all. As far as I understand, you took the lead on a POC development for the newly suggested approach. Unfortunately, I don't have time to support this now but I would be happy to give feedback when it is shaped already. If you decide at some point this work or part of it to be committed, let me know, I will complete whatever is outstanding. The main framework is in place. A couple of things to keep in mind: * As suggested by Benjamin, the patch can be simplified by making {{Converter}} an {{enum}}. It will allow to use the enum value directly into the {{Replaces}} annotation. Doing so will remove the need to use reflection to instantiate the converters and the use of a cache to avoid multiple instantiation. * In-jvm tests - loading config parameters should be reworked as currently they don't work with custom types (like the newly introduced Duration, etc). The suggested approach by [~dcapwell] would work but it requires also api changes. I suggest a separate ticket for this part to be opened. Instead of using reflection, the suggestion is to use SnakeYAML. In order not to slow down the tests, no yaml files will be introduced but there will be a function to build the yaml nodes for us. This was a quick POC by [~dcapwell] but there are parameters which will need additional work and attention: {code:java} public class MapToConfigTest { @Test public void test() { Map map = ImmutableMap.builder() .put("auto_bootstrap", false) .put("permissions_validity_in_ms", 10) .put("role_manager", "some value") .build(); Constructor constructor = new YamlConfigurationLoader.CustomConstructor(Config.class); PropertiesChecker propertiesChecker = new PropertiesChecker(); constructor.setPropertyUtils(propertiesChecker); constructor.setComposer(new Composer(null, null) { public Node getSingleNode() { return toNode(map); } }); Config config = (Config) constructor.getSingleData(Config.class); System.out.println("trap"); } public static Node toNode(Object object) { if (object instanceof Map) { List values = new ArrayList<>(); for (Map.Entry e : ((Map) object).entrySet()) { values.add(new NodeTuple(toNode(e.getKey()), toNode(e.getValue(; } return new MappingNode(FAKE_TAG, values, null); } else if (object instanceof Number || object instanceof String || object instanceof Boolean) { return new ScalarNode(FAKE_TAG, object.toString(), FAKE_MARK, FAKE_MARK, '\''); } else { throw new UnsupportedOperationException("unexpected type found: given " + object.getClass()); } } private static final Tag FAKE_TAG = new Tag("ignore"); private static final Mark FAKE_MARK = new Mark("ignore", 0, 0, 0, "", 0); } {code} The rest are small things which I also partially already handled. [~maedhroz], [~benedict], please, let me know if you have any other questions around this patch. was (Author: e.dimitrova): Good catch [~benedict] . Thank you. My bad, just pushed a fast commit [here|https://github.com/ekaterinadimitrova2/cassandra/commit/b4eebe080835da79d032f9314262c268b71172a8] for the three rate parameters we have. For the record, I stopped working on this patch until it is clear whether the code will be used at all. As far as I understand, you took the lead on a POC development for the newly suggested approach. Unfortunately, I don't have time to support this now but I would be happy to give feedback when it is shaped already. If you decide at some point this work or part of it to be committed, let me know, I will complete whatever is outstanding. The main framework is in place. A couple of things to keep in mind: * As suggested by Benjamin, the patch can be simplified by making {{Converter}} an {{enum}}. It will allow to use the en
[jira] [Comment Edited] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162209#comment-17162209 ] Ekaterina Dimitrova edited comment on CASSANDRA-15234 at 7/21/20, 5:53 PM: --- Good catch [~benedict] . Thank you. My bad, just pushed a fast commit [here|https://github.com/ekaterinadimitrova2/cassandra/commit/b4eebe080835da79d032f9314262c268b71172a8] for the three rate parameters we have. For the record, I stopped working on this patch until it is clear whether the code will be used at all. As far as I understand, you took the lead on a POC development for the newly suggested approach. Unfortunately, I don't have time to support this now but I would be happy to give feedback when it is shaped already. If you decide at some point this work or part of it to be committed, let me know, I will complete whatever is outstanding. The main framework is in place. A couple of things to keep in mind: * As suggested by Benjamin, the patch can be simplified by making {{Converter}} an {{enum}}. It will allow to use the enum value directly into the {{Replaces}} annotation. Doing so will remove the need to use reflection to instantiate the converters and the use of a cache to avoid multiple instantiation. * In-jvm tests - loading config parameters should be reworked as currently they don't work with custom types (like the newly introduced Duration, etc). The suggested approach by [~dcapwell] would work but it requires also api changes. I suggest a separate ticket for this part to be opened. Instead of using reflection, the suggestion is to use SnakeYAML. In order not to slow down the tests, no yaml files will be introduced but there will be a function to build the yaml nodes for us. This was a quick POC by [~dcapwell] but there are parameters which will need additional work and attention: {code:java} public class MapToConfigTest { @Test public void test() { Map map = ImmutableMap.builder() .put("auto_bootstrap", false) .put("permissions_validity_in_ms", 10) .put("role_manager", "some value") .build(); Constructor constructor = new YamlConfigurationLoader.CustomConstructor(Config.class); PropertiesChecker propertiesChecker = new PropertiesChecker(); constructor.setPropertyUtils(propertiesChecker); constructor.setComposer(new Composer(null, null) { public Node getSingleNode() { return toNode(map); } }); Config config = (Config) constructor.getSingleData(Config.class); System.out.println("trap"); } public static Node toNode(Object object) { if (object instanceof Map) { List values = new ArrayList<>(); for (Map.Entry e : ((Map) object).entrySet()) { values.add(new NodeTuple(toNode(e.getKey()), toNode(e.getValue(; } return new MappingNode(FAKE_TAG, values, null); } else if (object instanceof Number || object instanceof String || object instanceof Boolean) { return new ScalarNode(FAKE_TAG, object.toString(), FAKE_MARK, FAKE_MARK, '\''); } else { throw new UnsupportedOperationException("unexpected type found: given " + object.getClass()); } } private static final Tag FAKE_TAG = new Tag("ignore"); private static final Mark FAKE_MARK = new Mark("ignore", 0, 0, 0, "", 0); } {code} The rest are small things which I also partially already handled. [~maedhroz], [~benedict], please, let me know if you have any other questions around this patch. was (Author: e.dimitrova): Good catch [~benedict] . Thank you. My bad, just pushed a fast commit [here|https://github.com/ekaterinadimitrova2/cassandra/commit/b4eebe080835da79d032f9314262c268b71172a8] for the three rate parameters we have. For the record, I stopped working on this patch until it is clear whether the code will be used at all. As far as I understand, you took the lead on a POC development for the newly suggested approach. Unfortunately, I don't have time to support this now but I would be happy to give feedback when it is shaped already. If you decide at some point this work or part of it to be committed, let me know, I will complete whatever is outstanding. The main framework is in place. A couple of things to keep in mind: * As suggested by Benjamin, the patch can be simplified by making {{Converter}} an {{enum}}. It will allow to use the e
[jira] [Comment Edited] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162209#comment-17162209 ] Ekaterina Dimitrova edited comment on CASSANDRA-15234 at 7/21/20, 5:42 PM: --- Good catch [~benedict] . Thank you. My bad, just pushed a fast commit [here|https://github.com/ekaterinadimitrova2/cassandra/commit/b4eebe080835da79d032f9314262c268b71172a8] for the three rate parameters we have. For the record, I stopped working on this patch until it is clear whether the code will be used at all. As far as I understand, you took the lead on a POC development for the newly suggested approach. Unfortunately, I don't have time to support this now but I would be happy to give feedback when it is shaped already. If you decide at some point this work or part of it to be committed, let me know, I will complete whatever is outstanding. The main framework is in place. A couple of things to keep in mind: * As suggested by Benjamin, the patch can be simplified by making {{Converter}} an {{enum}}. It will allow to use the enum value directly into the {{Replaces}} annotation. Doing so will remove the need to use reflection to instantiate the converters and the use of a cache to avoid multiple instantiation. * In-jvm tests - loading config parameters should be reworked as currently they don't work with custom types (like the newly introduced Duration, etc). The suggested approach by [~dcapwell] would work but it requires also api changes. I suggest a separate ticket for this part to be opened. Instead of using reflection, the suggestion is to use SnakeYAML. In order not to slow down the tests, no yaml files will be introduced but there will be a function to build the yaml nodes for us. This was a quick POC by [~dcapwell] but there are parameters which will need additional work and attention: public class MapToConfigTest { @Test public void test() { Map map = ImmutableMap.builder() .put("auto_bootstrap", false) .put("permissions_validity_in_ms", 10) .put("role_manager", "some value") .build();Constructor constructor = new YamlConfigurationLoader.CustomConstructor(Config.class); PropertiesChecker propertiesChecker = new PropertiesChecker(); constructor.setPropertyUtils(propertiesChecker); constructor.setComposer(new Composer(null, null) { public Node getSingleNode() { return toNode(map); } });Config config = (Config) constructor.getSingleData(Config.class); System.out.println("trap"); }public static Node toNode(Object object) { if (object instanceof Map) { List values = new ArrayList<>(); for (Map.Entry e : ((Map) object).entrySet()) { values.add(new NodeTuple(toNode(e.getKey()), toNode(e.getValue(; } return new MappingNode(FAKE_TAG, values, null); } else if (object instanceof Number || object instanceof String || object instanceof Boolean) { return new ScalarNode(FAKE_TAG, object.toString(), FAKE_MARK, FAKE_MARK, '\''); } else { throw new UnsupportedOperationException("unexpected type found: given " + object.getClass()); } }private static final Tag FAKE_TAG = new Tag("ignore"); private static final Mark FAKE_MARK = new Mark("ignore", 0, 0, 0, "", 0); } The rest are small things which I also partially already handled. [~maedhroz], [~benedict], please, let me know if you have any other questions around this patch. was (Author: e.dimitrova): Good catch [~benedict] . Thank you. My bad, just pushed a fast commit [here|https://github.com/ekaterinadimitrova2/cassandra/commit/b4eebe080835da79d032f9314262c268b71172a8] for the three rate parameters we have. For the record, I stopped working on this patch until it is clear whether the code will be used at all. As far as I understand, you took the lead on a POC development for the newly suggested approach. Unfortunately, I don't have time to support this now but I would be happy to give feedback when it is shaped already. If you decide at some point this work or part of it to be committed, let me know, I will complete whatever is outstanding. The main framework is in place. A couple of things to keep in mind: * As suggested by Benjamin, the patch can be simplified by making {{Converter}} an {{enum}}. It will allow to use the enum value directly into the {{Replaces}} annotation. Doing so will remove the need to use reflection to instantiate the converters and the use of a cache to avoid multiple instantiation. * In-jvm tests
[jira] [Commented] (CASSANDRA-15234) Standardise config and JVM parameters
[ https://issues.apache.org/jira/browse/CASSANDRA-15234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162209#comment-17162209 ] Ekaterina Dimitrova commented on CASSANDRA-15234: - Good catch [~benedict] . Thank you. My bad, just pushed a fast commit [here|https://github.com/ekaterinadimitrova2/cassandra/commit/b4eebe080835da79d032f9314262c268b71172a8] for the three rate parameters we have. For the record, I stopped working on this patch until it is clear whether the code will be used at all. As far as I understand, you took the lead on a POC development for the newly suggested approach. Unfortunately, I don't have time to support this now but I would be happy to give feedback when it is shaped already. If you decide at some point this work or part of it to be committed, let me know, I will complete whatever is outstanding. The main framework is in place. A couple of things to keep in mind: * As suggested by Benjamin, the patch can be simplified by making {{Converter}} an {{enum}}. It will allow to use the enum value directly into the {{Replaces}} annotation. Doing so will remove the need to use reflection to instantiate the converters and the use of a cache to avoid multiple instantiation. * In-jvm tests - loading config parameters should be reworked as currently they don't work with custom types (like the newly introduced Duration, etc). The suggested approach by [~dcapwell] would work but it requires also api changes. I suggest a separate ticket for this part to be opened. Instead of using reflection, the suggestion is to use SnakeYAML. In order not to slow down the tests, no yaml files will be introduced but there will be a function to build the yaml nodes for us. This was a quick POC by [~dcapwell] but there are parameters which will need additional work and attention: public class MapToConfigTest { @Test public void test() { Map map = ImmutableMap.builder() .put("auto_bootstrap", false) .put("permissions_validity_in_ms", 10) .put("role_manager", "some value") .build();Constructor constructor = new YamlConfigurationLoader.CustomConstructor(Config.class); PropertiesChecker propertiesChecker = new PropertiesChecker(); constructor.setPropertyUtils(propertiesChecker); constructor.setComposer(new Composer(null, null) { public Node getSingleNode() { return toNode(map); } });Config config = (Config) constructor.getSingleData(Config.class); System.out.println("trap"); }public static Node toNode(Object object) { if (object instanceof Map) { List values = new ArrayList<>(); for (Map.Entry e : ((Map) object).entrySet()) { values.add(new NodeTuple(toNode(e.getKey()), toNode(e.getValue(; } return new MappingNode(FAKE_TAG, values, null); } else if (object instanceof Number || object instanceof String || object instanceof Boolean) { return new ScalarNode(FAKE_TAG, object.toString(), FAKE_MARK, FAKE_MARK, '\''); } else { throw new UnsupportedOperationException("unexpected type found: given " + object.getClass()); } }private static final Tag FAKE_TAG = new Tag("ignore"); private static final Mark FAKE_MARK = new Mark("ignore", 0, 0, 0, "", 0); } The rest are small things which I also partially already handled. [~maedhroz], [~benedict], please, let me know if you have any other questions around this patch. > Standardise config and JVM parameters > - > > Key: CASSANDRA-15234 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15234 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Benedict Elliott Smith >Assignee: Ekaterina Dimitrova >Priority: Normal > Fix For: 4.0-beta > > Attachments: CASSANDRA-15234-3-DTests-JAVA8.txt > > > We have a bunch of inconsistent names and config patterns in the codebase, > both from the yams and JVM properties. It would be nice to standardise the > naming (such as otc_ vs internode_) as well as the provision of values with > units - while maintaining perpetual backwards compatibility with the old > parameter names, of course. > For temporal units, I would propose parsing strings with suffixes of: > {{code}} > u|micros(econds?)? > ms|millis(econds?)? > s(econds?)? > m(inutes?)? > h(ours?)? > d(ays?)? > mo(nths?)? > {{code}} > For rate units, I would propose parsing any of the standard {{B/s, KiB/s, > MiB/s, GiB/s,
[jira] [Updated] (CASSANDRA-15191) stop_paranoid disk failure policy is ignored on CorruptSSTableException after node is up
[ https://issues.apache.org/jira/browse/CASSANDRA-15191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] David Capwell updated CASSANDRA-15191: -- Reviewers: David Capwell, David Capwell (was: David Capwell) David Capwell, David Capwell Status: Review In Progress (was: Patch Available) > stop_paranoid disk failure policy is ignored on CorruptSSTableException after > node is up > > > Key: CASSANDRA-15191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15191 > Project: Cassandra > Issue Type: Bug > Components: Local/Config >Reporter: Vincent White >Assignee: Stefan Miklosovic >Priority: Normal > Fix For: 3.11.x, 4.0-beta > > Attachments: log.txt > > Time Spent: 3h 20m > Remaining Estimate: 0h > > There is a bug when disk_failure_policy is set to stop_paranoid and > CorruptSSTableException is thrown after server is up. The problem is that > this setting is ignored. Normally, it should stop gossip and transport but it > just continues to serve requests and an exception is just logged. > > This patch unifies the exception handling in JVMStabilityInspector and code > is reworked in such way that this inspector acts as a central place where > such exceptions are inspected. > > The core reason for ignoring that exception is that thrown exception in > AbstractLocalAwareExecturorService is not CorruptSSTableException but it is > RuntimeException and that exception is as its cause. Hence it is better if we > handle this in JVMStabilityInspector which can recursively examine it, hence > act accordingly. > Behaviour before: > stop_paranoid of disk_failure_policy is ignored when CorruptSSTableException > is thrown, e.g. on a regular select statement > Behaviour after: > Gossip and transport (cql) is turned off, JVM is still up for further > investigation e.g. by jmx. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162188#comment-17162188 ] Olivier Michallat commented on CASSANDRA-15299: --- {quote}There's one more issue with a driver, which is easy to reproduce {quote} >From the line numbers in the stack trace it looks like an old commit, and the >same issue Sam reported >[here|https://github.com/datastax/java-driver/pull/1444#pullrequestreview-421159435]. > Try upgrading to the latest (aeb4066b9), the issue should be fixed. > CASSANDRA-13304 follow-up: improve checksumming and compression in protocol > v5-beta > --- > > Key: CASSANDRA-15299 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15299 > Project: Cassandra > Issue Type: Improvement > Components: Messaging/Client >Reporter: Aleksey Yeschenko >Assignee: Alex Petrov >Priority: Normal > Labels: protocolv5 > Fix For: 4.0-alpha > > > CASSANDRA-13304 made an important improvement to our native protocol: it > introduced checksumming/CRC32 to request and response bodies. It’s an > important step forward, but it doesn’t cover the entire stream. In > particular, the message header is not covered by a checksum or a crc, which > poses a correctness issue if, for example, {{streamId}} gets corrupted. > Additionally, we aren’t quite using CRC32 correctly, in two ways: > 1. We are calculating the CRC32 of the *decompressed* value instead of > computing the CRC32 on the bytes written on the wire - losing the properties > of the CRC32. In some cases, due to this sequencing, attempting to decompress > a corrupt stream can cause a segfault by LZ4. > 2. When using CRC32, the CRC32 value is written in the incorrect byte order, > also losing some of the protections. > See https://users.ece.cmu.edu/~koopman/pubs/KoopmanCRCWebinar9May2012.pdf for > explanation for the two points above. > Separately, there are some long-standing issues with the protocol - since > *way* before CASSANDRA-13304. Importantly, both checksumming and compression > operate on individual message bodies rather than frames of multiple complete > messages. In reality, this has several important additional downsides. To > name a couple: > # For compression, we are getting poor compression ratios for smaller > messages - when operating on tiny sequences of bytes. In reality, for most > small requests and responses we are discarding the compressed value as it’d > be smaller than the uncompressed one - incurring both redundant allocations > and compressions. > # For checksumming and CRC32 we pay a high overhead price for small messages. > 4 bytes extra is *a lot* for an empty write response, for example. > To address the correctness issue of {{streamId}} not being covered by the > checksum/CRC32 and the inefficiency in compression and checksumming/CRC32, we > should switch to a framing protocol with multiple messages in a single frame. > I suggest we reuse the framing protocol recently implemented for internode > messaging in CASSANDRA-15066 to the extent that its logic can be borrowed, > and that we do it before native protocol v5 graduates from beta. See > https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/FrameDecoderCrc.java > and > https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/net/FrameDecoderLZ4.java. -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15857) Frozen RawTuple is not annotated with frozen in the toString method
[ https://issues.apache.org/jira/browse/CASSANDRA-15857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162154#comment-17162154 ] Yifan Cai commented on CASSANDRA-15857: --- Got it. Just dropped the commit that upgrade junit version and fix the test. 3.11 PR: [https://github.com/apache/cassandra/pull/686] Test: [https://app.circleci.com/pipelines/github/yifan-c/cassandra?branch=CASSANDRA-15857-freeze-rawtuple-3_11] Trunk PR: [https://github.com/apache/cassandra/pull/669] Test: [https://app.circleci.com/pipelines/github/yifan-c/cassandra?branch=CASSANDRA-15857-rawtuple-tostring-miss-frozen] > Frozen RawTuple is not annotated with frozen in the toString method > --- > > Key: CASSANDRA-15857 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15857 > Project: Cassandra > Issue Type: Bug > Components: Legacy/CQL >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Fix For: 4.0, 3.11.x > > Time Spent: 1h 10m > Remaining Estimate: 0h > > All raw types (e.g. RawCollection, RawUT) that supports freezing wraps the > type name with 'frozen<>' in the toString method, except RawTuple. > Therefore, the RawTuple::toString output misses the frozen wrapper. > Tuple is always frozen. However since CASSANDRA-15035, it throws when the > inner tuple is not explicitly wrapped with frozen within a collection. > The method, CQL3Type.Raw::toString, is referenced at multiple places in the > source. For example, referenced in CreateTypeStatement.Raw and involved in > CQLSSTableWriter. Another example is that it is called to produce the > SchemaChange at several AlterSchemaStatement implementations. > A test can prove that missing the frozen wrapper causes exception when > building CQLSSTableWriter for user types defined like below. Note that the > inner tuple is wrapped with frozen in the initial CQL statement. > {code:java} > CREATE TYPE ks.fooType ( f list>> ) > {code} > {code:java} > org.apache.cassandra.exceptions.InvalidRequestException: Non-frozen tuples > are not allowed inside collections: list> > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.throwNestedNonFrozenError(CQL3Type.java:710) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepare(CQL3Type.java:669) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepareInternal(CQL3Type.java:661) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.lambda$prepare$1(Types.java:341) > at > java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) > at > java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) > at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) > at > java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) > at > java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) > at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) > at > java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.prepare(Types.java:342) > at org.apache.cassandra.schema.Types$RawBuilder.build(Types.java:291) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.createTypes(CQLSSTableWriter.java:551) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.build(CQLSSTableWriter.java:527) > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15814) order by descending on frozen list not working
[ https://issues.apache.org/jira/browse/CASSANDRA-15814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162129#comment-17162129 ] Andres de la Peña commented on CASSANDRA-15814: --- Great. I've rebased and run CI: ||PR||utest||dtest||CircleCI j8||CircleCI j11|| |[2.2|https://github.com/apache/cassandra/pull/655] |[193|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-test/193/]|[229|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/229/]|[link|https://app.circleci.com/pipelines/github/adelapena/cassandra/70/workflows/f69c8ddc-b243-438d-86a5-579c67b70091]|-| |[3.0|https://github.com/apache/cassandra/pull/656] |[194|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-test/194/]|[230|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/230/]|[link|https://app.circleci.com/pipelines/github/adelapena/cassandra/69/workflows/c5548c6d-da91-48c3-94f6-d171410a2337]|-| |[3.11|https://github.com/apache/cassandra/pull/657] |[195|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-test/195/]|[231|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/231/]|[link|https://app.circleci.com/pipelines/github/adelapena/cassandra/68/workflows/54c2b6b0-5eb9-4561-95f6-9b68b834dddb]|-| |[trunk|https://github.com/apache/cassandra/pull/658]|[196|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-test/196/]|[232|https://ci-cassandra.apache.org/view/all/job/Cassandra-devbranch-dtest/232/]|[link|https://app.circleci.com/pipelines/github/adelapena/cassandra/67/workflows/76d504cc-b1d1-42a3-bc45-b0b183194d21]|[link|https://app.circleci.com/pipelines/github/adelapena/cassandra/67/workflows/556d39f6-80b4-4342-bfa9-ec7fd12a25cd]| > order by descending on frozen list not working > -- > > Key: CASSANDRA-15814 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15814 > Project: Cassandra > Issue Type: Bug > Components: CQL/Interpreter >Reporter: Felipe Perez >Assignee: Andres de la Peña >Priority: Normal > Time Spent: 1h 40m > Remaining Estimate: 0h > > By creating a table like the following: > {code:java} > CREATE TABLE IF NOT EXISTS software ( > name ascii, > version frozen>, > data ascii, > PRIMARY KEY(name,version) > ) > {code} > It works and version is ordered in an ascending order. But when trying to > order in descending order: > {code:java} > CREATE TABLE IF NOT EXISTS software ( > name ascii, > version frozen>, > data ascii, > PRIMARY KEY(name,version) > ) WITH CLUSTERING ORDER BY (version DESC); > {code} > The table is created normally, but when trying to insert a row: > {code:java} > insert into software(name, version) values ('t1', [2,10,30,40,50]); > {code} > Cassandra throws an error: > {code:java} > InvalidRequest: Error from server: code=2200 [Invalid query] message="Invalid > list literal for version of type frozen>" > {code} > The goal here is that I would like to get the last version of a software. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-15792: - Since Version: 4.0-beta1 Source Control Link: https://github.com/apache/cassandra-dtest/commit/23944afa4f435bb726c37e8cb10311e2e9bbd792 (was: https://github.com/grighetto/cassandra-dtest/pull/2) Resolution: Fixed Status: Resolved (was: Ready to Commit) Committed, thanks. BTW, putting something in 'ready to commit' is just fine if you need a committer to review. > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-15792: - Status: Ready to Commit (was: Review In Progress) > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra-dtest] branch master updated: Make TestSpeculativeReadRepair.test_speculative_data_request deterministic
This is an automated email from the ASF dual-hosted git repository. brandonwilliams pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/cassandra-dtest.git The following commit(s) were added to refs/heads/master by this push: new 23944af Make TestSpeculativeReadRepair.test_speculative_data_request deterministic 23944af is described below commit 23944afa4f435bb726c37e8cb10311e2e9bbd792 Author: Gianluca Righetto AuthorDate: Tue Jul 21 11:31:31 2020 -0300 Make TestSpeculativeReadRepair.test_speculative_data_request deterministic Patch by Gianluca Righetto, reviewed by Bereguer Blasi and brandonwilliams for CASSANDRA-15792 --- byteman/request_verb_timing.btm | 12 read_repair_test.py | 40 +++- 2 files changed, 51 insertions(+), 1 deletion(-) diff --git a/byteman/request_verb_timing.btm b/byteman/request_verb_timing.btm new file mode 100644 index 000..e0dc68e --- /dev/null +++ b/byteman/request_verb_timing.btm @@ -0,0 +1,12 @@ +RULE timing of request messages broken down by verb +CLASS org.apache.cassandra.net.MessagingService +METHOD doSend +AT ENTRY +BIND prefix:String = "org.jboss.byteman."; # byteman in strict mode requires the o.j.b prefix +toHost:String = $to.address.toString(); +verb:String = $message.header.verb.toString(); +prop:String = prefix + "|request_verb_timing|" + toHost + "|" + verb; +IF true +DO +System.setProperty(prop, String.valueOf(System.currentTimeMillis())); +ENDRULE \ No newline at end of file diff --git a/read_repair_test.py b/read_repair_test.py index 64e3b55..0d725d0 100644 --- a/read_repair_test.py +++ b/read_repair_test.py @@ -53,6 +53,33 @@ def byteman_validate(node, script, verbose=False, opts=None): assert not has_errors, "byteman script didn't compile\n" + out +def build_byteman_submit_command(node, opts): +cdir = node.get_install_dir() +byteman_cmd = [os.path.join(os.environ['JAVA_HOME'], 'bin', 'java'), + '-cp', + glob.glob(os.path.join(cdir, 'build', 'lib', 'jars', 'byteman-submit-[0-9]*.jar'))[0], + 'org.jboss.byteman.agent.submit.Submit', + '-p', node.byteman_port, + '-h', node.network_interfaces['binary'][0]] + opts +return byteman_cmd + +def request_verb_timing(node): +# -y is byteman's built-in flag for reading system props +byteman_cmd = build_byteman_submit_command(node, ['-y']) +out = subprocess.check_output(byteman_cmd) +if (out is not None) and isinstance(out, bytes): +out = out.decode() +lines = out.splitlines() +props = {} +for line in lines: +# look for the custom separators, otherwise skip +if "=" in line and "|" in line: +key, value = line.split("=") +split_key = key.split("|") +ip = split_key[-2].replace("/", "") +verb = split_key[-1] +props.setdefault(ip, {}).update({verb: int(value)}) +return props class TestReadRepair(Tester): @@ -518,6 +545,8 @@ class TestSpeculativeReadRepair(Tester): node2.byteman_submit(['-u', './byteman/read_repair/stop_writes.btm']) node1.byteman_submit(['./byteman/read_repair/sorted_live_endpoints.btm']) +node1.byteman_submit(['./byteman/request_verb_timing.btm']) + with StorageProxy(node1) as storage_proxy: assert storage_proxy.blocking_read_repair == 0 assert storage_proxy.speculated_rr_read == 0 @@ -526,11 +555,20 @@ class TestSpeculativeReadRepair(Tester): session = self.get_cql_connection(node1) node2.byteman_submit(['./byteman/read_repair/stop_data_reads.btm']) results = session.execute(quorum("SELECT * FROM ks.tbl WHERE k=1")) + +timing = request_verb_timing(node1) +repair_req_node3 = timing[node3.ip_addr].get('READ_REPAIR_REQ') +repair_req_node2 = timing[node2.ip_addr].get('READ_REPAIR_REQ') assert listify(results) == [kcv(1, 0, 1), kcv(1, 1, 2)] assert storage_proxy.blocking_read_repair == 1 assert storage_proxy.speculated_rr_read == 1 -assert storage_proxy.speculated_rr_write == 0 + +# under normal circumstances we don't expect a speculated write here, +# but the repair request to node 3 may timeout due to CPU contention and +# then a speculated write is sent to node 2, so we just make sure that the +# request to node 2 didn't happen before the request to node 3 +assert storage_proxy.speculated_rr_write == 0 or repair_req_node2 > repair_req_node3 @since('4.0') def test_speculative_write(self): - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@c
[jira] [Updated] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-15792: - Reviewers: Berenguer Blasi, Brandon Williams (was: Berenguer Blasi) > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Gianluca Righetto updated CASSANDRA-15792: -- Source Control Link: https://github.com/grighetto/cassandra-dtest/pull/2 > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162074#comment-17162074 ] Gianluca Righetto commented on CASSANDRA-15792: --- Opened a clean PR to get rid of the merge noise of the first one (pushing then rebasing against two remotes messed that one up): https://github.com/grighetto/cassandra-dtest/pull/2 > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162062#comment-17162062 ] Alex Petrov edited comment on CASSANDRA-15299 at 7/21/20, 2:29 PM: --- Thank you for the changes. I'm mostly done with a second pass of the review. First, wanted to bring up a couple of important things: * Current memory management is rather hard to follow, and maybe we should make an effort to document and/or codify it somewhere. I've spent about a day following all paths and figuring out ins-and-outs of it. One thing that I've found is that in {{FrameSet#finish}}, we're releasing not only the frame that was encoded into the sending buffer, but also the one we flush. This is done in three places with {{writeAndFlush}}. I believe this was leading to crc mismatch bug that I was catching earlier (see stacktrace [1] below). What might have been happening is we were releasing the buffer too early, which would allow it to get recycled. * There were several places in {{SimpleClient}} where we were not accouting for bytes. One is that we were never releasing {{Frame}} of the response we were getting. I've implemented copying, which is not optimal, but should work for testing purposes. The other one is that we weren't giving the bytes back to the limits. They would be acquired via netty processing, so we need to release them eventually. I've added a simple hook to release those. An alterntive to this would be to not use limits at all, with a downside of loosing a bit of observability. * [this one was discussed privately and source of the leak suggested by Sam] Flusher takes care of calling release for the flush item, which releases the source buffer. However, _response_ frame is released only for small buffers in {{FrameSet}} Here's a branch with aforementioned changes. However, I haven't run CI on it, I'll check if it breaks anything later today: https://github.com/ifesdjeen/cassandra/pull/new/15299-alex {code} [1] org.apache.cassandra.net.Crc$InvalidCrc: Read -854589741, Computed 1432984585 at org.apache.cassandra.transport.CQLMessageHandler.processCorruptFrame(CQLMessageHandler.java:328) at org.apache.cassandra.net.AbstractMessageHandler.process(AbstractMessageHandler.java:217) at org.apache.cassandra.net.FrameDecoder.deliver(FrameDecoder.java:321) at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:285) at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:269) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at {code} Some small nits: * In {{CQLMessageHandler#processOneContainedMessage}}, when we can't acquire capacity and, subsequently, we're not passing the frame further down the line. Shouold we release the frame in this case, since usually we're releasing the source frame after flush. * {{ReusableBuffer}} is unused. * {{Server}} has a few unused imports and {{eventExecutorGroup}} which is unused. * I'm not sure if we currently handle releasing corrupted frames. * In {{FrameSet}}, we're currently relying on the fact that we'll be able to go through {{finish}} and release everything successfully. However, this might not always be the case. I couldn't trigger such error, but it might still be possible. Shouold we maybe make {{FrameSet}} auto-closeable and make sure we always release buffers in {{finally}}? I've also made a similar change to {{processItem}} which would add item to {{flushed}} to make sure it's released. That makes {{flushed}} variable name not quite right though. And some improvements that were done to simple client: * it now supports time outs * supports sending multiple messages in a single frame when using v5 * simpler pipelines * reuses code for frame processing There's one more issue with a driver, which is easy to reproduce (with attached jar and test), but here's a stack trace: {code} DEBUG [cluster1-nio-worker-1] 2020-07-21 11:54:20,474 Connection.java:1396 - Connection[/127.0.0.1:64050-2, inFlight=2, closed=false] connection error java.lang.AssertionError: null at com.datastax.driver.core.BytesToSegmentDecoder.decode(BytesToSegmentDecoder.java:56) at io.netty.handler.codec.LengthFieldBasedFrameDecoder.decode(LengthFieldBasedFrameDecoder.java:332) at io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:501) at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:440) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(Abstra
[jira] [Commented] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162066#comment-17162066 ] Berenguer Blasi commented on CASSANDRA-15792: - I moved to 'review in progress' as still sbdy that is a committer has to review it :-) > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Berenguer Blasi updated CASSANDRA-15792: Reviewers: Berenguer Blasi, Berenguer Blasi (was: Berenguer Blasi) Berenguer Blasi, Berenguer Blasi (was: Berenguer Blasi) Status: Review In Progress (was: Patch Available) > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15583) 4.0 quality testing: Tooling, Bundled and First Party
[ https://issues.apache.org/jira/browse/CASSANDRA-15583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162040#comment-17162040 ] Berenguer Blasi edited comment on CASSANDRA-15583 at 7/21/20, 2:11 PM: --- [~samt] I see you're shepherd here so I turn to you for some advice :-) We have added so far what enables us to test all internal tooling with samples. Now no PR touching tooling shouldn't include utests/dtests imo. The q I have is how does this relate to this ticket: Should we consider this ticket 'done' or should we start adding missing test cases for the tools which only have minimal testing in some cases? Adding all test cases missing in all tooling sounds like a mammoth task and there might be better usages of time for the project. I have thought a few options: A. Leave it here and all new PRs should include tooling testing if applicable B. Let's select a few 'most user impacting' scenarios, whatever that may be, add testing for those and then apply A C. Leave it here and create tickets per each tool, describing what's already there and what's missing, as LHF tickets. No particular order, finger in the air and thinking out loud. Wdyt? I like C for some reason, I think it will help ramp up people :shrug: :thinking: was (Author: bereng): [~samt] I see you're shepherd here so I turn to you for some advice :-) We have added so far what enables us to test all internal tooling with samples. Now no PR touching tooling shouldn't include utests/dtests imo. The q I have is how does this relate to this ticket: Should we consider this ticket 'done' or should we start adding missing test cases for the tools which only have minimal testing in some cases? Adding all test cases missing in all tooling sounds like a mammoth task and there might be better usages or time for the project. I have thought a few options: A. Leave it here and all new PRs should include tooling testing if applicable B. Let's select a few 'most user impacting' scenarios, whatever that may be, add testing for those and the apply A C. Leave it here and create tickets per each tool, describing what's already there and what's missing, as LHF tickets. No particular order, finger in the air and thinking out loud. Wdyt? I like C for some reason, I think it will help ramp up people :shrug: :thinking: > 4.0 quality testing: Tooling, Bundled and First Party > - > > Key: CASSANDRA-15583 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15583 > Project: Cassandra > Issue Type: Task > Components: Test/dtest, Test/unit >Reporter: Josh McKenzie >Assignee: Berenguer Blasi >Priority: Normal > Fix For: 4.0-beta > > > Reference [doc from > NGCC|https://docs.google.com/document/d/1uhUOp7wpE9ZXNDgxoCZHejHt5SO4Qw1dArZqqsJccyQ/edit#] > for context. > *Shepherd: Sam Tunnicliffe* > Test plans should cover bundled first-party tooling and CLIs such as > nodetool, cqlsh, and new tools supporting full query and audit logging > (CASSANDRA-13983, CASSANDRA-12151). -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162064#comment-17162064 ] Gianluca Righetto commented on CASSANDRA-15792: --- Thanks for the first pass, [~Bereng]. Yes, this has been waiting for a committer to do the final push, I'll check if there's anyone available. I just realized I never added a comment detailing the final solution I landed on, so here it goes: the original implementation didn't expect a {{speculative write}} to be performed to node2, but depending on timing and load of node3, that might actually happen. So, as long as we can guarantee the speculative write to node2 happened after the initial write attempt to node3, we're good. In order to achieve that, I implemented a byteman function that records the {{System.currentTimeMillis}} of each Message Verb, and with that I can reconstruct the order of the events in the test. > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15299) CASSANDRA-13304 follow-up: improve checksumming and compression in protocol v5-beta
[ https://issues.apache.org/jira/browse/CASSANDRA-15299?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162062#comment-17162062 ] Alex Petrov commented on CASSANDRA-15299: - Thank you for the changes. I'm mostly done with a second pass of the review. First, wanted to bring up a couple of important things: * Current memory management is rather hard to follow, and maybe we should make an effort to document and/or codify it somewhere. I've spent about a day following all paths and figuring out ins-and-outs of it. One thing that I've found is that in {{FrameSet#finish}}, we're releasing not only the frame that was encoded into the sending buffer, but also the one we flush. This is done in three places with {{writeAndFlush}}. I believe this was leading to crc mismatch bug that I was catching earlier (see stacktrace [1] below). What might have been happening is we were releasing the buffer too early, which would allow it to get recycled. * There were several places in {{SimpleClient}} where we were not accouting for bytes. One is that we were never releasing {{Frame}} of the response we were getting. I've implemented copying, which is not optimal, but should work for testing purposes. The other one is that we weren't giving the bytes back to the limits. They would be acquired via netty processing, so we need to release them eventually. I've added a simple hook to release those. An alterntive to this would be to not use limits at all, with a downside of loosing a bit of observability. Here's a branch with aforementioned changes. However, I haven't run CI on it, I'll check if it breaks anything later today: https://github.com/ifesdjeen/cassandra/pull/new/15299-alex {code} [1] org.apache.cassandra.net.Crc$InvalidCrc: Read -854589741, Computed 1432984585 at org.apache.cassandra.transport.CQLMessageHandler.processCorruptFrame(CQLMessageHandler.java:328) at org.apache.cassandra.net.AbstractMessageHandler.process(AbstractMessageHandler.java:217) at org.apache.cassandra.net.FrameDecoder.deliver(FrameDecoder.java:321) at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:285) at org.apache.cassandra.net.FrameDecoder.channelRead(FrameDecoder.java:269) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at {code} Some small nits: * In {{CQLMessageHandler#processOneContainedMessage}}, when we can't acquire capacity and, subsequently, we're not passing the frame further down the line. Shouold we release the frame in this case, since usually we're releasing the source frame after flush. * {{ReusableBuffer}} is unused. * {{Server}} has a few unused imports and {{eventExecutorGroup}} which is unused. * I'm not sure if we currently handle releasing corrupted frames. * In {{FrameSet}}, we're currently relying on the fact that we'll be able to go through {{finish}} and release everything successfully. However, this might not always be the case. I couldn't trigger such error, but it might still be possible. Shouold we maybe make {{FrameSet}} auto-closeable and make sure we always release buffers in {{finally}}? I've also made a similar change to {{processItem}} which would add item to {{flushed}} to make sure it's released. That makes {{flushed}} variable name not quite right though. And some improvements that were done to simple client: * it now supports time outs * supports sending multiple messages in a single frame when using v5 * simpler pipelines * reuses code for frame processing There's one more issue with a driver, which is easy to reproduce (with attached jar and test), but here's a stack trace: {code} DEBUG [cluster1-nio-worker-1] 2020-07-21 11:54:20,474 Connection.java:1396 - Connection[/127.0.0.1:64050-2, inFlight=2, closed=false] connection error java.lang.AssertionError: null at com.datastax.driver.core.BytesToSegmentDecoder.decode(BytesToSegmentDecoder.java:56) at io.netty.handler.codec.LengthFieldBasedFrameDecoder.decode(LengthFieldBasedFrameDecoder.java:332) at io.netty.handler.codec.ByteToMessageDecoder.decodeRemovalReentryProtection(ByteToMessageDecoder.java:501) at io.netty.handler.codec.ByteToMessageDecoder.callDecode(ByteToMessageDecoder.java:440) at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:276) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:379) at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:365) at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:357) at io.netty.channel.ChannelInboundHandlerAdapter.channelRead(ChannelInboundHandlerAdapter.java:93) at com.datastax.driver.core.InboundT
[jira] [Commented] (CASSANDRA-15583) 4.0 quality testing: Tooling, Bundled and First Party
[ https://issues.apache.org/jira/browse/CASSANDRA-15583?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162040#comment-17162040 ] Berenguer Blasi commented on CASSANDRA-15583: - [~samt] I see you're shepherd here so I turn to you for some advice :-) We have added so far what enables us to test all internal tooling with samples. Now no PR touching tooling shouldn't include utests/dtests imo. The q I have is how does this relate to this ticket: Should we consider this ticket 'done' or should we start adding missing test cases for the tools which only have minimal testing in some cases? Adding all test cases missing in all tooling sounds like a mammoth task and there might be better usages or time for the project. I have thought a few options: A. Leave it here and all new PRs should include tooling testing if applicable B. Let's select a few 'most user impacting' scenarios, whatever that may be, add testing for those and the apply A C. Leave it here and create tickets per each tool, describing what's already there and what's missing, as LHF tickets. No particular order, finger in the air and thinking out loud. Wdyt? I like C for some reason, I think it will help ramp up people :shrug: :thinking: > 4.0 quality testing: Tooling, Bundled and First Party > - > > Key: CASSANDRA-15583 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15583 > Project: Cassandra > Issue Type: Task > Components: Test/dtest, Test/unit >Reporter: Josh McKenzie >Assignee: Berenguer Blasi >Priority: Normal > Fix For: 4.0-beta > > > Reference [doc from > NGCC|https://docs.google.com/document/d/1uhUOp7wpE9ZXNDgxoCZHejHt5SO4Qw1dArZqqsJccyQ/edit#] > for context. > *Shepherd: Sam Tunnicliffe* > Test plans should cover bundled first-party tooling and CLIs such as > nodetool, cqlsh, and new tools supporting full query and audit logging > (CASSANDRA-13983, CASSANDRA-12151). -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15956) 15583 - Ensure tooling surface area coverage
[ https://issues.apache.org/jira/browse/CASSANDRA-15956?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Berenguer Blasi updated CASSANDRA-15956: Test and Documentation Plan: See PR Status: Patch Available (was: In Progress) > 15583 - Ensure tooling surface area coverage > > > Key: CASSANDRA-15956 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15956 > Project: Cassandra > Issue Type: Improvement > Components: Test/unit >Reporter: Berenguer Blasi >Assignee: Berenguer Blasi >Priority: Normal > Fix For: 4.0-beta > > > Given CASSANDRA-15942 POC seems ok so far. Now we'll extend that to other > tools and scenarios in a width first approach to make sure enough surface > area is covered. This should suffice to confirm our framework should enable > devs to test any tools when needed. > Tools: https://cassandra.apache.org/doc/latest/tools/index.html > ||Tool||Easy UT possible||UT Sample||Easy dtest possible||Has > dtest||Comments|| > |Nodetool|(/) Added/Enhanced|(/) Added/enhanced|(/)|(/)|Use UT if possible. > Otherwise dtests if you need a full multinode env| > |Cqlsh|(/) Added|(/) Added|(/)|(/)|Easy Cqlsh UT is now possible and an > example is included| > |Cassandra-stress|(/) Added and existing|(/) Added|(/)|(x)| > |debug-cql|(/)|(x)|(/)|(x)| > |fqltool|(/) Existing|(/) Existing|(/)|(/)| > |auditlogviewer|(/) Existing|(/) Existing|(/)|(/)| > |*Sstable utilities*|| > |sstabledump|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableexpiredblockers|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablelevelreset|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableloader|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablemetadata|(/) enhanced|(/) enhanced|(/)|(!)|Ran in dtests, no > dedicated test| > |sstableofflinerelevel|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablerepairedset|(/) enhanced|(/) enhanced|(/)|(!)|Ran in dtests, no > dedicated test| > |sstablescrub|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablesplit|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableupgrade|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableutil|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableverify|(/) enhanced|(/) enhanced|(/)|(/)|| -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15956) 15583 - Ensure tooling surface area coverage
[ https://issues.apache.org/jira/browse/CASSANDRA-15956?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162034#comment-17162034 ] Berenguer Blasi commented on CASSANDRA-15956: - At the end of the day this one turned out to be shorter than expected. Most tools have some seed testing which has been enhanced or added as a result of this ticket and CASSANDRA-15942. It looks like no developer is going to have an excuse not to add tests, be it unit or dtest, to any PR touching tooling. Coverage is a different story where tools mostly seem to have only minimal testing. Some like nodetool have extensive testing on some commands but none on others. Fqltool i.e. has some testing particular to itself, etc. Overall I consider what we have at this point good and now it's a matter of discussing how to best start adding more test cases. > 15583 - Ensure tooling surface area coverage > > > Key: CASSANDRA-15956 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15956 > Project: Cassandra > Issue Type: Improvement > Components: Test/unit >Reporter: Berenguer Blasi >Assignee: Berenguer Blasi >Priority: Normal > Fix For: 4.0-beta > > > Given CASSANDRA-15942 POC seems ok so far. Now we'll extend that to other > tools and scenarios in a width first approach to make sure enough surface > area is covered. This should suffice to confirm our framework should enable > devs to test any tools when needed. > Tools: https://cassandra.apache.org/doc/latest/tools/index.html > ||Tool||Easy UT possible||UT Sample||Easy dtest possible||Has > dtest||Comments|| > |Nodetool|(/) Added/Enhanced|(/) Added/enhanced|(/)|(/)|Use UT if possible. > Otherwise dtests if you need a full multinode env| > |Cqlsh|(/) Added|(/) Added|(/)|(/)|Easy Cqlsh UT is now possible and an > example is included| > |Cassandra-stress|(/) Added and existing|(/) Added|(/)|(x)| > |debug-cql|(/)|(x)|(/)|(x)| > |fqltool|(/) Existing|(/) Existing|(/)|(/)| > |auditlogviewer|(/) Existing|(/) Existing|(/)|(/)| > |*Sstable utilities*|| > |sstabledump|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableexpiredblockers|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablelevelreset|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableloader|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablemetadata|(/) enhanced|(/) enhanced|(/)|(!)|Ran in dtests, no > dedicated test| > |sstableofflinerelevel|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablerepairedset|(/) enhanced|(/) enhanced|(/)|(!)|Ran in dtests, no > dedicated test| > |sstablescrub|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstablesplit|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableupgrade|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableutil|(/) enhanced|(/) enhanced|(/)|(/)|| > |sstableverify|(/) enhanced|(/) enhanced|(/)|(/)|| -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15857) Frozen RawTuple is not annotated with frozen in the toString method
[ https://issues.apache.org/jira/browse/CASSANDRA-15857?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-15857: - Reviewers: Robert Stupp, Robert Stupp (was: Robert Stupp) Robert Stupp, Robert Stupp Status: Review In Progress (was: Patch Available) > Frozen RawTuple is not annotated with frozen in the toString method > --- > > Key: CASSANDRA-15857 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15857 > Project: Cassandra > Issue Type: Bug > Components: Legacy/CQL >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Fix For: 4.0, 3.11.x > > Time Spent: 1h 10m > Remaining Estimate: 0h > > All raw types (e.g. RawCollection, RawUT) that supports freezing wraps the > type name with 'frozen<>' in the toString method, except RawTuple. > Therefore, the RawTuple::toString output misses the frozen wrapper. > Tuple is always frozen. However since CASSANDRA-15035, it throws when the > inner tuple is not explicitly wrapped with frozen within a collection. > The method, CQL3Type.Raw::toString, is referenced at multiple places in the > source. For example, referenced in CreateTypeStatement.Raw and involved in > CQLSSTableWriter. Another example is that it is called to produce the > SchemaChange at several AlterSchemaStatement implementations. > A test can prove that missing the frozen wrapper causes exception when > building CQLSSTableWriter for user types defined like below. Note that the > inner tuple is wrapped with frozen in the initial CQL statement. > {code:java} > CREATE TYPE ks.fooType ( f list>> ) > {code} > {code:java} > org.apache.cassandra.exceptions.InvalidRequestException: Non-frozen tuples > are not allowed inside collections: list> > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.throwNestedNonFrozenError(CQL3Type.java:710) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepare(CQL3Type.java:669) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepareInternal(CQL3Type.java:661) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.lambda$prepare$1(Types.java:341) > at > java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) > at > java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) > at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) > at > java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) > at > java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) > at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) > at > java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.prepare(Types.java:342) > at org.apache.cassandra.schema.Types$RawBuilder.build(Types.java:291) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.createTypes(CQLSSTableWriter.java:551) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.build(CQLSSTableWriter.java:527) > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15857) Frozen RawTuple is not annotated with frozen in the toString method
[ https://issues.apache.org/jira/browse/CASSANDRA-15857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17161999#comment-17161999 ] Robert Stupp edited comment on CASSANDRA-15857 at 7/21/20, 12:25 PM: - Hm, not really a fan of sneaking unrelated changes in a 3.11-patch, although it's probably not an issue. Let's stick with the "old" (non-{{@Rule}}) approach in 3.11 for {{CQLSSTableWriterTest}}. Otherwise the PR looks good. was (Author: snazy): Hm, not really a fan of sneaking unrelated changes in a 3.11-patch, although it's probably not an issue. Let's stick with the "old" (non-{{@Rule}}) approach in 3.11 for {{CQLSSTableWriterTest}}. > Frozen RawTuple is not annotated with frozen in the toString method > --- > > Key: CASSANDRA-15857 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15857 > Project: Cassandra > Issue Type: Bug > Components: Legacy/CQL >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Fix For: 4.0, 3.11.x > > Time Spent: 1h 10m > Remaining Estimate: 0h > > All raw types (e.g. RawCollection, RawUT) that supports freezing wraps the > type name with 'frozen<>' in the toString method, except RawTuple. > Therefore, the RawTuple::toString output misses the frozen wrapper. > Tuple is always frozen. However since CASSANDRA-15035, it throws when the > inner tuple is not explicitly wrapped with frozen within a collection. > The method, CQL3Type.Raw::toString, is referenced at multiple places in the > source. For example, referenced in CreateTypeStatement.Raw and involved in > CQLSSTableWriter. Another example is that it is called to produce the > SchemaChange at several AlterSchemaStatement implementations. > A test can prove that missing the frozen wrapper causes exception when > building CQLSSTableWriter for user types defined like below. Note that the > inner tuple is wrapped with frozen in the initial CQL statement. > {code:java} > CREATE TYPE ks.fooType ( f list>> ) > {code} > {code:java} > org.apache.cassandra.exceptions.InvalidRequestException: Non-frozen tuples > are not allowed inside collections: list> > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.throwNestedNonFrozenError(CQL3Type.java:710) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepare(CQL3Type.java:669) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepareInternal(CQL3Type.java:661) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.lambda$prepare$1(Types.java:341) > at > java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) > at > java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) > at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) > at > java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) > at > java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) > at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) > at > java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.prepare(Types.java:342) > at org.apache.cassandra.schema.Types$RawBuilder.build(Types.java:291) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.createTypes(CQLSSTableWriter.java:551) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.build(CQLSSTableWriter.java:527) > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15857) Frozen RawTuple is not annotated with frozen in the toString method
[ https://issues.apache.org/jira/browse/CASSANDRA-15857?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17161999#comment-17161999 ] Robert Stupp commented on CASSANDRA-15857: -- Hm, not really a fan of sneaking unrelated changes in a 3.11-patch, although it's probably not an issue. Let's stick with the "old" (non-{{@Rule}}) approach in 3.11 for {{CQLSSTableWriterTest}}. > Frozen RawTuple is not annotated with frozen in the toString method > --- > > Key: CASSANDRA-15857 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15857 > Project: Cassandra > Issue Type: Bug > Components: Legacy/CQL >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Fix For: 4.0, 3.11.x > > Time Spent: 1h 10m > Remaining Estimate: 0h > > All raw types (e.g. RawCollection, RawUT) that supports freezing wraps the > type name with 'frozen<>' in the toString method, except RawTuple. > Therefore, the RawTuple::toString output misses the frozen wrapper. > Tuple is always frozen. However since CASSANDRA-15035, it throws when the > inner tuple is not explicitly wrapped with frozen within a collection. > The method, CQL3Type.Raw::toString, is referenced at multiple places in the > source. For example, referenced in CreateTypeStatement.Raw and involved in > CQLSSTableWriter. Another example is that it is called to produce the > SchemaChange at several AlterSchemaStatement implementations. > A test can prove that missing the frozen wrapper causes exception when > building CQLSSTableWriter for user types defined like below. Note that the > inner tuple is wrapped with frozen in the initial CQL statement. > {code:java} > CREATE TYPE ks.fooType ( f list>> ) > {code} > {code:java} > org.apache.cassandra.exceptions.InvalidRequestException: Non-frozen tuples > are not allowed inside collections: list> > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.throwNestedNonFrozenError(CQL3Type.java:710) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepare(CQL3Type.java:669) > at > org.apache.cassandra.cql3.CQL3Type$Raw$RawCollection.prepareInternal(CQL3Type.java:661) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.lambda$prepare$1(Types.java:341) > at > java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193) > at > java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382) > at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481) > at > java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471) > at > java.util.stream.ReduceOps$ReduceOp.evaluateSequential(ReduceOps.java:708) > at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) > at > java.util.stream.ReferencePipeline.collect(ReferencePipeline.java:499) > at > org.apache.cassandra.schema.Types$RawBuilder$RawUDT.prepare(Types.java:342) > at org.apache.cassandra.schema.Types$RawBuilder.build(Types.java:291) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.createTypes(CQLSSTableWriter.java:551) > at > org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.build(CQLSSTableWriter.java:527) > {code} -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15928) Throw FSWriteError upon write failures in order to apply DiskFailurePolicy
[ https://issues.apache.org/jira/browse/CASSANDRA-15928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-15928: -- Reviewers: Aleksey Yeschenko, Aleksey Yeschenko (was: Aleksey Yeschenko) Aleksey Yeschenko, Aleksey Yeschenko (was: Aleksey Yeschenko) Status: Review In Progress (was: Patch Available) > Throw FSWriteError upon write failures in order to apply DiskFailurePolicy > -- > > Key: CASSANDRA-15928 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15928 > Project: Cassandra > Issue Type: Bug > Components: Local/Other >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Attachments: intellij_custom_insepction.xml > > Time Spent: 10m > Remaining Estimate: 0h > > Active handling of FSError in AbstractLocalAwareExecutorService was > introduced in CASSANDRA-14993. > > However, in the current code base, there are places that catch the > IOException but not wrap it with FSError, hence not triggering the > DiskFailurePolicy. Especially when the bad disk no longer permits write > operations, it could leads to the mutation stage backup. Therefore I propose > to fix the IOException handling in those cases. > > From the code inspection, those 6 places are current not re-throwing an > IOException with FSWriteError. > # org.apache.cassandra.triggers.CustomClassLoader#addClassPath throws > IOError. Invoked in TriggerExecutor's constructor > # org.apache.cassandra.io.util.FileUtils#renameWithConfirm throws > RuntimeException > # org.apache.cassandra.io.util.FileUtils#truncate throws RuntimeException > # org.apache.cassandra.io.util.FileUtils#write throws RuntimeException > # org.apache.cassandra.db.compaction.LeveledManifest#sendBackToL0 throws > RuntimeException. Invokes rewriteSSTableMetadata > # org.apache.cassandra.io.sstable.SSTableHeaderFix#writeNewMetadata throws > RuntimeException. Invokes rewriteSSTableMetadata -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15928) Throw FSWriteError upon write failures in order to apply DiskFailurePolicy
[ https://issues.apache.org/jira/browse/CASSANDRA-15928?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-15928: -- Reviewers: Aleksey Yeschenko > Throw FSWriteError upon write failures in order to apply DiskFailurePolicy > -- > > Key: CASSANDRA-15928 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15928 > Project: Cassandra > Issue Type: Bug > Components: Local/Other >Reporter: Yifan Cai >Assignee: Yifan Cai >Priority: Normal > Attachments: intellij_custom_insepction.xml > > Time Spent: 10m > Remaining Estimate: 0h > > Active handling of FSError in AbstractLocalAwareExecutorService was > introduced in CASSANDRA-14993. > > However, in the current code base, there are places that catch the > IOException but not wrap it with FSError, hence not triggering the > DiskFailurePolicy. Especially when the bad disk no longer permits write > operations, it could leads to the mutation stage backup. Therefore I propose > to fix the IOException handling in those cases. > > From the code inspection, those 6 places are current not re-throwing an > IOException with FSWriteError. > # org.apache.cassandra.triggers.CustomClassLoader#addClassPath throws > IOError. Invoked in TriggerExecutor's constructor > # org.apache.cassandra.io.util.FileUtils#renameWithConfirm throws > RuntimeException > # org.apache.cassandra.io.util.FileUtils#truncate throws RuntimeException > # org.apache.cassandra.io.util.FileUtils#write throws RuntimeException > # org.apache.cassandra.db.compaction.LeveledManifest#sendBackToL0 throws > RuntimeException. Invokes rewriteSSTableMetadata > # org.apache.cassandra.io.sstable.SSTableHeaderFix#writeNewMetadata throws > RuntimeException. Invokes rewriteSSTableMetadata -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Berenguer Blasi updated CASSANDRA-15792: Reviewers: Berenguer Blasi, Berenguer Blasi (was: Berenguer Blasi) Berenguer Blasi, Berenguer Blasi (was: Berenguer Blasi) Status: Review In Progress (was: Patch Available) > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15792) test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair
[ https://issues.apache.org/jira/browse/CASSANDRA-15792?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Berenguer Blasi updated CASSANDRA-15792: Status: Patch Available (was: Review In Progress) Ready to commit imo, delegating to [~gianluca] > test_speculative_data_request - read_repair_test.TestSpeculativeReadRepair > -- > > Key: CASSANDRA-15792 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15792 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest >Reporter: Ekaterina Dimitrova >Assignee: Gianluca Righetto >Priority: Normal > Fix For: 4.0-beta > > > Failing on the latest trunk here: > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/127/workflows/dfba669d-4a5c-4553-b6a2-85647d0d8d2b/jobs/668/tests > Failing once in 30 times as per Jenkins: > https://jenkins-cm4.apache.org/job/Cassandra-trunk-dtest/69/testReport/dtest.read_repair_test/TestSpeculativeReadRepair/test_speculative_data_request/ -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15814) order by descending on frozen list not working
[ https://issues.apache.org/jira/browse/CASSANDRA-15814?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17161811#comment-17161811 ] Berenguer Blasi commented on CASSANDRA-15814: - Ha great [~adelapena] yes this is what I meant. Now we test all the collection types :-) Despite the wall of failures, which I bet are env related in 2.2, I am +1 on that. Next you should _rebase_ >2.2 imo as I read yesterday about a commit fixing a ton of flaky failures in 3.X. Once that's merged forward, rebased and CI looks good that should be it. We can skip compression and upgrade tests imo? > order by descending on frozen list not working > -- > > Key: CASSANDRA-15814 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15814 > Project: Cassandra > Issue Type: Bug > Components: CQL/Interpreter >Reporter: Felipe Perez >Assignee: Andres de la Peña >Priority: Normal > Time Spent: 1h 40m > Remaining Estimate: 0h > > By creating a table like the following: > {code:java} > CREATE TABLE IF NOT EXISTS software ( > name ascii, > version frozen>, > data ascii, > PRIMARY KEY(name,version) > ) > {code} > It works and version is ordered in an ascending order. But when trying to > order in descending order: > {code:java} > CREATE TABLE IF NOT EXISTS software ( > name ascii, > version frozen>, > data ascii, > PRIMARY KEY(name,version) > ) WITH CLUSTERING ORDER BY (version DESC); > {code} > The table is created normally, but when trying to insert a row: > {code:java} > insert into software(name, version) values ('t1', [2,10,30,40,50]); > {code} > Cassandra throws an error: > {code:java} > InvalidRequest: Error from server: code=2200 [Invalid query] message="Invalid > list literal for version of type frozen>" > {code} > The goal here is that I would like to get the last version of a software. > -- This message was sent by Atlassian Jira (v8.3.4#803005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org