[jira] [Commented] (CASSANDRA-15592) IllegalStateException in gossip after removing node

2020-02-21 Thread Marcus Olsson (Jira)


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

Marcus Olsson commented on CASSANDRA-15592:
---

The branches are updated. 3.11/4.0 contains an additional smaller patch with 
the difference in the test setup compared to 3.0.

Additionally I ran the test code on 3.0/3.11/4.0 without the patch and they all 
got the IllegalStateException mentioned in the ticket (example output from 3.11 
below):
{noformat}
INFO  [GossipStage:1] 2020-02-21 13:48:47,395 Gossiper.java:1106 - InetAddress 
/127.0.0.2 is now DOWN
DEBUG [main] 2020-02-21 13:48:47,398 Gossiper.java:1746 - adding expire time 
for endpoint : /127.0.0.2 (1582289327272)
DEBUG [main] 2020-02-21 13:48:47,399 Gossiper.java:894 - time is expiring for 
endpoint : /127.0.0.2 (1582289327272)

java.lang.IllegalStateException: Attempting gossip state mutation from illegal 
thread: main

at 
org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
at 
org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465)
at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:896)
at 
org.apache.cassandra.gms.ExpireEndpointTest.testExpireEndpoint(ExpireEndpointTest.java:59)
...
{noformat}

> IllegalStateException in gossip after removing node
> ---
>
> Key: CASSANDRA-15592
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15592
> Project: Cassandra
>  Issue Type: Bug
>  Components: Cluster/Gossip
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Normal
>
> In one of our test environments we encountered the following exception:
> {noformat}
> 2020-02-02T10:50:13.276+0100 [GossipTasks:1] ERROR 
> o.a.c.u.NoSpamLogger$NoSpamLogStatement:97 log 
> java.lang.IllegalStateException: Attempting gossip state mutation from 
> illegal thread: GossipTasks:1
>  at 
> org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
>  at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465)
>  at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895)
>  at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78)
>  at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240)
>  at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
>  at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>  at java.lang.Thread.run(Thread.java:748)
> java.lang.IllegalStateException: Attempting gossip state mutation from 
> illegal thread: GossipTasks:1
>  at 
> org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_231]
>  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
> [na:1.8.0_231]
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  [na:1.8.0_231]
>  at 
> 

[jira] [Commented] (CASSANDRA-15592) IllegalStateException in gossip after removing node

2020-02-21 Thread Marcus Olsson (Jira)


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

Marcus Olsson commented on CASSANDRA-15592:
---

Sure thing.

In order to get the test case small and to avoid starting up the full Gossiper 
I modified "doStatusCheck()" to package-private (with @VisibleForTesting).
The test case injects an application state for a "remote" node, marks it down, 
removes it and then adds an expiry time earlier than "now" before running 
"doStatusCheck()" so that it should be evicted.
I believe this should represent the state transition that has occurred but 
gossip state transitions are not my strong suite so please correct me if I'm 
wrong.

I'll update the branches shortly.

> IllegalStateException in gossip after removing node
> ---
>
> Key: CASSANDRA-15592
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15592
> Project: Cassandra
>  Issue Type: Bug
>  Components: Cluster/Gossip
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Normal
>
> In one of our test environments we encountered the following exception:
> {noformat}
> 2020-02-02T10:50:13.276+0100 [GossipTasks:1] ERROR 
> o.a.c.u.NoSpamLogger$NoSpamLogStatement:97 log 
> java.lang.IllegalStateException: Attempting gossip state mutation from 
> illegal thread: GossipTasks:1
>  at 
> org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
>  at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465)
>  at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895)
>  at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78)
>  at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240)
>  at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
>  at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>  at java.lang.Thread.run(Thread.java:748)
> java.lang.IllegalStateException: Attempting gossip state mutation from 
> illegal thread: GossipTasks:1
>  at 
> org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_231]
>  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
> [na:1.8.0_231]
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  [na:1.8.0_231]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>  ~[netty-all-4.1.42.Final.jar:4.1.42.Final]
>  at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_231]
> {noformat}
> Since CASSANDRA-15059 we check that all state changes are performed in the 
> GossipStage but it seems like it was still 

[jira] [Commented] (CASSANDRA-15592) IllegalStateException in gossip after removing node

2020-02-20 Thread Marcus Olsson (Jira)


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

Marcus Olsson commented on CASSANDRA-15592:
---

|[3.0|https://github.com/emolsson/cassandra/tree/CASSANDRA-15592-3.0]|
|[3.11|https://github.com/emolsson/cassandra/tree/CASSANDRA-15592-3.11]|
|[4.0|https://github.com/emolsson/cassandra/tree/CASSANDRA-15592-4.0]|

> IllegalStateException in gossip after removing node
> ---
>
> Key: CASSANDRA-15592
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15592
> Project: Cassandra
>  Issue Type: Bug
>  Components: Cluster/Gossip
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Normal
>
> In one of our test environments we encountered the following exception:
> {noformat}
> 2020-02-02T10:50:13.276+0100 [GossipTasks:1] ERROR 
> o.a.c.u.NoSpamLogger$NoSpamLogStatement:97 log 
> java.lang.IllegalStateException: Attempting gossip state mutation from 
> illegal thread: GossipTasks:1
>  at 
> org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
>  at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465)
>  at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895)
>  at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78)
>  at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240)
>  at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
>  at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>  at java.lang.Thread.run(Thread.java:748)
> java.lang.IllegalStateException: Attempting gossip state mutation from 
> illegal thread: GossipTasks:1
>  at 
> org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240) 
> [apache-cassandra-3.11.5.jar:3.11.5]
>  at 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_231]
>  at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
> [na:1.8.0_231]
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>  [na:1.8.0_231]
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>  [na:1.8.0_231]
>  at 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
>  [apache-cassandra-3.11.5.jar:3.11.5]
>  at 
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>  ~[netty-all-4.1.42.Final.jar:4.1.42.Final]
>  at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_231]
> {noformat}
> Since CASSANDRA-15059 we check that all state changes are performed in the 
> GossipStage but it seems like it was still performed in the "current" thread 
> [here|https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/gms/Gossiper.java#L895].
>  It should be as simple as adding a
> {code:java}
> runInGossipStageBlocking(() ->)
> {code}
> for it.
> I'll upload patches for 3.0, 3.11 and 4.0.
>   



--
This message was sent by Atlassian Jira

[jira] [Created] (CASSANDRA-15592) IllegalStateException in gossip after removing node

2020-02-20 Thread Marcus Olsson (Jira)
Marcus Olsson created CASSANDRA-15592:
-

 Summary: IllegalStateException in gossip after removing node
 Key: CASSANDRA-15592
 URL: https://issues.apache.org/jira/browse/CASSANDRA-15592
 Project: Cassandra
  Issue Type: Bug
  Components: Cluster/Gossip
Reporter: Marcus Olsson
Assignee: Marcus Olsson


In one of our test environments we encountered the following exception:
{noformat}
2020-02-02T10:50:13.276+0100 [GossipTasks:1] ERROR 
o.a.c.u.NoSpamLogger$NoSpamLogStatement:97 log java.lang.IllegalStateException: 
Attempting gossip state mutation from illegal thread: GossipTasks:1
 at 
org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
 at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465)
 at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895)
 at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78)
 at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240)
 at 
org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
 at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
 at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 at java.lang.Thread.run(Thread.java:748)
java.lang.IllegalStateException: Attempting gossip state mutation from illegal 
thread: GossipTasks:1
 at 
org.apache.cassandra.gms.Gossiper.checkProperThreadForStateMutation(Gossiper.java:178)
 [apache-cassandra-3.11.5.jar:3.11.5]
 at org.apache.cassandra.gms.Gossiper.evictFromMembership(Gossiper.java:465) 
[apache-cassandra-3.11.5.jar:3.11.5]
 at org.apache.cassandra.gms.Gossiper.doStatusCheck(Gossiper.java:895) 
[apache-cassandra-3.11.5.jar:3.11.5]
 at org.apache.cassandra.gms.Gossiper.access$700(Gossiper.java:78) 
[apache-cassandra-3.11.5.jar:3.11.5]
 at org.apache.cassandra.gms.Gossiper$GossipTask.run(Gossiper.java:240) 
[apache-cassandra-3.11.5.jar:3.11.5]
 at 
org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:118)
 [apache-cassandra-3.11.5.jar:3.11.5]
 at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
[na:1.8.0_231]
 at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
[na:1.8.0_231]
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
 [na:1.8.0_231]
 at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
 [na:1.8.0_231]
 at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
[na:1.8.0_231]
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
[na:1.8.0_231]
 at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:84)
 [apache-cassandra-3.11.5.jar:3.11.5]
 at 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
 ~[netty-all-4.1.42.Final.jar:4.1.42.Final]
 at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_231]
{noformat}
Since CASSANDRA-15059 we check that all state changes are performed in the 
GossipStage but it seems like it was still performed in the "current" thread 
[here|https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/gms/Gossiper.java#L895].
 It should be as simple as adding a
{code:java}
runInGossipStageBlocking(() ->)
{code}
for it.

I'll upload patches for 3.0, 3.11 and 4.0.
  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14629) Abstract Virtual Table for very large result sets

2019-10-07 Thread Marcus Olsson (Jira)


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

Marcus Olsson commented on CASSANDRA-14629:
---

I tried out the provided pull request (with a few modifications as mentioned in 
the review). It was quite trivial to add simple tables based on the 
AbstractIteratingTable which is good.
I used a structure as:

{code:sql}
CREATE TABLE system_views.testtable (
c1 bigint,
c2 bigint,
c3 bigint,
c4 bigint,
id int,
key text,
PRIMARY KEY (key, id)
) WITH CLUSTERING ORDER BY (id ASC)
AND compaction = {'class': 'None'}
AND compression = {};
{code}
and created a few different implementations of it.

Functionally it seems to be working as intended. Both 
DataRange/ClusteringIndexFilter are working as expected as far as I can tell.
To verify the paging functionality I added some extra logging and had a table 
setup with id values between 0 -> 11. 
For a request with 100 rows per page it should return all rows from pk 0 -> 7 
(96 rows) and then an additional 4 rows (0 -> 3) from pk 8. The logs showed the 
following when executing "SELECT * FROM system_views.testtableother": 
{noformat}
AbstractIteratingTable.java:150 - select range=(min(), min()) 
pfilter=slice(slices=ALL, reversed=false) - *
TestTableOther.java:74 - getPartitionKeys range=(min(), min()) 
pfilter=slice(slices=ALL, reversed=false)
TestTableOther.java:96 - getRows slice(slices=ALL, reversed=false) - 
DecoratedKey(mykey000, 6d796b657930303030303030) - *
...
TestTableOther.java:96 - getRows slice(slices=ALL, reversed=false) - 
DecoratedKey(mykey008, 6d796b657930303030303038) - *
# Next page request
AbstractIteratingTable.java:150 - select range=[mykey008, min()) (paging) 
pfilter=slice(slices=ALL, reversed=false) lastReturned=id=3 (excluded) - *
TestTableOther.java:74 - getPartitionKeys range=[mykey008, min()) (paging) 
pfilter=slice(slices=ALL, reversed=false) lastReturned=id=3 (excluded)
TestTableOther.java:96 - getRows slice(slices={(3, ]}, reversed=false) - 
DecoratedKey(mykey008, 6d796b657930303030303038) - *
TestTableOther.java:96 - getRows slice(slices=ALL, reversed=false) - 
DecoratedKey(mykey009, 6d796b657930303030303039) - *
{noformat}
The first request finished on pk8 row 3 and as expected the next continued with 
row 4->11 on pk 8 and then everything from key 9 etc.



I ran some performance tests on it to check what happens when this is scaled to 
millions of rows. This is also the main reason for the proposed changes to the 
#getRows() method.

*Test case 1*

The first test case was executed with a single partition key and millions of 
rows based on the provided table format at the top.
The id column was simply incremented for each new row.

There were three types of implementations tested:
# Iterate and create rows for everything (filter in AbstractIteratingTable) 
(possible with current solution)
# Iterate and create Clustering for everything (filter in sub-class before 
adding columns/building the row)
# Only generate rows that are read (based on the slice)

||Implementation||100,000||200,000||400,000||600,000||800,000||1,000,000||2,000,000||3,000,000||4,000,000|
||#1|1063|3658|13757|30214|53144|82827|-|-|-|
||#2|287|654|1705|3267|5189|7598|26574|55436|95842|
||#3|217|424|835|1319|1676|2250|4207|6307|8589|
The table shows the amount of time (ms) it took to iterate through the amount 
of rows specified in column header (always starting from the first row). Both 
this and the next test case used a page size of 5000.
Depending on how much data we are expecting to display in a table we should 
make careful considerations on how we generate it in the iterators.

I guess the downside here is that #3 is probably not trivial to implement for 
all virtual tables..

*Test case 2*

The second test case was executed with a million single-row partitions (same 
schema) with two implementations:
# Generate all partitions (filter in AbstractIteratingTable)
# Generate only partitions falling in the data range

||Implementation||100,000||200,000||400,000||600,000||800,000||1,000,000|
||#1|1308|3934|14280|30587|53233|82813|
||#2|405|794|1554|2318|3122|3805|
Same as the previous table this shows the time (ms) to iterate through the rows.
There is a comment on the #getPartitionKeys() method already that it's ok to 
generate all partitions for small data sets and I'd say that seems true.



Some comments/questions:
* Do we have a target for how many rows/partitions we should be able to 
support? Or should it be "unlimited"?
* In order to make table paging efficient I believe we need to include 
ClusteringIndexFilter (and optionally ColumnFilter) to #getRows().
** Clustering also needs to be exposed to the sub-class, either through 
RowBuilder#getClustering() or by making the sub-class create the Clustering 
object.
* Although this is quite 

[jira] [Created] (CASSANDRA-15186) InternodeOutboundMetrics overloaded bytes/count mixup

2019-06-27 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-15186:
-

 Summary: InternodeOutboundMetrics overloaded bytes/count mixup
 Key: CASSANDRA-15186
 URL: https://issues.apache.org/jira/browse/CASSANDRA-15186
 Project: Cassandra
  Issue Type: Bug
  Components: Observability/Metrics
Reporter: Marcus Olsson


In 
[https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java]
 there is a small mixup between overloaded count and bytes, in 
