[jira] [Comment Edited] (CASSANDRA-17466) Shut repair task executor down without interruption to avoid compromising shared channel proxies

2022-03-29 Thread Berenguer Blasi (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-17466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17513991#comment-17513991
 ] 

Berenguer Blasi edited comment on CASSANDRA-17466 at 3/29/22, 10:37 AM:


I haven't dug deep. But could sbdy in the know check if this commit could be 
related to the recent rise in trunk failures on commit log closure please?


was (Author: bereng):
I haven't dug deep. But could sbdy in the know check if this commit be related 
to the recent rise in trunk failures on commit log closure?

> Shut repair task executor down without interruption to avoid compromising 
> shared channel proxies
> 
>
> Key: CASSANDRA-17466
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17466
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Caleb Rackliffe
>Assignee: Caleb Rackliffe
>Priority: Normal
> Fix For: 4.1, 4.0.4
>
>  Time Spent: 2h
>  Remaining Estimate: 0h
>
> If a {{RepairJob}} gets past validation, it builds a list of {{SyncTask}} 
> items and fires them off. If any one of those fails, we grab the relevant 
> exception and throw it up from {{RepairJob}} to {{RepairSession}}.
> {noformat}
> ERROR 2022-03-09T23:53:36,721 
> [Stream-Deserializer-/10.246.3.102:7000-d97958c4] 
> org.apache.cassandra.streaming.StreamSession:1110 - [Stream 
> #07c55da0-a047-11ec-8122-ab911c7a993f] Remote peer /10.246.3.102:7000 failed 
> stream session.
> {noformat}
> {{RepairSession}} then marks itself as being terminated and clears its 
> internal maps of active validations and sync tasks, but immediately before it 
> does that, it calls {{shutdownNow()}} on the executor that executes those 
> tasks. In the case of our failing stream session, we may still have other 
> running stream tasks whose threads' interrupt flag has been set, and this can 
> have some unintended negative consequences, because any {{ChannelProxy}} 
> interrupted in the middle of a blocking operation will both be closed and 
> throw a {{ClosedByInterruptException}}. (Keep in mind that we share 
> {{ChannelProxy}} instances outside a few specific cases, like those 
> introduced in CASSANDRA-15666.)
> We've seen this manifest in production in a couple ways, both of them while 
> trying to read from the {{peers_v2}} system table:
> {noformat}
> Exception in thread Thread[RepairJobTask:23,5,main]"^M
> exception="FSReadError in 
> .../data/system/peers_v2-c4325fbb8e5e3bafbd070f9250ed818e/system-peers_v2-nb-99-big-Data.db
> at 
> org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:143)
> at 
> org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:115)
> at 
> org.apache.cassandra.io.util.BufferManagingRebufferer.rebuffer(BufferManagingRebufferer.java:79)
> at 
> org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:68)
> at 
> org.apache.cassandra.io.util.RandomAccessReader.seek(RandomAccessReader.java:210)
> at 
> org.apache.cassandra.io.util.FileHandle.createReader(FileHandle.java:151)
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(SSTableReader.java:1628)
> at 
> org.apache.cassandra.db.columniterator.AbstractSSTableIterator.(AbstractSSTableIterator.java:96)
> at 
> org.apache.cassandra.db.columniterator.SSTableIterator.(SSTableIterator.java:48)
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:75)
> at 
> org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(BigTableReader.java:67)
> at 
> org.apache.cassandra.db.StorageHook$1.makeRowIterator(StorageHook.java:87)
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndSSTablesInTimestampOrder(SinglePartitionReadCommand.java:897)
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDiskInternal(SinglePartitionReadCommand.java:605)
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryMemtableAndDisk(SinglePartitionReadCommand.java:578)
> at 
> org.apache.cassandra.db.SinglePartitionReadCommand.queryStorage(SinglePartitionReadCommand.java:412)
> at 
> org.apache.cassandra.db.ReadCommand.executeLocally(ReadCommand.java:414)
> at 
> org.apache.cassandra.db.SinglePartitionReadQuery$Group.executeLocally(SinglePartitionReadQuery.java:242)
> at 
> org.apache.cassandra.db.SinglePartitionReadQuery$Group.executeInternal(SinglePartitionReadQuery.java:216)
> at 
> org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:458)
> at 

