[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17826795#comment-17826795 ]
Brandon Williams edited comment on CASSANDRA-19465 at 3/13/24 6:48 PM: ----------------------------------------------------------------------- Running with the assumption that JMX is the culprit, I [rewrote the check|https://github.com/driftx/cassandra-dtest/commit/6a9f18943b711c378407af7630697cf3eb8b6932] to manually sum the commitlogs, and still no dice, so I added [debugging|https://github.com/driftx/cassandra-dtest/commit/b264a289c9e2f2cd90070e265f492a1f56cc49c3] to print all the commitlogs: {noformat} 16:38:17,433 configuration_test DEBUG added <DirEntry 'CommitLog-7-1710347893973.log'>, os.stat_result(st_mode=33188, st_ino=631414135, st_dev=66308, st_nlink=1, st_uid=1000, st_gid=1000, st_size=33554432, st_atime=1710347893, st_mtime=1710347896, st_ctime=1710347896) 16:38:17,434 configuration_test DEBUG added <DirEntry 'CommitLog-7-1710347893974.log'>, os.stat_result(st_mode=33188, st_ino=631415045, st_dev=66308, st_nlink=1, st_uid=1000, st_gid=1000, st_size=33554432, st_atime=1710347894, st_mtime=1710347894, st_ctime=1710347894) {noformat} Here we find the mystery 64MB, in two 32MB commitlogs. The timestamp in the filenames point to 16:38:13 3/13/2024, which is 4 seconds before these logs. During that time: {noformat} 16:38:10,579 conftest INFO Starting execution of test_change_durable_writes[12-20] at 2024-03-13 16:38:10.579693 16:38:11,471 dtest_setup INFO cluster ccm directory: /tmp/dtest-nf9xjaeg 16:38:11,537 ccm DEBUG Log-watching thread starting. -------------------------------- live log call --------------------------------- 16:38:11,537 ccm DEBUG Log-watching thread starting. 16:38:11,539 ccm DEBUG using balanced tokens for non-vnode cluster 16:38:11,539 ccm DEBUG using balanced tokens for non-vnode cluster ---circle stuff snipped--- 16:38:12,306 ccm INFO node1: using Java 17 for the current invocation 16:38:12,306 ccm INFO node1: using Java 17 for the current invocation 16:38:12,350 ccm INFO Starting node1 with JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 java_version=17 cassandra_version=5.0, install_dir=/home/cassandra/cassandra 16:38:12,350 ccm INFO Starting node1 with JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 java_version=17 cassandra_version=5.0, install_dir=/home/cassandra/cassandra {noformat} The node is simply starting up and nothing should be generating that much data. I'll keep digging on what seems to be a total circle-ism here. was (Author: brandon.williams): Running with the assumption that JMX is the culprit, I [rewrote the check|https://github.com/driftx/cassandra-dtest/commit/6a9f18943b711c378407af7630697cf3eb8b6932] to manually sum the commitlogs, and still no dice, so I added [debugging|https://github.com/driftx/cassandra-dtest/commit/b264a289c9e2f2cd90070e265f492a1f56cc49c3] to print all the commitlogs: {noformat} 16:38:17,433 configuration_test DEBUG added <DirEntry 'CommitLog-7-1710347893973.log'>, os.stat_result(st_mode=33188, st_ino=631414135, st_dev=66308, st_nlink=1, st_uid=1000, st_gid=1000, st_size=33554432, st_atime=1710347893, st_mtime=1710347896, st_ctime=1710347896) 16:38:17,434 configuration_test DEBUG added <DirEntry 'CommitLog-7-1710347893974.log'>, os.stat_result(st_mode=33188, st_ino=631415045, st_dev=66308, st_nlink=1, st_uid=1000, st_gid=1000, st_size=33554432, st_atime=1710347894, st_mtime=1710347894, st_ctime=1710347894) {noformat} Here we find the mystery 64MB, in two 32MB commitlogs. The timestamp in the filenames point to 16:38:13 3/13/2024, which is 4 seconds before these logs. During that time: {noformat} 16:38:10,579 conftest INFO Starting execution of test_change_durable_writes[12-20] at 2024-03-13 16:38:10.579693 16:38:11,471 dtest_setup INFO cluster ccm directory: /tmp/dtest-nf9xjaeg 16:38:11,537 ccm DEBUG Log-watching thread starting. -------------------------------- live log call --------------------------------- 16:38:11,537 ccm DEBUG Log-watching thread starting. 16:38:11,539 ccm DEBUG using balanced tokens for non-vnode cluster 16:38:11,539 ccm DEBUG using balanced tokens for non-vnode cluster ---circle stuff snipped--- 16:38:12,306 ccm INFO node1: using Java 17 for the current invocation 16:38:12,306 ccm INFO node1: using Java 17 for the current invocation 16:38:12,350 ccm INFO Starting node1 with JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 java_version=17 cassandra_version=5.0, install_dir=/home/cassandra/cassandra 16:38:12,350 ccm INFO Starting node1 with JAVA_HOME=/usr/lib/jvm/java-17-openjdk-amd64 java_version=17 cassandra_version=5.0, install_dir=/home/cassandra/cassandra {noformat} The node is simply starting up and nothing shouldn't be generating that much data. I'll keep digging on what seems to be a total circle-ism here. > Test Failure: configuration_test.TestConfiguration.test_change_durable_writes > ----------------------------------------------------------------------------- > > Key: CASSANDRA-19465 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19465 > Project: Cassandra > Issue Type: Bug > Components: Test/dtest/python > Reporter: Berenguer Blasi > Assignee: Brandon Williams > Priority: Normal > Fix For: 5.0-rc, 5.x > > > https://app.circleci.com/pipelines/github/bereng/cassandra/1181/workflows/fe2ac859-f6ba-4f1e-b0b1-e6923b16e874/jobs/39449/tests > {noformat} > self = <configuration_test.TestConfiguration object at 0x7ff4ecd33460> > @pytest.mark.timeout(60*30) > def test_change_durable_writes(self): > """ > @jira_ticket CASSANDRA-9560 > > Test that changes to the DURABLE_WRITES option on keyspaces is > respected in subsequent writes. > > This test starts by writing a dataset to a cluster and asserting that > the commitlogs have been written to. The subsequent test depends on > the assumption that this dataset triggers an fsync. > > After checking this assumption, the test destroys the cluster and > creates a fresh one. Then it tests that DURABLE_WRITES is respected > by: > > - creating a keyspace with DURABLE_WRITES set to false, > - using ALTER KEYSPACE to set its DURABLE_WRITES option to true, > - writing a dataset to this keyspace that is known to trigger a > commitlog fsync, > - asserting that the commitlog has grown in size since the data was > written. > """ > def new_commitlog_cluster_node(): > # writes should block on commitlog fsync > self.fixture_dtest_setup.cluster.populate(1) > node = self.fixture_dtest_setup.cluster.nodelist()[0] > > self.fixture_dtest_setup.cluster.set_batch_commitlog(enabled=True, > use_batch_window = self.fixture_dtest_setup.cluster.version() < '5.0') > > self.fixture_dtest_setup.cluster.start() > return node > > durable_node = new_commitlog_cluster_node() > durable_init_size = commitlog_size(durable_node) > durable_session = self.patient_exclusive_cql_connection(durable_node) > > # test assumption that write_to_trigger_fsync actually triggers a > commitlog fsync > durable_session.execute("CREATE KEYSPACE ks WITH REPLICATION = > {'class': 'SimpleStrategy', 'replication_factor': 1} " > "AND DURABLE_WRITES = true") > durable_session.execute('CREATE TABLE ks.tab (key int PRIMARY KEY, a > int, b int, c int)') > logger.debug('commitlog size diff = ' + > str(commitlog_size(durable_node) - durable_init_size)) > write_to_trigger_fsync(durable_session, 'ks', 'tab') > > assert commitlog_size(durable_node) > durable_init_size, \ > "This test will not work in this environment; > write_to_trigger_fsync does not trigger fsync." > > # get a fresh cluster to work on > durable_session.shutdown() > self.fixture_dtest_setup.cleanup_and_replace_cluster() > > node = new_commitlog_cluster_node() > init_size = commitlog_size(node) > session = self.patient_exclusive_cql_connection(node) > > # set up a keyspace without durable writes, then alter it to use them > session.execute("CREATE KEYSPACE ks WITH REPLICATION = {'class': > 'SimpleStrategy', 'replication_factor': 1} " > "AND DURABLE_WRITES = false") > session.execute('CREATE TABLE ks.tab (key int PRIMARY KEY, a int, b > int, c int)') > session.execute('ALTER KEYSPACE ks WITH DURABLE_WRITES=true') > > write_to_trigger_fsync(session, 'ks', 'tab') > configuration_test.py:113: > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > configuration_test.py:186: in write_to_trigger_fsync > execute_concurrent_with_args(session, > ../env3.8/src/cassandra-driver/cassandra/concurrent.py:238: in > execute_concurrent_with_args > return execute_concurrent(session, zip(cycle((statement,)), parameters), > *args, **kwargs) > ../env3.8/src/cassandra-driver/cassandra/concurrent.py:94: in > execute_concurrent > return executor.execute(concurrency, raise_on_first_error) > ../env3.8/src/cassandra-driver/cassandra/concurrent.py:201: in execute > return super(ConcurrentExecutorListResults, self).execute(concurrency, > fail_fast) > ../env3.8/src/cassandra-driver/cassandra/concurrent.py:120: in execute > return self._results() > ../env3.8/src/cassandra-driver/cassandra/concurrent.py:219: in _results > self._raise(self._exception) > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > exc = NoHostAvailable('Unable to complete the operation against any hosts', > {<Host: 127.0.0.1:9042 datacenter1>: ConnectionShutdown('Connection to > 127.0.0.1:9042 was closed')}) > @staticmethod > def _raise(exc): > if six.PY2 and isinstance(exc, tuple): > (exc_type, value, traceback) = exc > six.reraise(exc_type, value, traceback) > else: > > raise exc > E cassandra.cluster.NoHostAvailable: ('Unable to complete the > operation against any hosts', {<Host: 127.0.0.1:9042 datacenter1>: > ConnectionShutdown('Connection to 127.0.0.1:9042 was closed')}) > ../env3.8/src/cassandra-driver/cassandra/concurrent.py:167: NoHostAvailable > {noformat} > {noformat} > failed on teardown with "Failed: Unexpected error found in node logs (see > stdout for full details). Errors: [[node1] 'ERROR [MutationStage-3] > 2024-03-08 09:36:11,386 StorageProxy.java:1670 - Failed to apply mutation > locally : \njava.lang.IllegalArgumentException: newPosition > limit: (1048644 > > 1048576)\n\tat > java.base/java.nio.Buffer.createPositionException(Buffer.java:341)\n\tat > java.base/java.nio.Buffer.position(Buffer.java:316)\n\tat > java.base/java.nio.ByteBuffer.position(ByteBuffer.java:1516)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:321)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:73)\n\tat > org.apache.cassandra.db.commitlog.CommitLogSegment.allocate(CommitLogSegment.java:216)\n\tat > > org.apache.cassandra.db.commitlog.CommitLogSegmentManagerStandard.allocate(CommitLogSegmentManagerStandard.java:52)\n\tat > org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:307)\n\tat > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.addToCommitLog(CassandraKeyspaceWriteHandler.java:99)\n\tat > > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.beginWrite(CassandraKeyspaceWriteHandler.java:53)\n\tat > org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:612)\n\tat > org.apache.cassandra.db.Keyspace.apply(Keyspace.java:497)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:244)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:264)\n\tat > org.apache.cassandra.service.StorageProxy$4.runMayThrow(StorageProxy.java:1664)\n\tat > > org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2624)\n\tat > > org.apache.cassandra.concurrent.ExecutionFailure$2.run(ExecutionFailure.java:163)\n\tat > org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:143)\n\tat > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n\tat > java.base/java.lang.Thread.run(Thread.java:833)', [node1] 'ERROR > [MutationStage-4] 2024-03-08 09:36:11,386 StorageProxy.java:1670 - Failed to > apply mutation locally : \njava.lang.IllegalArgumentException: newPosition > > limit: (1048722 > 1048576)\n\tat > java.base/java.nio.Buffer.createPositionException(Buffer.java:341)\n\tat > java.base/java.nio.Buffer.position(Buffer.java:316)\n\tat > java.base/java.nio.ByteBuffer.position(ByteBuffer.java:1516)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:321)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:73)\n\tat > org.apache.cassandra.db.commitlog.CommitLogSegment.allocate(CommitLogSegment.java:216)\n\tat > > org.apache.cassandra.db.commitlog.CommitLogSegmentManagerStandard.allocate(CommitLogSegmentManagerStandard.java:52)\n\tat > org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:307)\n\tat > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.addToCommitLog(CassandraKeyspaceWriteHandler.java:99)\n\tat > > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.beginWrite(CassandraKeyspaceWriteHandler.java:53)\n\tat > org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:612)\n\tat > org.apache.cassandra.db.Keyspace.apply(Keyspace.java:497)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:244)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:264)\n\tat > org.apache.cassandra.service.StorageProxy$4.runMayThrow(StorageProxy.java:1664)\n\tat > > org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2624)\n\tat > > org.apache.cassandra.concurrent.ExecutionFailure$2.run(ExecutionFailure.java:163)\n\tat > org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:143)\n\tat > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n\tat > java.base/java.lang.Thread.run(Thread.java:833)', [node1] 'ERROR > [COMMIT-LOG-WRITER] 2024-03-08 09:36:11,387 StorageService.java:631 - > Stopping native transport', [node1] 'ERROR [COMMIT-LOG-WRITER] 2024-03-08 > 09:36:11,392 StorageService.java:636 - Stopping gossiper']" > Unexpected error found in node logs (see stdout for full details). Errors: > [[node1] 'ERROR [MutationStage-3] 2024-03-08 09:36:11,386 > StorageProxy.java:1670 - Failed to apply mutation locally : > \njava.lang.IllegalArgumentException: newPosition > limit: (1048644 > > 1048576)\n\tat > java.base/java.nio.Buffer.createPositionException(Buffer.java:341)\n\tat > java.base/java.nio.Buffer.position(Buffer.java:316)\n\tat > java.base/java.nio.ByteBuffer.position(ByteBuffer.java:1516)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:321)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:73)\n\tat > org.apache.cassandra.db.commitlog.CommitLogSegment.allocate(CommitLogSegment.java:216)\n\tat > > org.apache.cassandra.db.commitlog.CommitLogSegmentManagerStandard.allocate(CommitLogSegmentManagerStandard.java:52)\n\tat > org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:307)\n\tat > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.addToCommitLog(CassandraKeyspaceWriteHandler.java:99)\n\tat > > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.beginWrite(CassandraKeyspaceWriteHandler.java:53)\n\tat > org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:612)\n\tat > org.apache.cassandra.db.Keyspace.apply(Keyspace.java:497)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:244)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:264)\n\tat > org.apache.cassandra.service.StorageProxy$4.runMayThrow(StorageProxy.java:1664)\n\tat > > org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2624)\n\tat > > org.apache.cassandra.concurrent.ExecutionFailure$2.run(ExecutionFailure.java:163)\n\tat > org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:143)\n\tat > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n\tat > java.base/java.lang.Thread.run(Thread.java:833)', [node1] 'ERROR > [MutationStage-4] 2024-03-08 09:36:11,386 StorageProxy.java:1670 - Failed to > apply mutation locally : \njava.lang.IllegalArgumentException: newPosition > > limit: (1048722 > 1048576)\n\tat > java.base/java.nio.Buffer.createPositionException(Buffer.java:341)\n\tat > java.base/java.nio.Buffer.position(Buffer.java:316)\n\tat > java.base/java.nio.ByteBuffer.position(ByteBuffer.java:1516)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:321)\n\tat > java.base/java.nio.MappedByteBuffer.position(MappedByteBuffer.java:73)\n\tat > org.apache.cassandra.db.commitlog.CommitLogSegment.allocate(CommitLogSegment.java:216)\n\tat > > org.apache.cassandra.db.commitlog.CommitLogSegmentManagerStandard.allocate(CommitLogSegmentManagerStandard.java:52)\n\tat > org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:307)\n\tat > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.addToCommitLog(CassandraKeyspaceWriteHandler.java:99)\n\tat > > org.apache.cassandra.db.CassandraKeyspaceWriteHandler.beginWrite(CassandraKeyspaceWriteHandler.java:53)\n\tat > org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:612)\n\tat > org.apache.cassandra.db.Keyspace.apply(Keyspace.java:497)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:244)\n\tat > org.apache.cassandra.db.Mutation.apply(Mutation.java:264)\n\tat > org.apache.cassandra.service.StorageProxy$4.runMayThrow(StorageProxy.java:1664)\n\tat > > org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2624)\n\tat > > org.apache.cassandra.concurrent.ExecutionFailure$2.run(ExecutionFailure.java:163)\n\tat > org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:143)\n\tat > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n\tat > java.base/java.lang.Thread.run(Thread.java:833)', [node1] 'ERROR > [COMMIT-LOG-WRITER] 2024-03-08 09:36:11,387 StorageService.java:631 - > Stopping native transport', [node1] 'ERROR [COMMIT-LOG-WRITER] 2024-03-08 > 09:36:11,392 StorageService.java:636 - Stopping gossiper'] > {noformat} -- This message was sent by Atlassian Jira (v8.20.10#820010) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org