[LargeMessageDroppedTasksDueToOverload|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java#L129]
 and 
[UrgentMessageDroppedTasksDueToOverload|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/metrics/InternodeOutboundMetrics.java#L151].



--
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-15088) PagingTest failure on trunk

2019-06-20 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-15088:
---

[~eperott] No objections here

> PagingTest failure on trunk
> ---
>
> Key: CASSANDRA-15088
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15088
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/unit
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Normal
> Fix For: 4.0
>
> Attachments: 15088-trunk.txt
>
>
> [Example failure|https://circleci.com/gh/emolsson/cassandra/19]
> {noformat}
> java.lang.RuntimeException: Unable to gossip with any peers
>   at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1546)
>   at 
> org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:553)
>   at 
> org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:841)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:699)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:650)
>   at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:379)
>   at 
> org.apache.cassandra.service.CassandraDaemon.init(CassandraDaemon.java:501)
>   at 
> org.apache.cassandra.service.EmbeddedCassandraService.start(EmbeddedCassandraService.java:50)
>   at org.apache.cassandra.cql3.PagingTest.setup(PagingTest.java:63)
> {noformat}
> Running the test case by itself won't reproduce the issue:
> {noformat}
> ant test -Dtest.name=PagingTest
> {noformat}
> But running it in parallel with other tests will:
> {noformat}
> ant test -Dtest.name=cql3/*
> {noformat}
> From the logs the following can be observed:
> {noformat}
> INFO  [main] 2019-04-11 15:32:29,916 Node 
> configuration:[...seed_provider=org.apache.cassandra.locator.SimpleSeedProvider{seeds=127.0.0.1:7017};
>  storage_port=7027]
> ...
> DEBUG [main] 2019-04-17 10:11:55,418 connection attempt 0 to 127.0.0.1:7044 
> (GOSSIP)
> DEBUG [main] 2019-04-17 10:11:55,420 creating outbound bootstrap to peer 
> 127.0.0.1:7044, compression: false, encryption: disabled, coalesce: DISABLED, 
> protocolVersion: 12
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 connection attempt 1 to 
> 127.0.0.1:7044 (GOSSIP)
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 creating outbound 
> bootstrap to peer 127.0.0.1:7044, compression: false, encryption: disabled, 
> coalesce: DISABLED, protocolVersion: 12
> {noformat}
> It seems like we have an offset issue of 10 between seed/storage_port.
> The port 7044 does not seem to be defined anywhere though.



--
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-15119) Repair fails randomly, causing nodes to restart

2019-05-07 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-15119:
---

{quote}This does indeed seem promising (can't wait for 3.11.5 to try it) 
however, in the linked ticket, the reporter gets an error that it goes out of 
memory. We don't. We simply see that it gets dangerously close to the XMX limit 
in JProfiler and suddenly one or more nodes are down, quickly restarting 
afterwards. The only errors given is that streams failed.
{quote}
The ticket was created for Cassandra 3.11.1 and since CASSANDRA-13006 (3.11.2) 
OOMs are handled by the JVM rather than by Cassandra. Do you think this could 
be what you are experiencing and why there is a difference? If you do not have 
heap dumps enabled could you try enabling them to see if one would get 
generated when this happens (I suspect this could be the case since the nodes 
are restarting)? If you then also use a tool like Eclipse MAT to analyze one of 
the heap dumps it can generate a _leak suspect report_ which can be quite 
helpful to investigate what is causing it.

Are you running repairs in parallel or do you run it on one node at a time? 
CASSANDRA-14096 should cause most stress for the repair coordinating node as it 
stores merkle trees for all the replicas. So if you are running repair on one 
node but the other nodes are going down it could be something different.

bq. and can see that it would be busy streaming data to other nodes and 
suddenly they go down. We also sometimes see validation compaction happening 
but I believe it stops streaming then.
Part of CASSANDRA-14096 is about reducing the amount of time the MerkleTrees 
are on-heap. Before the fix these trees could be stored until the full keyspace 
was repaired (including during streaming). So it is possible be that the 
streaming + other potential activities in the cluster in combination with these 
stored MerkleTrees could tip it over.

> Repair fails randomly, causing nodes to restart
> ---
>
> Key: CASSANDRA-15119
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15119
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair, Consistency/Streaming
>Reporter: Brent
>Priority: Normal
>
> We have a cluster of 3 nodes (same dc) that is ~8GB on disk (per node). One 
> keyspace has two tables, combined having about 20m rows with around 20 colums 
> each. Whenever we try to run a repair (with or without cassandra-reaper, on 
> any setting) the repair causes certain nodes to fail and restart. Originally 
> these nodes had the default heap space calculation on a device with 12GB ram.
> We upscaled these to 24GB ram and 12GB XMX which seemed to make a difference 
> but still not quite enough. With JProfiler we can see that random nodes reach 
> the xmx limit, regardless of the size of the repair, while streaming data.
> I can't understand that such operations can cause servers to literally crash 
> rather than just say "no I can't do it". We've tried a lot of things 
> including setting up a fresh cluster and manually inserting all the data 
> (with the correct replication factor) and then run repairs.
> Sometimes they will work (barely) sometimes they will fail. I really don't 
> understand.
> We're running cassandra 3.11.4.  
> Could I receive some assistance in troubleshooting this?



--
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-15119) Repair fails randomly, causing nodes to restart

2019-05-07 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-15119:
---

I think CASSANDRA-14096 could be interesting to look at here, it should be 
fixed in *3.11.5*. As you say you have ~20m rows that gives me an additional 
indication for this as this could create large merkle trees.

You could try to run {{jmap -histo}} for the Cassandra process several times 
during one of these repairs as that could reveal what type of objects are 
building up. If you can see a lot of MerkleTree related objects 
(MerkleTree$Inner/Leaf etc) at the top of the histogram with large amounts of 
memory used this could be related to CASSANDRA-14096.

Also, are you using virtual nodes or single token per node?

*Note:*
I believe repair could start the next round of validation compactions 
(MerkleTree creation) in parallel with streaming the data files which could 
explain your observation of reaching Xmx during streaming.

> Repair fails randomly, causing nodes to restart
> ---
>
> Key: CASSANDRA-15119
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15119
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Repair, Consistency/Streaming
>Reporter: Brent
>Priority: Normal
>
> We have a cluster of 3 nodes (same dc) that is ~8GB on disk (per node). One 
> keyspace has two tables, combined having about 20m rows with around 20 colums 
> each. Whenever we try to run a repair (with or without cassandra-reaper, on 
> any setting) the repair causes certain nodes to fail and restart. Originally 
> these nodes had the default heap space calculation on a device with 12GB ram.
> We upscaled these to 24GB ram and 12GB XMX which seemed to make a difference 
> but still not quite enough. With JProfiler we can see that random nodes reach 
> the xmx limit, regardless of the size of the repair, while streaming data.
> I can't understand that such operations can cause servers to literally crash 
> rather than just say "no I can't do it". We've tried a lot of things 
> including setting up a fresh cluster and manually inserting all the data 
> (with the correct replication factor) and then run repairs.
> Sometimes they will work (barely) sometimes they will fail. I really don't 
> understand.
> We're running cassandra 3.11.4.  
> Could I receive some assistance in troubleshooting this?



--
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-15088) PagingTest failure on trunk

2019-05-02 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-15088:
---

Thanks for looking at it. Yes, the port was actually added later to 
cassandra-murmur.yaml during CASSANDRA-14956 as 
[27b3579|https://github.com/apache/cassandra/commit/27b35799a46dd5b649c4a172f4f8316b48615304]
 due to a test case failing in a similar way as described in this ticket. It 
seems like the port was removed again by a later merge from 3.11 -> trunk.

I can fix a patch that aligns the fields but I'm not sure that it's worth to do 
it for all fields (as long as the test cases are working as intended). Some 
test cases i.e. _test/unit/org/apache/cassandra/cql3/PstmtPersistenceTest.java_ 
uses the parameter _prepared_statements_cache_size_mb: 1_ from cassandra.yaml 
which I don't believe is necessary for test cases using cassandra-murmur.yaml 
(currently at least). To try to keep these two files aligned will probably be 
hard to remember as they have already diverged. We could of course add a 
comment in the yaml files saying that the fields should be aligned but that can 
still be missed.

But instead of keeping the two yaml files aligned perhaps a better approach 
would be to have only one yaml file that all test cases depend on and have a 
programmatic way to change specific fields required for the test cases. This 
could be done in such a way that the setup phase for all test cases that wants 
to use murmur3 would define the partitioner in the test case instead of 
specifying a yaml file to use. But that seems like a larger work and should 
probably be handled outside of this ticket.

> PagingTest failure on trunk
> ---
>
> Key: CASSANDRA-15088
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15088
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/unit
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Normal
> Fix For: 4.0
>
> Attachments: 15088-trunk.txt
>
>
> [Example failure|https://circleci.com/gh/emolsson/cassandra/19]
> {noformat}
> java.lang.RuntimeException: Unable to gossip with any peers
>   at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1546)
>   at 
> org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:553)
>   at 
> org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:841)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:699)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:650)
>   at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:379)
>   at 
> org.apache.cassandra.service.CassandraDaemon.init(CassandraDaemon.java:501)
>   at 
> org.apache.cassandra.service.EmbeddedCassandraService.start(EmbeddedCassandraService.java:50)
>   at org.apache.cassandra.cql3.PagingTest.setup(PagingTest.java:63)
> {noformat}
> Running the test case by itself won't reproduce the issue:
> {noformat}
> ant test -Dtest.name=PagingTest
> {noformat}
> But running it in parallel with other tests will:
> {noformat}
> ant test -Dtest.name=cql3/*
> {noformat}
> From the logs the following can be observed:
> {noformat}
> INFO  [main] 2019-04-11 15:32:29,916 Node 
> configuration:[...seed_provider=org.apache.cassandra.locator.SimpleSeedProvider{seeds=127.0.0.1:7017};
>  storage_port=7027]
> ...
> DEBUG [main] 2019-04-17 10:11:55,418 connection attempt 0 to 127.0.0.1:7044 
> (GOSSIP)
> DEBUG [main] 2019-04-17 10:11:55,420 creating outbound bootstrap to peer 
> 127.0.0.1:7044, compression: false, encryption: disabled, coalesce: DISABLED, 
> protocolVersion: 12
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 connection attempt 1 to 
> 127.0.0.1:7044 (GOSSIP)
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 creating outbound 
> bootstrap to peer 127.0.0.1:7044, compression: false, encryption: disabled, 
> coalesce: DISABLED, protocolVersion: 12
> {noformat}
> It seems like we have an offset issue of 10 between seed/storage_port.
> The port 7044 does not seem to be defined anywhere though.



--
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-14983) Local reads potentially blocking remote reads

2019-04-23 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14983:
---

bq. But, speculative retry is itself a guessing game, and one we don't play 
very well (since we assume uniform request duration), and one that wastes 
cluster resources when we get it wrong. Is it worth trading the possibility of 
a very unbalanced node serving as coordinator and failing to perform 
speculative retry, in exchange for capping the downside to incorrectly guessed 
speculative retries? Probably, I think.
Then I think we should keep the fast path as long as we enforce that the local 
request is performed last. I believe my patches should cover this.

bq. Could you expand on what you mean here?
Implementing speculative retries based on local query duration seems like a 
bigger work which may require some deeper analysis. I think that if it should 
be implemented it should be done in such a way that it covers both the fast 
path and the normal query path. So I think it would make sense to take it in a 
separate ticket and it sounds like this was your suggestion here as well.

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Low
> Attachments: graph_local_read.html, graph_local_read_trunk.html, 
> local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> "data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
>  as the type of request.
> I have tried to run performance measurements to see the impact of this in 3.0 
> (by moving local requests to the end of ARE#executeAsync()) but I haven't 
> seen any big difference yet. I'll continue to run some more tests to see if I 
> can find a use case affected by this.
> Attaching a trace (3.0) where this happens. Reproduction:
>  # Create a three node CCM cluster
>  # Provision data with stress (rf=3)
>  # In parallel:
>  ## Start stress read run
>  ## Run multiple manual read queries in cqlsh with tracing on and 
> local_quorum (as this does not always happen)



--
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-14983) Local reads potentially blocking remote reads

2019-04-17 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14983:
---

I don't necessarily think we should remove the fast path, I might have phrased 
my previous comment a bit ambiguous on that point. For me it depends on what 
expectations and guarantees we want to have for the requests (wrt. 
timeouts/speculative retries). In any case I think it would be really hard to 
have guarantees for both speculative retries and timeouts as there are multiple 
things affecting us here (gc pauses being a major one).

I think your suggestion makes sense, but there is still a problematic situation 
with cluster imbalance. If one node has a slow disk, more sstables or similar 
issues, should it act mostly as a proxy to other nodes or do we accept slow 
requests? If we accept that requests could be slow in those situations then I 
don't think there is any reason to remove the fast path. I'm not sure which 
behavior would be best here as there are arguments for both.

I also want to mention that if we do want to have a feature that waits for 
local requests before doing speculative retries I think we should keep it 
separate from the fast path. The requests can and will go through the normal 
path still and I guess we want to have a similar behavior when that happens.

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Low
> Attachments: graph_local_read.html, graph_local_read_trunk.html, 
> local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> "data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
>  as the type of request.
> I have tried to run performance measurements to see the impact of this in 3.0 
> (by moving local requests to the end of ARE#executeAsync()) but I haven't 
> seen any big difference yet. I'll continue to run some more tests to see if I 
> can find a use case affected by this.
> Attaching a trace (3.0) where this happens. Reproduction:
>  # Create a three node CCM cluster
>  # Provision data with stress (rf=3)
>  # In parallel:
>  ## Start stress read run
>  ## Run multiple manual read queries in cqlsh with tracing on and 
> local_quorum (as this does not always happen)



--
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-15088) PagingTest failure on trunk

2019-04-17 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-15088:
---

Attached a patch for trunk that adds the seed port back in 
cassandra-murmur.yaml that fixes the test case. It seems like it could have 
been missed during a merge from 3.11.

> PagingTest failure on trunk
> ---
>
> Key: CASSANDRA-15088
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15088
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/unit
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Normal
> Attachments: 15088-trunk.txt
>
>
> [Example failure|https://circleci.com/gh/emolsson/cassandra/19]
> {noformat}
> java.lang.RuntimeException: Unable to gossip with any peers
>   at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1546)
>   at 
> org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:553)
>   at 
> org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:841)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:699)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:650)
>   at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:379)
>   at 
> org.apache.cassandra.service.CassandraDaemon.init(CassandraDaemon.java:501)
>   at 
> org.apache.cassandra.service.EmbeddedCassandraService.start(EmbeddedCassandraService.java:50)
>   at org.apache.cassandra.cql3.PagingTest.setup(PagingTest.java:63)
> {noformat}
> Running the test case by itself won't reproduce the issue:
> {noformat}
> ant test -Dtest.name=PagingTest
> {noformat}
> But running it in parallel with other tests will:
> {noformat}
> ant test -Dtest.name=cql3/*
> {noformat}
> From the logs the following can be observed:
> {noformat}
> INFO  [main] 2019-04-11 15:32:29,916 Node 
> configuration:[...seed_provider=org.apache.cassandra.locator.SimpleSeedProvider{seeds=127.0.0.1:7017};
>  storage_port=7027]
> ...
> DEBUG [main] 2019-04-17 10:11:55,418 connection attempt 0 to 127.0.0.1:7044 
> (GOSSIP)
> DEBUG [main] 2019-04-17 10:11:55,420 creating outbound bootstrap to peer 
> 127.0.0.1:7044, compression: false, encryption: disabled, coalesce: DISABLED, 
> protocolVersion: 12
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 connection attempt 1 to 
> 127.0.0.1:7044 (GOSSIP)
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 creating outbound 
> bootstrap to peer 127.0.0.1:7044, compression: false, encryption: disabled, 
> coalesce: DISABLED, protocolVersion: 12
> {noformat}
> It seems like we have an offset issue of 10 between seed/storage_port.
> The port 7044 does not seem to be defined anywhere though.



--
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-15088) PagingTest failure on trunk

2019-04-17 Thread Marcus Olsson (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-15088?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-15088:
--
Attachment: 15088-trunk.txt

> PagingTest failure on trunk
> ---
>
> Key: CASSANDRA-15088
> URL: https://issues.apache.org/jira/browse/CASSANDRA-15088
> Project: Cassandra
>  Issue Type: Bug
>  Components: Test/unit
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Normal
> Attachments: 15088-trunk.txt
>
>
> [Example failure|https://circleci.com/gh/emolsson/cassandra/19]
> {noformat}
> java.lang.RuntimeException: Unable to gossip with any peers
>   at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1546)
>   at 
> org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:553)
>   at 
> org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:841)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:699)
>   at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:650)
>   at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:379)
>   at 
> org.apache.cassandra.service.CassandraDaemon.init(CassandraDaemon.java:501)
>   at 
> org.apache.cassandra.service.EmbeddedCassandraService.start(EmbeddedCassandraService.java:50)
>   at org.apache.cassandra.cql3.PagingTest.setup(PagingTest.java:63)
> {noformat}
> Running the test case by itself won't reproduce the issue:
> {noformat}
> ant test -Dtest.name=PagingTest
> {noformat}
> But running it in parallel with other tests will:
> {noformat}
> ant test -Dtest.name=cql3/*
> {noformat}
> From the logs the following can be observed:
> {noformat}
> INFO  [main] 2019-04-11 15:32:29,916 Node 
> configuration:[...seed_provider=org.apache.cassandra.locator.SimpleSeedProvider{seeds=127.0.0.1:7017};
>  storage_port=7027]
> ...
> DEBUG [main] 2019-04-17 10:11:55,418 connection attempt 0 to 127.0.0.1:7044 
> (GOSSIP)
> DEBUG [main] 2019-04-17 10:11:55,420 creating outbound bootstrap to peer 
> 127.0.0.1:7044, compression: false, encryption: disabled, coalesce: DISABLED, 
> protocolVersion: 12
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 connection attempt 1 to 
> 127.0.0.1:7044 (GOSSIP)
> DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 creating outbound 
> bootstrap to peer 127.0.0.1:7044, compression: false, encryption: disabled, 
> coalesce: DISABLED, protocolVersion: 12
> {noformat}
> It seems like we have an offset issue of 10 between seed/storage_port.
> The port 7044 does not seem to be defined anywhere though.



--
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] [Created] (CASSANDRA-15088) PagingTest failure on trunk

2019-04-17 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-15088:
-

 Summary: PagingTest failure on trunk
 Key: CASSANDRA-15088
 URL: https://issues.apache.org/jira/browse/CASSANDRA-15088
 Project: Cassandra
  Issue Type: Bug
  Components: Test/unit
Reporter: Marcus Olsson
Assignee: Marcus Olsson


[Example failure|https://circleci.com/gh/emolsson/cassandra/19]
{noformat}
java.lang.RuntimeException: Unable to gossip with any peers
at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1546)
at 
org.apache.cassandra.service.StorageService.checkForEndpointCollision(StorageService.java:553)
at 
org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:841)
at 
org.apache.cassandra.service.StorageService.initServer(StorageService.java:699)
at 
org.apache.cassandra.service.StorageService.initServer(StorageService.java:650)
at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:379)
at 
org.apache.cassandra.service.CassandraDaemon.init(CassandraDaemon.java:501)
at 
org.apache.cassandra.service.EmbeddedCassandraService.start(EmbeddedCassandraService.java:50)
at org.apache.cassandra.cql3.PagingTest.setup(PagingTest.java:63)
{noformat}

Running the test case by itself won't reproduce the issue:
{noformat}
ant test -Dtest.name=PagingTest
{noformat}

But running it in parallel with other tests will:
{noformat}
ant test -Dtest.name=cql3/*
{noformat}

>From the logs the following can be observed:
{noformat}
INFO  [main] 2019-04-11 15:32:29,916 Node 
configuration:[...seed_provider=org.apache.cassandra.locator.SimpleSeedProvider{seeds=127.0.0.1:7017};
 storage_port=7027]

...

DEBUG [main] 2019-04-17 10:11:55,418 connection attempt 0 to 127.0.0.1:7044 
(GOSSIP)
DEBUG [main] 2019-04-17 10:11:55,420 creating outbound bootstrap to peer 
127.0.0.1:7044, compression: false, encryption: disabled, coalesce: DISABLED, 
protocolVersion: 12
DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 connection attempt 1 to 
127.0.0.1:7044 (GOSSIP)
DEBUG [ScheduledFastTasks:1] 2019-04-17 10:11:55,538 creating outbound 
bootstrap to peer 127.0.0.1:7044, compression: false, encryption: disabled, 
coalesce: DISABLED, protocolVersion: 12
{noformat}

It seems like we have an offset issue of 10 between seed/storage_port.
The port 7044 does not seem to be defined anywhere though.



--
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-14983) Local reads potentially blocking remote reads

2019-04-11 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14983:
---

No worries, thanks for the update and suggestions for the testing.

I tried out 4 but testing the complete flow of this behavior still seems a bit 
problematic. I think this comes from a few different factors. One is that 
background requests are required to trigger this unless CASSANDRA-15022 gets 
in. Another problem is that when the fast path is used neither speculative 
retries nor timeouts seems to be properly respected as they are executed 
sequentially after the local read. If we have timeout defined as 100ms and the 
request takes 150ms locally then it won't throw an exception after either 
100/150 ms (unless the remote request was even slower). As we already have the 
data after 150ms I do think it makes sense to return it(rather than throw an 
exception) but IMO it should have thrown an exception after ~100ms. A similar 
scenario occurs for the speculative retries as well, they wouldn't be triggered 
until after the local request is finished.
I think this would leave using request latency as a basis for verifying the 
behavior but that could make the test case quite unstable and sensitive to the 
testing environment.

>From a correctness perspective this feature seems to have some edge cases with 
>request timeouts and speculative retries. In CASSANDRA-6995 it was suggested 
>to add this feature specifically for LOCAL_ONE/ONE requests which I think 
>would narrow some of the edge cases. And as this is probably a quite narrow 
>failure case as it is, I'm not sure if we should aim to solve it for all cases 
>as that might limit the use case significantly. I.e. I don't think we can 
>solve the timeout issue without a significant rework of the request path.

However I do think we should consider the effects this has for speculative 
retries and if this feature should be enabled in combination with that.

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Low
> Attachments: graph_local_read.html, graph_local_read_trunk.html, 
> local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> "data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
>  as the type of request.
> I have tried to run performance measurements to see the impact of this in 3.0 
> (by moving local requests to the end of ARE#executeAsync()) but I haven't 
> seen any big difference yet. I'll continue to run some more tests to see if I 
> can find a use case affected by this.
> Attaching a trace (3.0) where this happens. Reproduction:
>  # Create a three node CCM cluster
>  # Provision data with stress (rf=3)
>  # In parallel:
>  ## Start stress read run
>  ## Run multiple manual read queries in cqlsh with tracing on and 
> local_quorum (as this does not always happen)



--
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-14983) Local reads potentially blocking remote reads

2019-02-20 Thread Marcus Olsson (JIRA)


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

Marcus Olsson edited comment on CASSANDRA-14983 at 2/20/19 6:26 PM:


I created CASSANDRA-15022 for the SEP executor and have uploaded some patches 
for 3.0, 3.11 and 4.0:
|[3.0|https://github.com/emolsson/cassandra/tree/3.0-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/13]|
|[3.11|https://github.com/emolsson/cassandra/tree/3.11-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/16]|
|[4.0|https://github.com/emolsson/cassandra/tree/4.0-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/19]|

I seem to be having some issues running the tests on 4.0 though. When I tried 
to run the tests on a clean trunk branch locally I got similar issues as the 
ones in the build. But the 3.0 and 3.11 branches seems to pass the unit tests.

A note for the 4.0 patch is that all requests are still performed as full data 
requests. My understanding is that digest request are primarily used to avoid 
sending large amounts of data over the network. If we always perform data 
requests locally we might however keep multiple copies of the data in memory if 
we send data requests to remote nodes as well. If we want to change that 
behavior I think it should be in a separate JIRA ticket.

I also looked into the transient requests and I don't think anything would 
change from a logic perspective by executing the local request as a transient 
request as it is right now. Responses seems to be considered transient by 
checking the replica plan rather than the response. The 
ReadCommand#acceptsTransient()-method only seems to be used for validation 
before starting to execute the request and not affecting the actual request. 
Basically I don't think this is causing us any bugs right now. With that said I 
think the transient status should be kept even for local requests, at least 
from a correctness perspective. WDYT, should the local request be converted to 
a transient request when that is the case?

–

I also looked at the distributed tests but I'm not sure how to delay or lose 
the local requests in a good way without changing the actual code base. Do you 
know of a good place on the nodes read path to hook in to and delay it - I 
can't seem to find any?


was (Author: molsson):
I created CASSANDRA-15022 for the SEP executor and have uploaded some patches 
for 3.0, 3.11 and 4.0:

|[3.0|https://github.com/emolsson/cassandra/tree/3.0-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/13]|
|[3.11|https://github.com/emolsson/cassandra/tree/3.11-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/16]|
|[4.0|https://github.com/emolsson/cassandra/tree/4.0-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/19]|

I seem to be having some issues running the tests on 4.0 though. When I tried 
to run the tests on a clean trunk branch locally I got similar issues as the 
ones in the build. But the 3.0 and 3.11 branches seems to pass the unit tests.

I also looked at the distributed tests but I'm not sure how to delay or lose 
the local requests in a good way without changing the actual code base.

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Priority: Minor
> Attachments: graph_local_read.html, graph_local_read_trunk.html, 
> local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> 

[jira] [Commented] (CASSANDRA-14983) Local reads potentially blocking remote reads

2019-02-20 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14983:
---

I created CASSANDRA-15022 for the SEP executor and have uploaded some patches 
for 3.0, 3.11 and 4.0:

|[3.0|https://github.com/emolsson/cassandra/tree/3.0-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/13]|
|[3.11|https://github.com/emolsson/cassandra/tree/3.11-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/16]|
|[4.0|https://github.com/emolsson/cassandra/tree/4.0-CASSANDRA-14983]|[unit 
tests|https://circleci.com/gh/emolsson/cassandra/19]|

I seem to be having some issues running the tests on 4.0 though. When I tried 
to run the tests on a clean trunk branch locally I got similar issues as the 
ones in the build. But the 3.0 and 3.11 branches seems to pass the unit tests.

I also looked at the distributed tests but I'm not sure how to delay or lose 
the local requests in a good way without changing the actual code base.

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Priority: Minor
> Attachments: graph_local_read.html, graph_local_read_trunk.html, 
> local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> "data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
>  as the type of request.
> I have tried to run performance measurements to see the impact of this in 3.0 
> (by moving local requests to the end of ARE#executeAsync()) but I haven't 
> seen any big difference yet. I'll continue to run some more tests to see if I 
> can find a use case affected by this.
> Attaching a trace (3.0) where this happens. Reproduction:
>  # Create a three node CCM cluster
>  # Provision data with stress (rf=3)
>  # In parallel:
>  ## Start stress read run
>  ## Run multiple manual read queries in cqlsh with tracing on and 
> local_quorum (as this does not always happen)



--
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] [Created] (CASSANDRA-15022) SEPExecutor local requests not triggered often

2019-02-14 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-15022:
-

 Summary: SEPExecutor local requests not triggered often
 Key: CASSANDRA-15022
 URL: https://issues.apache.org/jira/browse/CASSANDRA-15022
 Project: Cassandra
  Issue Type: Improvement
Reporter: Marcus Olsson


Based on observations done in 
[CASSANDRA-14983|https://issues.apache.org/jira/browse/CASSANDRA-14983?focusedCommentId=16745238=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-16745238]
 SEPExecutor#maybeExecuteImmediately() does not necessarily run tasks in the 
same thread often.

The logic [as it is 
today|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPExecutor.java#L166]
 basically requires another task to be in the work queue for it to trigger. One 
option to make it trigger more often could be to change the logic to:
{code:java}
if (workPermits == 0 || (takeTaskPermit && taskPermits == 0))
 return false;
{code}
So that we only check the taskPermits if we want to take a taskPermit.



--
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-14983) Local reads potentially blocking remote reads

2019-02-11 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14983:
---

Great, I can create the patches for 3.0, 3.11 and 4.0 and also a separate 
ticket for the SEP executor. I can also try to write an in-jvm dtest, I haven't 
really tried it out yet so it will be interesting to see how it works.

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Priority: Minor
> Attachments: graph_local_read.html, graph_local_read_trunk.html, 
> local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> "data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
>  as the type of request.
> I have tried to run performance measurements to see the impact of this in 3.0 
> (by moving local requests to the end of ARE#executeAsync()) but I haven't 
> seen any big difference yet. I'll continue to run some more tests to see if I 
> can find a use case affected by this.
> Attaching a trace (3.0) where this happens. Reproduction:
>  # Create a three node CCM cluster
>  # Provision data with stress (rf=3)
>  # In parallel:
>  ## Start stress read run
>  ## Run multiple manual read queries in cqlsh with tracing on and 
> local_quorum (as this does not always happen)



--
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] [Created] (CASSANDRA-14994) Incorrect repair history when running repair

2019-01-22 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-14994:
-

 Summary: Incorrect repair history when running repair
 Key: CASSANDRA-14994
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14994
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Olsson


Since CASSANDRA-5220 there is an issue with *system_distributed.repair_history* 
when using virtual nodes. Performing a standard "nodetool repair" will create a 
lot less entries than it should.

Example:

{code}
$ ccm create test_repair -n 3 --vnodes -v 3.0.17

...

cqlsh> CREATE KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 3};
cqlsh> CREATE TABLE test.test(key PRIMARY KEY);

...

ccm node1 nodetool repair test

...

cqlsh> SELECT keyspace_name, columnfamily_name, id, range_begin, range_end FROM 
system_distributed.repair_history ;

 keyspace_name | columnfamily_name | id   | 
range_begin | range_end
---+---+--+-+-
  test |  test | 12f27830-1e53-11e9-93a0-2122ff85bd0a | 
6842951316968308632 | 6844625844103123572
{code}

In the above example the cluster is created with 256 tokens but the repair 
history only shows one entry.

The problem is that in CASSANDRA-5220 a single repair session can repair 
multiple token ranges but the insertion into the repair_history table is done 
with the same id for all of them.



--
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-14983) Local reads potentially blocking remote reads

2019-01-17 Thread Marcus Olsson (JIRA)


 [ 
https://issues.apache.org/jira/browse/CASSANDRA-14983?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-14983:
--
Attachment: graph_local_read_trunk.html
graph_local_read.html

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Priority: Minor
> Attachments: graph_local_read.html, graph_local_read_trunk.html, 
> local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> "data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
>  as the type of request.
> I have tried to run performance measurements to see the impact of this in 3.0 
> (by moving local requests to the end of ARE#executeAsync()) but I haven't 
> seen any big difference yet. I'll continue to run some more tests to see if I 
> can find a use case affected by this.
> Attaching a trace (3.0) where this happens. Reproduction:
>  # Create a three node CCM cluster
>  # Provision data with stress (rf=3)
>  # In parallel:
>  ## Start stress read run
>  ## Run multiple manual read queries in cqlsh with tracing on and 
> local_quorum (as this does not always happen)



--
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-14983) Local reads potentially blocking remote reads

2019-01-17 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14983:
---

I think I have found out why background traffic is required to reproduce this. 
In SEPExecutor#maybeExecuteImmediately() we try to take a work permit (and no 
task permit) but we 
[check|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/concurrent/SEPExecutor.java#L166]
 if there are task permits available in order to take the work permit. As the 
fast path does not add the task to the queue (#addTask() which also is the 
place where the task permits are increased) it might not have any task permits 
available. By changing the if statement to:
{code}
if (workPermits == 0 || (takeTaskPermit && taskPermits == 0))
 return false;
{code}
This made the fast path trigger more often. Actually when I added some metrics 
for this it seemed like the fast path was basically never used for the read 
path in my setup. With the if statement change I saw it used ~80% of the time 
with a thread count of 128 in stress. Unfortunately I did not see any 
performance difference when running tests on this with QUORUM.

But it did seem to have a large effect in a scenario with a single node and rf 
= 1 for 3.0 (graph_local_read.html). While running stress with 32 threads 
(pre-SEP-1, post-SEP-1) I could see a *~7%* throughput improvement locally.
When running stress with 128 threads (pre-SEP-128-1, post-SEP-128-1) the 
performance dropped slightly. The median latency seems lower but the higher 
percentiles are taking a hit.
For the final two tests (pre-SEP-128-cr128-1, post-SEP-128-cr128-1) I decided 
to increase the concurrent_read threads from 32 -> 128. The latency results are 
similar to the previous run but the throughput seems to have increased.
Note: I ran _echo 3 > /proc/sys/vm/drop_caches_ before these tests to clear the 
page cache, etc. which is why there is a large buildup in the beginning.

I also made a similar quick test for trunk (graph_local_read_trunk.html) where 
it seems to be a throughput improvement when using a low thread count. But the 
overall performance seems to have decreased with a default CCM setup (unless my 
environment was behaving oddly).

I think this could warrant it's own JIRA ticket to investigate more.

> Local reads potentially blocking remote reads
> -
>
> Key: CASSANDRA-14983
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
> Project: Cassandra
>  Issue Type: Bug
>  Components: Consistency/Coordination
>Reporter: Marcus Olsson
>Priority: Minor
> Attachments: local_read_trace.log
>
>
> Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
> to [work in the same 
> thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
>  rather than being sent over to the read stage.
> Based on the comment
> {code:java}
> // We delay the local (potentially blocking) read till the end to avoid 
> stalling remote requests.
> {code}
> it seems like this should be performed last in the chain to avoid blocking 
> remote requests but that does not seem to be the case when the local request 
> is a data request. The digest request(s) are sent after the data requests are 
> sent (and now the transient replica requests as well). When the fast path is 
> used for local data/transient data requests this will block the next type of 
> request from being sent away until the local read is finished and add 
> additional latency to the request.
> In addition to this it seems like local requests are *always* data requests 
> (might not be a problem), but the log message can say either ["digest" or 
> "data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
>  as the type of request.
> I have tried to run performance measurements to see the impact of this in 3.0 
> (by moving local requests to the end of ARE#executeAsync()) but I haven't 
> seen any big difference yet. I'll continue to run some more tests to see if I 
> can find a use case affected by this.
> Attaching a trace (3.0) where this happens. Reproduction:
>  # Create a three node CCM cluster
>  # Provision data with stress (rf=3)
>  # In parallel:
>  ## Start stress read run
>  ## Run multiple manual read queries in cqlsh with tracing on and 
> local_quorum (as this does not always happen)



--
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] [Created] (CASSANDRA-14983) Local reads potentially blocking remote reads

2019-01-15 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-14983:
-

 Summary: Local reads potentially blocking remote reads
 Key: CASSANDRA-14983
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14983
 Project: Cassandra
  Issue Type: Bug
  Components: Consistency/Coordination
Reporter: Marcus Olsson
 Attachments: local_read_trace.log

Since CASSANDRA-4718 there is a fast path allowing local requests to continue 
to [work in the same 
thread|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L157]
 rather than being sent over to the read stage.

Based on the comment
{code:java}
// We delay the local (potentially blocking) read till the end to avoid 
stalling remote requests.
{code}
it seems like this should be performed last in the chain to avoid blocking 
remote requests but that does not seem to be the case when the local request is 
a data request. The digest request(s) are sent after the data requests are sent 
(and now the transient replica requests as well). When the fast path is used 
for local data/transient data requests this will block the next type of request 
from being sent away until the local read is finished and add additional 
latency to the request.

In addition to this it seems like local requests are *always* data requests 
(might not be a problem), but the log message can say either ["digest" or 
"data"|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/reads/AbstractReadExecutor.java#L156]
 as the type of request.

I have tried to run performance measurements to see the impact of this in 3.0 
(by moving local requests to the end of ARE#executeAsync()) but I haven't seen 
any big difference yet. I'll continue to run some more tests to see if I can 
find a use case affected by this.

Attaching a trace (3.0) where this happens. Reproduction:
 # Create a three node CCM cluster
 # Provision data with stress (rf=3)
 # In parallel:
 ## Start stress read run
 ## Run multiple manual read queries in cqlsh with tracing on and local_quorum 
(as this does not always happen)



--
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-14659) Disable old protocol versions on demand

2018-08-23 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14659:
---

[~djoshi3], sure, as I said I don't really have a strong opinion on this and I 
think it would be useful as you said to have a test of client connections 
before performing a major upgrade.

 

I haven't tested/verified the patches but I did glance over them and found a 
few small things:
 * In the nodetool command for 
[DisableOldProtocolVersions|https://github.com/dineshjoshi/cassandra/blob/73f921d7f981f0be1efe688c6d02968eb784a15b/src/java/org/apache/cassandra/tools/nodetool/DisableOldProtocolVersions.java#L25]
 the description is "Enable old protocol versions" rather than "Disable old 
protocol versions".
 * In the dtest I think it would be good to have a verification that it's still 
possible to connect with the latest protocol version in between 
disabling/enabling the feature.

> Disable old protocol versions on demand
> ---
>
> Key: CASSANDRA-14659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14659
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: usability
>
> This patch allows the operators to disable older protocol versions on demand. 
> To use it, you can set {{native_transport_allow_older_protocols}} to false or 
> use nodetool disableolderprotocolversions. Cassandra will reject requests 
> from client coming in on any version except the current version. This will 
> help operators selectively reject connections from clients that do not 
> support the latest protoocol.



--
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-14659) Disable old protocol versions on demand

2018-08-22 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14659:
---

It seems like we support V3, V4 and V5 protocols on trunk as of today. Would it 
make sense to be able to be a bit more selective than to say only latest 
protocol or all of them? One option could be to specify the minimum protocol 
version to allow from clients.

 

I don't have a strong opinion on this but I thought it might potentially be 
useful to have a bit more flexibility with this feature when multiple protocol 
versions are supported.

> Disable old protocol versions on demand
> ---
>
> Key: CASSANDRA-14659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14659
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: usability
>
> This patch allows the operators to disable older protocol versions on demand. 
> To use it, you can set {{native_transport_allow_older_protocols}} to false or 
> use nodetool disableolderprotocolversions. Cassandra will reject requests 
> from client coming in on any version except the current version. This will 
> help operators selectively reject connections from clients that do not 
> support the latest protoocol.



--
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-13969) InterruptedException while running repair

2017-10-24 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13969?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-13969:
--
Attachment: executorexample.tar.gz

Attaching a small example project which has a simplified version of the 
described problem.

For mapping between the threadpools/tasks in the example and Apache Cassandra:
TP#1 - Outer threadpool executing RepairJob
SimpleTask - RepairJob
TP#2 - RepairSession.taskExecutor which is executing validation/sync requests
SubTask#1 - Validation request
SubTask#2 - Sync request

> InterruptedException while running repair
> -
>
> Key: CASSANDRA-13969
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13969
> Project: Cassandra
>  Issue Type: Bug
>  Components: Repair
> Environment: Cassandra 2.2.10, sub-range repairs
>Reporter: Marcus Olsson
>Priority: Minor
> Attachments: executorexample.tar.gz
>
>
> In one of our test clusters we observed the following error in system.log:
> {noformat}
> 2017-10-12T15:55:25.617+0200 ERROR [Repair#34:1] CassandraDaemon.java:195 
> Exception in thread Thread[Repair#34:1,5,RMI Runtime]
> java.lang.AssertionError: java.lang.InterruptedException
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:265)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logExceptionsAfterExecute(DebuggableThreadPoolExecutor.java:225)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:196)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1150)
>  ~[na:1.8.0_131]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  ~[na:1.8.0_131]
> at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131]
> Caused by: java.lang.InterruptedException: null
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1302)
>  ~[na:1.8.0_131]
> at 
> com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:285)
>  ~[guava-16.0.jar:na]
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) 
> ~[guava-16.0.jar:na]
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:261)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> ... 5 common frames omitted
> {noformat}
> Except for the exception there is no other odd outputs in system.log.
> From the repair history there is no indication of failed repairs and our 
> repair handler doesn't get any errors reported back through the progress 
> reporting either. One thing to note is that we utilize sub-range repairs and 
> repair one vnode at a time, which means that we effectively run several 
> hundreds of repair sessions for each table.
> From our repair handler the following is written in the logs:
> {noformat}
> 2017-10-12T15:55:25.611+0200 | INFO  | Repair of . - 
> [(8922822608060820611,8928269034264081622]] completed successfully
> 2017-10-12T15:55:25.678+0200 | INFO  | Repair of . - 
> [(-5406027845309604779,-5405899934869332173]] completed successfully
> 2017-10-12T15:55:25.744+0200 | INFO  | Repair of . - 
> [(1498725784389153529,1509146082320230540]] completed successfully
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13969) InterruptedException while running repair

2017-10-19 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-13969:
---

When looking at the code it seems to be the thread pool initialized 
[here|https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/repair/RepairRunnable.java#L220]
 (based on the name "Repair#XYZ"). This thread pool is shutdown using 
[executor.shutdownNow()|https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/repair/RepairRunnable.java#L348]
 when the repair is finished, which is the only place I could find where the 
threads running in it would be interrupted.

Based on the exception it seems like the thread is interrupted after having 
completed it's task, in the afterExcecute()-method of the thread pool. The task 
running in this thread pool is an instance of RepairJob, which spawns several 
subtasks in a separate thread pool. These subtasks include sending snapshot 
requests, validation requests and sync requests. Each of these tasks wait for 
the previous one to complete while the RepairJob is only waiting for the 
validation requests to complete.

Based on the structure of things it seems like the thread pool where an 
exception is thrown is shutdown by the thread pool executing the subtasks, 
after all subtasks have been completed. I believe that a race condition could 
happen when the sync-task is basically a no-op and happens to trigger the chain 
of events that leads to the shutdownNow() before the afterExecute() of the 
RepairJob has a chance to finish.

---

I have created a simplified scenario as described above (outside of Cassandra) 
and changing the shutdownNow() to a shutdown() seems to fix that situation. I 
have not yet found a good/easy way to test this behaviour in unit tests, so if 
anyone has any idea that would be great.

One question here is if the shutdownNow() is necessary for i.e. making sure 
that threads are cleaned up properly in case of a failure? Looking at 
RepairSession it seems like the sub thread pool is using 
[shutdown()|https://github.com/apache/cassandra/blob/cassandra-2.2/src/java/org/apache/cassandra/repair/RepairSession.java#L270]
 rather than shutdownNow(), so this might not be a problem?

> InterruptedException while running repair
> -
>
> Key: CASSANDRA-13969
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13969
> Project: Cassandra
>  Issue Type: Bug
>  Components: Repair
> Environment: Cassandra 2.2.10, sub-range repairs
>Reporter: Marcus Olsson
>Priority: Minor
>
> In one of our test clusters we observed the following error in system.log:
> {noformat}
> 2017-10-12T15:55:25.617+0200 ERROR [Repair#34:1] CassandraDaemon.java:195 
> Exception in thread Thread[Repair#34:1,5,RMI Runtime]
> java.lang.AssertionError: java.lang.InterruptedException
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:265)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logExceptionsAfterExecute(DebuggableThreadPoolExecutor.java:225)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:196)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1150)
>  ~[na:1.8.0_131]
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  ~[na:1.8.0_131]
> at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131]
> Caused by: java.lang.InterruptedException: null
> at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1302)
>  ~[na:1.8.0_131]
> at 
> com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:285)
>  ~[guava-16.0.jar:na]
> at 
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) 
> ~[guava-16.0.jar:na]
> at 
> org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:261)
>  ~[apache-cassandra-2.2.10.jar:2.2.10]
> ... 5 common frames omitted
> {noformat}
> Except for the exception there is no other odd outputs in system.log.
> From the repair history there is no indication of failed repairs and our 
> repair handler doesn't get any errors reported back through the progress 
> reporting either. One thing to note is that we utilize sub-range repairs and 
> repair one vnode at a time, which means that we effectively run several 
> hundreds of repair sessions for 

[jira] [Created] (CASSANDRA-13969) InterruptedException while running repair

2017-10-19 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-13969:
-

 Summary: InterruptedException while running repair
 Key: CASSANDRA-13969
 URL: https://issues.apache.org/jira/browse/CASSANDRA-13969
 Project: Cassandra
  Issue Type: Bug
  Components: Repair
 Environment: Cassandra 2.2.10, sub-range repairs
Reporter: Marcus Olsson
Priority: Minor


In one of our test clusters we observed the following error in system.log:
{noformat}
2017-10-12T15:55:25.617+0200 ERROR [Repair#34:1] CassandraDaemon.java:195 
Exception in thread Thread[Repair#34:1,5,RMI Runtime]
java.lang.AssertionError: java.lang.InterruptedException
at 
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:265)
 ~[apache-cassandra-2.2.10.jar:2.2.10]
at 
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.logExceptionsAfterExecute(DebuggableThreadPoolExecutor.java:225)
 ~[apache-cassandra-2.2.10.jar:2.2.10]
at 
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.afterExecute(DebuggableThreadPoolExecutor.java:196)
 ~[apache-cassandra-2.2.10.jar:2.2.10]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1150) 
~[na:1.8.0_131]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) 
~[na:1.8.0_131]
at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_131]
Caused by: java.lang.InterruptedException: null
at 
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1302)
 ~[na:1.8.0_131]
at 
com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:285)
 ~[guava-16.0.jar:na]
at 
com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:116) 
~[guava-16.0.jar:na]
at 
org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.extractThrowable(DebuggableThreadPoolExecutor.java:261)
 ~[apache-cassandra-2.2.10.jar:2.2.10]
... 5 common frames omitted
{noformat}
Except for the exception there is no other odd outputs in system.log.

>From the repair history there is no indication of failed repairs and our 
>repair handler doesn't get any errors reported back through the progress 
>reporting either. One thing to note is that we utilize sub-range repairs and 
>repair one vnode at a time, which means that we effectively run several 
>hundreds of repair sessions for each table.

>From our repair handler the following is written in the logs:
{noformat}
2017-10-12T15:55:25.611+0200 | INFO  | Repair of . - 
[(8922822608060820611,8928269034264081622]] completed successfully
2017-10-12T15:55:25.678+0200 | INFO  | Repair of . - 
[(-5406027845309604779,-5405899934869332173]] completed successfully
2017-10-12T15:55:25.744+0200 | INFO  | Repair of . - 
[(1498725784389153529,1509146082320230540]] completed successfully
{noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Created] (CASSANDRA-13886) OOM put node in limbo

2017-09-20 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-13886:
-

 Summary: OOM put node in limbo
 Key: CASSANDRA-13886
 URL: https://issues.apache.org/jira/browse/CASSANDRA-13886
 Project: Cassandra
  Issue Type: Bug
 Environment: Cassandra version 2.2.10
Reporter: Marcus Olsson
Priority: Minor


In one of our test clusters we have had some issues with OOM. While working on 
fixing this it was discovered that one of the nodes that got OOM actually 
wasn't shut down properly. Instead it went into a half-up-state where the 
affected node considered itself up while all other nodes considered it as down.

The following stacktrace was observed which seems to be the cause of this:
{noformat}
java.lang.NoClassDefFoundError: Could not initialize class java.lang.UNIXProcess
at java.lang.ProcessImpl.start(ProcessImpl.java:130) ~[na:1.8.0_131]
at java.lang.ProcessBuilder.start(ProcessBuilder.java:1029) 
~[na:1.8.0_131]
at java.lang.Runtime.exec(Runtime.java:620) ~[na:1.8.0_131]
at java.lang.Runtime.exec(Runtime.java:485) ~[na:1.8.0_131]
at 
org.apache.cassandra.utils.HeapUtils.generateHeapDump(HeapUtils.java:88) 
~[apache-cassandra-2.2.10.jar:2.2.10]
at 
org.apache.cassandra.utils.JVMStabilityInspector.inspectThrowable(JVMStabilityInspector.java:56)
 ~[apache-cassandra-2.2.10.jar:2.2.10]
at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:168)
 ~[apache-cassandra-2.2.10.jar:2.2.10]
at 
org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$LocalSessionFutureTask.run(AbstractLocalAwareExecutorService.java:136)
 ~[apache-cassandra-2.2.10.jar:2.2.10]
at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) 
~[apache-cassandra-2.2.10.jar:2.2.10]
at java.lang.Thread.run(Thread.java:748) [na:1.8.0_131]
{noformat}

It seems that if an unexpected exception/error is thrown inside 
JVMStabilityInspector.inspectThrowable the JVM is not actually shut down but 
instead keeps on running. My expectation is that the JVM should shut down in 
case OOM is thrown.

Potential workaround is to add:
{noformat}
JVM_OPTS="$JVM_OPTS -XX:+ExitOnOutOfMemoryError"
{noformat}
to cassandra-env.sh.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-13084) Avoid byte-array copy when key cache is disabled

2016-12-29 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-13084:
--
Status: Patch Available  (was: Open)

> Avoid byte-array copy when key cache is disabled
> 
>
> Key: CASSANDRA-13084
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13084
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Trivial
> Attachments: CASSANDRA-13084-3.0.patch, CASSANDRA-13084-3.X.patch
>
>
> While profiling a cluster with the key cache disabled some unnecessary heap 
> pressure was observed. It seems that even though the key cache is disabled we 
> still create the *KeyCacheKey* object which uses _ByteBufferUtil.getArray()_ 
> to copy the keys byte array.
> I don't believe it's a big issue but it's still unnecessary when not using 
> the key cache.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-13084) Avoid byte-array copy when key cache is disabled

2016-12-29 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-13084:
--
Attachment: CASSANDRA-13084-3.X.patch
CASSANDRA-13084-3.0.patch

Attaching patches for 3.0 and 3.X. The 3.X version should apply cleanly to 
trunk.

Unfortunately I have not been able to run the dtests locally with these patches 
as I seem to have some issues with my setup.

> Avoid byte-array copy when key cache is disabled
> 
>
> Key: CASSANDRA-13084
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13084
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Trivial
> Attachments: CASSANDRA-13084-3.0.patch, CASSANDRA-13084-3.X.patch
>
>
> While profiling a cluster with the key cache disabled some unnecessary heap 
> pressure was observed. It seems that even though the key cache is disabled we 
> still create the *KeyCacheKey* object which uses _ByteBufferUtil.getArray()_ 
> to copy the keys byte array.
> I don't believe it's a big issue but it's still unnecessary when not using 
> the key cache.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-13084) Avoid byte-array copy when key cache is disabled

2016-12-29 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-13084:
-

 Summary: Avoid byte-array copy when key cache is disabled
 Key: CASSANDRA-13084
 URL: https://issues.apache.org/jira/browse/CASSANDRA-13084
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Trivial


While profiling a cluster with the key cache disabled some unnecessary heap 
pressure was observed. It seems that even though the key cache is disabled we 
still create the *KeyCacheKey* object which uses _ByteBufferUtil.getArray()_ to 
copy the keys byte array.

I don't believe it's a big issue but it's still unnecessary when not using the 
key cache.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-13079) Repair doesn't work after several replication factor changes

2016-12-28 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-13079:
---

Good to hear that --full did the trick.

I think it would be a good idea for this type of scenario to change the repair 
state during replication altering, but I'm not sure if that's always the case.

In the scenario for adding a new data center I believe the recommended approach 
is to disable auto bootstrap and instead change the replication factor when the 
full data center is up and running. And then run "nodetool rebuild" to stream 
over the data from an existing data center. In that scenario it could be large 
amounts of data that would get marked as unrepaired and would have to be 
repaired again causing unnecessary load on the cluster.

Another scenario is reducing the replication factor, in that case I don't think 
there would be a need to alter the repair state since there should only be less 
replicas, but to me it feels like this scenario would be easier to cover than 
the multi-dc one.

Unless I'm missing something with the multi-dc scenario I'd say this would need 
to be implemented as an optional feature to avoid complexity(both operational 
and in code), but I'm not sure how this would be done or how feasible it is 
currently. Perhaps by adding some metadata to the schema altering statements?

> Repair doesn't work after several replication factor changes
> 
>
> Key: CASSANDRA-13079
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13079
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Debian 
>Reporter: Vladimir Yudovin
>Priority: Critical
>
> Scenario:
> Start two nodes cluster.
> Create keyspace with rep.factor *one*:
> CREATE KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> CREATE TABLE rep.data (str text PRIMARY KEY );
> INSERT INTO rep.data (str) VALUES ( 'qwerty');
> Run *nodetool flush* on all nodes. On one of them table files are created.
> Change replication factor to *two*:
> ALTER KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 2};
> Run repair, then *nodetool flush* on all nodes. On all nodes table files are 
> created.
> Change replication factor to *one*:
> ALTER KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> Then *nodetool cleanup*, only on initial node remained data files.
> Change replication factor to *two* again:
> ALTER KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 2};
> Run repair, then *nodetool flush* on all nodes. No data files on second node 
> (though expected, as after first repair/flush).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-13079) Repair doesn't work after several replication factor changes

2016-12-27 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-13079:
---

How are you running repair? If repair is run without the '--full' flag I 
believe this is the expected behaviour due to incremental repair being default 
since version 2.2. Incremental repair basically doesn't repair the data that 
has previously been repaired, which could explain the above situation.

> Repair doesn't work after several replication factor changes
> 
>
> Key: CASSANDRA-13079
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13079
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Debian 
>Reporter: Vladimir Yudovin
>Priority: Critical
>
> Scenario:
> Start two nodes cluster.
> Create keyspace with rep.factor *one*:
> CREATE KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> CREATE TABLE rep.data (str text PRIMARY KEY );
> INSERT INTO rep.data (str) VALUES ( 'qwerty');
> Run *nodetool flush* on all nodes. On one of them table files are created.
> Change replication factor to *two*:
> ALTER KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 2};
> Run repair, then *nodetool flush* on all nodes. On all nodes table files are 
> created.
> Change replication factor to *one*:
> ALTER KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 1};
> Then *nodetool cleanup*, only on initial node remained data files.
> Change replication factor to *two* again:
> ALTER KEYSPACE rep WITH replication = {'class': 'SimpleStrategy', 
> 'replication_factor': 2};
> Run repair, then *nodetool flush* on all nodes. No data files on second node 
> (though expected, as after first repair/flush).



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-10070) Automatic repair scheduling

2016-06-01 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-10070:
--
Attachment: Distributed Repair Scheduling_V2.doc

I have attached an updated feature draft, if anything seems unclear just let me 
know! :)

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc, Distributed Repair 
> Scheduling_V2.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-05-31 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

I'm currently working on it and hopefully I'll have it up in the next couple of 
days.

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Attachments: newDuration.patch
>
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-05-27 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

I've pushed some new patches to the [same 
branch|https://github.com/emolsson/cassandra/commits/CASSANDRA-11258] based on 
your comments/patches. :)

---

bq. How about renaming renew(int duration) to renew(int newDuration) to make 
this more explicit?
+1

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Attachments: newDuration.patch
>
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-05-24 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

Sorry for the delay. I've added the check for the existing lease in case we get 
a WTE 
[here|https://github.com/emolsson/cassandra/commit/cd6266ffc73be3a874d15ad0e2302652e316fed9].

bq. Sounds good! Perhaps we can add an expirationTime column to the lease table 
and compare it with the current value to see if it was updated, to avoid 
keeping track of TTLs? This would also provide a means for other nodes to check 
when the current lease will expire if necessary.
+1 I've added {{expirationTime}} to the patch. I've also changed from 
{{boolean}} to {{Optional}} for {{lease()}}, {{renewLease()}} and 
{{holdsLeaseUntil()}} so they return the stored {{expirationTime}}.

bq. Got it, thanks for the clarification! I was initially thinking renewLease 
would only extend the current lease period, but updating it for a smaller 
period might be a valid use case. We should probably make this more explicit in 
the renewLease javadoc.
I've added an example in the javadoc, hopefully that will make it clearer. I'm 
a bit unsure about the ["This duration is not added to the current 
duration."|https://github.com/emolsson/cassandra/commit/cd6266ffc73be3a874d15ad0e2302652e316fed9#diff-77d0ba5e1f9b13e38d862eec4591998aR39]
 part of the javadoc that I added, it might confuse more than it contributes, 
WDYT?

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-05-13 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

bq. I still don't see this in the latest commits, is there anything I'm 
missing? It seems that we return null on an exception on processSilent, and 
threat null responses as failure to acquire lock on renewLease. How about 
handling the exception on renewLease (instead of processSilent) and checking 
holdsLease before returning the lock?
Sorry, after I added that check I merged the code for update/insert into the 
{{renewLease()}}-method and it got lost in that merge, I'll add it back. Since 
{{renewLease()}} is both for creating new leases and renewing them I think it 
should check both that the lease is held by the local node and that the TTL has 
been updated, otherwise the call to {{lease.renew()}} might be successful even 
though the update of the lease was unsuccessful.

Also, in the case when we renew a lease and it fails by a WTE, what should be 
returned to the caller? Should we throw an exception or return a simple false? 
A lease renewal could be handled either as:
{code}
while(lease.isValid() && !lease.renew(30))
{
 Thread.sleep(1000);
}

if (!lease.isValid())
{
 // Stop repair
}
{code}
or:
{code}
boolean leaseRenewed = false;
while(lease.isValid() && !leaseRenewed)
{
 try {
  lease.renew(30);
  leaseRenewed = true;
 } catch (LeaseException e)
 {
  logger.warn("Unable to renew lease", e);
  Thread.sleep(1000);
 }
}

if (!lease.isValid())
{
 // Stop repair
}
{code}
I'm more for the "return false" approach here the code feels both easier and 
cleaner, WDYT?

---

bq. I don't quite get this case, why won't the row get updated when renewing 
the lease? Can you clarify with an example?
Sure, here's an example where some data is inserted and then updated it with a 
TTL of 1. When the updated data expires the row is still in the database.
{code}
cqlsh> CREATE KEYSPACE test WITH replication = {'class': 'SimpleStrategy', 
'replication_factor': 1};
cqlsh> CREATE TABLE test.row_ttl (a text, b text, c text, PRIMARY KEY(a));
cqlsh> INSERT INTO test.row_ttl (a, b, c) VALUES ('a', 'b', 'c') IF NOT EXISTS;

 [applied]
---
  True

cqlsh> SELECT * FROM test.row_ttl;
 a | b | c
---+---+---
 a | b | c

(1 rows)
cqlsh> UPDATE test.row_ttl USING TTL 1 SET b = 'b', c = 'c' WHERE a = 'a' IF b 
= 'b';

 [applied]
---
  True

# Wait until the data expires
cqlsh> SELECT * FROM test.row_ttl;

 a | b| c
---+--+--
 a | null | null

(1 rows)
{code}
As you can see the column {{a}} is set, while the other columns are null. This 
makes the next {{INSERT IF NOT EXISTS}} fail as:
{code}
cqlsh> INSERT INTO test.row_ttl (a, b, c) VALUES ('a', 'b', 'c') IF NOT EXISTS;

 [applied] | a | b| c
---+---+--+--
 False | a | null | null
{code}

If you try to update the column {{a}} with the other columns you get the 
following error:
{code}
cqlsh> UPDATE test.row_ttl USING TTL 1 SET a = 'a', b = 'b', c = 'c' WHERE a = 
'a';
InvalidRequest: code=2200 [Invalid query] message="PRIMARY KEY part a found in 
SET part"
{code}

In our case we have a TTL of 60 seconds for the primary key column so this is 
only a problem if you create the lease and renew it directly for a small 
duration like 10 seconds and let it expire. In that case {{lease.cancel()}} 
wouldn't try to clear the lease and this would make the primary key column 
remain for the full 60 seconds and prevent new leases from being obtained. One 
option could be to always delete the row in {{lease.cancel()}}, but that should 
be an unnecessary operation if the lease is expired. As I stated before, this 
is maybe a small enough issue that it might not make a difference but I'm not 
too fond of having the inconsistency that we sometimes have that column and 
sometimes not, since it disappears after 60 seconds.

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-05-04 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

@[~jbellis]
bq. How closely does this match the design doc from February? Is it worth 
posting an updated design for those of us joining late?
I'd say there have been enough changes for it to be a good idea to update the 
document, so I'll work on that! :)

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-05-04 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

I've added the check for if the lease was acquired in the case of a WTE and 
also added the mentioned tests to the same branch as before.

I also did some other changes:
* Added javadoc to package/classes (based on the ongoing proposal in the dev 
mailing list regarding mandatory comments).
* Changed from insert to update when doing CAS for the lease.

The reason for changing from {{insert}} to {{update}} was that with the 
previous {{insert}} a row marker is added with a TTL of 60(default TTL) and it 
doesn't get updated when renewing the lease since that uses an {{update}}. So 
if we would "forget" a lease or are unable to renew it in time we would have a 
row marker that is blocking us from acquiring the lease until the TTL has 
expired. Another alternative would be to do a {{SERIAL}} read and check if the 
host column isn't set and then either delete it and retry or use update with 
CAS. Always using update seemed cleaner for the implementation, but it might be 
that this is a small enough issue that the 60 second wait isn't a problem since 
it's only the first 60 seconds from when the lease was acquired.

bq. While it would be interesting to add dtests to make sure leases are working 
in a distributed environment, we would probably need to expose the LeaseFactory 
interface over JMX, but we want to keep this strictly internal to avoid 
external misuse. So it's probably better to move on and test this more 
extensively on dtests via scheduled repairs, for instance by triggering 
simultaneous scheduled repair requests and modifying the resource lease tables 
manually to test leases in scenarios with tampering, network partitions and 
nodes out of sync, so let's leave this for a future task.
I had the same thoughts about dtest and it makes more sense to wait until we 
have scheduled repairs until we test it like that.

---

Since the comments/implementation details regarding update vs insert shouldn't 
affect the interface as such I'll start implementing the repair scheduling. :)

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-04-28 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

bq. The only thing I mildly dislike about the interfaces is that they throw 
exception when it's not possible to acquire or renew the lock, but since this 
is quite a common case should we use Optional and boolean instead? WDYT about 
this definition?
Sounds good, it seems cleaner to use Optional and booleans instead of 
exceptions. :)

I've pushed a new commit 
[here|https://github.com/emolsson/cassandra/commits/CASSANDRA-11258].

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-04-21 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

bq. From the code it seems that when an LWT insert timeouts, the CasLockFactory 
assumes the lock was not acquired, but maybe the operation succeeded and there 
was a timeout, so we will not be able to re-acquire the lock before it expires. 
So we should perform a read at SERIAL level in this situation to make sure any 
previous in-progress operations are committed and we get the most recent value.
Good catch, I'll add that. 

bq. Is the sufficientNodesForLocking check necessary?
It is mostly to avoid trying to do CAS operations that we know will fail, 
however that check would be done later down in StorageProxy, so it might be 
redundant.

bq. I noticed that we are doing non-LWT reads at ONE, but we should use QUORUM 
instead and that check will be automatically done when reading or writing.
I'll change that.

bq. I think we should adjust our nomenclature and mindset from distributed 
locks to expiring leases, since this is what we are doing rather than 
distributed locking. If you agree, can you rename classes to reflect that?
I agree, leases seems to be a more reasonable term for it.

{quote}
When renewing the lease we should also insert the current lease holder priority 
into the resource_lock_priority table, otherwise other nodes might try to 
acquire the lease while it's being hold (the operation will fail, but the load 
on the system will be higher due to LWT).

We should also probably let lease holders renew leases explicitly rather than 
auto-renewing leases at the lease service, so for example the job scheduler can 
abort the job if it cannot renew the lease. For that matter, we should probably 
extend the DistributedLease interface with methods to renew the lease and/or 
check if it's still valid (perhaps we should have a look at the JINI lease spec 
for inspiration, although it looks a bit verbose).
{quote}
I've taken a look at the JINI lease spec and I think there are some parts of it 
that we wouldn't need, for instance {{setSerialFormat()}} and {{canBatch()}}. 
But the interface could perhaps look like this instead:
{code}
interface Lease {
 long getExpiration();
 void renew(long duration) throws LeaseException;
 void cancel(); throws LeaseException;
 boolean valid();
}

interface LeaseGrantor { // Or LeaseFactory
 Lease newLease(long duration, String resource, int priority, Map metadata); throws LeaseException
}
{code}
I think the {{LeaseMap}}(mentioned in the JINI lease spec) or a similar 
interface will be useful for locking multiple data centers. Maybe it's enough 
to create some kind of {{LeaseCollection}} that bundles the leases together and 
performs renew()/cancel() on all underlying leases?

--
I'll also change the keyspace name to {{system_leases}} and the tables to 
{{resource_lease}} and {{resource_lease_priority}}.

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-04-15 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

I've pushed a rebased patch where I addressed your two comments from github 
(the comments got lost in the rebase so should I avoid rebase until review is 
done in the future?). I also added a removal of the locked resource when the 
lock object is closed.

Regarding my previous comment I'm starting to think it would be cleaner(for the 
code using the locks at least) to have the implementation the way Java locks 
are used instead.

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11264) Repair scheduling - Failure handling and retry

2016-04-14 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11264:
---

bq. After having a look at your original patch I saw that a failed task will be 
re-prioritized against other scheduled jobs/tasks with a high priority (given 
its last run time will not be updated), so that's already a retry mechanism in 
itself.
While this is true, I believe that this part should probably be reworked a bit. 
If we have a scenario where one particular job will always fail, we will end up 
in a loop where that job would get retried constantly which leads to starvation 
of other jobs. One option is to keep it simple and only run it once (by 
removing the retry logic) and also add a flag for the job which is used to 
determine when the job is allowed to run again. Something like:
{code}
execute()
{
 runTasks();
 if (allTasksWasSuccessful())
 {
  nextRun = -1
  lastRunTime = now;
 }
 else
 {
  nextRun = now + defaultWaitTime;
 }
}
{code}
Then that flag would be used to avoid prioritizing the failing job against the 
other jobs until the {{defaultWaitTime}} has elapsed. This flag could also work 
nicely with the rejection policies (assuming that they estimate the time until 
the job can actually be run), especially if we would be able to reject repairs 
on a specific table rather than all tables. WDYT?

bq. Rather than cluttering the scheduled repair mechanism with retry logic, I 
think that it's better to add a retry option to (non-scheduled) repair job, and 
do more fine grained retry on individual steps such as validation and sync, 
since this will be more effective against transient failures rather than 
retrying the whole task and potentially losing work of non-failed tasks.
Great idea! If e.g. a validation would fail on one node, would we clean up the 
resources on that node by CASSANDRA-11190 (specifically about cleaning up 
resources, so that we can safely retry it) or would we need a separate way of 
doing that? 

bq. We can of course log warns and gather statistics when a scheduled task 
fails, but I think we should add retry support to repair independently of this. 
WDYT?
Sounds good!

> Repair scheduling - Failure handling and retry
> --
>
> Key: CASSANDRA-11264
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11264
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Make it possible for repairs to be run again if they fail and clean up the 
> associated resources (validations and streaming sessions) before retrying. 
> Log a warning for each re-attempt and an error if it can't complete in X 
> times. The number of retries before considering the repair a failure could be 
> configurable.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-04-12 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

[~pauloricardomg] Sorry for the delay in getting back to this ticket.

--

I've pushed a rebased branch 
[here|https://github.com/emolsson/cassandra/commits/CASSANDRA-11258]. The lock 
factory implementation is creating locks in a try-with-resources fashion:
{code}
interface DistributedLock extends Closeable
{
}

interface LockFactory
{
  DistributedLock tryLock(String resource, int priority, Map 
metadata);
}

...

try (DistributedLock lock = lockFactory.tryLock("RepairResource-DC1-1", 1, new 
HashMap<>()))
{
  // Do repair
}
catch (LockException e)
{
  // Unable to lock the resource
}
{code}
Which would keep the lock updated with a scheduled task until {{lock.close()}} 
is called.

--

Another alternative would be to go for the java 
[Lock|https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/locks/Lock.html]
 style of implementation and perhaps only implement {{tryLock()}} and 
{{unlock()}} on the locks and have the lock factory return a lockable object 
instead:
{code}
interface LockFactory
{
  Lock getLock(String resource, int parallelism, int priority, 
Map metadata);
}

...

Lock lock = lockFactory.getLock("RepairResource-DC1", 1, 1, new HashMap<>());
if (lock.tryLock())
{
  try
  {
// Do repair
  }
  finally
  {
lock.unlock();
  }
}
else
{
  // Unable to lock the resource
}
{code}

WDYT?

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-9621) Repair of the SystemDistributed keyspace creates a non-trivial amount of memory pressure

2016-04-12 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-9621?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-9621:
-
Assignee: Marcus Eriksson  (was: Marcus Olsson)

> Repair of the SystemDistributed keyspace creates a non-trivial amount of 
> memory pressure
> 
>
> Key: CASSANDRA-9621
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9621
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Sylvain Lebresne
>Assignee: Marcus Eriksson
>Priority: Minor
> Fix For: 2.2.0 rc2, 3.0 alpha 1
>
>
> When a repair without any particular option is triggered, the 
> {{SystemDistributed}} keyspace is repaired for all range, and in particular 
> the {{repair_history}} table. For every range, that table is written and 
> flushed (as part of normal repair), meaning that every range triggers the 
> creation of a new 1MB slab region (this also triggers quite a few compactions 
> that also write and flush {{compaction_progress}} at every start and end).
> I don't know how much of a big deal this will be in practice, but I wonder if 
> it's really useful to repair the {{repair_*}} tables by default so maybe we 
> could exclude the SystemDistributed keyspace from default repairs and only 
> repair it if asked explicitly?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Assigned] (CASSANDRA-9621) Repair of the SystemDistributed keyspace creates a non-trivial amount of memory pressure

2016-04-12 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-9621?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson reassigned CASSANDRA-9621:


Assignee: Marcus Olsson  (was: Marcus Eriksson)

> Repair of the SystemDistributed keyspace creates a non-trivial amount of 
> memory pressure
> 
>
> Key: CASSANDRA-9621
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9621
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Sylvain Lebresne
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 2.2.0 rc2, 3.0 alpha 1
>
>
> When a repair without any particular option is triggered, the 
> {{SystemDistributed}} keyspace is repaired for all range, and in particular 
> the {{repair_history}} table. For every range, that table is written and 
> flushed (as part of normal repair), meaning that every range triggers the 
> creation of a new 1MB slab region (this also triggers quite a few compactions 
> that also write and flush {{compaction_progress}} at every start and end).
> I don't know how much of a big deal this will be in practice, but I wonder if 
> it's really useful to repair the {{repair_*}} tables by default so maybe we 
> could exclude the SystemDistributed keyspace from default repairs and only 
> repair it if asked explicitly?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11486) Duplicate logging of merkle tree request

2016-04-04 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11486:
-

 Summary: Duplicate logging of merkle tree request
 Key: CASSANDRA-11486
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11486
 Project: Cassandra
  Issue Type: Improvement
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Trivial
 Attachments: duplicate_repair_logging-2.2.patch, 
duplicate_repair_logging-3.0.patch

When repairing a table using parallel repair the "Requesting merkle trees for 
x" message is logged twice:
{noformat}
2016-04-04T13:17:06.217+0200  INFO [Repair#47:1] RepairJob.java:107 [repair 
#c370f703-fa56-11e5-a98e-bbedcba36f16] requesting merkle trees for x (to 
[/x.x.x.1, /x.x.x.2, /x.x.x.3])
2016-04-04T13:17:06.217+0200  INFO [Repair#47:1] RepairJob.java:181 [repair 
#c370f703-fa56-11e5-a98e-bbedcba36f16] Requesting merkle trees for x (to 
[/x.x.x.1, /x.x.x.2, /x.x.x.3])
{noformat}

Attaching trivial patches that removes the duplicate logging as well as adding 
a repair trace message to sequential validations. The 3.0 version should apply 
cleanly to trunk.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11412) Many sstablescanners opened during repair

2016-03-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11412:
---

Hmm.. yes that seems to be a bug. Either we could implement hashCode() and 
equals() on all scanners or we could change (or add a separate) 
{{AbstractCompactionStrategy.getScanners()}} to take a collection of 
{{Range}}. I think the second option would be preferable since we would 
only call {{SSTableReader.getScanner()}} once for each sstable. There is 
already a {{getScanner()}} method that takes [a collection of 
ranges|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java#L1761]
 in SSTableReader. This would also make more sense with LCS since as it is now 
even if we check for equality the LeveledScanner might not always be equal for 
all ranges and create one scanner per range either way.

This would still make it so that we have one scanner per sstable opened at the 
same time though(which would pretty much be the same as pre-CASSANDRA-5220), 
except for when using leveled compaction. So it might be good to only open the 
scanners when needed depending on how many sstables we have. The implementation 
for that would probably be a bit more complex than for LCS since in 
LeveledScanner we work with the assumption that all sstables in a level are 
non-overlapping so we only open one scanner at a time. I think the partition 
iteration for overlapping sstables could be done as:
# Sort all the sstables based on tokens
# Open the first sstable scanner
# Read/merge first partition from all open scanners (and close exhausted 
scanners) and keep the first partition
# Compare the partition to the non-open sstables to see if they may contain a 
partition before or at the same token as the previously read partition, open a 
scanner if that's the case and then read the first partition
# Compare/merge all current partitions and return the first
# Continue at step 3 until all sstables are read

Unless I'm over-thinking this.

> Many sstablescanners opened during repair
> -
>
> Key: CASSANDRA-11412
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11412
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Eriksson
>
> Since CASSANDRA-5220 we open [one sstablescanner per range per 
> sstable|https://github.com/apache/cassandra/blob/cassandra-3.0/src/java/org/apache/cassandra/db/compaction/CompactionStrategyManager.java#L374].
>  If compaction gets behind and you are running vnodes with 256 tokens and 
> RF3, this could become a problem (ie, {{768 * number of sstables}} scanners)
> We could probably refactor this similar to the way we handle scanners with 
> LCS - only open the scanner once we need it



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11390) Too big MerkleTrees allocated during repair

2016-03-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11390:
---

There is a comment in the doValidationCompaction() log message saying
{code}
// MT serialize may take time
{code}
Which should probably be moved to createMerkleTrees() log message instead or 
removed, whichever you prefer.

After that +1 :)

> Too big MerkleTrees allocated during repair
> ---
>
> Key: CASSANDRA-11390
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11390
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
> Fix For: 3.0.x, 3.x
>
>
> Since CASSANDRA-5220 we create one merkle tree per range, but each of those 
> trees is allocated to hold all the keys on the node, taking up too much memory



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11390) Too big MerkleTrees allocated during repair

2016-03-22 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11390:
---

bq. I imagine this is what was always intended - perhaps we should open a new 
ticket to investigate if we should increase it
It would probably be good to test if it's a reasonable limit, but it might not 
have that high priority unless we see lots of over-streaming from the current 
one.

bq. Note that we don't care about the ranges when we calculate this, so we have 
to assume that gain within a range is the same as the total gain. Biggest 
problem is how to test this, will try to figure something out.
If it gets too complex to the test it might not be worth to have the compaction 
gain as part of the calculation. It would most probably reduce the MerkleTrees 
sizes, which is good unless the compaction gain comes from data that is not 
part of the repair. Capping the MerkleTrees total size might be good enough 
alone since the only thing the duplicate partitions should bring is 
unnecessarily large resolution, not the memory problems. It could possibly be a 
separate ticket to investigate if there would be a gain from using the 
compaction gain in the calculation.

---

For:
{code}
logger.trace("Created {} merkle trees, {} partitions, {} bytes", tree.size(), 
allPartitions, MerkleTrees.serializer.serializedSize(tree, 0));
{code}
The {{MerkleTrees.size()}} method returns the combined value from calling 
{{MerkleTree.size()}} on all MerkleTrees, which returns {{2^d}}. To get the 
number of merkle trees we could either create a new method in {{MerkleTrees}} 
(treeCount()?) or use {{MerkleTrees.ranges().size()}}. It could probably be 
good to have both the number of trees as well as the output from 
{{MerkleTrees.size()}} in the log output.

Other than that LGTM. :)

> Too big MerkleTrees allocated during repair
> ---
>
> Key: CASSANDRA-11390
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11390
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
> Fix For: 3.0.x, 3.x
>
>
> Since CASSANDRA-5220 we create one merkle tree per range, but each of those 
> trees is allocated to hold all the keys on the node, taking up too much memory



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11390) Too big MerkleTrees allocated during repair

2016-03-22 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11390:
---

Good point, the % responsibility should probably be in the calculation. So 
instead it could be:
{noformat}
2^d < percent * 2^20
<=>
d < log2(percent * 2^20)
<=>
d < log2(percent) + 20
{noformat}
Where {{log2\(percent)}} would be negative for each value such as {{0.0 < 
percent < 1.0}}.

Or in java:
{code}
int maxDepth = (int) Math.floor(20 + Math.log(percent) / Math.log(2));
{code}

Using this I'd say there are two options, either we base the percentage on the 
range sizes or on the estimated partitions for each range. Both would require 
us to iterate through all ranges once before estimating the depth for each 
tree, but using the estimated partition count would probably be more effective.

---

Another thing we should consider is if the total limit should be {{2^20}}. 
Before vnodes this was used for a single token range and after vnodes it was 
instead {{vnodes * 2^20}} per node. This gave us a much higher resolution in 
the merkle trees with vnodes. If we divide {{2^20}} between all ranges we go 
back to the pre-vnode merkle tree resolution for each node (possibly lower).

> Too big MerkleTrees allocated during repair
> ---
>
> Key: CASSANDRA-11390
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11390
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
> Fix For: 3.0.x, 3.x
>
>
> Since CASSANDRA-5220 we create one merkle tree per range, but each of those 
> trees is allocated to hold all the keys on the node, taking up too much memory



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11390) Too big MerkleTrees allocated during repair

2016-03-22 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11390:
---

Sure! :) I've taken a glance at the patch and it looks good and should reduce 
the merkle tree memory usage in most cases. However I'm worried that we still 
might be able to get into a situation where we allocate too much memory for the 
merkle trees since we have a max depth of {{20}} which was used as max when 
there was only one merkle tree. Due to the fact that we have a variable number 
of concurrently repairing ranges it would probably be ineffective to simply 
lower the max depth since that would reduce the resolution of the merkle trees 
when we are repairing a fewer number of ranges. Instead we could calculate the 
max depth each time. The number of nodes in a merkle tree is roughly {{2^d}} 
where {{d}} is the depth of the tree and before CASSANDRA-5220 the max number 
of nodes were {{2^20 = 1048576}}. If we want {{2^20}} to be the max total 
number of nodes then {{ranges * 2^d}} would have to be less than {{2^20}} which 
we could calculate as:

{code}
2^20 >= ranges * 2^d
<=>
log2(2^20) >= log2(ranges * 2^d)
<=>
20 >= log2(ranges) + d
<=>
d <= 20 - log2(ranges)
{code}

In java:
{code}
int maxDepth = (int) Math.floor(20 - Math.log(validator.desc.ranges.size()) / 
Math.log(2));
// And then calculate the depth using:
int depth = numPartitions > 0 ? (int) 
Math.min(Math.floor(Math.log(numPartitions)), maxDepth) : 0;
{code}

---

Another thing that also could reduce the merkle tree sizes is if we are able to 
estimate how much overlap there is between sstables for each range, since that 
could be used to reduce the estimated number of partitions. The effectiveness 
of this would probably depend on the compaction strategy and how accurately we 
can calculate the range overlap though.

> Too big MerkleTrees allocated during repair
> ---
>
> Key: CASSANDRA-11390
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11390
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
> Fix For: 3.0.x, 3.x
>
>
> Since CASSANDRA-5220 we create one merkle tree per range, but each of those 
> trees is allocated to hold all the keys on the node, taking up too much memory



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-03-14 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

I'm on vacation and I will be back in office 21st of March!

/Marcus Olsson


> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-03-11 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

Great, then I'll go for one branch per task.

I've created a branch for the resource locking API 
[here|https://github.com/emolsson/cassandra/commits/CASSANDRA-11258]. 
Unfortunately I will not make much progress next week, but after that I will be 
able to dedicate more time to this. :)

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-03-08 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

bq. Yes, I think we should advance the implementation based on SimpleStrategy 
for now and focus on single DC first, and think better about multi-DC support 
in a separate task (since it may become easier to reason about multi-DC details 
when we have a full scheduled repair cycle working for a single DC).
+1

I've started work with the lock factory. How should we proceed with the 
coding/reviewing? Should we have separate branches for each task and when 
review is done incrementally append them to a "master" branch? Or should we 
keep it all in a single branch?

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-03-04 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

bq. WDYT of this approach?
I like it, it would reduce the amount of work for the write operations while 
keeping a simple structure of the lock keyspace. However, *if* we want to 
visualize all data center locks to the user, it would be more complex to do so.

bq. If you like this approach, we could do the initial version assuming a 
single DC with SimpleStrategy replication + SERIAL consistency, while power 
users could still have multi-DC support by manually changing replication 
settings of the lock keyspace. We could later add transparent/efficient 
multi-DC support via CASSANDRA-11300 and LocalDataCenterReplicationStrategy.
+1

Should we start with {{SimpleStrategy}} and a replication of 3 then? If we only 
have one node there would be no need for repairs and if we have two it would be 
possible to do CAS requests. If we include the 
{{LocalDataCenterReplicationStrategy}} class directly in the first version but 
use SimpleStrategy until CASSANDRA-11300 is in place it should be possible to 
switch to LDCRS automatically later on, assuming that repairs would be paused 
during that upgrade. If we don't include LDCRS directly I think the switch 
would either have to be manual or done by first verifying the version of the 
other nodes, so that the "older" nodes wouldn't complain about an unknown 
replication class.

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-03-03 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

bq. While I think we could add a new VERB (REMOTE_CAS) to the messaging service 
without a protocol bump (by reusing the UNUSED_X verbs), I think we could this 
in a separate ticket to avoid losing focus here.
Great, I'll create a JIRA for it and link it to this one.

bq. So I propose we use a global CAS (SERIAL consistency) for each DC lock for 
the first version, which should make multi-dc schedule repairs work when there 
is no network partition, and improve later when the REMOTE_CAS verb is in 
place. WDYT?
+1

For this lock table to work correctly later on, it should be set up to have 
replicas in all data centers, right? Should this be automatically configured or 
should this be something that the user would have to configure when 
adding/removing data centers? From a usability point I think it would be great 
if this was handled automatically and it would probably not be too hard to 
create a replication strategy defined as "at most X replicas in each dc", but 
I'm not sure if this might cause problems if someone where to use it for other 
purposes?

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11300) Support for forwarding of CAS requests

2016-03-03 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11300:
-

 Summary: Support for forwarding of CAS requests
 Key: CASSANDRA-11300
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11300
 Project: Cassandra
  Issue Type: New Feature
Reporter: Marcus Olsson
Priority: Minor


For CASSANDRA-11258 to be able to lock a resource in a specific data center it 
would be needed to forward the CAS request to a node in that data center that 
would act as the coordinator for the request with LOCAL_SERIAL consistency.

Proposal is to add a Verb (REMOTE_CAS) that is used to forward the request to 
another node.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-03-02 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11258:
---

cc [~pauloricardomg]

bq. All data centers involved in a repair must be available for a repair to 
start/succeed, so if we make the lock resource dc-aware and try to create the 
lock by contacting a node in each involved data center with LOCAL_SERIAL 
consistency that should be sufficient to ensure correctness without the need 
for a global lock. This will also play along well with both dc_parallelism 
global option and with the --local or --dcs table repair options.

As it is right now I don't think it's possible to forward a LWT query to 
another node (at least not through the MessagingService). I guess we wouldn't 
want to do the access through the native layer since that would require that we 
have auth and encryption configured. So I think there are a couple of options 
here:
# More expressive consistency levels, perhaps CASSANDRA-8119? (e.g. "QUORUM in 
DC1")
# Make it possible to forward a LWT requests and have a node in the other dc 
act as a coordinator for the request.
# Extend the current LWT implementation to be able to coordinate "LOCAL_SERIAL" 
in a remote dc.
# Create separate lock keyspaces for each dc.

I think that 3 & 4 should be possible to implement directly but 1 & 2 would 
probably require a protocol change. It should be possible to start by extending 
the current LWT implementation and then switch to either 1 or 2 later on if 
needed.

WDYT?

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-29 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

I think I might have had a bug in my environment when I ran it, now it fails on 
3.0 for me as well. Should we include the null check in the 2.2 version as well?

The reason for putting the test case in a separate class in the repair_test.py 
was that it uses the ignore_log_patterns. I wasn't sure if that could affect 
the other tests as well, so I put it in a separate class to be safe.

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
> Fix For: 2.2.x, 3.0.x, 3.x
>
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-29 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

I've rebased the 2.2 version and added your synchronized change, the branches 
are available here:

||2.2||3.0||trunk
|[branch|https://github.com/emolsson/cassandra/commit/3f9d87329f367eb90706efe32afc390bf8171d06]|[branch|https://github.com/emolsson/cassandra/commit/dcb9c288665aa7b1e4a773d19c31f4c6b24e726b]|[branch|https://github.com/emolsson/cassandra/commit/f10faffa7fdd7bf2b8625e8116e6219d56a515b7]

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
> Fix For: 2.2.x, 3.0.x, 3.x
>
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-02-26 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

I've now created the sub-tasks and linked them to this issue. I didn't include 
the node configuration since it might be redundant but we might add it later on 
if we feel the need to.

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11264) Repair scheduling - Failure handling and retry

2016-02-26 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11264?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-11264:
--
Issue Type: Sub-task  (was: Task)
Parent: CASSANDRA-10070

> Repair scheduling - Failure handling and retry
> --
>
> Key: CASSANDRA-11264
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11264
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Make it possible for repairs to be run again if they fail and clean up the 
> associated resources (validations and streaming sessions) before retrying. 
> Log a warning for each re-attempt and an error if it can't complete in X 
> times. The number of retries before considering the repair a failure could be 
> configurable.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11264) Repair scheduling - Failure handling and retry

2016-02-26 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11264:
-

 Summary: Repair scheduling - Failure handling and retry
 Key: CASSANDRA-11264
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11264
 Project: Cassandra
  Issue Type: Task
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Minor


Make it possible for repairs to be run again if they fail and clean up the 
associated resources (validations and streaming sessions) before retrying. Log 
a warning for each re-attempt and an error if it can't complete in X times. The 
number of retries before considering the repair a failure could be configurable.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11263) Repair scheduling - Polling and monitoring module

2016-02-26 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11263:
-

 Summary: Repair scheduling - Polling and monitoring module
 Key: CASSANDRA-11263
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11263
 Project: Cassandra
  Issue Type: Task
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Minor


Create a module that keeps track of currently running repairs and make sure 
that they are either associated with a resource lock or are user-created. 
Otherwise they should be aborted and the on-going validations/streaming 
sessions should be stopped. A log warning should be issued as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11259) Repair scheduling - Maintenance scheduling API & basic repair scheduling

2016-02-26 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11259:
-

 Summary: Repair scheduling - Maintenance scheduling API & basic 
repair scheduling
 Key: CASSANDRA-11259
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11259
 Project: Cassandra
  Issue Type: Task
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Minor


Create a scheduling API that can run maintenance jobs. To determine which node 
should run a maintenance task the locking API from CASSANDRA-11258 should be 
used. These jobs should have a dynamic priority that is comparable between 
nodes.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11263) Repair scheduling - Polling and monitoring module

2016-02-26 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11263?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-11263:
--
Issue Type: Sub-task  (was: Task)
Parent: CASSANDRA-10070

> Repair scheduling - Polling and monitoring module
> -
>
> Key: CASSANDRA-11263
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11263
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a module that keeps track of currently running repairs and make sure 
> that they are either associated with a resource lock or are user-created. 
> Otherwise they should be aborted and the on-going validations/streaming 
> sessions should be stopped. A log warning should be issued as well.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11262) Repair scheduling - Global configuration

2016-02-26 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11262?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-11262:
--
Issue Type: Sub-task  (was: Task)
Parent: CASSANDRA-10070

> Repair scheduling - Global configuration
> 
>
> Key: CASSANDRA-11262
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11262
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Make it possible to configure the maintenance scheduler on all nodes. Should 
> include the possibility to pause repairs on all nodes.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11262) Repair scheduling - Global configuration

2016-02-26 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11262:
-

 Summary: Repair scheduling - Global configuration
 Key: CASSANDRA-11262
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11262
 Project: Cassandra
  Issue Type: Task
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Minor


Make it possible to configure the maintenance scheduler on all nodes. Should 
include the possibility to pause repairs on all nodes.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11261) Repair scheduling - Table configuration

2016-02-26 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11261:
-

 Summary: Repair scheduling - Table configuration
 Key: CASSANDRA-11261
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11261
 Project: Cassandra
  Issue Type: Task
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Minor


Make it possible to configure tables to run scheduled repairs. First 
configurations should be to define that the table should be scheduled for 
repair and how often.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11261) Repair scheduling - Table configuration

2016-02-26 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11261?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-11261:
--
Issue Type: Sub-task  (was: Task)
Parent: CASSANDRA-10070

> Repair scheduling - Table configuration
> ---
>
> Key: CASSANDRA-11261
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11261
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Make it possible to configure tables to run scheduled repairs. First 
> configurations should be to define that the table should be scheduled for 
> repair and how often.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11260) Repair scheduling - Rejection policy interface

2016-02-26 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11260:
-

 Summary: Repair scheduling - Rejection policy interface
 Key: CASSANDRA-11260
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11260
 Project: Cassandra
  Issue Type: Task
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Minor


Create an interface for rejection policies that could be used to specify 
maintenance windows. This interface should also be used to define system-known 
interruptions like upgrades. A default implementation that is strictly 
time-based should be included.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11260) Repair scheduling - Rejection policy interface

2016-02-26 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11260?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-11260:
--
Issue Type: Sub-task  (was: Task)
Parent: CASSANDRA-10070

> Repair scheduling - Rejection policy interface
> --
>
> Key: CASSANDRA-11260
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11260
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create an interface for rejection policies that could be used to specify 
> maintenance windows. This interface should also be used to define 
> system-known interruptions like upgrades. A default implementation that is 
> strictly time-based should be included.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11259) Repair scheduling - Maintenance scheduling API & basic repair scheduling

2016-02-26 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11259?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-11259:
--
Issue Type: Sub-task  (was: Task)
Parent: CASSANDRA-10070

> Repair scheduling - Maintenance scheduling API & basic repair scheduling
> 
>
> Key: CASSANDRA-11259
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11259
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a scheduling API that can run maintenance jobs. To determine which 
> node should run a maintenance task the locking API from CASSANDRA-11258 
> should be used. These jobs should have a dynamic priority that is comparable 
> between nodes.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Updated] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-02-26 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-11258?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-11258:
--
Issue Type: Sub-task  (was: Task)
Parent: CASSANDRA-10070

> Repair scheduling - Resource locking API
> 
>
> Key: CASSANDRA-11258
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
> Project: Cassandra
>  Issue Type: Sub-task
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
>
> Create a resource locking API & implementation that is able to lock a 
> resource in a specified data center. It should handle priorities to avoid 
> node starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11258) Repair scheduling - Resource locking API

2016-02-26 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11258:
-

 Summary: Repair scheduling - Resource locking API
 Key: CASSANDRA-11258
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11258
 Project: Cassandra
  Issue Type: Task
Reporter: Marcus Olsson
Assignee: Marcus Olsson
Priority: Minor


Create a resource locking API & implementation that is able to lock a resource 
in a specified data center. It should handle priorities to avoid node 
starvation.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-26 Thread Marcus Olsson (JIRA)

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

Marcus Olsson edited comment on CASSANDRA-11215 at 2/26/16 6:18 PM:


I ran the test four times with the patch on trunk and it passed every time, 
while failing on a non-patched trunk. So it seems to be working as it should 
now. :)


was (Author: molsson):
I ran the test four times with the patch on trunk and it passed every time, 
while failing twice on a non-patched trunk. So it seems to be working as it 
should now. :)

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
> Fix For: 2.2.x, 3.0.x, 3.x
>
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-26 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

I ran the test four times with the patch on trunk and it passed every time, 
while failing twice on a non-patched trunk. So it seems to be working as it 
should now. :)

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
> Fix For: 2.2.x, 3.0.x, 3.x
>
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-26 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

@[~krummas]
There seems to be one 
[dtest|https://cassci.datastax.com/view/Dev/view/krummas/job/krummas-emolsson-11215-dtest/lastCompletedBuild/testReport/repair_tests.incremental_repair_test/TestIncRepair/sstable_repairedset_test/]
 failing but it's also failing on 
[2.2|https://cassci.datastax.com/job/cassandra-2.2_dtest/lastCompletedBuild/testReport/repair_tests.incremental_repair_test/TestIncRepair/sstable_repairedset_test/]
 so it doesn't seem related. I'll try it with your patch locally against trunk 
to see if I can reproduce my earlier problems, thanks! :)

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
> Fix For: 2.2.x, 3.0.x, 3.x
>
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-25 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

I've created a dtest for it 
[here|https://github.com/emolsson/cassandra-dtest/commit/adbf51a8b17f6923b6fbd3d7a511399afd695738],
 which fails on 2.2.x and trunk with LEAK DETECTED, while working on 2.2 with 
the provided patch.

I've found one problem though, it doesn't seem like trunk always gets the 
"Cannot start multiple repairs" error message. It could be due to how it groups 
the ranges together and only has a single repair session while 2.2 has one 
repair session per range. On trunk the "LEAK DETECTED" is only logged once and 
on 2.2 it's logged multiple times, so it could be that the likelihood of 
getting the error is reduced on trunk since it only has one repair session? 
Should we handle this by flaky or should we increase the size of the data and 
make the validation run longer?

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-02-24 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

For the basic implementation I think the tasks could be broken down as:
- Resource locking API & implementation
-- Maintenance scheduling API & basic repair scheduling
--- Rejection policy interface & default implementations
--- Configuration support
 Table configuration
 Global configuration (for pausing repairs in the basic implementation)
 Node configuration
--- Aborting/interrupting repairs (Requires CASSANDRA-3486,CASSANDRA-11190)
 Polling and monitoring module
 Failure handling and retry 

So that we start with the resource locking and then move on to the maintenance 
scheduling API. And after that I think most tasks could be discussed in 
parallel. Also I removed the task for management commands since I think it 
would be easier to add them while implementing the features. WDYT?

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-02-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

bq. Sounds good! We could ask the user to pause, but I think doing that 
automatically via "system interrupts" is better. It just ocurred to me that 
both "the pause" or "system interrupts" will prevent new repairs from starting, 
but what about already running repairs? We will probably want to interrupt 
already running repairs as well in some situations. For this reason 
CASSANDRA-3486 is also relevant for this ticket (adding it as a dependency of 
this ticket).
+1

bq. Then I think we should either have timeout, or add an ability to 
cancel/interrupt a running scheduled repair in the initial version, to avoid 
hanging repairs to render the automatic repair scheduling useless.
I think the timeout would be good enough in the initial version. I guess the 
interruption of repairs would be handled by CASSANDRA-3486? Perhaps it would be 
possible to extend that feature later to be able to cancel a scheduled repair? 
Here I'm thinking that the interruption is stopping the running repair and 
allowing the scheduled job to retry it immediately, while cancelling it would 
prevent the scheduled job from retrying it immediately.

bq. WDYT? Feel free to update or break-up into smaller or larger subtasks, and 
then create the actual subtasks to start work on them.
Sounds good, I'll have a closer look on the subtasks tomorrow! I guess we will 
have sort of a dependency tree for some of the tasks.

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

After looking around a bit in the dtests I think that self.ignore_log_patterns 
could handle that, although our expected error is causing more errors than 
"Cannot start multiple repairs". There are errors logged from nodetool and the 
repair sessions as well. But for this test I guess the important part is that 
there are no "LEAK DETECTED" error messages, right? Assuming that is the case, 
could we simply ignore the other repair errors?

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

I could try to do it. I guess it more or less would be to go through the 
reproduction steps and grep in the logs for reference leak, right? I'll put it 
in the repair_test.py dtest. :)

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Comment Edited] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson edited comment on CASSANDRA-11215 at 2/23/16 2:50 PM:


Patch for 2.2 is available 
[here|https://github.com/emolsson/cassandra/commit/8b1b4317c43db648d54ce2e339a525e3fb324cab].

I think there will be some merge conflicts in 3.0/3.x should I apply separate 
patch sets for them directly or wait for the review of the 2.2 version first?

Edit: To make it clear what the fix is, the sstableCandidates are put in a 
try-with-resources to make sure that they are released. I felt that this 
clarification might be needed since the patch also moves the SSTable 
referencing code to a separate method to reduce complexity in the 
doValidationCompaction-method.


was (Author: molsson):
Patch for 2.2 is available 
[here|https://github.com/emolsson/cassandra/commit/8b1b4317c43db648d54ce2e339a525e3fb324cab].

I think there will be some merge conflicts in 3.0/3.x should I apply separate 
patch sets for them directly or wait for the review of the 2.2 version first?

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

Patch for 2.2 is available 
[here|https://github.com/emolsson/cassandra/commit/8b1b4317c43db648d54ce2e339a525e3fb324cab].

I think there will be some merge conflicts in 3.0/3.x should I apply separate 
patch sets for them directly or wait for the review of the 2.2 version first?

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-11215:
---

It seems that this issue is caused by not accepting parallel repairs on the 
same sstables anymore, where it throws a RuntimeException if that happens and 
fails to release the previously acquired references.

I'm currently working on a patch for this.

> Reference leak with parallel repairs on the same table
> --
>
> Key: CASSANDRA-11215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>
> When starting multiple repairs on the same table Cassandra starts to log 
> about reference leak as:
> {noformat}
> ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
> DETECTED: a reference 
> (org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
> org.apache.cassandra.io.sstable.format.SSTableReader
> $InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
>  was not released before the reference was garbage collected
> {noformat}
> Reproducible with:
> {noformat}
> ccm create repairtest -v 2.2.5 -n 3
> ccm start
> ccm stress write n=100 -schema 
> replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
> # And then perform two repairs concurrently with:
> ccm node1 nodetool repair testrepair
> {noformat}
> I know that starting multiple repairs in parallel on the same table isn't 
> very wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (CASSANDRA-11215) Reference leak with parallel repairs on the same table

2016-02-23 Thread Marcus Olsson (JIRA)
Marcus Olsson created CASSANDRA-11215:
-

 Summary: Reference leak with parallel repairs on the same table
 Key: CASSANDRA-11215
 URL: https://issues.apache.org/jira/browse/CASSANDRA-11215
 Project: Cassandra
  Issue Type: Bug
Reporter: Marcus Olsson
Assignee: Marcus Olsson


When starting multiple repairs on the same table Cassandra starts to log about 
reference leak as:
{noformat}
ERROR [Reference-Reaper:1] 2016-02-23 15:02:05,516 Ref.java:187 - LEAK 
DETECTED: a reference 
(org.apache.cassandra.utils.concurrent.Ref$State@5213f926) to class 
org.apache.cassandra.io.sstable.format.SSTableReader
$InstanceTidier@605893242:.../testrepair/standard1-dcf311a0da3411e5a5c0c1a39c091431/la-30-big
 was not released before the reference was garbage collected
{noformat}

Reproducible with:
{noformat}
ccm create repairtest -v 2.2.5 -n 3
ccm start
ccm stress write n=100 -schema 
replication(strategy=SimpleStrategy,factor=3) keyspace=testrepair
# And then perform two repairs concurrently with:
ccm node1 nodetool repair testrepair
{noformat}

I know that starting multiple repairs in parallel on the same table isn't very 
wise, but this shouldn't result in reference leaks.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-02-23 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

bq. We could probably replace the single resource lock 
('RepairResource\-{dc}\-{i}') with global ('Global\-{dc}\-{i}') or mutually 
exclusive resources ('CleanupAndRepairResource-{dc}-{i}') later if necessary. 
We'll probably only need some special care during upgrades when we introduce 
these new locks, but other than that I don't see any problem that could arise 
with renaming the resources later if necessary. Do you see any issue with this 
approach?
No that should probably work, so we can have it as 
'RepairResource-\{dc\}-\{i\}' for now. For the upgrades we could add a release 
note that says something like "pause/stop all scheduled repairs while upgrading 
from x.y to x.z". But in that case the pause/stop feature should be implemented 
as early as possible to avoid having an upgrade scenario that requires the user 
to upgrade to the version that introduces the pause feature before upgrading to 
the latest. Another way would be to have the "system interrupts" feature in 
place early, so that the repairs would be paused during an upgrade.

bq. Created CASSANDRA-11190 for failing repairs fast and linked as a 
requirement of this ticket.
Great!

bq. No unless there is a bug. Repair messages are undroppable, and the nodes 
report the coordinator on failure.
bq. We could probably handle explicit failures in CASSANDRA-11190 making sure 
all nodes are properly informed and abort their operations in case of failures 
in any of the nodes. The timeout in this context could be helpful in case of 
hangs in streaming or validation. But I suppose that as the protocol becomes 
more mature/correct and with fail fast in place these hanging situations will 
become more rare so I'm not sure timeouts would be required if we assume there 
are no hangs. I guess we can leave them out of the initial version for 
simplicity and add them later if necessary.
I think the timeout might be good to have to prevent a hang from stopping the 
entire repair process. But I think it would only work if the repair would only 
hang occasionally, otherwise the same repair would be retried until it is 
marked as a "fail". Another option is to have a "slow repair"-detector that 
would log a warning if a repair session is taking too long time, to avoid 
aborting it if it's actually repairing and leaving it up to the user to handle 
it. Either way I'd say it's out of the scope of the initial version.

---

We might also want to be able to detect if it would be impossible to repair the 
whole cluster within gc grace and report it to the user. This could happen for 
multiple reasons like too many tables, too many nodes, too few parallel repairs 
or simply overload. I guess it would be hard to make accurate predictions with 
all of these variables so it might be good enough to check through the history 
of the repairs, do an estimation of the time and compare it to gc grace? I 
think this is something out of scope for the first version, but I thought I'd 
just mention it here to remember it.

Should we maybe compile a list of  "features that should be in the initial 
version" and also a "improvements" list for future work to make the scope clear?

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-02-16 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

bq. Do we intend to reuse the lock table for other maintenance tasks as well? 
If so, we must add a generic "holder" column to the lock table so we can reuse 
to identify resources other than the parent repair session in the future. We 
could also add an "attributes" map in the lock table to store additional 
attributes such as status, or have a separate table to maintain status to keep 
the lock table simple.

I think it could be reused, so it's probably better to do it generic from the 
start. I think that as long as we don't put too much data in the attributes 
map, it could be stored in the lock table. Another thing is that it's tightly 
bound to the lock itself, since we will use it to clean up repairs without a 
lock, which means keeping it in a single table is probably the easiest solution.

Another thing we should probably consider is whether or not multiple types of 
maintenance work should run simultaneously. If we need to add this constraint, 
should they use the same lock resources?

bq. Ideally all repairs would go through this interface, but this would 
probably add complexity at this stage. So we should probably just add a 
"lockResource" attribute to each repair session object, and each node would go 
through all repairs currently running checking if it still holds the lock in 
case the "lockResource" field is set.

Sounds good, let's start with the lockResource field in the repair session and 
move to scheduled repairs all together later on (maybe optionally scheduled via 
JMX at first?).

{quote}
It would probably be safe to abort ongoing validation and stream background 
tasks and cleanup repair state on all involved nodes before starting a new 
repair session in the same ranges. This doesn't seem to be done currently. As 
far as I understood, if there are nodes A, B, C running repair, A is the 
coordinator. If validation or streaming fails on node B, the coordinator (A) is 
notified and fails the repair session, but node C will remain doing validation 
and/or streaming, what could cause problems (or increased load) if we start 
another repair session on the same range. 

We will probably need to extend the repair protocol to perform this 
cleanup/abort step on failure. We already have a legacy cleanup message that 
doesn't seem to be used in the current protocol that we could maybe reuse to 
cleanup repair state after a failure. This repair abortion will probably have 
intersection with CASSANDRA-3486. In any case, this is a separate (but related) 
issue and we should address it in an independent ticket, and make this ticket 
dependent on that.
{quote}

Right now it seems that the cleanup message is only used to remove the parent 
repair session from the ActiveRepairService's map. I guess that if we should 
use it we would have to rewrite it to stop validation and streaming as well. 
But as you said, it should be done in a separate ticket.

bq. Another unrelated option that we should probably include in the future is a 
timeout, and abort repair sessions running longer than that.

Agreed. Do we have any time out scenarios that we could foresee before they 
occur? Would it be possible for a node to "drop" a validation/streaming without 
notifying the repair coordinator? If we could detect that, it would be good to 
abort the repair as early as possible, assuming that the timeout would be set 
rather high.

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-02-15 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

{quote}
All data centers involved in a repair must be available for a repair to 
start/succeed, so if we make the lock resource dc-aware and try to create the 
lock by contacting a node in each involved data center with LOCAL_SERIAL 
consistency that should be sufficient to ensure correctness without the need 
for a global lock. This will also play along well with both dc_parallelism 
global option and with the --local or --dcs table repair options.
{quote}

{quote}
The second alternative is probably the most desireable. Actually dc_parallelism 
by itself might cause problems, since we can have a situation where all repairs 
run in a single node or range, overloading those nodes. If we are to support 
concurrent repairs in the first pass, I think we need both dc_parallelism and 
node_parallelism options together.
{quote}

{quote}
This is becoming a bit complex and there probably are some edge cases and/or 
starvation scenarios so we should think carefully about before jumping into 
implementation. What do you think about this approach? Should we stick to a 
simpler non-parallel version in the first pass or think this through and 
already support parallelism in the first version?
{quote}

I like the approach with using local serial for each dc and having specialized 
keys. I think we could include the dc parallelism lock with 
"RepairResource-\{dc}-\{i}" but only allow one repair per data center by 
hardcoding "i" to 1 in the first pass. This should make the upgrades easier 
when we do allow parallel repairs. I like the node locks approach as well, but 
as you say there are probably some edge cases so we could wait with adding them 
until we allow parallel repairs and I don't think it would break the upgrades 
by introducing them later.

{quote}
We should also think better about possible failure scenarios and network 
partitions. What happens if the node cannot renew locks in a remote DC due to a 
temporary network partition but the repair is still running ? We should 
probably cancel a repair if not able to renew the lock and also have some kind 
of garbage collector to kill ongoing repair sessions without associated locks 
to protect from disrespecting the configured dc_parallelism and 
node_paralellism.
{quote}
I agree and we could probably store the parent repair session id in an extra 
column of the lock table and have a thread wake up periodically to see if there 
are repair sessions without locks. But then we must somehow be able to 
differentiate user-defined and automatically scheduled repair sessions. It 
could be done by having all repairs go through this scheduling interface, which 
also would reduce user mistakes with multiple repairs in parallel. Another 
alternative is to have a custom flag in the parent repair that makes the 
garbage collector ignore it if it's user-defined. I think that the garbage 
collector/cancel repairs when unable to lock feature is something that should 
be included in the first pass.

The most basic failure scenarios should be covered by retrying a repair if it 
fails and log a warning/error based on how many times it failed. Could the 
retry behaviour cause some unexpected consequences?

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-02-05 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

[~yukim] [~pauloricardomg] Thanks for the comments, great questions/suggestions!

Regarding your questions about the locking:
{quote}
* What would "lock resource" be like for repair scheduling? I think the value 
controls number of repair jobs running at given time in the whole cluster, and 
we don't want to run as many repair jobs at once.
* I second Yuki Morishita's first question above, in that we need to better 
specify how is cluster-wide repair parallelism handled: is it fixed or 
configurable? can a node run repair for multiple ranges in parallel? Perhaps we 
should have a node_repair_paralellism (default 1) and dc_repair_parallelism 
(default 1) global config and reject starting repairs above those thresholds.
{quote}
The thought with the lock resource was that it could be something simple, like 
a table defined as:
{noformat}
CREATE TABLE lock (
resource text PRIMARY KEY
)
{noformat}
And then the different nodes would try to get the lock using LWT with TTL:
{noformat}
INSERT INTO lock (resource) VALUES ('RepairResource') IF NOT EXISTS USING TTL 
30;
{noformat}
After that the node would have to continue to update the locked resource while 
running the repair to prevent that someone else gets the locked resource. The 
value "RepairResource" could just as easily be defined as "RepairResource-N", 
so that it would be possible to allow repairs to run in parallel.

A problem with this table is that if we have a setup with two data centers and 
three replicas in each data center, then we have a total of six replicas and 
QUORUM would require four replicas to succeed. This would require that both 
data centers are available to be able to run repair. Since some of the 
keyspaces might not be replicated across both data centers we would still have 
to be able to run repair even if one of the data centers is unavailable. This 
also applies if we should "force" local dc repairs if a data center has been 
unavailable too long. There are two options as I see it on how to solve this:
* Get the lock with local_serial during these scenarios.
* Have a separate lock table for each data center *and* a global one.

I guess the easiest solution would be to use local_serial, but I'm not sure if 
it might cause some unexpected behavior. If we would go for the other option 
with separate tables it would probably increase the overall complexity, but it 
would make it easier to restrict the number of parallel repairs in a single 
data center.

Just a questions regarding your suggestion with the node_repair_parallelism. 
Should it be used to specify the number of repairs a node can initiate or how 
many repairs the node can be an active part of in parallel? I guess the second 
alternative would be harder to implement, but it is probably what one would 
expect.

---

{quote}
* It seems the scheduling only makes sense for repairing primary range of the 
node ('nodetool -pr') since we end up repairing all nodes eventually. Are you 
considering other options like subrange ('nodetool -st -et') repair?
* For subrange repair, we could maybe have something similar to reaper's 
segmentCount option, but since this would add more complexity we could leave 
for a separate ticket.
{quote}

It should be possible to extend the repair scheduler with subrange repairs, 
either by having it as an option per table or by having a separate scheduler 
for it. The separate scheduler would just be another plugin that could replace 
the default repair scheduler. If we go for a table configuration it could be 
that the user either specifies pr or the number of segments to divide the token 
range in, something like:
{noformat}
repair_options = {..., token_division='pr'}; // Use primary range repair
or
repair_options = {..., token_division='2048'}; // Divide the token range in 
2048 slices
{noformat}
If we would have a separate scheduler it could just be a configuration for it. 
Personally I would prefer to have it all in a single scheduler and I agree that 
it should probably be a separate ticket to keep the complexity of the base 
scheduler to a minimum. But I think this is a feature that will be very much 
needed both with non-vnode token assignment and also with the possibility to 
reduce the number of vnodes as of CASSANDRA-7032.

---

{quote}
* While pausing repair is a nice future for user-based interruptions, we could 
probably embed system known interruptions (such as when a bootstrap or upgrade 
is going on) in the default rejection logic.
{quote}

Agreed, are there any other scenarios that we might have to take into account?

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: 

[jira] [Updated] (CASSANDRA-10070) Automatic repair scheduling

2016-02-01 Thread Marcus Olsson (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-10070?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Marcus Olsson updated CASSANDRA-10070:
--
Attachment: Distributed Repair Scheduling.doc

This is a draft of the proposal, it would be great to get some comments on it! 
:)

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
> Attachments: Distributed Repair Scheduling.doc
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2016-01-20 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

I completely agree, I should create a document describing these things.

I've also thought about making a high level document for the whole proposal, so 
as to see if everyone agrees that this is the way to go about the distributed 
scheduling. Then we can take it from there and revise the proposal and 
hopefully later on break the JIRA into several tasks to make it easier to 
review and develop this feature.

I think this document should contain:
* High level description of proposal (flow charts, etc.)
* Problems that could occur and possible solutions

Any thoughts or ideas on this?

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2015-12-10 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

{quote}
While it may intuitively seem like you want to kick-off a repair as soon as a 
node comes back online, it can be very dangerous in a production environment.

Starting the most resource intensive process on a node that is already 
problematic, in a cluster that is already having issues can exacerbate the 
issue and lead to a longer outage, or degradation, than anticipated. 
{quote}
True, it should probably be a feature enabled by the user and maybe with a 
configurable delay before it actually performs the repair?

{quote}
Network reliability is also another aspect of this. Lets say you have 3 nodes, 
RF=3 and there is a partition dividing node A and node B. All nodes are still 
actually, up, but in this case node A will start a repair on B and B will start 
a repair on A. Now 2/3 of your cluster is un-needly repairing which can cause 
serious performance problems, especially when running a loaded cluster.
{quote}
The repairs are still executed with respect to the distributed locking, so 
there would only be one node running repair at a time. But they would send the 
job information to each other in parallel.

{quote}
Also:
Other times you might not want a repair automatically started:
* The cluster is in the middle of a rolling upgrade where streaming is broken 
between versions.
* Heavily loaded clusters during normal operation (some users schedule repairs 
at night to not affect performance during normal hours of operation)
* Clusters where the read-consistency is high enough to account for the hints 
beyond the window allowing the user to schedule the repair for a time that 
makes sense for their cluster and use-case.
{quote}
* This is something that the repair scheduler should be handling either way, to 
avoiding repairing if the cluster is unable to perform it. (version 
incompatibility, nodes are down, etc.)
* There is a plug-in point for schedule policies that can be used to decide if 
repairs should run, so it would be possible to prevent repairs due to some 
condition(s). The conditions could be based on what the user wants, be it 
maintenance windows or resource usage. It would also be possible to prevent 
normal scheduled repairs during some hours, but allow manually scheduled 
repairs at all times.
* This would be possible by making this feature optional.

---

{quote}
I don't know much about Cassandra internals, so one of the regular devs would 
know better, buy my thought would be during a restart, somewhere it figures out 
that it needs to replay part of the commit log to rebuild memtables that hadn't 
been flushed to disk. The timestamp of the last thing in the commit log might 
be a good estimate of when the node went down, and you could compare that to 
the current time to figure out how long the node was down.

I wouldn't worry about the second case since it would be hard to get that right.
{quote}
Looking at the commitlog might be a good enough approach. I'll look in to that.

---

Overall I'd say that if this feature(exceeding hint window repairs) should 
exist, it should probably be something that is enabled per table, but disabled 
by default.

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2015-12-07 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

I agree that it would probably be safer for node A to check how long it has 
been down itself, but I'm not sure how that can be done reliably. But also if 
node A & B couldn't communicate for a time period longer than the hint window 
they will not have hints. So in that case they should do a repair even if both 
were up the whole time.

Note that I'm not against having the check on the node that was down, it's just 
that I think that both the case that a node was down and that two nodes was 
unable to communicate should require a repair. If the second case is not 
required do you have any suggestions on how the self-check could be implemented?

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (CASSANDRA-10070) Automatic repair scheduling

2015-12-07 Thread Marcus Olsson (JIRA)

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

Marcus Olsson commented on CASSANDRA-10070:
---

[~zemeyer] I've added the possibility to schedule a job remotely, so that one 
node can tell another node to run a certain job. Right now it's used for when a 
node discovers that another node has been down longer than the possible hint 
window, and then tells that node to repair it's ranges ASAP. The remote 
scheduling is using the distributed locking mechanism to avoid that multiple 
nodes try to tell the same node to run the repair at the same time.

So a simple flow could be:
Node A goes down at 12:00
Node B recognizes it and saves "Node A DOWN @ 12:00" locally
Node A comes back up at 16:00
Node B sees Node A as online again at 16:00 and sees that Node A has been down 
since 12:00, 4 hours.
Node B sends a repair job to Node A for each table that has a hint window that 
is 4 hours or less.
Node A runs all repairs

---

I'll continue to work on the feature of pausing all repairs and also the 
prevention mechanism. I've done some work for the prevention mechanism for jobs 
in that it checks the job history for repairs and only returns that it *can* 
run a repair if any range hasn't been repaired within the hint window (it's 
still based on the interval though, so the repair shouldn't run more than once 
per interval in the normal case).

To the prevention mechanism I should probably add a way for it to avoid doing 
multiple repairs for a single node at the same time. After that I'll add the 
possibility to run parallel repair tasks over the cluster.

---

The git branch is [here|https://github.com/emolsson/cassandra/commits/10070].

> Automatic repair scheduling
> ---
>
> Key: CASSANDRA-10070
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10070
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Marcus Olsson
>Assignee: Marcus Olsson
>Priority: Minor
> Fix For: 3.x
>
>
> Scheduling and running repairs in a Cassandra cluster is most often a 
> required task, but this can both be hard for new users and it also requires a 
> bit of manual configuration. There are good tools out there that can be used 
> to simplify things, but wouldn't this be a good feature to have inside of 
> Cassandra? To automatically schedule and run repairs, so that when you start 
> up your cluster it basically maintains itself in terms of normal 
> anti-entropy, with the possibility for manual configuration.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


  1   2   >