[jira] [Comment Edited] (CASSANDRA-17466) Shut repair task executor down without interruption to avoid compromising shared channel proxies

2022-03-24 Thread Caleb Rackliffe (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-17466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17511394#comment-17511394
 ] 

Caleb Rackliffe edited comment on CASSANDRA-17466 at 3/25/22, 3:06 AM:
---

I've submitted a patch that addresses the problem by 1.) avoiding abrupt 
{{shutdownNow()}} calls on repair failure and 2.) adds an abort mechanism for 
outstanding local repair sync tasks. There may have been a viable way to fix 
this particular issue by pulling the {{getPreferredIP()}} call out of repair 
task threads, but using an interrupting shutdown when we don't have a 
comprehensive understanding of our interrupt handling downstream seems risky. 
(Also, there is a fix contained here that actually makes sure the task executor 
pool shuts down. It was trying to shut itself down before, which wasn't going 
so well...)

|4.0|trunk|
|[patch|https://github.com/apache/cassandra/pull/1521]|[patch|https://github.com/apache/cassandra/pull/1522]|
|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-4.0=all]|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-trunk=all]|
|[Apache 
CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1540/]|[Apache
 CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1542/]|


was (Author: maedhroz):
I've submitted a patch that addresses the problem by 1.) avoiding abrupt 
{{shutdownNow()}} calls on repair failure and 2.) adds an abort mechanism for 
outstanding local repair sync tasks. There may have been a viable way to fix 
this particular issue by pulling the {{getPreferredIP()}} call out of repair 
task threads, but using an interrupting shutdown when we don't have a 
comprehensive understanding of our interrupt handling downstream seems risky. 
(Also, there is a fix contained here that actually makes sure the task executor 
pool shuts down. It was trying to shut itself down before, which wasn't going 
so well...)

|4.0|trunk|
|[patch|https://github.com/apache/cassandra/pull/1521]|[patch|https://github.com/apache/cassandra/pull/1522]|
|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-4.0=all]|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-trunk=all]|
|[Apache 
CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1540/]|[Apache
 CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1541/]|

> Shut repair task executor down without interruption to avoid compromising 
> shared channel proxies
> 
>
> Key: CASSANDRA-17466
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17466
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Caleb Rackliffe
>Assignee: Caleb Rackliffe
>Priority: Normal
> Fix For: 4.1, 4.0.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> If a {{RepairJob}} gets past validation, it builds a list of {{SyncTask}} 
> items and fires them off. If any one of those fails, we grab the relevant 
> exception and throw it up from {{RepairJob}} to {{RepairSession}}.
> {noformat}
> ERROR 2022-03-09T23:53:36,721 
> [Stream-Deserializer-/10.246.3.102:7000-d97958c4] 
> org.apache.cassandra.streaming.StreamSession:1110 - [Stream 
> #07c55da0-a047-11ec-8122-ab911c7a993f] Remote peer /10.246.3.102:7000 failed 
> stream session.
> {noformat}
> {{RepairSession}} then marks itself as being terminated and clears its 
> internal maps of active validations and sync tasks, but immediately before it 
> does that, it calls {{shutdownNow()}} on the executor that executes those 
> tasks. In the case of our failing stream session, we may still have other 
> running stream tasks whose threads' interrupt flag has been set, and this can 
> have some unintended negative consequences, because any {{ChannelProxy}} 
> interrupted in the middle of a blocking operation will both be closed and 
> throw a {{ClosedByInterruptException}}. (Keep in mind that we share 
> {{ChannelProxy}} instances outside a few specific cases, like those 
> introduced in CASSANDRA-15666.)
> We've seen this manifest in production in a couple ways, both of them while 
> trying to read from the {{peers_v2}} system table:
> {noformat}
> Exception in thread Thread[RepairJobTask:23,5,main]"^M
> exception="FSReadError in 
> .../data/system/peers_v2-c4325fbb8e5e3bafbd070f9250ed818e/system-peers_v2-nb-99-big-Data.db
> at 
> org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:143)
> at 
> org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:115)
> at 

[jira] [Comment Edited] (CASSANDRA-17466) Shut repair task executor down without interruption to avoid compromising shared channel proxies

2022-03-24 Thread Caleb Rackliffe (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-17466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17511394#comment-17511394
 ] 

