[jira] [Commented] (CASSANDRA-15059) Gossiper#markAlive can race with Gossiper#markDead
[ https://issues.apache.org/jira/browse/CASSANDRA-15059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16803221#comment-16803221 ] Ariel Weisberg commented on CASSANDRA-15059: Well as part of this ticket. There is a question of scope. Yes this fixes the bug, but it arguably doesn't address the reason the bug exists. > Gossiper#markAlive can race with Gossiper#markDead > -- > > Key: CASSANDRA-15059 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15059 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Gossip >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Normal > > The Gossiper class is not threadsafe and assumes all state changes happen in > a single thread (the gossip stage). Gossiper#convict, however, can be called > from the GossipTasks thread. This creates a race where calls to > Gossiper#markAlive and Gossiper#markDead can interleave, corrupting gossip > state. Gossiper#assassinateEndpoint has a similar problem, being called from > the mbean server thread. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15059) Gossiper#markAlive can race with Gossiper#markDead
[ https://issues.apache.org/jira/browse/CASSANDRA-15059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16803112#comment-16803112 ] Blake Eggleston commented on CASSANDRA-15059: - {quote} Future wise this doesn't do anything to address the underlying fragility in how Gossiper doesn't document what is safe to call from outside the Gossip thread and what isn't. It also doesn't validate the correct thread is running a given method. I think we want to add two (or three?) interfaces that Gossiper can implement. One is for methods that are non-mutating and safe to call from any thread. The other is for methods for that should only be called from the Gossip stage thread. And maybe a third which contains methods that mutate, but block on the Gossip stage. So Gossiper.instance would go away and you would have a reference to the Gossiper via one of the 2-3 interfaces. Additionally any method that should only run in Gossip stage interface should have a Preconditions.checkState checking that it's actually running in the Gossip stage. {quote} Do you mean as part of this ticket, or as future improvements to gossip? > Gossiper#markAlive can race with Gossiper#markDead > -- > > Key: CASSANDRA-15059 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15059 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Gossip >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Normal > > The Gossiper class is not threadsafe and assumes all state changes happen in > a single thread (the gossip stage). Gossiper#convict, however, can be called > from the GossipTasks thread. This creates a race where calls to > Gossiper#markAlive and Gossiper#markDead can interleave, corrupting gossip > state. Gossiper#assassinateEndpoint has a similar problem, being called from > the mbean server thread. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15059) Gossiper#markAlive can race with Gossiper#markDead
[ https://issues.apache.org/jira/browse/CASSANDRA-15059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16802986#comment-16802986 ] Ariel Weisberg edited comment on CASSANDRA-15059 at 3/27/19 4:23 PM: - Finished my review. [Should this always wrap in AssertionError?|https://github.com/apache/cassandra/compare/cassandra-3.0...bdeggleston:15059-3.0?expand=1#diff-8be666c70553b1f0017a01458c490f47R340]. I get that InterruptedException is probably an Error, but ExecutionException is just a run of the mill exception. It's not super important as it won't have a functional difference. I think this fixes the problem it sets out to fix. I checked the call hierarchy for {{Gossiper.runInGossipStageBlocking}} and I think that every caller is not holding any locks or resources that might stop the Gossip thread from making progress. All of them don't appear to be holding anything obvious. Documentation wise definitely document {{runInGossipStageBlocking}}, {{assassinate}}, and {{convict}} warning about the risk of deadlock if the Gossip thread ends up needing to acquire a resource via a listener (or FD's listener if there is a path for that) the calling thread holds. Future wise this doesn't do anything to address the underlying fragility in how Gossiper doesn't document what is safe to call from outside the Gossip thread and what isn't. It also doesn't validate the correct thread is running a given method. I think we want to add two (or three?) interfaces that Gossiper can implement. One is for methods that are non-mutating and safe to call from any thread. The other is for methods for that should only be called from the Gossip stage thread. And maybe a third which contains methods that mutate, but block on the Gossip stage. So {{Gossiper.instance}} would go away and you would have a reference to the Gossiper via one of the 2-3 interfaces. Additionally any method that should only run in Gossip stage interface should have a {{Preconditions.checkState}} checking that it's actually running in the Gossip stage. WDYT? was (Author: aweisberg): Finished my review. [Should this always wrap in AssertionError?|https://github.com/apache/cassandra/compare/cassandra-3.0...bdeggleston:15059-3.0?expand=1#diff-8be666c70553b1f0017a01458c490f47R340]. I get that InterruptedException is probably an Error, but ExecutionException is just a run of the mill exception. It's not super important as it won't have a functional difference. I think this fixes the problem it sets out to fix. I checked the call hierarchy for {{Gossiper.runInGossipStageBlocking}} and I think that every caller is not holding any locks or resources that might stop the Gossip thread from making progress. All of them don't appear to be holding anything obvious. Documentation wise definitely document {{runInGossipStageBlocking}}, {{assassinate}}, and {{convict}} warning about the risk of deadlock if the Gossip thread ends up needing to acquire a resource via a listener (or FD's listener if there is a path for that) the calling thread holds. Future wise this doesn't do anything to address the underlying fragility in how Gossiper doesn't document what is safe to call from outside the Gossip thread and what isn't. It also doesn't validate the correct thread is running a given method. I think we want to add two (or three?) interfaces that Gossiper can implement. One is for methods that are non-mutating and safe to call from any thread. The other is for methods for that should only be called from the Gossip stage thread. And maybe a third which contains methods that mutate, but block on the Gossip stage. So Gossiper.instance would go away and you would have a reference to the Gossiper via one of the 2-3 interfaces. Additionally any method that should only run in Gossip stage interface has a {{Preconditions.checkState}} checking that it's actually running in the Gossip stage. WDYT? > Gossiper#markAlive can race with Gossiper#markDead > -- > > Key: CASSANDRA-15059 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15059 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Gossip >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Normal > > The Gossiper class is not threadsafe and assumes all state changes happen in > a single thread (the gossip stage). Gossiper#convict, however, can be called > from the GossipTasks thread. This creates a race where calls to > Gossiper#markAlive and Gossiper#markDead can interleave, corrupting gossip > state. Gossiper#assassinateEndpoint has a similar problem, being called from > the mbean server thread. -- This message was sent by Atlassian JIRA (v7.6.3#76005)
[jira] [Comment Edited] (CASSANDRA-15059) Gossiper#markAlive can race with Gossiper#markDead
[ https://issues.apache.org/jira/browse/CASSANDRA-15059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16802986#comment-16802986 ] Ariel Weisberg edited comment on CASSANDRA-15059 at 3/27/19 4:22 PM: - Finished my review. [Should this always wrap in AssertionError?|https://github.com/apache/cassandra/compare/cassandra-3.0...bdeggleston:15059-3.0?expand=1#diff-8be666c70553b1f0017a01458c490f47R340]. I get that InterruptedException is probably an Error, but ExecutionException is just a run of the mill exception. It's not super important as it won't have a functional difference. I think this fixes the problem it sets out to fix. I checked the call hierarchy for {{Gossiper.runInGossipStageBlocking}} and I think that every caller is not holding any locks or resources that might stop the Gossip thread from making progress. All of them don't appear to be holding anything obvious. Documentation wise definitely document {{runInGossipStageBlocking}}, {{assassinate}}, and {{convict}} warning about the risk of deadlock if the Gossip thread ends up needing to acquire a resource via a listener (or FD's listener if there is a path for that) the calling thread holds. Future wise this doesn't do anything to address the underlying fragility in how Gossiper doesn't document what is safe to call from outside the Gossip thread and what isn't. It also doesn't validate the correct thread is running a given method. I think we want to add two (or three?) interfaces that Gossiper can implement. One is for methods that are non-mutating and safe to call from any thread. The other is for methods for that should only be called from the Gossip stage thread. And maybe a third which contains methods that mutate, but block on the Gossip stage. So Gossiper.instance would go away and you would have a reference to the Gossiper via one of the 2-3 interfaces. Additionally any method that should only run in Gossip stage interface has a {{Preconditions.checkState}} checking that it's actually running in the Gossip stage. WDYT? was (Author: aweisberg): Finished my review. I can't tell what happened with CircleCI as it's quite complicated now, but it doesn't look like everything ran and/or passed. [Should this always wrap in AssertionError?|https://github.com/apache/cassandra/compare/cassandra-3.0...bdeggleston:15059-3.0?expand=1#diff-8be666c70553b1f0017a01458c490f47R340]. I get that InterruptedException is probably an Error, but ExecutionException is just a run of the mill exception. It's not super important as it won't have a functional difference. I think this fixes the problem it sets out to fix. I checked the call hierarchy for {{Gossiper.runInGossipStageBlocking}} and I think that every caller is not holding any locks or resources that might stop the Gossip thread from making progress. All of them don't appear to be holding anything obvious. Documentation wise definitely document {{runInGossipStageBlocking}}, {{assassinate}}, and {{convict}} warning about the risk of deadlock if the Gossip thread ends up needing to acquire a resource via a listener (or FD's listener if there is a path for that) the calling thread holds. Future wise this doesn't do anything to address the underlying fragility in how Gossiper doesn't document what is safe to call from outside the Gossip thread and what isn't. It also doesn't validate the correct thread is running a given method. I think we want to add two (or three?) interfaces that Gossiper can implement. One is for methods that are non-mutating and safe to call from any thread. The other is for methods for that should only be called from the Gossip stage thread. And maybe a third which contains methods that mutate, but block on the Gossip stage. So Gossiper.instance would go away and you would have a reference to the Gossiper via one of the 2-3 interfaces. Additionally any method that should only run in Gossip stage interface has a {{Preconditions.checkState}} checking that it's actually running in the Gossip stage. WDYT? > Gossiper#markAlive can race with Gossiper#markDead > -- > > Key: CASSANDRA-15059 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15059 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Gossip >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Normal > > The Gossiper class is not threadsafe and assumes all state changes happen in > a single thread (the gossip stage). Gossiper#convict, however, can be called > from the GossipTasks thread. This creates a race where calls to > Gossiper#markAlive and Gossiper#markDead can interleave, corrupting gossip > state. Gossiper#assassinateEndpoint has a similar problem, being called from > the mbean s
[jira] [Commented] (CASSANDRA-15059) Gossiper#markAlive can race with Gossiper#markDead
[ https://issues.apache.org/jira/browse/CASSANDRA-15059?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16802986#comment-16802986 ] Ariel Weisberg commented on CASSANDRA-15059: Finished my review. I can't tell what happened with CircleCI as it's quite complicated now, but it doesn't look like everything ran and/or passed. [Should this always wrap in AssertionError?|https://github.com/apache/cassandra/compare/cassandra-3.0...bdeggleston:15059-3.0?expand=1#diff-8be666c70553b1f0017a01458c490f47R340]. I get that InterruptedException is probably an Error, but ExecutionException is just a run of the mill exception. It's not super important as it won't have a functional difference. I think this fixes the problem it sets out to fix. I checked the call hierarchy for {{Gossiper.runInGossipStageBlocking}} and I think that every caller is not holding any locks or resources that might stop the Gossip thread from making progress. All of them don't appear to be holding anything obvious. Documentation wise definitely document {{runInGossipStageBlocking}}, {{assassinate}}, and {{convict}} warning about the risk of deadlock if the Gossip thread ends up needing to acquire a resource via a listener (or FD's listener if there is a path for that) the calling thread holds. Future wise this doesn't do anything to address the underlying fragility in how Gossiper doesn't document what is safe to call from outside the Gossip thread and what isn't. It also doesn't validate the correct thread is running a given method. I think we want to add two (or three?) interfaces that Gossiper can implement. One is for methods that are non-mutating and safe to call from any thread. The other is for methods for that should only be called from the Gossip stage thread. And maybe a third which contains methods that mutate, but block on the Gossip stage. So Gossiper.instance would go away and you would have a reference to the Gossiper via one of the 2-3 interfaces. Additionally any method that should only run in Gossip stage interface has a {{Preconditions.checkState}} checking that it's actually running in the Gossip stage. WDYT? > Gossiper#markAlive can race with Gossiper#markDead > -- > > Key: CASSANDRA-15059 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15059 > Project: Cassandra > Issue Type: Bug > Components: Cluster/Gossip >Reporter: Blake Eggleston >Assignee: Blake Eggleston >Priority: Normal > > The Gossiper class is not threadsafe and assumes all state changes happen in > a single thread (the gossip stage). Gossiper#convict, however, can be called > from the GossipTasks thread. This creates a race where calls to > Gossiper#markAlive and Gossiper#markDead can interleave, corrupting gossip > state. Gossiper#assassinateEndpoint has a similar problem, being called from > the mbean server thread. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15045) Fix index summary redistribution compaction cancellation issues
[ https://issues.apache.org/jira/browse/CASSANDRA-15045?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16802972#comment-16802972 ] Jordan West commented on CASSANDRA-15045: - Good catch on the Java 7 needs in 2.2. Thanks! > Fix index summary redistribution compaction cancellation issues > --- > > Key: CASSANDRA-15045 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15045 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Normal > Fix For: 2.2.15, 3.0.19, 3.11.5, 4.0 > > > We can't cancel ongoing index summary redistributions currently due to > {{CompactionInfo}} returning null for {{getTableMetadata/getCFMetaData}} > [here|https://github.com/apache/cassandra/blob/67d613204fa4fb9584f11ec9886a0e7a0d622e92/src/java/org/apache/cassandra/db/compaction/CompactionManager.java#L1814] > for index summary redistributions > CASSANDRA-14935 also introduced a bug where we track the wrong sstables for > index summary redistributions -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-11066) Static column with secondary index on clustering column causes ArrayIndexOutOfBoundsException
[ https://issues.apache.org/jira/browse/CASSANDRA-11066?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Artem Soloviov updated CASSANDRA-11066: --- Description: Enabling secondary index on clustering column breaks static column support: {code:java} Connected to Cassandra Cluster at 127.0.0.1:9042. [cqlsh 5.0.1 | Cassandra 3.2.1 | CQL spec 3.4.0 | Native protocol v4] Use HELP for help. cqlsh> CREATE KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; cqlsh> USE test; cqlsh:test> CREATE TABLE t (k text , s text static, i int, PRIMARY KEY (k, i)); cqlsh:test> CREATE INDEX t_idx ON t(i); cqlsh:test> INSERT INTO t (k,s,i) VALUES ('k','static value', 0); {code} causes {noformat} java.lang.ArrayIndexOutOfBoundsException: 0 at org.apache.cassandra.db.AbstractClusteringPrefix.get(AbstractClusteringPrefix.java:59) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.composites.ClusteringColumnIndex.getIndexedValue(ClusteringColumnIndex.java:60) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex.getIndexedValue(CassandraIndex.java:598) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex.insert(CassandraIndex.java:490) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex.access$100(CassandraIndex.java:53) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex$1.indexPrimaryKey(CassandraIndex.java:437) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex$1.insertRow(CassandraIndex.java:347) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.SecondaryIndexManager$WriteTimeTransaction.onInserted(SecondaryIndexManager.java:803) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.db.partitions.AtomicBTreePartition$RowUpdater.apply(AtomicBTreePartition.java:275) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.db.partitions.AtomicBTreePartition.addAllWithSizeDelta(AtomicBTreePartition.java:154) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.db.Memtable.put(Memtable.java:239) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.db.ColumnFamilyStore.apply(ColumnFamilyStore.java:1225) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:494) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:384) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.db.Mutation.apply(Mutation.java:202) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.service.StorageProxy$$Lambda$223/885149282.run(Unknown Source) ~[na:na] at org.apache.cassandra.service.StorageProxy$7.runMayThrow(StorageProxy.java:1276) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.service.StorageProxy$LocalMutationRunnable.run(StorageProxy.java:2433) [apache-cassandra-3.2.1.jar:3.2.1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_51] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:164) [apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) [apache-cassandra-3.2.1.jar:3.2.1] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_51] {noformat} was: Enabling secondary index on clustering column breaks static column support: {code} Connected to Exodentity Cassandra Cluster at 127.0.0.1:9042. [cqlsh 5.0.1 | Cassandra 3.2.1 | CQL spec 3.4.0 | Native protocol v4] Use HELP for help. cqlsh> CREATE KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; cqlsh> USE test; cqlsh:test> CREATE TABLE t (k text , s text static, i int, PRIMARY KEY (k, i)); cqlsh:test> CREATE INDEX t_idx ON t(i); cqlsh:test> INSERT INTO t (k,s,i) VALUES ('k','static value', 0); {code} causes {noformat} java.lang.ArrayIndexOutOfBoundsException: 0 at org.apache.cassandra.db.AbstractClusteringPrefix.get(AbstractClusteringPrefix.java:59) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.composites.ClusteringColumnIndex.getIndexedValue(ClusteringColumnIndex.java:60) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex.getIndexedValue(CassandraIndex.java:598) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex.insert(CassandraIndex.java:490) ~[apache-cassandra-3.2.1.jar:3.2.1] at org.apache.cassandra.index.internal.CassandraIndex.access$100(CassandraIndex.java:53) ~[apache-cassandra-3.2.1.
[jira] [Updated] (CASSANDRA-15053) Fix handling FS errors on writing and reading flat files - LogTransaction and hints
[ https://issues.apache.org/jira/browse/CASSANDRA-15053?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-15053: -- Status: Review In Progress (was: Patch Available) > Fix handling FS errors on writing and reading flat files - LogTransaction and > hints > --- > > Key: CASSANDRA-15053 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15053 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Hints >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa >Priority: Normal > Fix For: 4.0, 3.0.x, 3.11.x > > > We currently fail to handle and propagate IO errors when dealing with > transaction log and hints. It's trivial to fix this behaviour to ensure that > disk failure policy is properly invoked in error scenarios. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15053) Fix handling FS errors on writing and reading flat files - LogTransaction and hints
[ https://issues.apache.org/jira/browse/CASSANDRA-15053?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-15053: -- Fix Version/s: (was: 3.11.x) (was: 3.0.x) 3.11.5 3.0.19 Since Version: 3.0.0 Status: Resolved (was: Ready to Commit) Resolution: Fixed > Fix handling FS errors on writing and reading flat files - LogTransaction and > hints > --- > > Key: CASSANDRA-15053 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15053 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Hints >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa >Priority: Normal > Fix For: 3.0.19, 3.11.5, 4.0 > > > We currently fail to handle and propagate IO errors when dealing with > transaction log and hints. It's trivial to fix this behaviour to ensure that > disk failure policy is properly invoked in error scenarios. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Updated] (CASSANDRA-15053) Fix handling FS errors on writing and reading flat files - LogTransaction and hints
[ https://issues.apache.org/jira/browse/CASSANDRA-15053?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-15053: -- Status: Ready to Commit (was: Review In Progress) > Fix handling FS errors on writing and reading flat files - LogTransaction and > hints > --- > > Key: CASSANDRA-15053 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15053 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Hints >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa >Priority: Normal > Fix For: 4.0, 3.0.x, 3.11.x > > > We currently fail to handle and propagate IO errors when dealing with > transaction log and hints. It's trivial to fix this behaviour to ensure that > disk failure policy is properly invoked in error scenarios. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Commented] (CASSANDRA-15053) Fix handling FS errors on writing and reading flat files - LogTransaction and hints
[ https://issues.apache.org/jira/browse/CASSANDRA-15053?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16802881#comment-16802881 ] Aleksey Yeschenko commented on CASSANDRA-15053: --- Committed as [ba325955463076987eca476309043fcdd59dad7c|https://github.com/apache/cassandra/commit/ba325955463076987eca476309043fcdd59dad7c] to 3.0 and merged upwards, thanks. > Fix handling FS errors on writing and reading flat files - LogTransaction and > hints > --- > > Key: CASSANDRA-15053 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15053 > Project: Cassandra > Issue Type: Bug > Components: Consistency/Hints >Reporter: Jeff Jirsa >Assignee: Jeff Jirsa >Priority: Normal > Fix For: 4.0, 3.0.x, 3.11.x > > > We currently fail to handle and propagate IO errors when dealing with > transaction log and hints. It's trivial to fix this behaviour to ensure that > disk failure policy is properly invoked in error scenarios. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] branch cassandra-3.11 updated (08018ab -> 1152417)
This is an automated email from the ASF dual-hosted git repository. aleksey pushed a change to branch cassandra-3.11 in repository https://gitbox.apache.org/repos/asf/cassandra.git. from 08018ab Merge branch 'cassandra-3.0' into cassandra-3.11 add ba32595 Fix handling FS errors on writing and reading flat files - LogTransaction and hints add 1152417 Merge branch 'cassandra-3.0' into cassandra-3.11 No new revisions were added by this update. Summary of changes: CHANGES.txt| 1 + .../apache/cassandra/db/lifecycle/LogReplica.java | 42 +++--- .../cassandra/db/lifecycle/LogReplicaSet.java | 34 -- .../cassandra/db/lifecycle/LogTransaction.java | 11 -- .../org/apache/cassandra/hints/HintsCatalog.java | 24 +++-- .../apache/cassandra/hints/HintsWriteExecutor.java | 17 - .../org/apache/cassandra/io/util/FileUtils.java| 15 .../org/apache/cassandra/utils/NativeLibrary.java | 10 -- 8 files changed, 131 insertions(+), 23 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] branch cassandra-3.0 updated (b48ba8a -> ba32595)
This is an automated email from the ASF dual-hosted git repository. aleksey pushed a change to branch cassandra-3.0 in repository https://gitbox.apache.org/repos/asf/cassandra.git. from b48ba8a Merge branch 'cassandra-2.2' into cassandra-3.0 add ba32595 Fix handling FS errors on writing and reading flat files - LogTransaction and hints No new revisions were added by this update. Summary of changes: CHANGES.txt| 1 + .../apache/cassandra/db/lifecycle/LogReplica.java | 42 +++--- .../cassandra/db/lifecycle/LogReplicaSet.java | 33 - .../cassandra/db/lifecycle/LogTransaction.java | 8 +++-- .../org/apache/cassandra/hints/HintsCatalog.java | 24 +++-- .../apache/cassandra/hints/HintsWriteExecutor.java | 17 - .../org/apache/cassandra/io/util/FileUtils.java| 16 + .../org/apache/cassandra/utils/NativeLibrary.java | 10 -- 8 files changed, 130 insertions(+), 21 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[cassandra] branch trunk updated (8fdd6c7 -> df8e068)
This is an automated email from the ASF dual-hosted git repository. aleksey pushed a change to branch trunk in repository https://gitbox.apache.org/repos/asf/cassandra.git. from 8fdd6c7 Merge branch 'cassandra-3.11' into trunk new ba32595 Fix handling FS errors on writing and reading flat files - LogTransaction and hints new 1152417 Merge branch 'cassandra-3.0' into cassandra-3.11 new df8e068 Merge branch 'cassandra-3.11' into trunk The 24553 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: CHANGES.txt| 1 + .../apache/cassandra/db/lifecycle/LogReplica.java | 42 +++--- .../cassandra/db/lifecycle/LogReplicaSet.java | 34 -- .../cassandra/db/lifecycle/LogTransaction.java | 10 -- .../org/apache/cassandra/hints/HintsCatalog.java | 24 +++-- .../apache/cassandra/hints/HintsWriteExecutor.java | 17 - .../org/apache/cassandra/io/util/FileUtils.java| 15 .../org/apache/cassandra/utils/NativeLibrary.java | 10 -- 8 files changed, 130 insertions(+), 23 deletions(-) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org
[jira] [Comment Edited] (CASSANDRA-15069) Tombstone/Partition not purged after gc_grace_seconds
[ https://issues.apache.org/jira/browse/CASSANDRA-15069?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16802281#comment-16802281 ] Pedro Gordo edited comment on CASSANDRA-15069 at 3/27/19 8:53 AM: -- I have been trying to debug this but I'm stuck at the CompactionTask.runMayThrow method. I see that it's during this method execution that the new SSTable files are created, but I can't pinpoint the exact place where the data is scanned and the partition 96 is selected for the new SSTables files (generated from running garbagecollect). I wanted to see why this partition is being select for the new SSTable file when the gc_grace_seconds has already passed. If anyone would be willing to give me a hand to get past this point and find the place in the code where this criterion exists, I could investigate further and submit a patch. On the basis that this is a bug ofc, I might be missing something... was (Author: pedro_gordo): I have been trying to debug this but I'm stuck at the CompactionTask.runMayThrow method. I see that it's during this method execution that the new SSTable files are created, but I can't pinpoint the exact place where the data is scanned and the partition 96 is selected for the new SSTables files (generated from running garbagecollect). I wanted to see why this partition is being select for the new SSTable file when the gc_grace_seconds as already passed. If anyone would be willing to give me a hand to get past this point and find the place in the code where this criterion exists, I could investigate further and submit a patch. On the basis that this is a bug ofc, I might be missing something... > Tombstone/Partition not purged after gc_grace_seconds > - > > Key: CASSANDRA-15069 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15069 > Project: Cassandra > Issue Type: Bug > Components: Local/Compaction, Local/SSTable >Reporter: Pedro Gordo >Priority: Normal > Fix For: 3.11.3 > > Attachments: schema.cql, sstables.tar > > > During a tombstone purge (reducing gc_grace_seconds to zero and running > `nodetool garbagecollect`), I noticed that when doing a dump of the SSTable, > sometimes, there are a few partitions that do not get completely purged, even > with gc_grace_seconds set to zero. I was able to replicate this in a small > test dataset, from which I have attached the SSTable files and the schema to > this ticket so that you can verify this as well. > Doing a dump of the mc-51-big-Data.db file, you'll notice the following > partition: > { > "partition" : { > "key" : [ "96" ], > "position" : 285, > "deletion_info" : { "marked_deleted" : "2019-03-14T21:31:55.244490Z", > "local_delete_time" : "2019-03-14T21:31:55Z" } > }, > "rows" : [ ] > }, > As you can see, the rows were removed correctly by the garbagecollect, but > the partition record, continues there, and never goes away. > From the client side, no data is returned, so it's good there. But regardless > of that, this partition should not be present in the SSTable file. -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org