[jira] [Updated] (CASSANDRA-19633) Replaced node is stuck in a loop calculating ranges
[ https://issues.apache.org/jira/browse/CASSANDRA-19633?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19633: Test and Documentation Plan: ci runs Status: Patch Available (was: Open) opened PRs for 4.0,4.1,5.0,trunk will post CI runs soon > Replaced node is stuck in a loop calculating ranges > --- > > Key: CASSANDRA-19633 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19633 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jai Bheemsen Rao Dhanwada >Assignee: Marcus Eriksson >Priority: Normal > Labels: Bootstrap > Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x > > Attachments: result1.html > > Time Spent: 40m > Remaining Estimate: 0h > > Hello, > > I am running into an issue where in a node that is replacing a dead > (non-seed) node is stuck in calculating ranges forever. It eventually > succeeds, however the time taken for calculating the ranges is not constant. > I do sometimes see that it takes 24 hours to calculate ranges for each > keyspace. Attached the flume graph of the cassandra process during this time, > which points to the below code. > {code:java} > Multimap> > getRangeFetchMapForNonTrivialRanges() > { > //Get the graph with edges between ranges and their source endpoints > MutableCapacityGraph graph = getGraph(); > //Add source and destination vertex and edges > addSourceAndDestination(graph, getDestinationLinkCapacity(graph)); > int flow = 0; > MaximumFlowAlgorithmResult> result = > null; > //We might not be working on all ranges > while (flow < getTotalRangeVertices(graph)) > { > if (flow > 0) > { //We could not find a path with previous graph. Bump the capacity b/w > endpoint vertices and destination by 1 incrementCapacity(graph, 1); } > MaximumFlowAlgorithm fordFulkerson = > FordFulkersonAlgorithm.getInstance(DFSPathFinder.getInstance()); > result = fordFulkerson.calc(graph, sourceVertex, destinationVertex, > IntegerNumberSystem.getInstance()); > int newFlow = result.calcTotalFlow(); > assert newFlow > flow; //We are not making progress which should not happen > flow = newFlow; > } > return getRangeFetchMapFromGraphResult(graph, result); > } > {code} > Digging through the logs, I see the below log line for a given keyspace > `system_auth` > {code:java} > INFO [main] 2024-05-10 17:35:02,489 RangeStreamer.java:330 - Bootstrap: range > Full(/10.135.56.214:7000,(5080189126057290696,5081324396311791613]) exists on > Full(/10.135.56.157:7000,(5080189126057290696,5081324396311791613]) for > keyspace system_auth{code} > corresponding code: > {code:java} > for (Map.Entry entry : fetchMap.flattenEntries()) > logger.info("{}: range {} exists on {} for keyspace {}", description, > entry.getKey(), entry.getValue(), keyspaceName);{code} > BUT do not see the below line for the corresponding keyspace > {code:java} > RangeStreamer.java:606 - Output from RangeFetchMapCalculator for > keyspace{code} > this means the code it's stuck in `getRangeFetchMap();` > {code:java} > Multimap> rangeFetchMapMap = > calculator.getRangeFetchMap(); > logger.info("Output from RangeFetchMapCalculator for keyspace {}", > keyspace);{code} > Here is the cluster topology: > * Cassandra version: 4.0.12 > * # of nodes: 190 > * Tokens (vnodes): 128 > Initial hypothesis was that the graph calculation was taking longer due to > the combination of nodes + tokens + tables but in the same cluster I see one > of the node joined without any issues. > wondering if I am hitting a bug causing it to work sometimes but get into an > infinite loop some times? > Please let me know if you need any other details and appreciate any pointers > to debug this further. -- 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
[jira] [Commented] (CASSANDRA-19599) Remove unused config params for out of range token requests
[ https://issues.apache.org/jira/browse/CASSANDRA-19599?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17846655#comment-17846655 ] Marcus Eriksson commented on CASSANDRA-19599: - +1 > Remove unused config params for out of range token requests > --- > > Key: CASSANDRA-19599 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19599 > Project: Cassandra > Issue Type: Task > Components: Local/Config >Reporter: Sam Tunnicliffe >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html > > > The fields {{log_out_of_token_range_requests}} and > {{reject_out_of_token_range_requests}} in {{Config.java}} have never actually > been used and are just vestiges from early development on CEP-21. > We should remove them and the related accessors in {{DatabaseDescriptor}}. -- 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
[jira] [Comment Edited] (CASSANDRA-19633) Replaced node is stuck in a loop calculating ranges
[ https://issues.apache.org/jira/browse/CASSANDRA-19633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17846595#comment-17846595 ] Marcus Eriksson edited comment on CASSANDRA-19633 at 5/15/24 12:54 PM: --- I now have this reproducing locally using tokens/datacenters provided by [~jaid], thanks! This problem is indeed that we only provide one source for each range post CASSANDRA-14405. The 4650-optimisation is done by setting up a graph where each source -> destination edge has the capacity of one - optimally we would stream each range from a unique source. If we can't find a flow using the capacity of one, we bump the capacity on all edges to two and try to calculate the flow again, and repeat this bumping until we have found a flow where we can [stream all ranges|https://github.com/apache/cassandra/blob/75794540573b6f0c39094b5448fe73326e14e058/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java#L134-L148]. The problem is though that with only a single source per range we are very far away from being able to find a flow with edge-capacities of 1, so in this cluster this calculation is done 1000+ times, and each calculation takes several minutes (per keyspace). And the result is terrible anyway because we end up streaming from only two sources. Removing [this|https://github.com/apache/cassandra/blob/6bae4f76fb043b4c3a3886178b5650b280e9a50b/src/java/org/apache/cassandra/dht/RangeStreamer.java#L531] line allows us to only do the calculation once and we stream from 191 sources. It still takes several minutes to do the calculation, but we most likely save more time due to quicker streaming later. But as the comment says some downstream uses of {{sources}} requires it to be only a single node, so I'll need to fix those places before submitting a patch. was (Author: krummas): I now have this reproducing locally using tokens/datacenters provided by [~jaid], thanks! This problem is indeed that we only provide one source for each range post CASSANDRA-14405. The 4650-optimisation is done by setting up a graph where each source -> destination edge has the capacity of one - optimally we would stream each range from a unique source. If we can't find a flow using the capacity of one, we bump the capacity on all edges to two and try to calculate the flow again, and repeat this bumping until we have found a flow where we can [stream all ranges|https://github.com/apache/cassandra/blob/75794540573b6f0c39094b5448fe73326e14e058/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java#L134-L148]. The problem is though that with only a single source per range we are very far away from being able to find a flow with edge-capacities of 1, so in this cluster do this calculation 1000+ times, and each calculation takes several minutes (per keyspace). And the result is terrible anyway because we end up streaming from only two sources. Removing [this|https://github.com/apache/cassandra/blob/6bae4f76fb043b4c3a3886178b5650b280e9a50b/src/java/org/apache/cassandra/dht/RangeStreamer.java#L531] line allows us to only do the calculation once and we stream from 191 sources. It still takes several minutes to do the calculation, but we most likely save more time due to quicker streaming later. But as the comment says some downstream uses of {{sources}} requires it to be only a single node, so I'll need to fix those places before submitting a patch. > Replaced node is stuck in a loop calculating ranges > --- > > Key: CASSANDRA-19633 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19633 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jai Bheemsen Rao Dhanwada >Assignee: Marcus Eriksson >Priority: Normal > Labels: Bootstrap > Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x > > Attachments: result1.html > > > Hello, > > I am running into an issue where in a node that is replacing a dead > (non-seed) node is stuck in calculating ranges forever. It eventually > succeeds, however the time taken for calculating the ranges is not constant. > I do sometimes see that it takes 24 hours to calculate ranges for each > keyspace. Attached the flume graph of the cassandra process during this time, > which points to the below code. > {code:java} > Multimap> > getRangeFetchMapForNonTrivialRanges() > { > //Get the graph with edges between ranges and their source endpoints > MutableCapacityGraph graph = getGraph(); > //Add source and destination vertex and edges > addSourceAndDestination(graph, getDestinationLinkCapacity(graph)); > int flow = 0; > MaximumFlowAlgorithmResult> result = > null; > //We might not be working on all ranges > while (flow <
[jira] [Commented] (CASSANDRA-19633) Replaced node is stuck in a loop calculating ranges
[ https://issues.apache.org/jira/browse/CASSANDRA-19633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17846595#comment-17846595 ] Marcus Eriksson commented on CASSANDRA-19633: - I now have this reproducing locally using tokens/datacenters provided by [~jaid], thanks! This problem is indeed that we only provide one source for each range post CASSANDRA-14405. The 4650-optimisation is done by setting up a graph where each source -> destination edge has the capacity of one - optimally we would stream each range from a unique source. If we can't find a flow using the capacity of one, we bump the capacity on all edges to two and try to calculate the flow again, and repeat this bumping until we have found a flow where we can [stream all ranges|https://github.com/apache/cassandra/blob/75794540573b6f0c39094b5448fe73326e14e058/src/java/org/apache/cassandra/dht/RangeFetchMapCalculator.java#L134-L148]. The problem is though that with only a single source per range we are very far away from being able to find a flow with edge-capacities of 1, so in this cluster do this calculation 1000+ times, and each calculation takes several minutes (per keyspace). And the result is terrible anyway because we end up streaming from only two sources. Removing [this|https://github.com/apache/cassandra/blob/6bae4f76fb043b4c3a3886178b5650b280e9a50b/src/java/org/apache/cassandra/dht/RangeStreamer.java#L531] line allows us to only do the calculation once and we stream from 191 sources. It still takes several minutes to do the calculation, but we most likely save more time due to quicker streaming later. But as the comment says some downstream uses of {{sources}} requires it to be only a single node, so I'll need to fix those places before submitting a patch. > Replaced node is stuck in a loop calculating ranges > --- > > Key: CASSANDRA-19633 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19633 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jai Bheemsen Rao Dhanwada >Assignee: Marcus Eriksson >Priority: Normal > Labels: Bootstrap > Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x > > Attachments: result1.html > > > Hello, > > I am running into an issue where in a node that is replacing a dead > (non-seed) node is stuck in calculating ranges forever. It eventually > succeeds, however the time taken for calculating the ranges is not constant. > I do sometimes see that it takes 24 hours to calculate ranges for each > keyspace. Attached the flume graph of the cassandra process during this time, > which points to the below code. > {code:java} > Multimap> > getRangeFetchMapForNonTrivialRanges() > { > //Get the graph with edges between ranges and their source endpoints > MutableCapacityGraph graph = getGraph(); > //Add source and destination vertex and edges > addSourceAndDestination(graph, getDestinationLinkCapacity(graph)); > int flow = 0; > MaximumFlowAlgorithmResult> result = > null; > //We might not be working on all ranges > while (flow < getTotalRangeVertices(graph)) > { > if (flow > 0) > { //We could not find a path with previous graph. Bump the capacity b/w > endpoint vertices and destination by 1 incrementCapacity(graph, 1); } > MaximumFlowAlgorithm fordFulkerson = > FordFulkersonAlgorithm.getInstance(DFSPathFinder.getInstance()); > result = fordFulkerson.calc(graph, sourceVertex, destinationVertex, > IntegerNumberSystem.getInstance()); > int newFlow = result.calcTotalFlow(); > assert newFlow > flow; //We are not making progress which should not happen > flow = newFlow; > } > return getRangeFetchMapFromGraphResult(graph, result); > } > {code} > Digging through the logs, I see the below log line for a given keyspace > `system_auth` > {code:java} > INFO [main] 2024-05-10 17:35:02,489 RangeStreamer.java:330 - Bootstrap: range > Full(/10.135.56.214:7000,(5080189126057290696,5081324396311791613]) exists on > Full(/10.135.56.157:7000,(5080189126057290696,5081324396311791613]) for > keyspace system_auth{code} > corresponding code: > {code:java} > for (Map.Entry entry : fetchMap.flattenEntries()) > logger.info("{}: range {} exists on {} for keyspace {}", description, > entry.getKey(), entry.getValue(), keyspaceName);{code} > BUT do not see the below line for the corresponding keyspace > {code:java} > RangeStreamer.java:606 - Output from RangeFetchMapCalculator for > keyspace{code} > this means the code it's stuck in `getRangeFetchMap();` > {code:java} > Multimap> rangeFetchMapMap = > calculator.getRangeFetchMap(); > logger.info("Output from RangeFetchMapCalculator for keyspace {}", > keyspace);{code} > Here is the cluster topology: > * Cassandra version: 4.0.12 > * # of nodes: 190 > * Tokens
[jira] [Commented] (CASSANDRA-19633) Replaced node is stuck in a loop calculating ranges
[ https://issues.apache.org/jira/browse/CASSANDRA-19633?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17846275#comment-17846275 ] Marcus Eriksson commented on CASSANDRA-19633: - I've started looking at this and locally it produces a similar flame graph if I have a 200 node cluster, with 128 tokens/node and a keyspace with a large RF, I wrote up a temporary workaround patch to avoid doing the optimisation when on vnodes: https://github.com/krummas/cassandra/commit/70e0d9972a94fa63fa991315412e79ef033561b2 But as I looked in to this I think the optimisation from CASSANDRA-4650 was broken by CASSANDRA-14405 (transient replication) - before that we made the 4650-optimisation based on all sources available for a range [1], now we only supply a single source per range we need to stream [2] I'll try to find some time to look in to this soon. [1] https://github.com/apache/cassandra/blob/bf911cc6a852f9ef068318a3545611d9daa5112c/src/java/org/apache/cassandra/dht/RangeStreamer.java#L189-L197 [2] https://github.com/apache/cassandra/blob/6bae4f76fb043b4c3a3886178b5650b280e9a50b/src/java/org/apache/cassandra/dht/RangeStreamer.java#L531 > Replaced node is stuck in a loop calculating ranges > --- > > Key: CASSANDRA-19633 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19633 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jai Bheemsen Rao Dhanwada >Assignee: Marcus Eriksson >Priority: Normal > Labels: Bootstrap > Fix For: 4.0.x, 4.1.x, 5.0.x, 5.1-alpha1 > > Attachments: result1.html > > > Hello, > > I am running into an issue where in a node that is replacing a dead > (non-seed) node is stuck in calculating ranges forever. It eventually > succeeds, however the time taken for calculating the ranges is not constant. > I do sometimes see that it takes 24 hours to calculate ranges for each > keyspace. Attached the flume graph of the cassandra process during this time, > which points to the below code. > {code:java} > Multimap> > getRangeFetchMapForNonTrivialRanges() > { > //Get the graph with edges between ranges and their source endpoints > MutableCapacityGraph graph = getGraph(); > //Add source and destination vertex and edges > addSourceAndDestination(graph, getDestinationLinkCapacity(graph)); > int flow = 0; > MaximumFlowAlgorithmResult> result = > null; > //We might not be working on all ranges > while (flow < getTotalRangeVertices(graph)) > { > if (flow > 0) > { //We could not find a path with previous graph. Bump the capacity b/w > endpoint vertices and destination by 1 incrementCapacity(graph, 1); } > MaximumFlowAlgorithm fordFulkerson = > FordFulkersonAlgorithm.getInstance(DFSPathFinder.getInstance()); > result = fordFulkerson.calc(graph, sourceVertex, destinationVertex, > IntegerNumberSystem.getInstance()); > int newFlow = result.calcTotalFlow(); > assert newFlow > flow; //We are not making progress which should not happen > flow = newFlow; > } > return getRangeFetchMapFromGraphResult(graph, result); > } > {code} > Digging through the logs, I see the below log line for a given keyspace > `system_auth` > {code:java} > INFO [main] 2024-05-10 17:35:02,489 RangeStreamer.java:330 - Bootstrap: range > Full(/10.135.56.214:7000,(5080189126057290696,5081324396311791613]) exists on > Full(/10.135.56.157:7000,(5080189126057290696,5081324396311791613]) for > keyspace system_auth{code} > corresponding code: > {code:java} > for (Map.Entry entry : fetchMap.flattenEntries()) > logger.info("{}: range {} exists on {} for keyspace {}", description, > entry.getKey(), entry.getValue(), keyspaceName);{code} > BUT do not see the below line for the corresponding keyspace > {code:java} > RangeStreamer.java:606 - Output from RangeFetchMapCalculator for > keyspace{code} > this means the code it's stuck in `getRangeFetchMap();` > {code:java} > Multimap> rangeFetchMapMap = > calculator.getRangeFetchMap(); > logger.info("Output from RangeFetchMapCalculator for keyspace {}", > keyspace);{code} > Here is the cluster topology: > * Cassandra version: 4.0.12 > * # of nodes: 190 > * Tokens (vnodes): 128 > Initial hypothesis was that the graph calculation was taking longer due to > the combination of nodes + tokens + tables but in the same cluster I see one > of the node joined without any issues. > wondering if I am hitting a bug causing it to work sometimes but get into an > infinite loop some times? > Please let me know if you need any other details and appreciate any pointers > to debug this further. -- This message was sent by Atlassian Jira (v8.20.10#820010) - To unsubscribe, e-mail:
[jira] [Assigned] (CASSANDRA-19633) Replaced node is stuck in a loop calculating ranges
[ https://issues.apache.org/jira/browse/CASSANDRA-19633?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson reassigned CASSANDRA-19633: --- Assignee: Marcus Eriksson > Replaced node is stuck in a loop calculating ranges > --- > > Key: CASSANDRA-19633 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19633 > Project: Cassandra > Issue Type: Bug >Reporter: Jai Bheemsen Rao Dhanwada >Assignee: Marcus Eriksson >Priority: Normal > Labels: Bootstrap > Attachments: result1.html > > > Hello, > > I am running into an issue where in a node that is replacing a dead > (non-seed) node is stuck in calculating ranges forever. It eventually > succeeds, however the time taken for calculating the ranges is not constant. > I do sometimes see that it takes 24 hours to calculate ranges for each > keyspace. Attached the flume graph of the cassandra process during this time, > which points to the below code. > {code:java} > Multimap> > getRangeFetchMapForNonTrivialRanges() > { > //Get the graph with edges between ranges and their source endpoints > MutableCapacityGraph graph = getGraph(); > //Add source and destination vertex and edges > addSourceAndDestination(graph, getDestinationLinkCapacity(graph)); > int flow = 0; > MaximumFlowAlgorithmResult> result = > null; > //We might not be working on all ranges > while (flow < getTotalRangeVertices(graph)) > { > if (flow > 0) > { //We could not find a path with previous graph. Bump the capacity b/w > endpoint vertices and destination by 1 incrementCapacity(graph, 1); } > MaximumFlowAlgorithm fordFulkerson = > FordFulkersonAlgorithm.getInstance(DFSPathFinder.getInstance()); > result = fordFulkerson.calc(graph, sourceVertex, destinationVertex, > IntegerNumberSystem.getInstance()); > int newFlow = result.calcTotalFlow(); > assert newFlow > flow; //We are not making progress which should not happen > flow = newFlow; > } > return getRangeFetchMapFromGraphResult(graph, result); > } > {code} > Digging through the logs, I see the below log line for a given keyspace > `system_auth` > {code:java} > INFO [main] 2024-05-10 17:35:02,489 RangeStreamer.java:330 - Bootstrap: range > Full(/10.135.56.214:7000,(5080189126057290696,5081324396311791613]) exists on > Full(/10.135.56.157:7000,(5080189126057290696,5081324396311791613]) for > keyspace system_auth{code} > corresponding code: > {code:java} > for (Map.Entry entry : fetchMap.flattenEntries()) > logger.info("{}: range {} exists on {} for keyspace {}", description, > entry.getKey(), entry.getValue(), keyspaceName);{code} > BUT do not see the below line for the corresponding keyspace > {code:java} > RangeStreamer.java:606 - Output from RangeFetchMapCalculator for > keyspace{code} > this means the code it's stuck in `getRangeFetchMap();` > {code:java} > Multimap> rangeFetchMapMap = > calculator.getRangeFetchMap(); > logger.info("Output from RangeFetchMapCalculator for keyspace {}", > keyspace);{code} > Here is the cluster topology: > * Cassandra version: 4.0.12 > * # of nodes: 190 > * Tokens (vnodes): 128 > Initial hypothesis was that the graph calculation was taking longer due to > the combination of nodes + tokens + tables but in the same cluster I see one > of the node joined without any issues. > wondering if I am hitting a bug causing it to work sometimes but get into an > infinite loop some times? > Please let me know if you need any other details and appreciate any pointers > to debug this further. -- 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
[jira] [Updated] (CASSANDRA-19633) Replaced node is stuck in a loop calculating ranges
[ https://issues.apache.org/jira/browse/CASSANDRA-19633?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19633: Bug Category: Parent values: Degradation(12984)Level 1 values: Slow Use Case(12996) (was: Parent values: Degradation(12984)) Complexity: Normal Component/s: Consistency/Bootstrap and Decommission Fix Version/s: 4.0.x 4.1.x 5.0.x 5.1-alpha1 Severity: Normal Status: Open (was: Triage Needed) > Replaced node is stuck in a loop calculating ranges > --- > > Key: CASSANDRA-19633 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19633 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Bootstrap and Decommission >Reporter: Jai Bheemsen Rao Dhanwada >Assignee: Marcus Eriksson >Priority: Normal > Labels: Bootstrap > Fix For: 4.0.x, 4.1.x, 5.0.x, 5.1-alpha1 > > Attachments: result1.html > > > Hello, > > I am running into an issue where in a node that is replacing a dead > (non-seed) node is stuck in calculating ranges forever. It eventually > succeeds, however the time taken for calculating the ranges is not constant. > I do sometimes see that it takes 24 hours to calculate ranges for each > keyspace. Attached the flume graph of the cassandra process during this time, > which points to the below code. > {code:java} > Multimap> > getRangeFetchMapForNonTrivialRanges() > { > //Get the graph with edges between ranges and their source endpoints > MutableCapacityGraph graph = getGraph(); > //Add source and destination vertex and edges > addSourceAndDestination(graph, getDestinationLinkCapacity(graph)); > int flow = 0; > MaximumFlowAlgorithmResult> result = > null; > //We might not be working on all ranges > while (flow < getTotalRangeVertices(graph)) > { > if (flow > 0) > { //We could not find a path with previous graph. Bump the capacity b/w > endpoint vertices and destination by 1 incrementCapacity(graph, 1); } > MaximumFlowAlgorithm fordFulkerson = > FordFulkersonAlgorithm.getInstance(DFSPathFinder.getInstance()); > result = fordFulkerson.calc(graph, sourceVertex, destinationVertex, > IntegerNumberSystem.getInstance()); > int newFlow = result.calcTotalFlow(); > assert newFlow > flow; //We are not making progress which should not happen > flow = newFlow; > } > return getRangeFetchMapFromGraphResult(graph, result); > } > {code} > Digging through the logs, I see the below log line for a given keyspace > `system_auth` > {code:java} > INFO [main] 2024-05-10 17:35:02,489 RangeStreamer.java:330 - Bootstrap: range > Full(/10.135.56.214:7000,(5080189126057290696,5081324396311791613]) exists on > Full(/10.135.56.157:7000,(5080189126057290696,5081324396311791613]) for > keyspace system_auth{code} > corresponding code: > {code:java} > for (Map.Entry entry : fetchMap.flattenEntries()) > logger.info("{}: range {} exists on {} for keyspace {}", description, > entry.getKey(), entry.getValue(), keyspaceName);{code} > BUT do not see the below line for the corresponding keyspace > {code:java} > RangeStreamer.java:606 - Output from RangeFetchMapCalculator for > keyspace{code} > this means the code it's stuck in `getRangeFetchMap();` > {code:java} > Multimap> rangeFetchMapMap = > calculator.getRangeFetchMap(); > logger.info("Output from RangeFetchMapCalculator for keyspace {}", > keyspace);{code} > Here is the cluster topology: > * Cassandra version: 4.0.12 > * # of nodes: 190 > * Tokens (vnodes): 128 > Initial hypothesis was that the graph calculation was taking longer due to > the combination of nodes + tokens + tables but in the same cluster I see one > of the node joined without any issues. > wondering if I am hitting a bug causing it to work sometimes but get into an > infinite loop some times? > Please let me know if you need any other details and appreciate any pointers > to debug this further. -- 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
[jira] [Updated] (CASSANDRA-19517) Raise priority of TCM internode messages during critical operations
[ https://issues.apache.org/jira/browse/CASSANDRA-19517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19517: Fix Version/s: 5.1-alpha1 Source Control Link: https://github.com/apache/cassandra/commit/67139d5c334e58fdc8d9f09f9288155448666cd3 Resolution: Fixed Status: Resolved (was: Ready to Commit) > Raise priority of TCM internode messages during critical operations > --- > > Key: CASSANDRA-19517 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19517 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Alex Petrov >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-1.html, ci_summary.html, result_details.tar.gz > > > In a busy cluster, TCM messages may not get propagated throughout the > cluster, since they will be ordered together with other P1 messages (for > {{TCM_}} prefixed verbs), and with P2 with all Paxos operations. > To avoid this, and make sure we can continue cluster metadata changes, all > {{TCM_}}-prefixed verbs should have {{P0}} priority, just like Gossip > messages used to. All Paxos messages that involve distributed metadata > keyspace should now get an {{URGENT}} flag, which will instruct internode > messaging to schedule them on the {{URGENT_MESSAGES}} connection. -- 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
[jira] [Updated] (CASSANDRA-19517) Raise priority of TCM internode messages during critical operations
[ https://issues.apache.org/jira/browse/CASSANDRA-19517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19517: Reviewers: Marcus Eriksson, Sam Tunnicliffe (was: Marcus Eriksson) Status: Review In Progress (was: Patch Available) > Raise priority of TCM internode messages during critical operations > --- > > Key: CASSANDRA-19517 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19517 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Alex Petrov >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html, result_details.tar.gz > > > In a busy cluster, TCM messages may not get propagated throughout the > cluster, since they will be ordered together with other P1 messages (for > {{TCM_}} prefixed verbs), and with P2 with all Paxos operations. > To avoid this, and make sure we can continue cluster metadata changes, all > {{TCM_}}-prefixed verbs should have {{P0}} priority, just like Gossip > messages used to. All Paxos messages that involve distributed metadata > keyspace should now get an {{URGENT}} flag, which will instruct internode > messaging to schedule them on the {{URGENT_MESSAGES}} connection. -- 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
[jira] [Updated] (CASSANDRA-19517) Raise priority of TCM internode messages during critical operations
[ https://issues.apache.org/jira/browse/CASSANDRA-19517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19517: Status: Ready to Commit (was: Review In Progress) > Raise priority of TCM internode messages during critical operations > --- > > Key: CASSANDRA-19517 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19517 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Alex Petrov >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html, result_details.tar.gz > > > In a busy cluster, TCM messages may not get propagated throughout the > cluster, since they will be ordered together with other P1 messages (for > {{TCM_}} prefixed verbs), and with P2 with all Paxos operations. > To avoid this, and make sure we can continue cluster metadata changes, all > {{TCM_}}-prefixed verbs should have {{P0}} priority, just like Gossip > messages used to. All Paxos messages that involve distributed metadata > keyspace should now get an {{URGENT}} flag, which will instruct internode > messaging to schedule them on the {{URGENT_MESSAGES}} connection. -- 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
[jira] [Updated] (CASSANDRA-19615) Merge pre-existing schema with the system defined one during upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-19615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19615: Source Control Link: https://github.com/apache/cassandra/commit/4d29a1c75e4c7a5493d5601c7a907b75e042c5f9 Resolution: Fixed Status: Resolved (was: Ready to Commit) and committed with the comment fixed > Merge pre-existing schema with the system defined one during upgrade > > > Key: CASSANDRA-19615 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19615 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html > > Time Spent: 20m > Remaining Estimate: 0h > > When upgrading we should merge the pre-existing schema with the > system-defined schema. For example, if a table was defined in 5.0 in > system_distributed, but then removed from SystemDistributedKeyspace.java in > 5.1 we should still be able to read it (until manually dropped). -- 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
[jira] [Updated] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-19581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19581: Fix Version/s: 5.1-alpha1 Source Control Link: https://github.com/apache/cassandra/commit/7694d901529e9b78ad3e4c548fbe1c03c305b929 Resolution: Fixed Status: Resolved (was: Ready to Commit) and committed, thanks > Add nodetool command to unregister LEFT nodes > - > > Key: CASSANDRA-19581 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-1.html, ci_summary-2.html, ci_summary.html > > Time Spent: 20m > Remaining Estimate: 0h > > When decommissioning a node it still remains in ClusterMetadata with state = > LEFT. We should provide a nodetool command to unregister such nodes > completely. -- 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
[jira] [Commented] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-19581?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17843213#comment-17843213 ] Marcus Eriksson commented on CASSANDRA-19581: - and another CI run, one python dtest failure - {{auth_test.TestAuthUnavailable::test_authorization_handle_unavailable}} which is hopefully fixed by CASSANDRA-19135 > Add nodetool command to unregister LEFT nodes > - > > Key: CASSANDRA-19581 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary-1.html, ci_summary-2.html, ci_summary.html > > Time Spent: 20m > Remaining Estimate: 0h > > When decommissioning a node it still remains in ClusterMetadata with state = > LEFT. We should provide a nodetool command to unregister such nodes > completely. -- 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
[jira] [Updated] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-19581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19581: Attachment: ci_summary-2.html > Add nodetool command to unregister LEFT nodes > - > > Key: CASSANDRA-19581 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary-1.html, ci_summary-2.html, ci_summary.html > > Time Spent: 20m > Remaining Estimate: 0h > > When decommissioning a node it still remains in ClusterMetadata with state = > LEFT. We should provide a nodetool command to unregister such nodes > completely. -- 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
[jira] [Updated] (CASSANDRA-19615) Merge pre-existing schema with the system defined one during upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-19615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19615: Change Category: Semantic Complexity: Low Hanging Fruit Fix Version/s: 5.1-alpha1 Reviewers: Sam Tunnicliffe Status: Open (was: Triage Needed) > Merge pre-existing schema with the system defined one during upgrade > > > Key: CASSANDRA-19615 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19615 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > When upgrading we should merge the pre-existing schema with the > system-defined schema. For example, if a table was defined in 5.0 in > system_distributed, but then removed from SystemDistributedKeyspace.java in > 5.1 we should still be able to read it (until manually dropped). -- 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
[jira] [Updated] (CASSANDRA-19615) Merge pre-existing schema with the system defined one during upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-19615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19615: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Merge pre-existing schema with the system defined one during upgrade > > > Key: CASSANDRA-19615 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19615 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > When upgrading we should merge the pre-existing schema with the > system-defined schema. For example, if a table was defined in 5.0 in > system_distributed, but then removed from SystemDistributedKeyspace.java in > 5.1 we should still be able to read it (until manually dropped). -- 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
[jira] [Updated] (CASSANDRA-19615) Merge pre-existing schema with the system defined one during upgrade
[ https://issues.apache.org/jira/browse/CASSANDRA-19615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19615: Attachment: ci_summary.html > Merge pre-existing schema with the system defined one during upgrade > > > Key: CASSANDRA-19615 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19615 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > When upgrading we should merge the pre-existing schema with the > system-defined schema. For example, if a table was defined in 5.0 in > system_distributed, but then removed from SystemDistributedKeyspace.java in > 5.1 we should still be able to read it (until manually dropped). -- 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
[jira] [Commented] (CASSANDRA-19517) Raise priority of TCM internode messages during critical operations
[ https://issues.apache.org/jira/browse/CASSANDRA-19517?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17843154#comment-17843154 ] Marcus Eriksson commented on CASSANDRA-19517: - ci run on rebased + slightly fixed up patch [here|https://github.com/apache/cassandra/compare/trunk...krummas:cassandra:CASSANDRA-19517?expand=1] attached > Raise priority of TCM internode messages during critical operations > --- > > Key: CASSANDRA-19517 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19517 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Alex Petrov >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html, result_details.tar.gz > > > In a busy cluster, TCM messages may not get propagated throughout the > cluster, since they will be ordered together with other P1 messages (for > {{TCM_}} prefixed verbs), and with P2 with all Paxos operations. > To avoid this, and make sure we can continue cluster metadata changes, all > {{TCM_}}-prefixed verbs should have {{P0}} priority, just like Gossip > messages used to. All Paxos messages that involve distributed metadata > keyspace should now get an {{URGENT}} flag, which will instruct internode > messaging to schedule them on the {{URGENT_MESSAGES}} connection. -- 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
[jira] [Updated] (CASSANDRA-19517) Raise priority of TCM internode messages during critical operations
[ https://issues.apache.org/jira/browse/CASSANDRA-19517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19517: Attachment: ci_summary-1.html > Raise priority of TCM internode messages during critical operations > --- > > Key: CASSANDRA-19517 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19517 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Alex Petrov >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html, result_details.tar.gz > > > In a busy cluster, TCM messages may not get propagated throughout the > cluster, since they will be ordered together with other P1 messages (for > {{TCM_}} prefixed verbs), and with P2 with all Paxos operations. > To avoid this, and make sure we can continue cluster metadata changes, all > {{TCM_}}-prefixed verbs should have {{P0}} priority, just like Gossip > messages used to. All Paxos messages that involve distributed metadata > keyspace should now get an {{URGENT}} flag, which will instruct internode > messaging to schedule them on the {{URGENT_MESSAGES}} connection. -- 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
[jira] [Created] (CASSANDRA-19615) Merge pre-existing schema with the system defined one during upgrade
Marcus Eriksson created CASSANDRA-19615: --- Summary: Merge pre-existing schema with the system defined one during upgrade Key: CASSANDRA-19615 URL: https://issues.apache.org/jira/browse/CASSANDRA-19615 Project: Cassandra Issue Type: Improvement Components: Transactional Cluster Metadata Reporter: Marcus Eriksson Assignee: Marcus Eriksson When upgrading we should merge the pre-existing schema with the system-defined schema. For example, if a table was defined in 5.0 in system_distributed, but then removed from SystemDistributedKeyspace.java in 5.1 we should still be able to read it (until manually dropped). -- 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
[jira] [Updated] (CASSANDRA-19613) Add ClusterMetadata.metadataIdentifier to GossipDigestSyn messages
[ https://issues.apache.org/jira/browse/CASSANDRA-19613?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19613: Status: Ready to Commit (was: Review In Progress) +1 > Add ClusterMetadata.metadataIdentifier to GossipDigestSyn messages > -- > > Key: CASSANDRA-19613 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19613 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Gossip, Transactional Cluster Metadata >Reporter: Sam Tunnicliffe >Assignee: Sam Tunnicliffe >Priority: Normal > Attachments: ci_summary.html > > > We should add \{{ClusterMetadata.instance().metadataIdentifier}} to > \{{GossipDigestSyn}} messages and compare with the local one, rejecting > anything that has the wrong identifier like we do with cluster name. -- 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
[jira] [Commented] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-19581?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17843142#comment-17843142 ] Marcus Eriksson commented on CASSANDRA-19581: - comments addressed and new rebased ci run attached > Add nodetool command to unregister LEFT nodes > - > > Key: CASSANDRA-19581 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 20m > Remaining Estimate: 0h > > When decommissioning a node it still remains in ClusterMetadata with state = > LEFT. We should provide a nodetool command to unregister such nodes > completely. -- 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
[jira] [Updated] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-19581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19581: Attachment: ci_summary-1.html > Add nodetool command to unregister LEFT nodes > - > > Key: CASSANDRA-19581 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 20m > Remaining Estimate: 0h > > When decommissioning a node it still remains in ClusterMetadata with state = > LEFT. We should provide a nodetool command to unregister such nodes > completely. -- 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
[jira] [Updated] (CASSANDRA-19587) Remove leftover period column from system.metadata_snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-19587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19587: Fix Version/s: 5.1-alpha1 Source Control Link: https://github.com/apache/cassandra/commit/b7eafa94f3b4f5361e891fde5c8a24d089b52d4c Resolution: Fixed Status: Resolved (was: Ready to Commit) committed, thanks > Remove leftover period column from system.metadata_snapshots > > > Key: CASSANDRA-19587 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19587 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Seems we left a period column in metadata_snapshots in > CASSANDRA-19189/CASSANDRA-19482 - it should be removed -- 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
[jira] [Updated] (CASSANDRA-19587) Remove leftover period column from system.metadata_snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-19587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19587: Change Category: Semantic Complexity: Low Hanging Fruit Reviewers: Sam Tunnicliffe Status: Open (was: Triage Needed) > Remove leftover period column from system.metadata_snapshots > > > Key: CASSANDRA-19587 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19587 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Seems we left a period column in metadata_snapshots in > CASSANDRA-19189/CASSANDRA-19482 - it should be removed -- 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
[jira] [Updated] (CASSANDRA-19587) Remove leftover period column from system.metadata_snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-19587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19587: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Remove leftover period column from system.metadata_snapshots > > > Key: CASSANDRA-19587 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19587 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Seems we left a period column in metadata_snapshots in > CASSANDRA-19189/CASSANDRA-19482 - it should be removed -- 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
[jira] [Updated] (CASSANDRA-19587) Remove leftover period column from system.metadata_snapshots
[ https://issues.apache.org/jira/browse/CASSANDRA-19587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19587: Attachment: ci_summary.html > Remove leftover period column from system.metadata_snapshots > > > Key: CASSANDRA-19587 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19587 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary.html > > > Seems we left a period column in metadata_snapshots in > CASSANDRA-19189/CASSANDRA-19482 - it should be removed -- 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
[jira] [Created] (CASSANDRA-19587) Remove leftover period column from system.metadata_snapshots
Marcus Eriksson created CASSANDRA-19587: --- Summary: Remove leftover period column from system.metadata_snapshots Key: CASSANDRA-19587 URL: https://issues.apache.org/jira/browse/CASSANDRA-19587 Project: Cassandra Issue Type: Improvement Components: Transactional Cluster Metadata Reporter: Marcus Eriksson Assignee: Marcus Eriksson Seems we left a period column in metadata_snapshots in CASSANDRA-19189/CASSANDRA-19482 - it should be removed -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Status: Ready to Commit (was: Review In Progress) > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-2.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Reviewers: Marcus Eriksson Status: Review In Progress (was: Patch Available) > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-2.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Source Control Link: https://github.com/apache/cassandra/commit/17ecece5437ab39aaeaa0eb4b42434cddd9960b5 Resolution: Fixed Status: Resolved (was: Ready to Commit) > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-2.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Commented] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17840045#comment-17840045 ] Marcus Eriksson commented on CASSANDRA-19190: - attaching new ci run, two failures, CASSANDRA-17339 and a counter mismatch, so I'm +1 here, will get it committed > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-2.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Attachment: (was: ci_summary-1.html) > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-2.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Attachment: (was: ci_summary.html) > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-2.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Attachment: ci_summary-2.html > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-2.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Commented] (CASSANDRA-19572) Test failure: org.apache.cassandra.db.ImportTest flakiness
[ https://issues.apache.org/jira/browse/CASSANDRA-19572?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17839994#comment-17839994 ] Marcus Eriksson commented on CASSANDRA-19572: - sorry, don't remember seeing these errors > Test failure: org.apache.cassandra.db.ImportTest flakiness > -- > > Key: CASSANDRA-19572 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19572 > Project: Cassandra > Issue Type: Bug > Components: Tool/bulk load >Reporter: Brandon Williams >Priority: Normal > Fix For: 4.0.x, 4.1.x, 5.0.x, 5.x > > > As discovered on CASSANDRA-19401, the tests in this class are flaky, at least > the following: > * testImportCorruptWithoutValidationWithCopying > * testImportInvalidateCache > * testImportCorruptWithCopying > * testImportCacheEnabledWithoutSrcDir > * testImportInvalidateCache > [https://app.circleci.com/pipelines/github/instaclustr/cassandra/4199/workflows/a70b41d8-f848-4114-9349-9a01ac082281/jobs/223621/tests] -- 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
[jira] [Updated] (CASSANDRA-19191) Optimisations to PlacementForRange, improve lookup on r/w path
[ https://issues.apache.org/jira/browse/CASSANDRA-19191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19191: Source Control Link: https://github.com/apache/cassandra/commit/34d999c47a4da6d43a67910354fb9888184b23ab Resolution: Fixed Status: Resolved (was: Ready to Commit) and committed, thanks > Optimisations to PlacementForRange, improve lookup on r/w path > -- > > Key: CASSANDRA-19191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19191 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-1.html, ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > The lookup used when selecting the appropriate replica group for a range or > token while peforming reads and writes is extremely simplistic and > inefficient. There is plenty of scope to improve {{PlacementsForRange}} to by > replacing the current naive iteration with use a more efficient lookup. -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Attachment: ci_summary-1.html > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-19581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19581: Change Category: Operability Complexity: Low Hanging Fruit Reviewers: Alex Petrov, Sam Tunnicliffe Status: Open (was: Triage Needed) > Add nodetool command to unregister LEFT nodes > - > > Key: CASSANDRA-19581 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > When decommissioning a node it still remains in ClusterMetadata with state = > LEFT. We should provide a nodetool command to unregister such nodes > completely. -- 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
[jira] [Updated] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-19581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19581: Attachment: ci_summary.html > Add nodetool command to unregister LEFT nodes > - > > Key: CASSANDRA-19581 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > When decommissioning a node it still remains in ClusterMetadata with state = > LEFT. We should provide a nodetool command to unregister such nodes > completely. -- 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
[jira] [Updated] (CASSANDRA-19517) Raise priority of TCM internode messages during critical operations
[ https://issues.apache.org/jira/browse/CASSANDRA-19517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19517: Reviewers: Marcus Eriksson > Raise priority of TCM internode messages during critical operations > --- > > Key: CASSANDRA-19517 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19517 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Alex Petrov >Priority: Normal > Attachments: ci_summary.html, result_details.tar.gz > > > In a busy cluster, TCM messages may not get propagated throughout the > cluster, since they will be ordered together with other P1 messages (for > {{TCM_}} prefixed verbs), and with P2 with all Paxos operations. > To avoid this, and make sure we can continue cluster metadata changes, all > {{TCM_}}-prefixed verbs should have {{P0}} priority, just like Gossip > messages used to. All Paxos messages that involve distributed metadata > keyspace should now get an {{URGENT}} flag, which will instruct internode > messaging to schedule them on the {{URGENT_MESSAGES}} connection. -- 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
[jira] [Created] (CASSANDRA-19581) Add nodetool command to unregister LEFT nodes
Marcus Eriksson created CASSANDRA-19581: --- Summary: Add nodetool command to unregister LEFT nodes Key: CASSANDRA-19581 URL: https://issues.apache.org/jira/browse/CASSANDRA-19581 Project: Cassandra Issue Type: Improvement Components: Transactional Cluster Metadata Reporter: Marcus Eriksson Assignee: Marcus Eriksson When decommissioning a node it still remains in ClusterMetadata with state = LEFT. We should provide a nodetool command to unregister such nodes completely. -- 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
[jira] [Updated] (CASSANDRA-19191) Optimisations to PlacementForRange, improve lookup on r/w path
[ https://issues.apache.org/jira/browse/CASSANDRA-19191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19191: Attachment: ci_summary-1.html > Optimisations to PlacementForRange, improve lookup on r/w path > -- > > Key: CASSANDRA-19191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19191 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-1.html, ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > The lookup used when selecting the appropriate replica group for a range or > token while peforming reads and writes is extremely simplistic and > inefficient. There is plenty of scope to improve {{PlacementsForRange}} to by > replacing the current naive iteration with use a more efficient lookup. -- 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
[jira] [Updated] (CASSANDRA-19132) Update use of transition plan in PrepareReplace
[ https://issues.apache.org/jira/browse/CASSANDRA-19132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19132: Source Control Link: https://github.com/apache/cassandra/commit/c5c4cd4e57515785d91ebaa918d533baf315 Resolution: Fixed Status: Resolved (was: Ready to Commit) and committed, thanks > Update use of transition plan in PrepareReplace > --- > > Key: CASSANDRA-19132 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19132 > Project: Cassandra > Issue Type: Task > Components: Cluster/Membership >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > When PlacementTransitionPlan was reworked to make its use more consistent > across join and leave operations, PrepareReplace was not updated. This could > now be simplified in line with the other operations. -- 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
[jira] [Commented] (CASSANDRA-19344) Range movements involving transient replicas must safely enact changes to read and write replica sets
[ https://issues.apache.org/jira/browse/CASSANDRA-19344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838909#comment-17838909 ] Marcus Eriksson commented on CASSANDRA-19344: - +1 > Range movements involving transient replicas must safely enact changes to > read and write replica sets > - > > Key: CASSANDRA-19344 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19344 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary-1.html, ci_summary.html, > remove-n4-post-19344.txt, remove-n4-pre-19344.txt, result_details.tar.gz > > Time Spent: 1h 40m > Remaining Estimate: 0h > > (edit) This was originally opened due to a flaky test > {{org.apache.cassandra.distributed.test.TransientRangeMovementTest.testRemoveNode-_jdk17}} > The test can fail in two different ways: > {code:java} > junit.framework.AssertionFailedError: NOT IN CURRENT: 31 -- [(00,20), > (31,50)] at > org.apache.cassandra.distributed.test.TransientRangeMovementTest.assertAllContained(TransientRangeMovementTest.java:203) > at > org.apache.cassandra.distributed.test.TransientRangeMovementTest.testRemoveNode(TransientRangeMovementTest.java:183) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > as in here - > [https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2639/workflows/32b92ce7-5e9d-4efb-8362-d200d2414597/jobs/55139/tests#failed-test-0] > and > {code:java} > junit.framework.AssertionFailedError: nodetool command [removenode, > 6d194555-f6eb-41d0-c000-0003, --force] was not successful stdout: > stderr: error: Node /127.0.0.4:7012 is alive and owns this ID. Use > decommission command to remove it from the ring -- StackTrace -- > java.lang.UnsupportedOperationException: Node /127.0.0.4:7012 is alive and > owns this ID. Use decommission command to remove it from the ring at > org.apache.cassandra.tcm.sequences.SingleNodeSequences.removeNode(SingleNodeSequences.java:110) > at > org.apache.cassandra.service.StorageService.removeNode(StorageService.java:3682) > at org.apache.cassandra.tools.NodeProbe.removeNode(NodeProbe.java:1020) at > org.apache.cassandra.tools.nodetool.RemoveNode.execute(RemoveNode.java:51) at > org.apache.cassandra.tools.NodeTool$NodeToolCmd.runInternal(NodeTool.java:388) > at org.apache.cassandra.tools.NodeTool$NodeToolCmd.run(NodeTool.java:373) at > org.apache.cassandra.tools.NodeTool.execute(NodeTool.java:272) at > org.apache.cassandra.distributed.impl.Instance$DTestNodeTool.execute(Instance.java:1129) > at > org.apache.cassandra.distributed.impl.Instance.lambda$nodetoolResult$51(Instance.java:1038) > at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61) at > org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71) at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635) > at > io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) > at java.base/java.lang.Thread.run(Thread.java:833) Notifications: Error: > java.lang.UnsupportedOperationException: Node /127.0.0.4:7012 is alive and > owns this ID. Use decommission command to remove it from the ring at > org.apache.cassandra.tcm.sequences.SingleNodeSequences.removeNode(SingleNodeSequences.java:110) > at > org.apache.cassandra.service.StorageService.removeNode(StorageService.java:3682) > at org.apache.cassandra.tools.NodeProbe.removeNode(NodeProbe.java:1020) at > org.apache.cassandra.tools.nodetool.RemoveNode.execute(RemoveNode.java:51) at > org.apache.cassandra.tools.NodeTool$NodeToolCmd.runInternal(NodeTool.java:388) > at org.apache.cassandra.tools.NodeTool$NodeToolCmd.run(NodeTool.java:373) at > org.apache.cassandra.tools.NodeTool.execute(NodeTool.java:272) at > org.apache.cassandra.distributed.impl.Instance$DTestNodeTool.execute(Instance.java:1129) > at > org.apache.cassandra.distributed.impl.Instance.lambda$nodetoolResult$51(Instance.java:1038) > at org.apache.cassandra.concurrent.FutureTask.call(FutureTask.java:61) at > org.apache.cassandra.concurrent.FutureTask.run(FutureTask.java:71) at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136) > at >
[jira] [Commented] (CASSANDRA-19571) Avoid ConcurrentModificationException when removing metrics from CassandraMetricsRegistry
[ https://issues.apache.org/jira/browse/CASSANDRA-19571?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17838722#comment-17838722 ] Marcus Eriksson commented on CASSANDRA-19571: - cc [~mmuzaf] > Avoid ConcurrentModificationException when removing metrics from > CassandraMetricsRegistry > - > > Key: CASSANDRA-19571 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19571 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Priority: Normal > > Multiple threads can modify the lists in {{CassandraMetricsRegistry}} > {{ALIASES}} map values. We have seen this CME when removing metrics, but > could possibly affect other paths using those lists > Adding a small delay (10ms) when map:ing to {{getMetricName}} in > {{CassandraMetricsRegistry.remove(name)}} makes this reproduce when running > {{TransientRangeMovementTest.testLeave}} > Caused by CASSANDRA-14572 -- 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
[jira] [Created] (CASSANDRA-19571) Avoid ConcurrentModificationException when removing metrics from CassandraMetricsRegistry
Marcus Eriksson created CASSANDRA-19571: --- Summary: Avoid ConcurrentModificationException when removing metrics from CassandraMetricsRegistry Key: CASSANDRA-19571 URL: https://issues.apache.org/jira/browse/CASSANDRA-19571 Project: Cassandra Issue Type: Bug Reporter: Marcus Eriksson Multiple threads can modify the lists in {{CassandraMetricsRegistry}} {{ALIASES}} map values. We have seen this CME when removing metrics, but could possibly affect other paths using those lists Adding a small delay (10ms) when map:ing to {{getMetricName}} in {{CassandraMetricsRegistry.remove(name)}} makes this reproduce when running {{TransientRangeMovementTest.testLeave}} Caused by CASSANDRA-14572 -- 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
[jira] [Updated] (CASSANDRA-19538) Test Failure: test_assassinate_valid_node
[ https://issues.apache.org/jira/browse/CASSANDRA-19538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19538: Attachment: ci_summary-1.html > Test Failure: test_assassinate_valid_node > - > > Key: CASSANDRA-19538 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19538 > Project: Cassandra > Issue Type: Bug > Components: CI, Test/dtest/python >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Failing consistently on trunk: > {code:java} > ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after 120.11/120 > seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log: > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > Tail: ... some nodes were not ready > INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 > - Setup task failed with error, rescheduling > self = > def test_assassinate_valid_node(self): > """ > @jira_ticket CASSANDRA-16588 > Test that after taking two non-seed nodes down and assassinating > one of them, the other can come back up. > """ > cluster = self.cluster > > cluster.populate(5).start() > node1 = cluster.nodelist()[0] > node3 = cluster.nodelist()[2] > > self.cluster.set_configuration_options({ > 'seed_provider': [{'class_name': > 'org.apache.cassandra.locator.SimpleSeedProvider', >'parameters': [{'seeds': node1.address()}] > }] > }) > > non_seed_nodes = cluster.nodelist()[-2:] > for node in non_seed_nodes: > node.stop() > > assassination_target = non_seed_nodes[0] > logger.debug("Assassinating non-seed node > {}".format(assassination_target.address())) > out, err, _ = node1.nodetool("assassinate > {}".format(assassination_target.address())) > assert_stderr_clean(err) > > logger.debug("Starting non-seed nodes") > for node in non_seed_nodes: > > node.start() > gossip_test.py:78: > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:915: in start > node.watch_log_for_alive(self, from_mark=mark) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:684: in > watch_log_for_alive > self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, > filename=filename) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:608: in watch_log_for > TimeoutError.raise_if_passed(start=start, timeout=timeout, node=self.name, > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > start = 1712173052.8186479, timeout = 120 > msg = "Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log:\n > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 1...[OptionalTasks:1] > 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 - Setup task failed > with error, rescheduling\n" > node = 'node1' > @staticmethod > def raise_if_passed(start, timeout, msg, node=None): > if start + timeout < time.time(): > > raise TimeoutError.create(start, timeout, msg, node) > E ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after > 120.11/120 seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in > system.log: > EHead: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > ETail: ... some nodes were not ready > E INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 > CassandraRoleManager.java:484 - Setup task failed with error, rescheduling > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:56: TimeoutError > {code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2680/workflows/8b1c0d0a-7458-4b43-9bba-ac96b9bfe64f/jobs/58929/tests#failed-test-0 > https://ci-cassandra.apache.org/job/Cassandra-trunk/1859/#showFailuresLink -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Attachment: ci_summary.html > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19190) ForceSnapshot transformations should not be persisted in the local log table
[ https://issues.apache.org/jira/browse/CASSANDRA-19190?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19190: Test and Documentation Plan: ci run Status: Patch Available (was: Open) ci looks bad right now, will rerun after CASSANDRA-19514 has been committed > ForceSnapshot transformations should not be persisted in the local log table > > > Key: CASSANDRA-19190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19190 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Per its inline comments, ForceSnapshot is a synthetic transformation whose > purpose it to enable the local log to jump missing epochs. A common use for > this is when replaying persisted events from the metadata log at startup. The > log is initialised with {{Epoch.EMPTY}} but rather that replaying every > single entry since the beginning of history, we select the most recent > snapshot held locally and start the replay from that point. Likewise, when > catching up from a peer, a node may receive a snapshot plus subsequent log > entries. In order to bring local metadata to the same state as the snapshot, > a {{ForceSnapshot}} with the same epoch as the snapshot is inserted into the > {{LocalLog}} and enacted like any other other transformation. These synthetic > transformations should not be persisted in the `system.local_metadata_log`, > as they do not exist in the distributed metadata log. We _should_ persist the > snapshot itself in {{system.metadata_snapshots}} so that we can avoid having > to re-fetch remote snapshots (i.e. if a node were to restart shortly after > receiving a catchup from a peer). -- 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
[jira] [Updated] (CASSANDRA-19538) Test Failure: test_assassinate_valid_node
[ https://issues.apache.org/jira/browse/CASSANDRA-19538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19538: Attachment: ci_summary.html > Test Failure: test_assassinate_valid_node > - > > Key: CASSANDRA-19538 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19538 > Project: Cassandra > Issue Type: Bug > Components: CI, Test/dtest/python >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Failing consistently on trunk: > {code:java} > ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after 120.11/120 > seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log: > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > Tail: ... some nodes were not ready > INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 > - Setup task failed with error, rescheduling > self = > def test_assassinate_valid_node(self): > """ > @jira_ticket CASSANDRA-16588 > Test that after taking two non-seed nodes down and assassinating > one of them, the other can come back up. > """ > cluster = self.cluster > > cluster.populate(5).start() > node1 = cluster.nodelist()[0] > node3 = cluster.nodelist()[2] > > self.cluster.set_configuration_options({ > 'seed_provider': [{'class_name': > 'org.apache.cassandra.locator.SimpleSeedProvider', >'parameters': [{'seeds': node1.address()}] > }] > }) > > non_seed_nodes = cluster.nodelist()[-2:] > for node in non_seed_nodes: > node.stop() > > assassination_target = non_seed_nodes[0] > logger.debug("Assassinating non-seed node > {}".format(assassination_target.address())) > out, err, _ = node1.nodetool("assassinate > {}".format(assassination_target.address())) > assert_stderr_clean(err) > > logger.debug("Starting non-seed nodes") > for node in non_seed_nodes: > > node.start() > gossip_test.py:78: > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:915: in start > node.watch_log_for_alive(self, from_mark=mark) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:684: in > watch_log_for_alive > self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, > filename=filename) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:608: in watch_log_for > TimeoutError.raise_if_passed(start=start, timeout=timeout, node=self.name, > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > start = 1712173052.8186479, timeout = 120 > msg = "Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log:\n > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 1...[OptionalTasks:1] > 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 - Setup task failed > with error, rescheduling\n" > node = 'node1' > @staticmethod > def raise_if_passed(start, timeout, msg, node=None): > if start + timeout < time.time(): > > raise TimeoutError.create(start, timeout, msg, node) > E ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after > 120.11/120 seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in > system.log: > EHead: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > ETail: ... some nodes were not ready > E INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 > CassandraRoleManager.java:484 - Setup task failed with error, rescheduling > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:56: TimeoutError > {code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2680/workflows/8b1c0d0a-7458-4b43-9bba-ac96b9bfe64f/jobs/58929/tests#failed-test-0 > https://ci-cassandra.apache.org/job/Cassandra-trunk/1859/#showFailuresLink -- 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
[jira] [Updated] (CASSANDRA-19538) Test Failure: test_assassinate_valid_node
[ https://issues.apache.org/jira/browse/CASSANDRA-19538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19538: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Test Failure: test_assassinate_valid_node > - > > Key: CASSANDRA-19538 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19538 > Project: Cassandra > Issue Type: Bug > Components: CI, Test/dtest/python >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > Failing consistently on trunk: > {code:java} > ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after 120.11/120 > seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log: > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > Tail: ... some nodes were not ready > INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 > - Setup task failed with error, rescheduling > self = > def test_assassinate_valid_node(self): > """ > @jira_ticket CASSANDRA-16588 > Test that after taking two non-seed nodes down and assassinating > one of them, the other can come back up. > """ > cluster = self.cluster > > cluster.populate(5).start() > node1 = cluster.nodelist()[0] > node3 = cluster.nodelist()[2] > > self.cluster.set_configuration_options({ > 'seed_provider': [{'class_name': > 'org.apache.cassandra.locator.SimpleSeedProvider', >'parameters': [{'seeds': node1.address()}] > }] > }) > > non_seed_nodes = cluster.nodelist()[-2:] > for node in non_seed_nodes: > node.stop() > > assassination_target = non_seed_nodes[0] > logger.debug("Assassinating non-seed node > {}".format(assassination_target.address())) > out, err, _ = node1.nodetool("assassinate > {}".format(assassination_target.address())) > assert_stderr_clean(err) > > logger.debug("Starting non-seed nodes") > for node in non_seed_nodes: > > node.start() > gossip_test.py:78: > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:915: in start > node.watch_log_for_alive(self, from_mark=mark) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:684: in > watch_log_for_alive > self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, > filename=filename) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:608: in watch_log_for > TimeoutError.raise_if_passed(start=start, timeout=timeout, node=self.name, > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > start = 1712173052.8186479, timeout = 120 > msg = "Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log:\n > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 1...[OptionalTasks:1] > 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 - Setup task failed > with error, rescheduling\n" > node = 'node1' > @staticmethod > def raise_if_passed(start, timeout, msg, node=None): > if start + timeout < time.time(): > > raise TimeoutError.create(start, timeout, msg, node) > E ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after > 120.11/120 seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in > system.log: > EHead: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > ETail: ... some nodes were not ready > E INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 > CassandraRoleManager.java:484 - Setup task failed with error, rescheduling > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:56: TimeoutError > {code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2680/workflows/8b1c0d0a-7458-4b43-9bba-ac96b9bfe64f/jobs/58929/tests#failed-test-0 > https://ci-cassandra.apache.org/job/Cassandra-trunk/1859/#showFailuresLink -- 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
[jira] [Assigned] (CASSANDRA-19538) Test Failure: test_assassinate_valid_node
[ https://issues.apache.org/jira/browse/CASSANDRA-19538?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson reassigned CASSANDRA-19538: --- Assignee: Marcus Eriksson > Test Failure: test_assassinate_valid_node > - > > Key: CASSANDRA-19538 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19538 > Project: Cassandra > Issue Type: Bug > Components: CI, Test/dtest/python >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > > Failing consistently on trunk: > {code:java} > ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after 120.11/120 > seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log: > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > Tail: ... some nodes were not ready > INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 > - Setup task failed with error, rescheduling > self = > def test_assassinate_valid_node(self): > """ > @jira_ticket CASSANDRA-16588 > Test that after taking two non-seed nodes down and assassinating > one of them, the other can come back up. > """ > cluster = self.cluster > > cluster.populate(5).start() > node1 = cluster.nodelist()[0] > node3 = cluster.nodelist()[2] > > self.cluster.set_configuration_options({ > 'seed_provider': [{'class_name': > 'org.apache.cassandra.locator.SimpleSeedProvider', >'parameters': [{'seeds': node1.address()}] > }] > }) > > non_seed_nodes = cluster.nodelist()[-2:] > for node in non_seed_nodes: > node.stop() > > assassination_target = non_seed_nodes[0] > logger.debug("Assassinating non-seed node > {}".format(assassination_target.address())) > out, err, _ = node1.nodetool("assassinate > {}".format(assassination_target.address())) > assert_stderr_clean(err) > > logger.debug("Starting non-seed nodes") > for node in non_seed_nodes: > > node.start() > gossip_test.py:78: > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:915: in start > node.watch_log_for_alive(self, from_mark=mark) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:684: in > watch_log_for_alive > self.watch_log_for(tofind, from_mark=from_mark, timeout=timeout, > filename=filename) > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:608: in watch_log_for > TimeoutError.raise_if_passed(start=start, timeout=timeout, node=self.name, > _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ _ > _ > start = 1712173052.8186479, timeout = 120 > msg = "Missing: ['127.0.0.4:7000.* is now UP'] not found in system.log:\n > Head: INFO [Messaging-EventLoop-3-1] 2024-04-03 1...[OptionalTasks:1] > 2024-04-03 19:39:30,454 CassandraRoleManager.java:484 - Setup task failed > with error, rescheduling\n" > node = 'node1' > @staticmethod > def raise_if_passed(start, timeout, msg, node=None): > if start + timeout < time.time(): > > raise TimeoutError.create(start, timeout, msg, node) > E ccmlib.node.TimeoutError: 03 Apr 2024 19:39:32 [node1] after > 120.11/120 seconds Missing: ['127.0.0.4:7000.* is now UP'] not found in > system.log: > EHead: INFO [Messaging-EventLoop-3-1] 2024-04-03 19:37:3 > ETail: ... some nodes were not ready > E INFO [OptionalTasks:1] 2024-04-03 19:39:30,454 > CassandraRoleManager.java:484 - Setup task failed with error, rescheduling > ../env3.8/lib/python3.8/site-packages/ccmlib/node.py:56: TimeoutError > {code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2680/workflows/8b1c0d0a-7458-4b43-9bba-ac96b9bfe64f/jobs/58929/tests#failed-test-0 > https://ci-cassandra.apache.org/job/Cassandra-trunk/1859/#showFailuresLink -- 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
[jira] [Updated] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19516: Fix Version/s: 5.1-alpha1 Source Control Link: https://github.com/apache/cassandra/commit/d548396597897012c0b8ed83264f8dd911fb84bf Resolution: Fixed Status: Resolved (was: Ready to Commit) > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19516 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > We should store {{Kind.id}} added in CASSANDRA-19390 in the local and > distributed log tables. Virtual table will still do the id -> string lookup > for easier reading -- 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
[jira] [Commented] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19516?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17836427#comment-17836427 ] Marcus Eriksson commented on CASSANDRA-19516: - a bit unclean ci run, but mostly timeouts which we think are fixed by CASSANDRA-19514 > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19516 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > We should store {{Kind.id}} added in CASSANDRA-19390 in the local and > distributed log tables. Virtual table will still do the id -> string lookup > for easier reading -- 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
[jira] [Updated] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19516: Status: Ready to Commit (was: Review In Progress) > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19516 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > We should store {{Kind.id}} added in CASSANDRA-19390 in the local and > distributed log tables. Virtual table will still do the id -> string lookup > for easier reading -- 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
[jira] [Updated] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19516: Attachment: ci_summary-1.html > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19516 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary-1.html, ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > We should store {{Kind.id}} added in CASSANDRA-19390 in the local and > distributed log tables. Virtual table will still do the id -> string lookup > for easier reading -- 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
[jira] [Updated] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19516: Attachment: ci_summary.html > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19516 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Attachments: ci_summary.html > > Time Spent: 10m > Remaining Estimate: 0h > > We should store {{Kind.id}} added in CASSANDRA-19390 in the local and > distributed log tables. Virtual table will still do the id -> string lookup > for easier reading -- 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
[jira] [Updated] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19516: Change Category: Performance Complexity: Low Hanging Fruit Component/s: Transactional Cluster Metadata Reviewers: Alex Petrov, Sam Tunnicliffe Status: Open (was: Triage Needed) > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19516 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Time Spent: 10m > Remaining Estimate: 0h > > We should store {{Kind.id}} added in CASSANDRA-19390 in the local and > distributed log tables. Virtual table will still do the id -> string lookup > for easier reading -- 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
[jira] [Updated] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19516?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19516: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19516 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Time Spent: 10m > Remaining Estimate: 0h > > We should store {{Kind.id}} added in CASSANDRA-19390 in the local and > distributed log tables. Virtual table will still do the id -> string lookup > for easier reading -- 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
[jira] [Commented] (CASSANDRA-19482) Simplify metadata log implementation using custom partitioner
[ https://issues.apache.org/jira/browse/CASSANDRA-19482?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17835807#comment-17835807 ] Marcus Eriksson commented on CASSANDRA-19482: - +1 > Simplify metadata log implementation using custom partitioner > - > > Key: CASSANDRA-19482 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19482 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Sam Tunnicliffe >Assignee: Sam Tunnicliffe >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 1.5h > Remaining Estimate: 0h > > The distributed metadata log table can be simplified by leveraging the fact > that replicas are all responsible for the entire token range. Given this > assumption, we can then use {{ReversedLongLocalPartitioner}} introduced in > CASSANDRA-19391 to make it much easier to append to/read from the tail of the > log, effectively removing the need for the {{Period}} construct. This will > also improve apply to the local metadata log used at startup. -- 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
[jira] [Created] (CASSANDRA-19533) Use Transformation.Kind.id in local and distributed log tables
Marcus Eriksson created CASSANDRA-19533: --- Summary: Use Transformation.Kind.id in local and distributed log tables Key: CASSANDRA-19533 URL: https://issues.apache.org/jira/browse/CASSANDRA-19533 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Assignee: Marcus Eriksson Following on from CASSANDRA-19390, the numeric id should be used to represent the Kind in both the local and distributed log tables. The virtual log table can translate from id to name for a more readable option. -- 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
[jira] [Updated] (CASSANDRA-19533) Use Transformation.Kind.id in local and distributed log tables
[ https://issues.apache.org/jira/browse/CASSANDRA-19533?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19533: Change Category: Code Clarity Complexity: Low Hanging Fruit Component/s: Transactional Cluster Metadata Fix Version/s: 5.x Status: Open (was: Triage Needed) > Use Transformation.Kind.id in local and distributed log tables > -- > > Key: CASSANDRA-19533 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19533 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > > Following on from CASSANDRA-19390, the numeric id should be used to represent > the Kind in both the local and distributed log tables. The virtual log table > can translate from id to name for a more readable option. -- 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
[jira] [Updated] (CASSANDRA-19132) Update use of transition plan in PrepareReplace
[ https://issues.apache.org/jira/browse/CASSANDRA-19132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19132: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Update use of transition plan in PrepareReplace > --- > > Key: CASSANDRA-19132 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19132 > Project: Cassandra > Issue Type: Task > Components: Cluster/Membership >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Time Spent: 10m > Remaining Estimate: 0h > > When PlacementTransitionPlan was reworked to make its use more consistent > across join and leave operations, PrepareReplace was not updated. This could > now be simplified in line with the other operations. -- 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
[jira] [Updated] (CASSANDRA-19132) Update use of transition plan in PrepareReplace
[ https://issues.apache.org/jira/browse/CASSANDRA-19132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19132: Change Category: Code Clarity Complexity: Low Hanging Fruit Reviewers: Sam Tunnicliffe Status: Open (was: Triage Needed) > Update use of transition plan in PrepareReplace > --- > > Key: CASSANDRA-19132 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19132 > Project: Cassandra > Issue Type: Task > Components: Cluster/Membership >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > > When PlacementTransitionPlan was reworked to make its use more consistent > across join and leave operations, PrepareReplace was not updated. This could > now be simplified in line with the other operations. -- 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
[jira] [Assigned] (CASSANDRA-19132) Update use of transition plan in PrepareReplace
[ https://issues.apache.org/jira/browse/CASSANDRA-19132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson reassigned CASSANDRA-19132: --- Assignee: Marcus Eriksson > Update use of transition plan in PrepareReplace > --- > > Key: CASSANDRA-19132 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19132 > Project: Cassandra > Issue Type: Task > Components: Cluster/Membership >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > > When PlacementTransitionPlan was reworked to make its use more consistent > across join and leave operations, PrepareReplace was not updated. This could > now be simplified in line with the other operations. -- 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
[jira] [Created] (CASSANDRA-19516) Use Transformation.Kind.id in local and distributed log tables
Marcus Eriksson created CASSANDRA-19516: --- Summary: Use Transformation.Kind.id in local and distributed log tables Key: CASSANDRA-19516 URL: https://issues.apache.org/jira/browse/CASSANDRA-19516 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Assignee: Marcus Eriksson We should store {{Kind.id}} added in CASSANDRA-19390 in the local and distributed log tables. Virtual table will still do the id -> string lookup for easier reading -- 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
[jira] [Updated] (CASSANDRA-19343) Test Failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19343: Fix Version/s: 5.1-alpha1 (was: 5.x) Since Version: 5.x Source Control Link: https://github.com/apache/cassandra/commit/8d8c6fbc37899ff77be6b3431f99f6951c4c05c2 Resolution: Fixed Status: Resolved (was: Ready to Commit) and committed, thanks! > Test Failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19343 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19343 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > {code:java} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2636/workflows/93adbf3e-acf8-4a62-a1f6-baf4f4689347/jobs/54912/tests -- 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
[jira] [Updated] (CASSANDRA-19343) Test Failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19343: Status: Ready to Commit (was: Review In Progress) > Test Failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19343 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19343 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > {code:java} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2636/workflows/93adbf3e-acf8-4a62-a1f6-baf4f4689347/jobs/54912/tests -- 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
[jira] [Updated] (CASSANDRA-19343) Test Failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19343: Reviewers: Alex Petrov, Marcus Eriksson (was: Alex Petrov) Status: Review In Progress (was: Patch Available) > Test Failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19343 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19343 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > {code:java} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2636/workflows/93adbf3e-acf8-4a62-a1f6-baf4f4689347/jobs/54912/tests -- 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
[jira] [Updated] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19193: Status: Ready to Commit (was: Review In Progress) > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Updated] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19193: Reviewers: Alex Petrov (was: Alex Petrov, Sam Tunnicliffe) Status: Review In Progress (was: Patch Available) > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Updated] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19193: Source Control Link: https://github.com/apache/cassandra/commit/63c62618560ad65b5b3e9f4d34b70b8b6dd0a75b Resolution: Fixed Status: Resolved (was: Ready to Commit) and committed, thanks > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Updated] (CASSANDRA-19153) Preclude irrecoverable log corruption in case split-brain situation during leader election with absent seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-19153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19153: Status: Review In Progress (was: Patch Available) > Preclude irrecoverable log corruption in case split-brain situation during > leader election with absent seeds > > > Key: CASSANDRA-19153 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19153 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 20m > Remaining Estimate: 0h > > It should be possible to detect a scenario where two partitioned nodes > independently elect themselves as the first CMS nodes in a brand new cluster. > In such a case, metadata changes should not be applied so that the conflict > can be resolved. -- 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
[jira] [Updated] (CASSANDRA-19153) Preclude irrecoverable log corruption in case split-brain situation during leader election with absent seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-19153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19153: Source Control Link: https://github.com/apache/cassandra/commit/0ec5ef2c7035fc93323816140994617a9d953956 Resolution: Fixed Status: Resolved (was: Ready to Commit) and committed > Preclude irrecoverable log corruption in case split-brain situation during > leader election with absent seeds > > > Key: CASSANDRA-19153 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19153 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 20m > Remaining Estimate: 0h > > It should be possible to detect a scenario where two partitioned nodes > independently elect themselves as the first CMS nodes in a brand new cluster. > In such a case, metadata changes should not be applied so that the conflict > can be resolved. -- 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
[jira] [Updated] (CASSANDRA-19153) Preclude irrecoverable log corruption in case split-brain situation during leader election with absent seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-19153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19153: Status: Ready to Commit (was: Review In Progress) > Preclude irrecoverable log corruption in case split-brain situation during > leader election with absent seeds > > > Key: CASSANDRA-19153 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19153 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 20m > Remaining Estimate: 0h > > It should be possible to detect a scenario where two partitioned nodes > independently elect themselves as the first CMS nodes in a brand new cluster. > In such a case, metadata changes should not be applied so that the conflict > can be resolved. -- 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
[jira] [Updated] (CASSANDRA-19343) Test Failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19343: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Test Failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19343 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19343 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > {code:java} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2636/workflows/93adbf3e-acf8-4a62-a1f6-baf4f4689347/jobs/54912/tests -- 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
[jira] [Updated] (CASSANDRA-19343) Test Failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19343: Reviewers: Alex Petrov > Test Failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19343 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19343 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > {code:java} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2636/workflows/93adbf3e-acf8-4a62-a1f6-baf4f4689347/jobs/54912/tests -- 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
[jira] [Assigned] (CASSANDRA-19343) Test Failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson reassigned CASSANDRA-19343: --- Assignee: Marcus Eriksson > Test Failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19343 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19343 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > {code:java} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2636/workflows/93adbf3e-acf8-4a62-a1f6-baf4f4689347/jobs/54912/tests -- 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
[jira] [Updated] (CASSANDRA-19343) Test Failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19343?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19343: Attachment: ci_summary.html result_details.tar.gz > Test Failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19343 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19343 > Project: Cassandra > Issue Type: Bug > Components: CI >Reporter: Ekaterina Dimitrova >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > {code:java} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native > Method) at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:77) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43){code} > https://app.circleci.com/pipelines/github/ekaterinadimitrova2/cassandra/2636/workflows/93adbf3e-acf8-4a62-a1f6-baf4f4689347/jobs/54912/tests -- 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
[jira] [Commented] (CASSANDRA-19255) StorageService.getRangeToEndpointMap() MBean operation is running into NPE for LocalStrategy keysapces
[ https://issues.apache.org/jira/browse/CASSANDRA-19255?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17829814#comment-17829814 ] Marcus Eriksson commented on CASSANDRA-19255: - +1 > StorageService.getRangeToEndpointMap() MBean operation is running into NPE > for LocalStrategy keysapces > -- > > Key: CASSANDRA-19255 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19255 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Membership >Reporter: n.v.harikrishna >Assignee: n.v.harikrishna >Priority: Normal > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > When the StorageService's MBean operation getRangeToEndpointMap is called for > LocalStrategy keyspaces, then it is running into NPE. It is working in > earlier major version, but failing in trunk. It can be reproduced in local > using JConsole or using a tool like `jmxterm` (unfortunately these tools are > not giving full stacktrace). Observed the same behavior with > getRangeToEndpointWithPortMap operation too. -- 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
[jira] [Updated] (CASSANDRA-19191) Optimisations to PlacementForRange, improve lookup on r/w path
[ https://issues.apache.org/jira/browse/CASSANDRA-19191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19191: Attachment: ci_summary.html result_details.tar.gz > Optimisations to PlacementForRange, improve lookup on r/w path > -- > > Key: CASSANDRA-19191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19191 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > The lookup used when selecting the appropriate replica group for a range or > token while peforming reads and writes is extremely simplistic and > inefficient. There is plenty of scope to improve {{PlacementsForRange}} to by > replacing the current naive iteration with use a more efficient lookup. -- 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
[jira] [Updated] (CASSANDRA-19191) Optimisations to PlacementForRange, improve lookup on r/w path
[ https://issues.apache.org/jira/browse/CASSANDRA-19191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19191: Test and Documentation Plan: ci run, existing tests Status: Patch Available (was: Open) > Optimisations to PlacementForRange, improve lookup on r/w path > -- > > Key: CASSANDRA-19191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19191 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > The lookup used when selecting the appropriate replica group for a range or > token while peforming reads and writes is extremely simplistic and > inefficient. There is plenty of scope to improve {{PlacementsForRange}} to by > replacing the current naive iteration with use a more efficient lookup. -- 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
[jira] [Updated] (CASSANDRA-19191) Optimisations to PlacementForRange, improve lookup on r/w path
[ https://issues.apache.org/jira/browse/CASSANDRA-19191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19191: Change Category: Performance Complexity: Normal Reviewers: Alex Petrov, Sam Tunnicliffe Status: Open (was: Triage Needed) > Optimisations to PlacementForRange, improve lookup on r/w path > -- > > Key: CASSANDRA-19191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19191 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > > The lookup used when selecting the appropriate replica group for a range or > token while peforming reads and writes is extremely simplistic and > inefficient. There is plenty of scope to improve {{PlacementsForRange}} to by > replacing the current naive iteration with use a more efficient lookup. -- 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
[jira] [Commented] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17829801#comment-17829801 ] Marcus Eriksson commented on CASSANDRA-19193: - pushed a test > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Updated] (CASSANDRA-19463) Test failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
[ https://issues.apache.org/jira/browse/CASSANDRA-19463?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19463: Resolution: Duplicate Status: Resolved (was: Triage Needed) > Test failure: > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest > > > Key: CASSANDRA-19463 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19463 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Priority: Normal > > Looks like this was broken by CASSANDRA-18275 > Timeouts in the log and then: > {code} > java.lang.IllegalStateException: Can't use shutdown instances, delegate is > null > at > org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) > at > org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) > at > org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) > at > org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) > at > org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) > at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.base/java.lang.reflect.Method.invoke(Method.java:566) > at > org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) > at > org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) > at > org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) > at > org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) > at > org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) > at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) > at > org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) > at > org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) > at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) > at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) > at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) > at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) > at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) > at > org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) > at org.junit.runners.ParentRunner.run(ParentRunner.java:363) > at org.junit.runner.JUnitCore.run(JUnitCore.java:137) > at > com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) > at > com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) > at > com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) > at > com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) > at > com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) > at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) > {code} -- 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
[jira] [Assigned] (CASSANDRA-19191) Optimisations to PlacementForRange, improve lookup on r/w path
[ https://issues.apache.org/jira/browse/CASSANDRA-19191?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson reassigned CASSANDRA-19191: --- Assignee: Marcus Eriksson > Optimisations to PlacementForRange, improve lookup on r/w path > -- > > Key: CASSANDRA-19191 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19191 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > > The lookup used when selecting the appropriate replica group for a range or > token while peforming reads and writes is extremely simplistic and > inefficient. There is plenty of scope to improve {{PlacementsForRange}} to by > replacing the current naive iteration with use a more efficient lookup. -- 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
[jira] [Updated] (CASSANDRA-19255) StorageService.getRangeToEndpointMap() MBean operation is running into NPE for LocalStrategy keysapces
[ https://issues.apache.org/jira/browse/CASSANDRA-19255?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19255: Bug Category: Parent values: Correctness(12982)Level 1 values: API / Semantic Implementation(12988) Complexity: Normal Component/s: Cluster/Membership Discovered By: Adhoc Test Reviewers: Marcus Eriksson Severity: Normal Status: Open (was: Triage Needed) > StorageService.getRangeToEndpointMap() MBean operation is running into NPE > for LocalStrategy keysapces > -- > > Key: CASSANDRA-19255 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19255 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Membership >Reporter: n.v.harikrishna >Assignee: n.v.harikrishna >Priority: Normal > Time Spent: 10m > Remaining Estimate: 0h > > When the StorageService's MBean operation getRangeToEndpointMap is called for > LocalStrategy keyspaces, then it is running into NPE. It is working in > earlier major version, but failing in trunk. It can be reproduced in local > using JConsole or using a tool like `jmxterm` (unfortunately these tools are > not giving full stacktrace). Observed the same behavior with > getRangeToEndpointWithPortMap operation too. -- 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
[jira] [Updated] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19193: Attachment: ci_summary.html result_details.tar.gz > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Updated] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19193: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Updated] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19193: Change Category: Semantic Complexity: Normal Component/s: Transactional Cluster Metadata Reviewers: Alex Petrov, Sam Tunnicliffe Status: Open (was: Triage Needed) > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > Time Spent: 10m > Remaining Estimate: 0h > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Assigned] (CASSANDRA-19193) Reimplement ClusterMetadata::writePlacementAllSettled
[ https://issues.apache.org/jira/browse/CASSANDRA-19193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson reassigned CASSANDRA-19193: --- Assignee: Marcus Eriksson > Reimplement ClusterMetadata::writePlacementAllSettled > - > > Key: CASSANDRA-19193 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19193 > Project: Cassandra > Issue Type: Improvement >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.1-alpha1 > > > This should step through InProgressSequences to determine state when > finished, rather than emulating the logic inline. -- 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
[jira] [Updated] (CASSANDRA-19393) nodetool: group CMS-related commands into one command
[ https://issues.apache.org/jira/browse/CASSANDRA-19393?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19393: Status: Ready to Commit (was: Review In Progress) > nodetool: group CMS-related commands into one command > - > > Key: CASSANDRA-19393 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19393 > Project: Cassandra > Issue Type: Improvement > Components: Tool/nodetool, Transactional Cluster Metadata >Reporter: n.v.harikrishna >Assignee: n.v.harikrishna >Priority: Normal > Fix For: 5.x > > Time Spent: 1.5h > Remaining Estimate: 0h > > The purpose of this ticket is to group all CMS-related commands under one > "nodetool cms" command where existing command would be subcommands of it. -- 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
[jira] [Commented] (CASSANDRA-19393) nodetool: group CMS-related commands into one command
[ https://issues.apache.org/jira/browse/CASSANDRA-19393?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17825221#comment-17825221 ] Marcus Eriksson commented on CASSANDRA-19393: - +1 > nodetool: group CMS-related commands into one command > - > > Key: CASSANDRA-19393 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19393 > Project: Cassandra > Issue Type: Improvement > Components: Tool/nodetool, Transactional Cluster Metadata >Reporter: n.v.harikrishna >Assignee: n.v.harikrishna >Priority: Normal > Fix For: 5.x > > Time Spent: 1.5h > Remaining Estimate: 0h > > The purpose of this ticket is to group all CMS-related commands under one > "nodetool cms" command where existing command would be subcommands of it. -- 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
[jira] [Updated] (CASSANDRA-19384) Avoid exposing intermediate node state during startup
[ https://issues.apache.org/jira/browse/CASSANDRA-19384?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19384: Fix Version/s: 5.x Since Version: 5.x Source Control Link: https://github.com/apache/cassandra/commit/5d4bcc797af882c64736b3f842cbf8bedbba184b Resolution: Fixed Status: Resolved (was: Ready to Commit) committed, thanks > Avoid exposing intermediate node state during startup > - > > Key: CASSANDRA-19384 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19384 > Project: Cassandra > Issue Type: Bug > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary-1.html, ci_summary.html, > result_details.tar-1.gz, result_details.tar.gz > > Time Spent: 20m > Remaining Estimate: 0h > > During startup we replay the local log, during this time we might expose > intermediate node states (via JMX for example). -- 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
[jira] [Updated] (CASSANDRA-19391) Flush metadata snapshot table on every write
[ https://issues.apache.org/jira/browse/CASSANDRA-19391?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19391: Source Control Link: https://github.com/apache/cassandra/commit/f31d72e8758ee1018435612775c73f7addbb6ec4 Resolution: Fixed Status: Resolved (was: Ready to Commit) committed, thanks > Flush metadata snapshot table on every write > > > Key: CASSANDRA-19391 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19391 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Low > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > > We depend on the latest snapshot when starting up, flushing avoids gaps > between latest snapshot and the most recent local log entry -- 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
[jira] [Updated] (CASSANDRA-19348) Fix serialization version check in InProgressSequences
[ https://issues.apache.org/jira/browse/CASSANDRA-19348?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19348: Since Version: 5.x Source Control Link: https://github.com/apache/cassandra/commit/60fe2dc61d3ea3d241e7e624f7a3ef94bd427b2f Resolution: Fixed Status: Resolved (was: Ready to Commit) committed, thanks > Fix serialization version check in InProgressSequences > -- > > Key: CASSANDRA-19348 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19348 > Project: Cassandra > Issue Type: Bug > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > -- 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
[jira] [Updated] (CASSANDRA-19390) Transformation.Kind should contain an explicit integer id
[ https://issues.apache.org/jira/browse/CASSANDRA-19390?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19390: Source Control Link: https://github.com/apache/cassandra/commit/0e5fb78c018245e1dfb25317d695665cb3317f3e Resolution: Fixed Status: Resolved (was: Ready to Commit) committed, thanks > Transformation.Kind should contain an explicit integer id > - > > Key: CASSANDRA-19390 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19390 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Low > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > -- 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
[jira] [Created] (CASSANDRA-19463) Test failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest
Marcus Eriksson created CASSANDRA-19463: --- Summary: Test failure: org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest Key: CASSANDRA-19463 URL: https://issues.apache.org/jira/browse/CASSANDRA-19463 Project: Cassandra Issue Type: Bug Reporter: Marcus Eriksson Looks like this was broken by CASSANDRA-18275 Timeouts in the log and then: {code} java.lang.IllegalStateException: Can't use shutdown instances, delegate is null at org.apache.cassandra.distributed.impl.AbstractCluster$Wrapper.delegate(AbstractCluster.java:283) at org.apache.cassandra.distributed.impl.DelegatingInvokableInstance.transfer(DelegatingInvokableInstance.java:49) at org.apache.cassandra.distributed.api.IInvokableInstance.runsOnInstance(IInvokableInstance.java:45) at org.apache.cassandra.distributed.api.IInvokableInstance.runOnInstance(IInvokableInstance.java:46) at org.apache.cassandra.distributed.shared.ClusterUtils.unpauseCommits(ClusterUtils.java:548) at org.apache.cassandra.fuzz.ring.ConsistentBootstrapTest.coordinatorIsBehindTest(ConsistentBootstrapTest.java:227) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.base/java.lang.reflect.Method.invoke(Method.java:566) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at com.intellij.junit4.JUnit4IdeaTestRunner.startRunnerWithArgs(JUnit4IdeaTestRunner.java:69) at com.intellij.rt.junit.IdeaTestRunner$Repeater$1.execute(IdeaTestRunner.java:38) at com.intellij.rt.execution.junit.TestsRepeater.repeat(TestsRepeater.java:11) at com.intellij.rt.junit.IdeaTestRunner$Repeater.startRunnerWithArgs(IdeaTestRunner.java:35) at com.intellij.rt.junit.JUnitStarter.prepareStreamsAndStart(JUnitStarter.java:232) at com.intellij.rt.junit.JUnitStarter.main(JUnitStarter.java:55) {code} -- 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
[jira] [Updated] (CASSANDRA-19153) Preclude irrecoverable log corruption in case split-brain situation during leader election with absent seeds
[ https://issues.apache.org/jira/browse/CASSANDRA-19153?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-19153: Test and Documentation Plan: ci run Status: Patch Available (was: Open) > Preclude irrecoverable log corruption in case split-brain situation during > leader election with absent seeds > > > Key: CASSANDRA-19153 > URL: https://issues.apache.org/jira/browse/CASSANDRA-19153 > Project: Cassandra > Issue Type: Improvement > Components: Transactional Cluster Metadata >Reporter: Alex Petrov >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 5.x > > Attachments: ci_summary.html, result_details.tar.gz > > Time Spent: 10m > Remaining Estimate: 0h > > It should be possible to detect a scenario where two partitioned nodes > independently elect themselves as the first CMS nodes in a brand new cluster. > In such a case, metadata changes should not be applied so that the conflict > can be resolved. -- 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