Caleb Rackliffe edited comment on CASSANDRA-17466 at 3/25/22, 2:58 AM:
---

I've submitted a patch that addresses the problem by 1.) avoiding abrupt 
{{shutdownNow()}} calls on repair failure and 2.) adds an abort mechanism for 
outstanding local repair sync tasks. There may have been a viable way to fix 
this particular issue by pulling the {{getPreferredIP()}} call out of repair 
task threads, but using an interrupting shutdown when we don't have a 
comprehensive understanding of our interrupt handling downstream seems risky. 
(Also, there is a fix contained here that actually makes sure the task executor 
pool shuts down. It was trying to shut itself down before, which wasn't going 
so well...)

|4.0|trunk|
|[patch|https://github.com/apache/cassandra/pull/1521]|[patch|https://github.com/apache/cassandra/pull/1522]|
|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-4.0=all]|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-trunk=all]|
|[Apache 
CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1540/]|[Apache
 CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1541/]|


was (Author: maedhroz):
I've submitted a patch that addresses the problem by 1.) avoiding abrupt 
{{shutdownNow()}} calls on repair failure and 2.) adds an abort mechanism for 
outstanding local repair sync tasks. There may have been a viable way to fix 
this particular issue by pulling the {{getPreferredIP()}} call out of repair 
task threads, but using an interrupting shutdown when we don't have a 
comprehensive understanding of our interrupt handling downstream seems risky. 
(Also, there is a fix contained here that actually makes sure the task executor 
pool shuts down. It was trying to shut itself down before, which wasn't going 
so well...)

|4.0|trunk|
|[patch|https://github.com/apache/cassandra/pull/1521]|[patch|https://github.com/apache/cassandra/pull/1522]|
|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-4.0=all]|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-trunk=all]|
|[Apache 
CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1540/]|n/a|

> Shut repair task executor down without interruption to avoid compromising 
> shared channel proxies
> 
>
> Key: CASSANDRA-17466
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17466
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Caleb Rackliffe
>Assignee: Caleb Rackliffe
>Priority: Normal
> Fix For: 4.1, 4.0.x
>
>  Time Spent: 1h 20m
>  Remaining Estimate: 0h
>
> If a {{RepairJob}} gets past validation, it builds a list of {{SyncTask}} 
> items and fires them off. If any one of those fails, we grab the relevant 
> exception and throw it up from {{RepairJob}} to {{RepairSession}}.
> {noformat}
> ERROR 2022-03-09T23:53:36,721 
> [Stream-Deserializer-/10.246.3.102:7000-d97958c4] 
> org.apache.cassandra.streaming.StreamSession:1110 - [Stream 
> #07c55da0-a047-11ec-8122-ab911c7a993f] Remote peer /10.246.3.102:7000 failed 
> stream session.
> {noformat}
> {{RepairSession}} then marks itself as being terminated and clears its 
> internal maps of active validations and sync tasks, but immediately before it 
> does that, it calls {{shutdownNow()}} on the executor that executes those 
> tasks. In the case of our failing stream session, we may still have other 
> running stream tasks whose threads' interrupt flag has been set, and this can 
> have some unintended negative consequences, because any {{ChannelProxy}} 
> interrupted in the middle of a blocking operation will both be closed and 
> throw a {{ClosedByInterruptException}}. (Keep in mind that we share 
> {{ChannelProxy}} instances outside a few specific cases, like those 
> introduced in CASSANDRA-15666.)
> We've seen this manifest in production in a couple ways, both of them while 
> trying to read from the {{peers_v2}} system table:
> {noformat}
> Exception in thread Thread[RepairJobTask:23,5,main]"^M
> exception="FSReadError in 
> .../data/system/peers_v2-c4325fbb8e5e3bafbd070f9250ed818e/system-peers_v2-nb-99-big-Data.db
> at 
> org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:143)
> at 
> org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:115)
> at 
> 

[jira] [Comment Edited] (CASSANDRA-17466) Shut repair task executor down without interruption to avoid compromising shared channel proxies

2022-03-24 Thread Caleb Rackliffe (Jira)


[ 
https://issues.apache.org/jira/browse/CASSANDRA-17466?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17511394#comment-17511394
 ] 

Caleb Rackliffe edited comment on CASSANDRA-17466 at 3/24/22, 3:05 PM:
---

