[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17827105#comment-17827105 ] Stefan Miklosovic commented on CASSANDRA-19465: --- +1 > 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 = > @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() >
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17827092#comment-17827092 ] Brandon Williams commented on CASSANDRA-19465: -- To solve these issues: * pin python packages (CASSANDRA-19469) * allow oversized mutation test to run everywhere (CASSANDRA-19470) * rewrite test to actually test durable writes For that last point, the test was doing normal writes, tearing down the cluster and rebuilding without durable writes, enabling durable writes, and testing normal writes again. I think it makes more sense to leave durable writes disabled and ensure the commitlog was not written, which I've done [here|https://github.com/driftx/cassandra-dtest/commit/2305bd2c819fc80f552c9a6dd3861ed6a961060a], as well as removing the unneeded complexity of fixtures. ||Branch||CI|| |[4.0|https://github.com/driftx/cassandra/tree/CASSANDRA-19465-4.0]|[repeats|https://app.circleci.com/pipelines/github/driftx/cassandra/1514/workflows/c85ba6e7-1657-4d9e-8e52-480aa9d346a6/jobs/76850]| |[4.1|https://github.com/driftx/cassandra/tree/CASSANDRA-19465-4.1]|[repeats|https://app.circleci.com/pipelines/github/driftx/cassandra/1515/workflows/0c650aed-cc33-47f7-b05f-e7c2976e314f/jobs/76851]| |[5.0|https://github.com/driftx/cassandra/tree/CASSANDRA-19465-5.0]|[repeats|https://app.circleci.com/pipelines/github/driftx/cassandra/1513/workflows/31639484-078b-4dbb-85a9-b36b60cf13bf/jobs/76852]| > 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 = > @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
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17827038#comment-17827038 ] Stefan Miklosovic commented on CASSANDRA-19465: --- So we made some progress after all when it comes to test itself at least. The problem is that, we do not know exactly when yet, pytest broke this thing (1). It seems that when the original patch arrives here (2) what we see from the logs, with help of Brandon's debugging changes, it will have a commit log segment of size 32MB instead of 1MB. Huh right? Yesterday the test was privately rewritten in such a way that (1) is removed and we just simply set 1MB manually, it just all passes (3). I think what happened is that as we had a range version for pytest in requirements.txt like >= 6.5.0, CircleCI was constantly resolving the latest pytest available because it was downloading it all the time from scratch, but if somebody had already installed pytest of version >=6.5.0, then whatever goes, and that version might still have unbroken integration with the fixture in (1). Brandon confirmed that had locally pytest of 7.4.4 where this fixture thing was still doing its job. I think we should not have ranges in requirements.txt and it should be on some concrete version, like 8.0.2 for now, to avoid differences like this. When it comes to the error itself, that mutation was not oversized because it would throw way before that, as Brandon already mentioned. The exception is thrown here (4) in {code} buffer.duplicate().position(position).limit(position + size)); {code} as it says that position is > then limit in ByteBuffer (newPosition > limit: (1048644 > 1048576)). This should never happen in the first place because in the method in (4), we are trying to allocate some space in a commit log: {code} int position = allocate(size); if (position < 0) { opGroup.close(); return null; } {code} and the deal here is that if position is -1, saying it was not able to allocate it because it would not fit into the current segment, then it would return null and it keeps trying (5) until it can put it into a new segment. I personally think that as it can not be oversize (it would throw) there is some bug in (4), but given the fact that this was in production for years without any problem, maybe it had something to do with the fact that we were not actually setting the commit log segment size to 1MB properly for both cases in the test ... of if the fixture was broken, that it was not clearing commitlogs for the second node etc ... (1) https://github.com/apache/cassandra-dtest/blob/trunk/configuration_test.py#L18-L23 (2) https://github.com/apache/cassandra-dtest/blob/trunk/configuration_test.py#L101-L114 (3) https://app.circleci.com/pipelines/github/driftx/cassandra/1512/workflows/eb88ec9d-8ef7-41e0-b1d9-7ea6eafe9a80/jobs/76562 (4) https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java#L216 (5) https://github.com/apache/cassandra/blob/cassandra-5.0/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerStandard.java#L52 > 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 = > @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
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17826957#comment-17826957 ] Berenguer Blasi commented on CASSANDRA-19465: - I looked a bit into this yesterday and made very little/no progress. Not sure what is going on either... > 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 = > @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) >
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17826861#comment-17826861 ] Brandon Williams commented on CASSANDRA-19465: -- Discussed this with [~stefan.miklosovic] a bit, and it looks like he repro'd the 64MB problem with no test modification [here|https://app.circleci.com/pipelines/github/instaclustr/cassandra/4001/workflows/a628b12c-f6d9-4898-9507-45e3c3d222f0/jobs/204086/parallel-runs/23?filterBy=ALL]. So now I guess we have two problems, but even more he pointed out the original exception shouldn't happen from a large mutation, it should be caught sooner. It looks like that is correct since the [large mutation|https://github.com/apache/cassandra-dtest/blob/trunk/write_failures_test.py#L242] test is passing and doesn't account for that exception. This is some kind of progress but we're still stumped. > 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 = > @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: > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ >
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17826824#comment-17826824 ] Brandon Williams commented on CASSANDRA-19465: -- On a fresh start we will have two 32MB (by default) commitlogs present, but then...why doesn't this fail everywhere, all the time? I don't understand why this only fails in circle. > 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 = > @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
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17826795#comment-17826795 ] Brandon Williams commented on CASSANDRA-19465: -- 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 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 , 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 , 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 = > @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':
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17825722#comment-17825722 ] Brandon Williams commented on CASSANDRA-19465: -- >From the error we can tell that a mutation > 512k was sent, but with the >commitlog segment size set to 1MB, that is too large to be written. The >simplest thing to do would be increase the segment size to 2MB, but that's >where things get strange. This Just Works on my machines, but circle >[complains|https://app.circleci.com/pipelines/github/driftx/cassandra/1505/workflows/cd6734d4-f95a-45b2-bc65-d6341ae8484d/jobs/75819] > that not enough data is written, saying that we are on the cusp of the >segment size (assert 67108864 > 67108864). I thought maybe we need more iterations, but [nope|https://app.circleci.com/pipelines/github/driftx/cassandra/1505/workflows/9d313011-cd3b-4e87-9862-837846e67f57/jobs/75821], still on the cusp. Let's try more iterations with larger values, and add a debug to see how much leeway we show locally... [fails|https://app.circleci.com/pipelines/github/driftx/cassandra/1507/workflows/7483d772-45ca-4b67-83be-d8081656cbd1/jobs/76033] still on the cusp in circle, locally I see: {noformat} DEBUGconfiguration_test:configuration_test.py:94 commitlog size diff = 0 INFO tools.jmxutils:jmxutils.py:208 Port 8778 open for jolokia INFO tools.jmxutils:jmxutils.py:220 Jolokia successful on try 0 DEBUGconfiguration_test:configuration_test.py:96 commitlog size diff = 2097152 {noformat} And that's where I'm currently at - ~2MB over when run locally, but nothing can get off the 1MB cusp in circle. > 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 = > @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() > >
[jira] [Commented] (CASSANDRA-19465) Test Failure: configuration_test.TestConfiguration.test_change_durable_writes
[ https://issues.apache.org/jira/browse/CASSANDRA-19465?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17825636#comment-17825636 ] Brandon Williams commented on CASSANDRA-19465: -- I think the possibility for flakiness has always been there, since it appears what has happened here is that we've sent a request over 512k when the commitlog is set to 1MB, so the mutation can't be written. The simplest thing to do is bump the commitlog to 2MB, which I've done and repeat CI is running [here|https://app.circleci.com/pipelines/github/driftx/cassandra/1505/workflows/cd6734d4-f95a-45b2-bc65-d6341ae8484d/jobs/75819]. > 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 = > @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), >