I've submitted a patch that addresses the problem by 1.) avoiding abrupt 
{{shutdownNow()}} calls on repair failure and 2.) adds an abort mechanism for 
outstanding local repair sync tasks. There may have been a viable way to fix 
this particular issue by pulling the {{getPreferredIP()}} call out of repair 
task threads, but using an interrupting shutdown when we don't have a 
comprehensive understanding of our interrupt handling downstream seems risky. 
(Also, there is a fix contained here that actually makes sure the task executor 
pool shuts down. It was trying to shut itself down before, which wasn't going 
so well...)

|4.0|trunk|
|[patch|https://github.com/apache/cassandra/pull/1521]|[patch|https://github.com/apache/cassandra/pull/1522]|
|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-4.0=all]|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-trunk=all]|
|[Apache 
CI|https://ci-cassandra.apache.org/view/patches/job/Cassandra-devbranch/1540/]|n/a|


was (Author: maedhroz):
I've submitted a patch that addresses the problem by 1.) avoiding abrupt 
{{shutdownNow()}} calls on repair failure and 2.) adds an abort mechanism for 
outstanding local repair sync tasks. There may have been a viable way to fix 
this particular issue by pulling the {{getPreferredIP()}} call out of repair 
task threads, but using an interrupting shutdown when we don't have a 
comprehensive understanding of our interrupt handling downstream seems risky. 
(Also, there is a fix contained here that actually makes sure the task executor 
pool shuts down. It was trying to shut itself down before, which wasn't going 
so well...)

|4.0|trunk|
|[patch|https://github.com/apache/cassandra/pull/1521]|[patch|https://github.com/apache/cassandra/pull/1522]|
|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-4.0=all]|[CircleCI|https://app.circleci.com/pipelines/github/maedhroz/cassandra?branch=CASSANDRA-17466-trunk=all]|

> Shut repair task executor down without interruption to avoid compromising 
> shared channel proxies
> 
>
> Key: CASSANDRA-17466
> URL: https://issues.apache.org/jira/browse/CASSANDRA-17466
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair
>Reporter: Caleb Rackliffe
>Assignee: Caleb Rackliffe
>Priority: Normal
> Fix For: 4.1, 4.0.x
>
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> If a {{RepairJob}} gets past validation, it builds a list of {{SyncTask}} 
> items and fires them off. If any one of those fails, we grab the relevant 
> exception and throw it up from {{RepairJob}} to {{RepairSession}}.
> {noformat}
> ERROR 2022-03-09T23:53:36,721 
> [Stream-Deserializer-/10.246.3.102:7000-d97958c4] 
> org.apache.cassandra.streaming.StreamSession:1110 - [Stream 
> #07c55da0-a047-11ec-8122-ab911c7a993f] Remote peer /10.246.3.102:7000 failed 
> stream session.
> {noformat}
> {{RepairSession}} then marks itself as being terminated and clears its 
> internal maps of active validations and sync tasks, but immediately before it 
> does that, it calls {{shutdownNow()}} on the executor that executes those 
> tasks. In the case of our failing stream session, we may still have other 
> running stream tasks whose threads' interrupt flag has been set, and this can 
> have some unintended negative consequences, because any {{ChannelProxy}} 
> interrupted in the middle of a blocking operation will both be closed and 
> throw a {{ClosedByInterruptException}}. (Keep in mind that we share 
> {{ChannelProxy}} instances outside a few specific cases, like those 
> introduced in CASSANDRA-15666.)
> We've seen this manifest in production in a couple ways, both of them while 
> trying to read from the {{peers_v2}} system table:
> {noformat}
> Exception in thread Thread[RepairJobTask:23,5,main]"^M
> exception="FSReadError in 
> .../data/system/peers_v2-c4325fbb8e5e3bafbd070f9250ed818e/system-peers_v2-nb-99-big-Data.db
> at 
> org.apache.cassandra.io.util.ChannelProxy.read(ChannelProxy.java:143)
> at 
> org.apache.cassandra.io.util.CompressedChunkReader$Standard.readChunk(CompressedChunkReader.java:115)
> at 
> org.apache.cassandra.io.util.BufferManagingRebufferer.rebuffer(BufferManagingRebufferer.java:79)
> at 
> org.apache.cassandra.io.util.RandomAccessReader.reBufferAt(RandomAccessReader.java:68)
> at 
>