[jira] [Resolved] (CASSANDRA-9440) Bootstrap fails without any hint of prior stream failure

2015-05-29 Thread Omid Aladini (JIRA)

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

Omid Aladini resolved CASSANDRA-9440.
-
Resolution: Not A Problem

> Bootstrap fails without any hint of prior stream failure
> 
>
> Key: CASSANDRA-9440
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9440
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Cassandra 2.0.14
> 2 DCs on EC2
>Reporter: Omid Aladini
>
> I'm working on a cluster running Cassandra 2.0.14 and the bootstrap fails but 
> there is no prior hint of failed streams:
> {code}
>  WARN [StreamReceiveTask:177] 2015-05-20 04:20:55,251 StreamResultFuture.java 
> (line 215) [Stream #0b42c640-fe03-11e4-8a6f-dd5dc9b30af4] Stream failed
> ERROR [main] 2015-05-20 04:20:55,252 CassandraDaemon.java (line 584) 
> Exception encountered during startup
> java.lang.RuntimeException: Error during boostrap: Stream failed
> at 
> org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86)
> at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1005)
> at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:808)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:621)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:510)
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:437)
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:567)
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:656)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
> at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
> at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at 
> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
> at 
> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:216)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:377)
> at 
> org.apache.cassandra.streaming.StreamSession.maybeCompleted(StreamSession.java:662)
> at 
> org.apache.cassandra.streaming.StreamSession.taskCompleted(StreamSession.java:613)
> at 
> org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:143)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
>  INFO [StorageServiceShutdownHook] 2015-05-20 04:20:55,286 Gossiper.java 
> (line 1330) Announcing shutdown
> {code}
> There are no WARN or ERROR prior to this in the log files of the 
> bootstrapping node or other nodes in the cluster. Only relevant log lines are 
> "Session with 11.22.33.44/11.22.33.44 is complete"
> Is it possible that individual stream sessions fail silently? 



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


[jira] [Commented] (CASSANDRA-9440) Bootstrap fails without any hint of prior stream failure

2015-05-29 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-9440:
-

Enabling TCP keep alive and having streaming_socket_timeout_in_ms = 0 fixed the 
bootstrap failures.

> Bootstrap fails without any hint of prior stream failure
> 
>
> Key: CASSANDRA-9440
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9440
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Cassandra 2.0.14
> 2 DCs on EC2
>Reporter: Omid Aladini
>
> I'm working on a cluster running Cassandra 2.0.14 and the bootstrap fails but 
> there is no prior hint of failed streams:
> {code}
>  WARN [StreamReceiveTask:177] 2015-05-20 04:20:55,251 StreamResultFuture.java 
> (line 215) [Stream #0b42c640-fe03-11e4-8a6f-dd5dc9b30af4] Stream failed
> ERROR [main] 2015-05-20 04:20:55,252 CassandraDaemon.java (line 584) 
> Exception encountered during startup
> java.lang.RuntimeException: Error during boostrap: Stream failed
> at 
> org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86)
> at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1005)
> at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:808)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:621)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:510)
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:437)
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:567)
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:656)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
> at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
> at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at 
> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
> at 
> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:216)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:377)
> at 
> org.apache.cassandra.streaming.StreamSession.maybeCompleted(StreamSession.java:662)
> at 
> org.apache.cassandra.streaming.StreamSession.taskCompleted(StreamSession.java:613)
> at 
> org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:143)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
>  INFO [StorageServiceShutdownHook] 2015-05-20 04:20:55,286 Gossiper.java 
> (line 1330) Announcing shutdown
> {code}
> There are no WARN or ERROR prior to this in the log files of the 
> bootstrapping node or other nodes in the cluster. Only relevant log lines are 
> "Session with 11.22.33.44/11.22.33.44 is complete"
> Is it possible that individual stream sessions fail silently? 



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


[jira] [Commented] (CASSANDRA-9458) Race condition causing StreamSession to get stuck in WAIT_COMPLETE

2015-05-29 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-9458:
-

[~yukim] This was indeed due to secondary index build. thanks.

> Race condition causing StreamSession to get stuck in WAIT_COMPLETE
> --
>
> Key: CASSANDRA-9458
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9458
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Omid Aladini
>Assignee: Omid Aladini
>Priority: Critical
> Fix For: 2.1.x, 2.0.x
>
> Attachments: 9458-v1.txt
>
>
> I think there is a race condition in StreamSession where one side of the 
> stream could get stuck in WAIT_COMPLETE although both have sent COMPLETE 
> messages. Consider a scenario that node B is being bootstrapped and it only 
> receives files during the session:
> 1- During a stream session A sends some files to B and B sends no files to A.
> 2- Once B completes the last task (receiving), StreamSession::maybeComplete 
> is invoked.
> 3- While B is sending the COMPLETE message via StreamSession::maybeComplete, 
> it also receives the COMPLETE message from A and therefore 
> StreamSession::complete() is invoked.
> 4- Therefore both maybeComplete() and complete() functions have branched into 
> the state != State.WAIT_COMPLETE case and both set the state to WAIT_COMPLETE.
> 5- Now B is waiting to receive COMPLETE although it's already received it and 
> nothing triggers checking the state again, until it times out after 
> streaming_socket_timeout_in_ms.
> In the log below:
> https://gist.github.com/omidaladini/003de259958ad8dfb07e
> although the node has received COMPLETE, "SocketTimeoutException" is thrown 
> after streaming_socket_timeout_in_ms (30 minutes here).



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


[jira] [Commented] (CASSANDRA-9458) Race condition causing StreamSession to get stuck in WAIT_COMPLETE

2015-05-27 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-9458:
-

Thanks for checking the log and the patch. You're right as all the relevant 
calls to maybeCompleted are synchronised on the object.

{quote}
Do you have secondary indexes? Right now, streaming is considered completed 
after secondary indexes are built in that finalise phase(CASSANDRA-9308).
{quote}

There are secondary indexes and I see a bunch of "submitting index build of" in 
the full log so I guess it's possible that the index build is just taking 
longer than the timeout. I'll disable the timeout (and enable TCP keep-alive 
via CASSANDRA-9455) to see if it gets resolved.

> Race condition causing StreamSession to get stuck in WAIT_COMPLETE
> --
>
> Key: CASSANDRA-9458
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9458
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Omid Aladini
>Assignee: Omid Aladini
>Priority: Critical
> Fix For: 2.1.x, 2.0.x
>
> Attachments: 9458-v1.txt
>
>
> I think there is a race condition in StreamSession where one side of the 
> stream could get stuck in WAIT_COMPLETE although both have sent COMPLETE 
> messages. Consider a scenario that node B is being bootstrapped and it only 
> receives files during the session:
> 1- During a stream session A sends some files to B and B sends no files to A.
> 2- Once B completes the last task (receiving), StreamSession::maybeComplete 
> is invoked.
> 3- While B is sending the COMPLETE message via StreamSession::maybeComplete, 
> it also receives the COMPLETE message from A and therefore 
> StreamSession::complete() is invoked.
> 4- Therefore both maybeComplete() and complete() functions have branched into 
> the state != State.WAIT_COMPLETE case and both set the state to WAIT_COMPLETE.
> 5- Now B is waiting to receive COMPLETE although it's already received it and 
> nothing triggers checking the state again, until it times out after 
> streaming_socket_timeout_in_ms.
> In the log below:
> https://gist.github.com/omidaladini/003de259958ad8dfb07e
> although the node has received COMPLETE, "SocketTimeoutException" is thrown 
> after streaming_socket_timeout_in_ms (30 minutes here).



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


[jira] [Updated] (CASSANDRA-9458) Race condition causing StreamSession to get stuck in WAIT_COMPLETE

2015-05-22 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-9458:

Fix Version/s: 2.0.x
   2.1.x

> Race condition causing StreamSession to get stuck in WAIT_COMPLETE
> --
>
> Key: CASSANDRA-9458
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9458
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Omid Aladini
>Priority: Critical
> Fix For: 2.1.x, 2.0.x
>
> Attachments: 9458-v1.txt
>
>
> I think there is a race condition in StreamSession where one side of the 
> stream could get stuck in WAIT_COMPLETE although both have sent COMPLETE 
> messages. Consider a scenario that node B is being bootstrapped and it only 
> receives files during the session:
> 1- During a stream session A sends some files to B and B sends no files to A.
> 2- Once B completes the last task (receiving), StreamSession::maybeComplete 
> is invoked.
> 3- While B is sending the COMPLETE message via StreamSession::maybeComplete, 
> it also receives the COMPLETE message from A and therefore 
> StreamSession::complete() is invoked.
> 4- Therefore both maybeComplete() and complete() functions have branched into 
> the state != State.WAIT_COMPLETE case and both set the state to WAIT_COMPLETE.
> 5- Now B is waiting to receive COMPLETE although it's already received it and 
> nothing triggers checking the state again, until it times out after 
> streaming_socket_timeout_in_ms.
> In the log below:
> https://gist.github.com/omidaladini/003de259958ad8dfb07e
> although the node has received COMPLETE, "SocketTimeoutException" is thrown 
> after streaming_socket_timeout_in_ms (30 minutes here).



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


[jira] [Updated] (CASSANDRA-9458) Race condition causing StreamSession to get stuck in WAIT_COMPLETE

2015-05-22 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-9458:

Attachment: 9458-v1.txt

This patch may address the race condition (in case I've actually understood the 
problem correctly! :)

> Race condition causing StreamSession to get stuck in WAIT_COMPLETE
> --
>
> Key: CASSANDRA-9458
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9458
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Omid Aladini
>Priority: Critical
> Attachments: 9458-v1.txt
>
>
> I think there is a race condition in StreamSession where one side of the 
> stream could get stuck in WAIT_COMPLETE although both have sent COMPLETE 
> messages. Consider a scenario that node B is being bootstrapped and it only 
> receives files during the session:
> 1- During a stream session A sends some files to B and B sends no files to A.
> 2- Once B completes the last task (receiving), StreamSession::maybeComplete 
> is invoked.
> 3- While B is sending the COMPLETE message via StreamSession::maybeComplete, 
> it also receives the COMPLETE message from A and therefore 
> StreamSession::complete() is invoked.
> 4- Therefore both maybeComplete() and complete() functions have branched into 
> the state != State.WAIT_COMPLETE case and both set the state to WAIT_COMPLETE.
> 5- Now B is waiting to receive COMPLETE although it's already received it and 
> nothing triggers checking the state again, until it times out after 
> streaming_socket_timeout_in_ms.
> In the log below:
> https://gist.github.com/omidaladini/003de259958ad8dfb07e
> although the node has received COMPLETE, "SocketTimeoutException" is thrown 
> after streaming_socket_timeout_in_ms (30 minutes here).



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


[jira] [Updated] (CASSANDRA-9458) Race condition causing StreamSession to get stuck in WAIT_COMPLETE

2015-05-22 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-9458:

Fix Version/s: (was: 2.0.16)

> Race condition causing StreamSession to get stuck in WAIT_COMPLETE
> --
>
> Key: CASSANDRA-9458
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9458
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Omid Aladini
>Priority: Critical
>
> I think there is a race condition in StreamSession where one side of the 
> stream could get stuck in WAIT_COMPLETE although both have sent COMPLETE 
> messages. Consider a scenario that node B is being bootstrapped and it only 
> receives files during the session:
> 1- During a stream session A sends some files to B and B sends no files to A.
> 2- Once B completes the last task (receiving), StreamSession::maybeComplete 
> is invoked.
> 3- While B is sending the COMPLETE message via StreamSession::maybeComplete, 
> it also receives the COMPLETE message from A and therefore 
> StreamSession::complete() is invoked.
> 4- Therefore both maybeComplete() and complete() functions have branched into 
> the state != State.WAIT_COMPLETE case and both set the state to WAIT_COMPLETE.
> 5- Now B is waiting to receive COMPLETE although it's already received it and 
> nothing triggers checking the state again, until it times out after 
> streaming_socket_timeout_in_ms.
> In the log below:
> https://gist.github.com/omidaladini/003de259958ad8dfb07e
> although the node has received COMPLETE, "SocketTimeoutException" is thrown 
> after streaming_socket_timeout_in_ms (30 minutes here).



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


[jira] [Created] (CASSANDRA-9458) Race condition causing StreamSession to get stuck in WAIT_COMPLETE

2015-05-22 Thread Omid Aladini (JIRA)
Omid Aladini created CASSANDRA-9458:
---

 Summary: Race condition causing StreamSession to get stuck in 
WAIT_COMPLETE
 Key: CASSANDRA-9458
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9458
 Project: Cassandra
  Issue Type: Bug
Reporter: Omid Aladini
Priority: Critical
 Fix For: 2.0.16


I think there is a race condition in StreamSession where one side of the stream 
could get stuck in WAIT_COMPLETE although both have sent COMPLETE messages. 
Consider a scenario that node B is being bootstrapped and it only receives 
files during the session:

1- During a stream session A sends some files to B and B sends no files to A.
2- Once B completes the last task (receiving), StreamSession::maybeComplete is 
invoked.
3- While B is sending the COMPLETE message via StreamSession::maybeComplete, it 
also receives the COMPLETE message from A and therefore 
StreamSession::complete() is invoked.
4- Therefore both maybeComplete() and complete() functions have branched into 
the state != State.WAIT_COMPLETE case and both set the state to WAIT_COMPLETE.
5- Now B is waiting to receive COMPLETE although it's already received it and 
nothing triggers checking the state again, until it times out after 
streaming_socket_timeout_in_ms.

In the log below:

https://gist.github.com/omidaladini/003de259958ad8dfb07e

although the node has received COMPLETE, "SocketTimeoutException" is thrown 
after streaming_socket_timeout_in_ms (30 minutes here).



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


[jira] [Commented] (CASSANDRA-9440) Bootstrap fails without any hint of prior stream failure

2015-05-22 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-9440:
-

[~rkuris] Possibly related, thank you.

> Bootstrap fails without any hint of prior stream failure
> 
>
> Key: CASSANDRA-9440
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9440
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Cassandra 2.0.14
> 2 DCs on EC2
>Reporter: Omid Aladini
>
> I'm working on a cluster running Cassandra 2.0.14 and the bootstrap fails but 
> there is no prior hint of failed streams:
> {code}
>  WARN [StreamReceiveTask:177] 2015-05-20 04:20:55,251 StreamResultFuture.java 
> (line 215) [Stream #0b42c640-fe03-11e4-8a6f-dd5dc9b30af4] Stream failed
> ERROR [main] 2015-05-20 04:20:55,252 CassandraDaemon.java (line 584) 
> Exception encountered during startup
> java.lang.RuntimeException: Error during boostrap: Stream failed
> at 
> org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86)
> at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1005)
> at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:808)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:621)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:510)
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:437)
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:567)
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:656)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
> at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
> at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at 
> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
> at 
> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:216)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:377)
> at 
> org.apache.cassandra.streaming.StreamSession.maybeCompleted(StreamSession.java:662)
> at 
> org.apache.cassandra.streaming.StreamSession.taskCompleted(StreamSession.java:613)
> at 
> org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:143)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
>  INFO [StorageServiceShutdownHook] 2015-05-20 04:20:55,286 Gossiper.java 
> (line 1330) Announcing shutdown
> {code}
> There are no WARN or ERROR prior to this in the log files of the 
> bootstrapping node or other nodes in the cluster. Only relevant log lines are 
> "Session with 11.22.33.44/11.22.33.44 is complete"
> Is it possible that individual stream sessions fail silently? 



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


[jira] [Commented] (CASSANDRA-3569) Failure detector downs should not break streams

2015-05-22 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-3569:
-

[~yukim]: Here CASSANDRA-9455

> Failure detector downs should not break streams
> ---
>
> Key: CASSANDRA-3569
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3569
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Peter Schuller
>Assignee: Joshua McKenzie
> Fix For: 2.1.1
>
> Attachments: 3569-2.0.txt, 3569_v1.txt
>
>
> CASSANDRA-2433 introduced this behavior just to get repairs to don't sit 
> there waiting forever. In my opinion the correct fix to that problem is to 
> use TCP keep alive. Unfortunately the TCP keep alive period is insanely high 
> by default on a modern Linux, so just doing that is not entirely good either.
> But using the failure detector seems non-sensicle to me. We have a 
> communication method which is the TCP transport, that we know is used for 
> long-running processes that you don't want to incorrectly be killed for no 
> good reason, and we are using a failure detector tuned to detecting when not 
> to send real-time sensitive request to nodes in order to actively kill a 
> working connection.
> So, rather than add complexity with protocol based ping/pongs and such, I 
> propose that we simply just use TCP keep alive for streaming connections and 
> instruct operators of production clusters to tweak 
> net.ipv4.tcp_keepalive_{probes,intvl} as appropriate (or whatever equivalent 
> on their OS).
> I can submit the patch. Awaiting opinions.



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


[jira] [Updated] (CASSANDRA-9455) Rely on TCP keepalive vs failure detector for streaming connections

2015-05-22 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-9455:

Attachment: 9455.txt

> Rely on TCP keepalive vs failure detector for streaming connections
> ---
>
> Key: CASSANDRA-9455
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9455
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Omid Aladini
>Assignee: Omid Aladini
> Fix For: 2.0.16
>
> Attachments: 9455.txt
>
>
> The patch applies the streaming-related parts of CASSANDRA-3569 into the 
> current 2.0. The rest is already backported in CASSANDRA-7560.



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


[jira] [Created] (CASSANDRA-9455) Rely on TCP keepalive vs failure detector for streaming connections

2015-05-22 Thread Omid Aladini (JIRA)
Omid Aladini created CASSANDRA-9455:
---

 Summary: Rely on TCP keepalive vs failure detector for streaming 
connections
 Key: CASSANDRA-9455
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9455
 Project: Cassandra
  Issue Type: New Feature
Reporter: Omid Aladini
Assignee: Omid Aladini
 Fix For: 2.0.16


The patch applies the streaming-related parts of CASSANDRA-3569 into the 
current 2.0. The rest is already backported in CASSANDRA-7560.



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


[jira] [Comment Edited] (CASSANDRA-3569) Failure detector downs should not break streams

2015-05-21 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-3569 at 5/21/15 11:19 PM:
---

{quote}
May have better luck asking on CASSANDRA-7560 - I'm sure Yuki can better speak 
to why he decided to only partially back-port this ticket to 2.0; my guess is 
that he did the minimum necessary to the stable branch to rectify the hang that 
7560 is supposed to address.
{quote}
Right. Thought I'd keep the conversation here as the rest of the patch 
(streaming part) isn't relevant to that thread. 

If everyone's ok with that, I can submit a partial patch that applies the rest 
of this change-set on the current 2.0 before it becomes frozen.

[~yukim]: thoughts? 


was (Author: omid):
{quote}
May have better luck asking on CASSANDRA-7560 - I'm sure Yuki can better speak 
to why he decided to only partially back-port this ticket to 2.0; my guess is 
that he did the minimum necessary to the stable branch to rectify the hang that 
7560 is supposed to address.
{quote}
Right. Thought I'd keep the conversation here as the rest of the patch 
(streaming part) isn't relevant to that thread. 

If anyone's ok with that, I can submit a partial patch that applies the rest of 
this change-set on the current 2.0 before it becomes frozen.

[~yukim]: thoughts? 

> Failure detector downs should not break streams
> ---
>
> Key: CASSANDRA-3569
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3569
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Peter Schuller
>Assignee: Joshua McKenzie
> Fix For: 2.1.1
>
> Attachments: 3569-2.0.txt, 3569_v1.txt
>
>
> CASSANDRA-2433 introduced this behavior just to get repairs to don't sit 
> there waiting forever. In my opinion the correct fix to that problem is to 
> use TCP keep alive. Unfortunately the TCP keep alive period is insanely high 
> by default on a modern Linux, so just doing that is not entirely good either.
> But using the failure detector seems non-sensicle to me. We have a 
> communication method which is the TCP transport, that we know is used for 
> long-running processes that you don't want to incorrectly be killed for no 
> good reason, and we are using a failure detector tuned to detecting when not 
> to send real-time sensitive request to nodes in order to actively kill a 
> working connection.
> So, rather than add complexity with protocol based ping/pongs and such, I 
> propose that we simply just use TCP keep alive for streaming connections and 
> instruct operators of production clusters to tweak 
> net.ipv4.tcp_keepalive_{probes,intvl} as appropriate (or whatever equivalent 
> on their OS).
> I can submit the patch. Awaiting opinions.



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


[jira] [Commented] (CASSANDRA-3569) Failure detector downs should not break streams

2015-05-21 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-3569:
-

{quote}
May have better luck asking on CASSANDRA-7560 - I'm sure Yuki can better speak 
to why he decided to only partially back-port this ticket to 2.0; my guess is 
that he did the minimum necessary to the stable branch to rectify the hang that 
7560 is supposed to address.
{quote}
Right. Thought I'd keep the conversation here as the rest of the patch 
(streaming part) isn't relevant to that thread. 

If anyone's ok with that, I can submit a partial patch that applies the rest of 
this change-set on the current 2.0 before it becomes frozen.

[~yukim]: thoughts? 

> Failure detector downs should not break streams
> ---
>
> Key: CASSANDRA-3569
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3569
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Peter Schuller
>Assignee: Joshua McKenzie
> Fix For: 2.1.1
>
> Attachments: 3569-2.0.txt, 3569_v1.txt
>
>
> CASSANDRA-2433 introduced this behavior just to get repairs to don't sit 
> there waiting forever. In my opinion the correct fix to that problem is to 
> use TCP keep alive. Unfortunately the TCP keep alive period is insanely high 
> by default on a modern Linux, so just doing that is not entirely good either.
> But using the failure detector seems non-sensicle to me. We have a 
> communication method which is the TCP transport, that we know is used for 
> long-running processes that you don't want to incorrectly be killed for no 
> good reason, and we are using a failure detector tuned to detecting when not 
> to send real-time sensitive request to nodes in order to actively kill a 
> working connection.
> So, rather than add complexity with protocol based ping/pongs and such, I 
> propose that we simply just use TCP keep alive for streaming connections and 
> instruct operators of production clusters to tweak 
> net.ipv4.tcp_keepalive_{probes,intvl} as appropriate (or whatever equivalent 
> on their OS).
> I can submit the patch. Awaiting opinions.



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


[jira] [Comment Edited] (CASSANDRA-3569) Failure detector downs should not break streams

2015-05-21 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-3569 at 5/21/15 1:06 PM:
--

CASSANDRA-7560 partially backports the changes introduced here into 2.0 but not 
the removal of failure detector registration for stream sessions and enabling 
tcp keepalive. Can we backport the rest to 2.0 as well? Wondering if it's 
intentional.


was (Author: omid):
CASSANDRA-7560 partially backports the changes introduced here into 2.0 but not 
the removal of failure detection registration for stream sessions and enabling 
tcp keepalive. Can we backport the rest to 2.0 as well? Wondering if it's 
intentional.

> Failure detector downs should not break streams
> ---
>
> Key: CASSANDRA-3569
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3569
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Peter Schuller
>Assignee: Joshua McKenzie
> Fix For: 2.1.1
>
> Attachments: 3569-2.0.txt, 3569_v1.txt
>
>
> CASSANDRA-2433 introduced this behavior just to get repairs to don't sit 
> there waiting forever. In my opinion the correct fix to that problem is to 
> use TCP keep alive. Unfortunately the TCP keep alive period is insanely high 
> by default on a modern Linux, so just doing that is not entirely good either.
> But using the failure detector seems non-sensicle to me. We have a 
> communication method which is the TCP transport, that we know is used for 
> long-running processes that you don't want to incorrectly be killed for no 
> good reason, and we are using a failure detector tuned to detecting when not 
> to send real-time sensitive request to nodes in order to actively kill a 
> working connection.
> So, rather than add complexity with protocol based ping/pongs and such, I 
> propose that we simply just use TCP keep alive for streaming connections and 
> instruct operators of production clusters to tweak 
> net.ipv4.tcp_keepalive_{probes,intvl} as appropriate (or whatever equivalent 
> on their OS).
> I can submit the patch. Awaiting opinions.



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


[jira] [Commented] (CASSANDRA-3569) Failure detector downs should not break streams

2015-05-21 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-3569:
-

CASSANDRA-7560 partially backports the changes introduced here into 2.0 but not 
the removal of failure detection registration for stream sessions and enabling 
tcp keepalive. Can we backport the rest to 2.0 as well? Wondering if it's 
intentional.

> Failure detector downs should not break streams
> ---
>
> Key: CASSANDRA-3569
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3569
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Peter Schuller
>Assignee: Joshua McKenzie
> Fix For: 2.1.1
>
> Attachments: 3569-2.0.txt, 3569_v1.txt
>
>
> CASSANDRA-2433 introduced this behavior just to get repairs to don't sit 
> there waiting forever. In my opinion the correct fix to that problem is to 
> use TCP keep alive. Unfortunately the TCP keep alive period is insanely high 
> by default on a modern Linux, so just doing that is not entirely good either.
> But using the failure detector seems non-sensicle to me. We have a 
> communication method which is the TCP transport, that we know is used for 
> long-running processes that you don't want to incorrectly be killed for no 
> good reason, and we are using a failure detector tuned to detecting when not 
> to send real-time sensitive request to nodes in order to actively kill a 
> working connection.
> So, rather than add complexity with protocol based ping/pongs and such, I 
> propose that we simply just use TCP keep alive for streaming connections and 
> instruct operators of production clusters to tweak 
> net.ipv4.tcp_keepalive_{probes,intvl} as appropriate (or whatever equivalent 
> on their OS).
> I can submit the patch. Awaiting opinions.



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


[jira] [Updated] (CASSANDRA-9440) Bootstrap fails without any hint of prior stream failure

2015-05-20 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-9440:

Component/s: Core

> Bootstrap fails without any hint of prior stream failure
> 
>
> Key: CASSANDRA-9440
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9440
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Cassandra 2.0.14
> 2 DCs on EC2
>Reporter: Omid Aladini
>
> I'm working on a cluster running Cassandra 2.0.14 and the bootstrap fails but 
> there is no prior hint of failed streams:
> {code}
>  WARN [StreamReceiveTask:177] 2015-05-20 04:20:55,251 StreamResultFuture.java 
> (line 215) [Stream #0b42c640-fe03-11e4-8a6f-dd5dc9b30af4] Stream failed
> ERROR [main] 2015-05-20 04:20:55,252 CassandraDaemon.java (line 584) 
> Exception encountered during startup
> java.lang.RuntimeException: Error during boostrap: Stream failed
> at 
> org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86)
> at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1005)
> at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:808)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:621)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:510)
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:437)
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:567)
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:656)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
> at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
> at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at 
> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
> at 
> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:216)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:377)
> at 
> org.apache.cassandra.streaming.StreamSession.maybeCompleted(StreamSession.java:662)
> at 
> org.apache.cassandra.streaming.StreamSession.taskCompleted(StreamSession.java:613)
> at 
> org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:143)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
>  INFO [StorageServiceShutdownHook] 2015-05-20 04:20:55,286 Gossiper.java 
> (line 1330) Announcing shutdown
> {code}
> There are no WARN or ERROR prior to this in the log files of the 
> bootstrapping node or other nodes in the cluster. Only relevant log lines are 
> "Session with 11.22.33.44/11.22.33.44 is complete"
> Is it possible that individual stream sessions fail silently? 



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


[jira] [Commented] (CASSANDRA-9440) Bootstrap fails without any hint of prior stream failure

2015-05-20 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-9440:
-

It might be relevant that the cluster was running with the default 
streaming_socket_timeout_in_ms = 0, but if that's the cause of the failure, 
shouldn't the bootstrap just get stuck?

> Bootstrap fails without any hint of prior stream failure
> 
>
> Key: CASSANDRA-9440
> URL: https://issues.apache.org/jira/browse/CASSANDRA-9440
> Project: Cassandra
>  Issue Type: Bug
> Environment: Cassandra 2.0.14
> 2 DCs on EC2
>Reporter: Omid Aladini
>
> I'm working on a cluster running Cassandra 2.0.14 and the bootstrap fails but 
> there is no prior hint of failed streams:
> {code}
>  WARN [StreamReceiveTask:177] 2015-05-20 04:20:55,251 StreamResultFuture.java 
> (line 215) [Stream #0b42c640-fe03-11e4-8a6f-dd5dc9b30af4] Stream failed
> ERROR [main] 2015-05-20 04:20:55,252 CassandraDaemon.java (line 584) 
> Exception encountered during startup
> java.lang.RuntimeException: Error during boostrap: Stream failed
> at 
> org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86)
> at 
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1005)
> at 
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:808)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:621)
> at 
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:510)
> at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:437)
> at 
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:567)
> at 
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:656)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
> at 
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
> at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
> at 
> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
> at 
> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
> at 
> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
> at 
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:216)
> at 
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
> at 
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:377)
> at 
> org.apache.cassandra.streaming.StreamSession.maybeCompleted(StreamSession.java:662)
> at 
> org.apache.cassandra.streaming.StreamSession.taskCompleted(StreamSession.java:613)
> at 
> org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:143)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask.run(FutureTask.java:262)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
> at java.lang.Thread.run(Thread.java:744)
>  INFO [StorageServiceShutdownHook] 2015-05-20 04:20:55,286 Gossiper.java 
> (line 1330) Announcing shutdown
> {code}
> There are no WARN or ERROR prior to this in the log files of the 
> bootstrapping node or other nodes in the cluster. Only relevant log lines are 
> "Session with 11.22.33.44/11.22.33.44 is complete"
> Is it possible that individual stream sessions fail silently? 



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


[jira] [Created] (CASSANDRA-9440) Bootstrap fails without any hint of prior stream failure

2015-05-20 Thread Omid Aladini (JIRA)
Omid Aladini created CASSANDRA-9440:
---

 Summary: Bootstrap fails without any hint of prior stream failure
 Key: CASSANDRA-9440
 URL: https://issues.apache.org/jira/browse/CASSANDRA-9440
 Project: Cassandra
  Issue Type: Bug
 Environment: Cassandra 2.0.14
2 DCs on EC2
Reporter: Omid Aladini


I'm working on a cluster running Cassandra 2.0.14 and the bootstrap fails but 
there is no prior hint of failed streams:

{code}
 WARN [StreamReceiveTask:177] 2015-05-20 04:20:55,251 StreamResultFuture.java 
(line 215) [Stream #0b42c640-fe03-11e4-8a6f-dd5dc9b30af4] Stream failed
ERROR [main] 2015-05-20 04:20:55,252 CassandraDaemon.java (line 584) Exception 
encountered during startup
java.lang.RuntimeException: Error during boostrap: Stream failed
at org.apache.cassandra.dht.BootStrapper.bootstrap(BootStrapper.java:86)
at 
org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1005)
at 
org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:808)
at 
org.apache.cassandra.service.StorageService.initServer(StorageService.java:621)
at 
org.apache.cassandra.service.StorageService.initServer(StorageService.java:510)
at 
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:437)
at 
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:567)
at 
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:656)
Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
at 
org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
at com.google.common.util.concurrent.Futures$4.run(Futures.java:1160)
at 
com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
at 
com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
at 
com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
at 
com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
at 
org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:216)
at 
org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
at 
org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:377)
at 
org.apache.cassandra.streaming.StreamSession.maybeCompleted(StreamSession.java:662)
at 
org.apache.cassandra.streaming.StreamSession.taskCompleted(StreamSession.java:613)
at 
org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:143)
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
at java.util.concurrent.FutureTask.run(FutureTask.java:262)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:744)
 INFO [StorageServiceShutdownHook] 2015-05-20 04:20:55,286 Gossiper.java (line 
1330) Announcing shutdown

{code}

There are no WARN or ERROR prior to this in the log files of the bootstrapping 
node or other nodes in the cluster. Only relevant log lines are "Session with 
11.22.33.44/11.22.33.44 is complete"

Is it possible that individual stream sessions fail silently? 



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


[jira] [Commented] (CASSANDRA-8149) bump metrics-reporter-config dependency to 2.3.1

2014-10-24 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-8149:
-

I just added 2.0.12 to "fix version", not sure if it's ok to do so after it's 
already been assigned to. But as there is another reason (making Graphite 
reporter actually useful) besides "new reporters" mentioned in the issue 
description, thought it may deserve to be patched to the same branch as it was 
first introduced.

> bump metrics-reporter-config dependency to 2.3.1
> 
>
> Key: CASSANDRA-8149
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8149
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Pierre-Yves Ritschard
> Fix For: 2.0.12, 3.0
>
>
> It would be nice to be able to take advantage of the new reporters available 
> in
> metrics-reporter-config 2.3.1 which is now available on maven central.
> If my understanding is correct, this only entails bumping the dependency in 
> build.xml.



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


[jira] [Updated] (CASSANDRA-8149) bump metrics-reporter-config dependency to 2.3.1

2014-10-24 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-8149:

Fix Version/s: 2.0.12

> bump metrics-reporter-config dependency to 2.3.1
> 
>
> Key: CASSANDRA-8149
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8149
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Pierre-Yves Ritschard
> Fix For: 2.0.12, 3.0
>
>
> It would be nice to be able to take advantage of the new reporters available 
> in
> metrics-reporter-config 2.3.1 which is now available on maven central.
> If my understanding is correct, this only entails bumping the dependency in 
> build.xml.



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


[jira] [Updated] (CASSANDRA-8149) bump metrics-reporter-config dependency to 2.3.1

2014-10-23 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-8149:

Fix Version/s: 2.1.2
   2.0.12

> bump metrics-reporter-config dependency to 2.3.1
> 
>
> Key: CASSANDRA-8149
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8149
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Pierre-Yves Ritschard
> Fix For: 3.0
>
>
> It would be nice to be able to take advantage of the new reporters available 
> in
> metrics-reporter-config 2.3.1 which is now available on maven central.
> If my understanding is correct, this only entails bumping the dependency in 
> build.xml.



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


[jira] [Updated] (CASSANDRA-8149) bump metrics-reporter-config dependency to 2.3.1

2014-10-23 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-8149:

Fix Version/s: (was: 2.0.12)
   (was: 2.1.2)

> bump metrics-reporter-config dependency to 2.3.1
> 
>
> Key: CASSANDRA-8149
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8149
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Pierre-Yves Ritschard
> Fix For: 3.0
>
>
> It would be nice to be able to take advantage of the new reporters available 
> in
> metrics-reporter-config 2.3.1 which is now available on maven central.
> If my understanding is correct, this only entails bumping the dependency in 
> build.xml.



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


[jira] [Commented] (CASSANDRA-8149) bump metrics-reporter-config dependency to 2.3.1

2014-10-23 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-8149:
-

Would be great if this could also be applied to 2.0 and 2.1 releases. Prefixing 
metrics by hostname/ip/fqdn/etc for Graphite is introduced in 
metrics-reporter-config-2.2 and without it it's not possible to distinguish 
between nodes.

> bump metrics-reporter-config dependency to 2.3.1
> 
>
> Key: CASSANDRA-8149
> URL: https://issues.apache.org/jira/browse/CASSANDRA-8149
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Pierre-Yves Ritschard
> Fix For: 3.0
>
>
> It would be nice to be able to take advantage of the new reporters available 
> in
> metrics-reporter-config 2.3.1 which is now available on maven central.
> If my understanding is correct, this only entails bumping the dependency in 
> build.xml.



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


[jira] [Comment Edited] (CASSANDRA-5195) Offline scrub does not migrate the directory structure on migration from 1.0.x to 1.1.x and causes the keyspace to disappear

2013-03-16 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-5195 at 3/16/13 7:24 PM:
--

I think upon loading the schema via offline scrubber, DefsTable.loadFromStorage 
migrates the old system tables to the new format. Therefore it drops the old 
ones but doesn't flush the commitlogs of new ones. Offline scrubber exits and 
on the next start, schema_keyspaces, schema_columnfamilies and schema_columns 
CFs have no persisted sstables, and the commitlog only gets replayed after 
Cassandra tries to load CF schemas, therefore finds none. This explains why 
after second restart, column families appear again (something force-flushes 
system CFs?).

A new patch (0001-Flush-newly-migrated-system-CFs.patch) is attached to fix the 
problem (although I'm not sure if there is a more proper fix for this or if 
there is a better place to put the foced flush.)

  was (Author: omid):
I think upon loading the schema via offline scrubber, 
DefsTable.loadFromStorage migrates the old system tables to the new format. 
Therefore it drops the old ones but doesn't flush the commitlogs of new ones. 
OfflineScrubber exits and on the next start, schema _keyspaces, _columnfamilies 
and _columns CFs have no persisted sstables, and the commitlog gets replayed 
after Cassandra tries to load CF schemas, therefore finds none. This explains 
why after second restart, column families appear again (something force-flushes 
system CFs?).

A new patch (0001-Flush-newly-migrated-system-CFs.patch) is attached to fix the 
problem (although I'm not sure if there is a more proper fix for this or if 
there is a better place to put the foced flush.)
  
> Offline scrub does not migrate the directory structure on migration from 
> 1.0.x to 1.1.x and causes the keyspace to disappear
> 
>
> Key: CASSANDRA-5195
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5195
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.9
>Reporter: Omid Aladini
> Fix For: 1.1.11
>
> Attachments: 0001-Flush-newly-migrated-system-CFs.patch, 5195.patch
>
>
> Due to CASSANDRA-4411, upon migration from 1.0.x to 1.1.x containing 
> LCS-compacted sstables, an offline scrub should be run before Cassandra 1.1.x 
> is started. But Cassandra 1.1.x uses a new directory structure 
> (CASSANDRA-2749) that offline scrubber doesn't detect or try to migrate.
> How to reproduce:
> 1- Run cassandra 1.0.12.
> 2- Run stress tool, let Cassandra flush Keyspace1 or flush manually.
> 3- Stop cassandra 1.0.12
> 4- Run ./bin/sstablescrub Keyspace1 Standard1
>   which returns "Unknown keyspace/columnFamily Keyspace1.Standard1" and 
> notice the data directory isn't migrated.
> 5- Run cassandra 1.1.9. Keyspace1 doesn't get loaded and Cassandra doesn't 
> try to migrate the directory structure. Also commitlog entries get skipped: 
> "Skipped X mutations from unknown (probably removed) CF with id 1000"
> Without the unsuccessful step 4, Cassandra 1.1.9 loads and migrates the 
> Keyspace correctly.
>   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-5195) Offline scrub does not migrate the directory structure on migration from 1.0.x to 1.1.x and causes the keyspace to disappear

2013-03-16 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-5195 at 3/16/13 7:21 PM:
--

I think upon loading the schema via offline scrubber, DefsTable.loadFromStorage 
migrates the old system tables to the new format. Therefore it drops the old 
ones but doesn't flush the commitlogs of new ones. OfflineScrubber exits and on 
the next start, schema _keyspaces, _columnfamilies and _columns CFs have no 
persisted sstables, and the commitlog gets replayed after Cassandra tries to 
load CF schemas, therefore finds none. This explains why after second restart, 
column families appear again (something force-flushes system CFs?).

A new patch (0001-Flush-newly-migrated-system-CFs.patch) is attached to fix the 
problem (although I'm not sure if there is a more proper fix for this or if 
there is a better place to put the foced flush.)

  was (Author: omid):
I think upon loading the schema via OfflineScrubber, 
DefsTable.loadFromStorage migrates the old system tables to the new format. 
Therefore it drops the old ones but doesn't flush the commitlogs of new ones. 
OfflineScrubber exits and on the next start, schema _keyspaces, _columnfamilies 
and _columns CFs have no persisted sstables, and the commitlog gets replayed 
after Cassandra tries to load CF schemas, therefore finds none. This explains 
why after second restart, column families appear again (something force-flushes 
system CFs?).

A new patch (0001-Flush-newly-migrated-system-CFs.patch) is attached to fix the 
problem (although I'm not sure if there is a more proper fix for this or if 
there is a better place to put the foced flush.)
  
> Offline scrub does not migrate the directory structure on migration from 
> 1.0.x to 1.1.x and causes the keyspace to disappear
> 
>
> Key: CASSANDRA-5195
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5195
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.9
>Reporter: Omid Aladini
> Fix For: 1.1.11
>
> Attachments: 0001-Flush-newly-migrated-system-CFs.patch, 5195.patch
>
>
> Due to CASSANDRA-4411, upon migration from 1.0.x to 1.1.x containing 
> LCS-compacted sstables, an offline scrub should be run before Cassandra 1.1.x 
> is started. But Cassandra 1.1.x uses a new directory structure 
> (CASSANDRA-2749) that offline scrubber doesn't detect or try to migrate.
> How to reproduce:
> 1- Run cassandra 1.0.12.
> 2- Run stress tool, let Cassandra flush Keyspace1 or flush manually.
> 3- Stop cassandra 1.0.12
> 4- Run ./bin/sstablescrub Keyspace1 Standard1
>   which returns "Unknown keyspace/columnFamily Keyspace1.Standard1" and 
> notice the data directory isn't migrated.
> 5- Run cassandra 1.1.9. Keyspace1 doesn't get loaded and Cassandra doesn't 
> try to migrate the directory structure. Also commitlog entries get skipped: 
> "Skipped X mutations from unknown (probably removed) CF with id 1000"
> Without the unsuccessful step 4, Cassandra 1.1.9 loads and migrates the 
> Keyspace correctly.
>   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-5195) Offline scrub does not migrate the directory structure on migration from 1.0.x to 1.1.x and causes the keyspace to disappear

2013-03-16 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-5195 at 3/16/13 3:36 PM:
--

I think upon loading the schema via OfflineScrubber, DefsTable.loadFromStorage 
migrates the old system tables to the new format. Therefore it drops the old 
ones but doesn't flush the commitlogs of new ones. OfflineScrubber exits and on 
the next start, schema _keyspaces, _columnfamilies and _columns CFs have no 
persisted sstables, and the commitlog gets replayed after Cassandra tries to 
load CF schemas, therefore finds none. This explains why after second restart, 
column families appear again (something force-flushes system CFs?).

A new patch (0001-Flush-newly-migrated-system-CFs.patch) is attached to fix the 
problem (although I'm not sure if there is a more proper fix for this or if 
there is a better place to put the foced flush.)

  was (Author: omid):
I think upon loading the schema via OfflineScrubber, 
DefsTable.loadFromStorage migrates the old system tables to the new format. 
Therefore it drops the old ones but doesn't flush the commitlogs of new ones. 
OfflineScrubber exits and on the next start, schema_{keyspaces, columnfamilies, 
columns} have no persisted sstables, and the commitlog gets replayed after 
Cassandra tries to load CF schemas, therefore finds none. This explains why 
after second restart, column families appear again (something force-flushes 
system CFs?).

A new patch (0001-Flush-newly-migrated-system-CFs.patch) is attached to fix the 
problem (although I'm not sure if there is a more proper fix for this or if 
there is a better place to put the foced flush.)
  
> Offline scrub does not migrate the directory structure on migration from 
> 1.0.x to 1.1.x and causes the keyspace to disappear
> 
>
> Key: CASSANDRA-5195
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5195
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.9
>Reporter: Omid Aladini
> Fix For: 1.1.11
>
> Attachments: 0001-Flush-newly-migrated-system-CFs.patch, 5195.patch
>
>
> Due to CASSANDRA-4411, upon migration from 1.0.x to 1.1.x containing 
> LCS-compacted sstables, an offline scrub should be run before Cassandra 1.1.x 
> is started. But Cassandra 1.1.x uses a new directory structure 
> (CASSANDRA-2749) that offline scrubber doesn't detect or try to migrate.
> How to reproduce:
> 1- Run cassandra 1.0.12.
> 2- Run stress tool, let Cassandra flush Keyspace1 or flush manually.
> 3- Stop cassandra 1.0.12
> 4- Run ./bin/sstablescrub Keyspace1 Standard1
>   which returns "Unknown keyspace/columnFamily Keyspace1.Standard1" and 
> notice the data directory isn't migrated.
> 5- Run cassandra 1.1.9. Keyspace1 doesn't get loaded and Cassandra doesn't 
> try to migrate the directory structure. Also commitlog entries get skipped: 
> "Skipped X mutations from unknown (probably removed) CF with id 1000"
> Without the unsuccessful step 4, Cassandra 1.1.9 loads and migrates the 
> Keyspace correctly.
>   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-5195) Offline scrub does not migrate the directory structure on migration from 1.0.x to 1.1.x and causes the keyspace to disappear

2013-03-16 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-5195:
-

I think upon loading the schema via OfflineScrubber, DefsTable.loadFromStorage 
migrates the old system tables to the new format. Therefore it drops the old 
ones but doesn't flush the commitlogs of new ones. OfflineScrubber exits and on 
the next start, schema_{keyspaces, columnfamilies, columns} have no persisted 
sstables, and the commitlog gets replayed after Cassandra tries to load CF 
schemas, therefore finds none. This explains why after second restart, column 
families appear again (something force-flushes system CFs?).

A new patch (0001-Flush-newly-migrated-system-CFs.patch) is attached to fix the 
problem (although I'm not sure if there is a more proper fix for this or if 
there is a better place to put the foced flush.)

> Offline scrub does not migrate the directory structure on migration from 
> 1.0.x to 1.1.x and causes the keyspace to disappear
> 
>
> Key: CASSANDRA-5195
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5195
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.9
>Reporter: Omid Aladini
> Fix For: 1.1.11
>
> Attachments: 0001-Flush-newly-migrated-system-CFs.patch, 5195.patch
>
>
> Due to CASSANDRA-4411, upon migration from 1.0.x to 1.1.x containing 
> LCS-compacted sstables, an offline scrub should be run before Cassandra 1.1.x 
> is started. But Cassandra 1.1.x uses a new directory structure 
> (CASSANDRA-2749) that offline scrubber doesn't detect or try to migrate.
> How to reproduce:
> 1- Run cassandra 1.0.12.
> 2- Run stress tool, let Cassandra flush Keyspace1 or flush manually.
> 3- Stop cassandra 1.0.12
> 4- Run ./bin/sstablescrub Keyspace1 Standard1
>   which returns "Unknown keyspace/columnFamily Keyspace1.Standard1" and 
> notice the data directory isn't migrated.
> 5- Run cassandra 1.1.9. Keyspace1 doesn't get loaded and Cassandra doesn't 
> try to migrate the directory structure. Also commitlog entries get skipped: 
> "Skipped X mutations from unknown (probably removed) CF with id 1000"
> Without the unsuccessful step 4, Cassandra 1.1.9 loads and migrates the 
> Keyspace correctly.
>   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5195) Offline scrub does not migrate the directory structure on migration from 1.0.x to 1.1.x and causes the keyspace to disappear

2013-03-16 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-5195:


Attachment: 0001-Flush-newly-migrated-system-CFs.patch

New patch to fix the condition that CFs show up after second restart.

> Offline scrub does not migrate the directory structure on migration from 
> 1.0.x to 1.1.x and causes the keyspace to disappear
> 
>
> Key: CASSANDRA-5195
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5195
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.9
>Reporter: Omid Aladini
> Fix For: 1.1.11
>
> Attachments: 0001-Flush-newly-migrated-system-CFs.patch, 5195.patch
>
>
> Due to CASSANDRA-4411, upon migration from 1.0.x to 1.1.x containing 
> LCS-compacted sstables, an offline scrub should be run before Cassandra 1.1.x 
> is started. But Cassandra 1.1.x uses a new directory structure 
> (CASSANDRA-2749) that offline scrubber doesn't detect or try to migrate.
> How to reproduce:
> 1- Run cassandra 1.0.12.
> 2- Run stress tool, let Cassandra flush Keyspace1 or flush manually.
> 3- Stop cassandra 1.0.12
> 4- Run ./bin/sstablescrub Keyspace1 Standard1
>   which returns "Unknown keyspace/columnFamily Keyspace1.Standard1" and 
> notice the data directory isn't migrated.
> 5- Run cassandra 1.1.9. Keyspace1 doesn't get loaded and Cassandra doesn't 
> try to migrate the directory structure. Also commitlog entries get skipped: 
> "Skipped X mutations from unknown (probably removed) CF with id 1000"
> Without the unsuccessful step 4, Cassandra 1.1.9 loads and migrates the 
> Keyspace correctly.
>   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Updated] (CASSANDRA-5195) Offline scrub does not migrate the directory structure on migration from 1.0.x to 1.1.x and causes the keyspace to disappear

2013-01-29 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-5195:


Attachment: 5195.patch

I tried to fix the issue in offline-scrub but the patch doesn't fully fix the 
issue. Cassandra 1.1.9 with this patch only loads the migrated keyspaces after 
2nd restart after offine-scrub has applies the migration.

> Offline scrub does not migrate the directory structure on migration from 
> 1.0.x to 1.1.x and causes the keyspace to disappear
> 
>
> Key: CASSANDRA-5195
> URL: https://issues.apache.org/jira/browse/CASSANDRA-5195
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.9
>Reporter: Omid Aladini
> Fix For: 1.1.9
>
> Attachments: 5195.patch
>
>
> Due to CASSANDRA-4411, upon migration from 1.0.x to 1.1.x containing 
> LCS-compacted sstables, an offline scrub should be run before Cassandra 1.1.x 
> is started. But Cassandra 1.1.x uses a new directory structure 
> (CASSANDRA-2749) that offline scrubber doesn't detect or try to migrate.
> How to reproduce:
> 1- Run cassandra 1.0.12.
> 2- Run stress tool, let Cassandra flush Keyspace1 or flush manually.
> 3- Stop cassandra 1.0.12
> 4- Run ./bin/sstablescrub Keyspace1 Standard1
>   which returns "Unknown keyspace/columnFamily Keyspace1.Standard1" and 
> notice the data directory isn't migrated.
> 5- Run cassandra 1.1.9. Keyspace1 doesn't get loaded and Cassandra doesn't 
> try to migrate the directory structure. Also commitlog entries get skipped: 
> "Skipped X mutations from unknown (probably removed) CF with id 1000"
> Without the unsuccessful step 4, Cassandra 1.1.9 loads and migrates the 
> Keyspace correctly.
>   

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (CASSANDRA-5195) Offline scrub does not migrate the directory structure on migration from 1.0.x to 1.1.x and causes the keyspace to disappear

2013-01-29 Thread Omid Aladini (JIRA)
Omid Aladini created CASSANDRA-5195:
---

 Summary: Offline scrub does not migrate the directory structure on 
migration from 1.0.x to 1.1.x and causes the keyspace to disappear
 Key: CASSANDRA-5195
 URL: https://issues.apache.org/jira/browse/CASSANDRA-5195
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.1.9
Reporter: Omid Aladini


Due to CASSANDRA-4411, upon migration from 1.0.x to 1.1.x containing 
LCS-compacted sstables, an offline scrub should be run before Cassandra 1.1.x 
is started. But Cassandra 1.1.x uses a new directory structure (CASSANDRA-2749) 
that offline scrubber doesn't detect or try to migrate.

How to reproduce:

1- Run cassandra 1.0.12.
2- Run stress tool, let Cassandra flush Keyspace1 or flush manually.
3- Stop cassandra 1.0.12
4- Run ./bin/sstablescrub Keyspace1 Standard1
  which returns "Unknown keyspace/columnFamily Keyspace1.Standard1" and notice 
the data directory isn't migrated.
5- Run cassandra 1.1.9. Keyspace1 doesn't get loaded and Cassandra doesn't try 
to migrate the directory structure. Also commitlog entries get skipped: 
"Skipped X mutations from unknown (probably removed) CF with id 1000"

Without the unsuccessful step 4, Cassandra 1.1.9 loads and migrates the 
Keyspace correctly.

  

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-4765) StackOverflowError in CompactionExecutor thread

2012-10-18 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4765 at 10/18/12 1:03 PM:
---

I could only reproduce the error by mistakenly running size tiered compaction 
in test environment on originally LCS-compacted data and the patch fixes the 
stack overflow. 

  was (Author: omid):
I could only reproduce the error by mistakenly running size tiered 
compaction in test environment on originally LCS-compacted data and the patch 
fixes it. 
  
> StackOverflowError in CompactionExecutor thread
> ---
>
> Key: CASSANDRA-4765
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4765
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.1.4
>Reporter: Ahmed Bashir
>Assignee: Jonathan Ellis
>  Labels: compaction
> Fix For: 1.1.7, 1.2.0 beta 2
>
> Attachments: 4765.txt
>
>
> Seeing the following error:
> Exception in thread Thread[CompactionExecutor:21,1,RMI Runtime]
> java.lang.StackOverflowError
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4765) StackOverflowError in CompactionExecutor thread

2012-10-18 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4765:
-

I could only reproduce the error by mistakenly running size tiered compaction 
in test environment on originally LCS-compacted data and the patch fixes it. 

> StackOverflowError in CompactionExecutor thread
> ---
>
> Key: CASSANDRA-4765
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4765
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.1.4
>Reporter: Ahmed Bashir
>Assignee: Jonathan Ellis
>  Labels: compaction
> Fix For: 1.1.7, 1.2.0 beta 2
>
> Attachments: 4765.txt
>
>
> Seeing the following error:
> Exception in thread Thread[CompactionExecutor:21,1,RMI Runtime]
> java.lang.StackOverflowError
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-4446) nodetool drain sometimes doesn't mark commitlog fully flushed

2012-10-18 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4446 at 10/18/12 12:52 PM:


I also experience this every time I drain / restart (up until latest 1.1.6 but 
not on 1.1.6 itself any more) and getting this message in log:

{quote}
2012-10-12_15:50:36.92191  INFO 15:50:36,921 Log replay complete, N replayed 
mutations   
{quote}

with N being non-zero. I wonder if this is a cause of double-counts for Counter 
mutations.

  was (Author: omid):
I also experience this every time I drain / restart (up until latest 1.1.6) 
and getting this message in log:

{quote}
2012-10-12_15:50:36.92191  INFO 15:50:36,921 Log replay complete, N replayed 
mutations   
{quote}

with N being non-zero. I wonder if this is a cause of double-counts for Counter 
mutations.
  
> nodetool drain sometimes doesn't mark commitlog fully flushed
> -
>
> Key: CASSANDRA-4446
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4446
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.0.10
> Environment: ubuntu 10.04 64bit
> Linux HOSTNAME 2.6.32-345-ec2 #48-Ubuntu SMP Wed May 2 19:29:55 UTC 2012 
> x86_64 GNU/Linux
> sun JVM
> cassandra 1.0.10 installed from apache deb
>Reporter: Robert Coli
> Attachments: 
> cassandra.1.0.10.replaying.log.after.exception.during.drain.txt
>
>
> I recently wiped a customer's QA cluster. I drained each node and verified 
> that they were drained. When I restarted the nodes, I saw the commitlog 
> replay create a memtable and then flush it. I have attached a sanitized log 
> snippet from a representative node at the time. 
> It appears to show the following :
> 1) Drain begins
> 2) Drain triggers flush
> 3) Flush triggers compaction
> 4) StorageService logs DRAINED message
> 5) compaction thread excepts
> 6) on restart, same CF creates a memtable
> 7) and then flushes it [1]
> The columnfamily involved in the replay in 7) is the CF for which the 
> compaction thread excepted in 5). This seems to suggest a timing issue 
> whereby the exception in 5) prevents the flush in 3) from marking all the 
> segments flushed, causing them to replay after restart.
> In case it might be relevant, I did an online change of compaction strategy 
> from Leveled to SizeTiered during the uptime period preceding this drain.
> [1] Isn't commitlog replay not supposed to automatically trigger a flush in 
> modern cassandra?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-2610) Have the repair of a range repair *all* the replica for that range

2012-10-17 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-2610:
-

That's great, didn't know about that. This means by throttling the 
sequentialized streaming, I might be able to let pending compactions resolve on 
the previously-repaired replica, although tuning this would be a challenge 
(possibly by dynamically changing streamthroughput according to pending 
compactions, although doesn't seem ideal).

> Have the repair of a range repair *all* the replica for that range
> --
>
> Key: CASSANDRA-2610
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2610
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Affects Versions: 0.8 beta 1
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
>Priority: Minor
> Fix For: 1.0.0
>
> Attachments: 0001-Make-repair-repair-all-hosts.patch, 
> 0001-Make-repair-repair-all-hosts-v2.patch, 
> 0002-Cleanup-log-messages-v2.patch, 
> 0003-cleanup-and-fix-private-reference.patch
>
>   Original Estimate: 8h
>  Remaining Estimate: 8h
>
> Say you have a range R whose replica for that range are A, B and C. If you 
> run repair on node A for that range R, when the repair end you only know that 
> A is fully repaired. B and C are not. That is B and C are up to date with A 
> before the repair, but are not up to date with one another.
> It makes it a pain to schedule "optimal" cluster repairs, that is repairing a 
> full cluster without doing work twice (because you would have still have to 
> run a repair on B or C, which will make A, B and C redo a validation 
> compaction on R, and with more replica it's even more annoying).
> However it is fairly easy during the first repair on A to have him compare 
> all the merkle trees, i.e the ones for B and C, and ask to B or C to stream 
> between them whichever the differences they have. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4765) StackOverflowError in CompactionExecutor thread

2012-10-17 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4765:
-

I just got a similar StackOverflowError but on Iterators.java:

{code}
2012-10-17_14:35:09.95258 ERROR 14:35:09,942 Exception in thread 
Thread[CompactionExecutor:2773,1,main]
2012-10-17_14:35:09.95260 java.lang.StackOverflowError
2012-10-17_14:35:09.95261   at java.util.AbstractList$Itr.hasNext(Unknown 
Source)
2012-10-17_14:35:09.95269   at 
com.google.common.collect.Iterators$5.hasNext(Iterators.java:517)
2012-10-17_14:35:09.95281   at 
com.google.common.collect.Iterators$3.hasNext(Iterators.java:114)
2012-10-17_14:35:09.95293   at 
com.google.common.collect.Iterators$5.hasNext(Iterators.java:517)
2012-10-17_14:35:09.95305   at 
com.google.common.collect.Iterators$3.hasNext(Iterators.java:114)
2012-10-17_14:35:09.95320   at 
com.google.common.collect.Iterators$5.hasNext(Iterators.java:517)
2012-10-17_14:35:09.95331   at 
com.google.common.collect.Iterators$3.hasNext(Iterators.java:114)
{code}

And iterators.java:517 and :514 belong to concat call that appears on the 
SetView returned by Sets::union.

> StackOverflowError in CompactionExecutor thread
> ---
>
> Key: CASSANDRA-4765
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4765
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.1.4
>Reporter: Ahmed Bashir
> Attachments: 4765.txt
>
>
> Seeing the following error:
> Exception in thread Thread[CompactionExecutor:21,1,RMI Runtime]
> java.lang.StackOverflowError
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)
> at com.google.common.collect.Sets$1.iterator(Sets.java:578)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-2610) Have the repair of a range repair *all* the replica for that range

2012-10-17 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-2610:
-

This indeed makes repair across a cluster easier to manage, specially together 
with -pr (CASSANDRA-2606), but the downside is all replica for a range would be 
affected once the data is streamed. In my case repair transfers huge amount of 
data each time (possibly due to Merkle tree precision CASSANDRA-2698) causing 
hundreds of pending compactions that affects reads and counter-writes for the 
affected range. I'd prefer to have cassandra calculate Merkle trees multiple 
times (which is possible to throttle) and have faster quorum reads when only 
one replica is slowed down. Given that incremental repair (CASSANDRA-2699) is 
still in progress, do you think it makes sense to make repair-on-all-replica 
optional? Possibly via a flag on the node that the repair is run?

> Have the repair of a range repair *all* the replica for that range
> --
>
> Key: CASSANDRA-2610
> URL: https://issues.apache.org/jira/browse/CASSANDRA-2610
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Affects Versions: 0.8 beta 1
>Reporter: Sylvain Lebresne
>Assignee: Sylvain Lebresne
>Priority: Minor
> Fix For: 1.0.0
>
> Attachments: 0001-Make-repair-repair-all-hosts.patch, 
> 0001-Make-repair-repair-all-hosts-v2.patch, 
> 0002-Cleanup-log-messages-v2.patch, 
> 0003-cleanup-and-fix-private-reference.patch
>
>   Original Estimate: 8h
>  Remaining Estimate: 8h
>
> Say you have a range R whose replica for that range are A, B and C. If you 
> run repair on node A for that range R, when the repair end you only know that 
> A is fully repaired. B and C are not. That is B and C are up to date with A 
> before the repair, but are not up to date with one another.
> It makes it a pain to schedule "optimal" cluster repairs, that is repairing a 
> full cluster without doing work twice (because you would have still have to 
> run a repair on B or C, which will make A, B and C redo a validation 
> compaction on R, and with more replica it's even more annoying).
> However it is fairly easy during the first repair on A to have him compare 
> all the merkle trees, i.e the ones for B and C, and ask to B or C to stream 
> between them whichever the differences they have. 

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4807) Compaction progress counts more than 100%

2012-10-16 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4807:
-

Thanks Yuki. It also probably affects compaction throttling, by throttling more 
than it should 
(https://github.com/apache/cassandra/blob/cassandra-1.1.6/src/java/org/apache/cassandra/db/compaction/CompactionManager.java#L612)

> Compaction progress counts more than 100%
> -
>
> Key: CASSANDRA-4807
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4807
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.6
>Reporter: Omid Aladini
>Assignee: Yuki Morishita
>Priority: Minor
> Attachments: 4807-1.1.txt
>
>
> 'nodetool compactionstats' compaction progress counts more than 100%:
> {code}
> pending tasks: 74
>   compaction typekeyspace   column family bytes compacted 
> bytes total  progress
>ValidationKSPCF1   56192578305 
> 8465276891766.38%
>CompactionKSPCF2   162018591   
> 119913592 135.11%
> {code}
> Hadn't experienced this before 1.1.3. Is it due to changes in 1.1.4-1.1.6 ?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4807) Compaction progress counts more than 100%

2012-10-16 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4807:
-

[~yukim] LCS with snappy with chunk_length_kb = 64, sstable_size_in_mb = 10 and 
multithreaded compaction disabled on Cassandra 1.1.6.

> Compaction progress counts more than 100%
> -
>
> Key: CASSANDRA-4807
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4807
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.6
>Reporter: Omid Aladini
>Assignee: Yuki Morishita
>Priority: Minor
>
> 'nodetool compactionstats' compaction progress counts more than 100%:
> {code}
> pending tasks: 74
>   compaction typekeyspace   column family bytes compacted 
> bytes total  progress
>ValidationKSPCF1   56192578305 
> 8465276891766.38%
>CompactionKSPCF2   162018591   
> 119913592 135.11%
> {code}
> Hadn't experienced this before 1.1.3. Is it due to changes in 1.1.4-1.1.6 ?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4782) Commitlog not replayed after restart

2012-10-16 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4782:
-

CASSANDRA-4446 does not happen to me any more (so far) when restarting 1.1.6 
into 1.1.6. I could observe CASSANDRA-4446 on upgrade from 1.1.3 to 1.1.6 
though.

> Commitlog not replayed after restart
> 
>
> Key: CASSANDRA-4782
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4782
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.1.0
>Reporter: Fabien Rousseau
>Assignee: Jonathan Ellis
>Priority: Critical
> Fix For: 1.1.6
>
> Attachments: 4782.txt
>
>
> It seems that there are two corner cases where commitlog is not replayed 
> after a restart :
>  - After a reboot of a server + restart of cassandra (1.1.0 to 1.1.4)
>  - After doing an upgrade from cassandra 1.1.X to cassandra 1.1.5
> This is due to the fact that the commitlog segment id should always be an  
> incrementing number (see this condition : 
> https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java#L247
>  )
> But this assertion can be broken :
> In the first case, it is generated by System.nanoTime() but it seems that 
> System.nanoTime() is using the boot time as the base/reference (at least on 
> java6 & linux), thus after a reboot, System.nanoTime() can return a lower 
> number than before the reboot (and the javadoc says the reference is a 
> relative point in time...)
> In the second case, this was introduced by #4601 (which changes 
> System.nanoTime() by System.currentTimeMillis() thus people starting with 
> 1.1.5 are safe)
> This could explain the following tickets : #4741 and #4481

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4446) nodetool drain sometimes doesn't mark commitlog fully flushed

2012-10-15 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4446:
-

I also experience this every time I drain / restart (up until latest 1.1.6) and 
getting this message in log:

{quote}
2012-10-12_15:50:36.92191  INFO 15:50:36,921 Log replay complete, N replayed 
mutations   
{quote}

with N being non-zero. I wonder if this is a cause of double-counts for Counter 
mutations.

> nodetool drain sometimes doesn't mark commitlog fully flushed
> -
>
> Key: CASSANDRA-4446
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4446
> Project: Cassandra
>  Issue Type: Bug
>Affects Versions: 1.0.10
> Environment: ubuntu 10.04 64bit
> Linux HOSTNAME 2.6.32-345-ec2 #48-Ubuntu SMP Wed May 2 19:29:55 UTC 2012 
> x86_64 GNU/Linux
> sun JVM
> cassandra 1.0.10 installed from apache deb
>Reporter: Robert Coli
> Attachments: 
> cassandra.1.0.10.replaying.log.after.exception.during.drain.txt
>
>
> I recently wiped a customer's QA cluster. I drained each node and verified 
> that they were drained. When I restarted the nodes, I saw the commitlog 
> replay create a memtable and then flush it. I have attached a sanitized log 
> snippet from a representative node at the time. 
> It appears to show the following :
> 1) Drain begins
> 2) Drain triggers flush
> 3) Flush triggers compaction
> 4) StorageService logs DRAINED message
> 5) compaction thread excepts
> 6) on restart, same CF creates a memtable
> 7) and then flushes it [1]
> The columnfamily involved in the replay in 7) is the CF for which the 
> compaction thread excepted in 5). This seems to suggest a timing issue 
> whereby the exception in 5) prevents the flush in 3) from marking all the 
> segments flushed, causing them to replay after restart.
> In case it might be relevant, I did an online change of compaction strategy 
> from Leveled to SizeTiered during the uptime period preceding this drain.
> [1] Isn't commitlog replay not supposed to automatically trigger a flush in 
> modern cassandra?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (CASSANDRA-4807) Compaction progress counts more than 100%

2012-10-15 Thread Omid Aladini (JIRA)
Omid Aladini created CASSANDRA-4807:
---

 Summary: Compaction progress counts more than 100%
 Key: CASSANDRA-4807
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4807
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.1.6
Reporter: Omid Aladini
Priority: Minor


'nodetool compactionstats' compaction progress counts more than 100%:

{code}
pending tasks: 74
  compaction typekeyspace   column family bytes compacted 
bytes total  progress
   ValidationKSPCF1   56192578305 
8465276891766.38%
   CompactionKSPCF2   162018591   
119913592 135.11%
{code}

Hadn't experienced this before 1.1.3. Is it due to changes in 1.1.4-1.1.6 ?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4798) ValidationExecutor throws StackOverflowError during repair with LCS

2012-10-12 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4798:
-

Great! Thanks!

> ValidationExecutor throws StackOverflowError during repair with LCS
> ---
>
> Key: CASSANDRA-4798
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4798
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.4
> Environment: JRE 1.6.0_31 on Debian Squeeze
>Reporter: Omid Aladini
>
> During a repair, I get StackOverflowError originating from 
> ValidationExecutor. All CFs had been offline-scrubed after CASSANDRA-4411 fix.
> {code}
> 2012-10-12_13:07:39.12921 ERROR 13:07:39,120 Exception in thread 
> Thread[ValidationExecutor:2,1,main]
> 2012-10-12_13:07:39.12929 java.lang.StackOverflowError
> 2012-10-12_13:07:39.12934 at 
> sun.nio.cs.US_ASCII$Encoder.encodeLoop(Unknown Source)
> 2012-10-12_13:07:39.12942 at 
> java.nio.charset.CharsetEncoder.encode(Unknown Source)
> 2012-10-12_13:07:39.12950 at 
> java.lang.StringCoding$StringEncoder.encode(Unknown Source)
> 2012-10-12_13:07:39.12958 at java.lang.StringCoding.encode(Unknown Source)
> 2012-10-12_13:07:39.12964 at java.lang.String.getBytes(Unknown Source)
> 2012-10-12_13:07:39.12969 at java.io.RandomAccessFile.open(Native Method)
> 2012-10-12_13:07:39.12976 at java.io.RandomAccessFile.(Unknown 
> Source)
> 2012-10-12_13:07:39.12981 at 
> org.apache.cassandra.io.util.RandomAccessReader.(RandomAccessReader.java:67)
> 2012-10-12_13:07:39.12990 at 
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.(CompressedRandomAccessReader.java:64)
> 2012-10-12_13:07:39.13003 at 
> org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:46)
> 2012-10-12_13:07:39.13014 at 
> org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1007)
> 2012-10-12_13:07:39.13024 at 
> org.apache.cassandra.io.sstable.SSTableScanner.(SSTableScanner.java:56)
> 2012-10-12_13:07:39.13032 at 
> org.apache.cassandra.io.sstable.SSTableBoundedScanner.(SSTableBoundedScanner.java:41)
> 2012-10-12_13:07:39.13043 at 
> org.apache.cassandra.io.sstable.SSTableReader.getDirectScanner(SSTableReader.java:869)
> 2012-10-12_13:07:39.13053 at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:247)
> 2012-10-12_13:07:39.13066 at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:240)
> // More stack recursion goes here
> 2012-10-12_13:07:39.25061 at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:240)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Created] (CASSANDRA-4798) ValidationExecutor throws StackOverflowError during repair with LCS

2012-10-12 Thread Omid Aladini (JIRA)
Omid Aladini created CASSANDRA-4798:
---

 Summary: ValidationExecutor throws StackOverflowError during 
repair with LCS
 Key: CASSANDRA-4798
 URL: https://issues.apache.org/jira/browse/CASSANDRA-4798
 Project: Cassandra
  Issue Type: Bug
  Components: Core
Affects Versions: 1.1.4
 Environment: JRE 1.6.0_31 on Debian Squeeze
Reporter: Omid Aladini


During a repair, I get StackOverflowError originating from ValidationExecutor. 
All CFs had been offline-scrubed after CASSANDRA-4411 fix.

{code}
2012-10-12_13:07:39.12921 ERROR 13:07:39,120 Exception in thread 
Thread[ValidationExecutor:2,1,main]
2012-10-12_13:07:39.12929 java.lang.StackOverflowError
2012-10-12_13:07:39.12934   at 
sun.nio.cs.US_ASCII$Encoder.encodeLoop(Unknown Source)
2012-10-12_13:07:39.12942   at 
java.nio.charset.CharsetEncoder.encode(Unknown Source)
2012-10-12_13:07:39.12950   at 
java.lang.StringCoding$StringEncoder.encode(Unknown Source)
2012-10-12_13:07:39.12958   at java.lang.StringCoding.encode(Unknown Source)
2012-10-12_13:07:39.12964   at java.lang.String.getBytes(Unknown Source)
2012-10-12_13:07:39.12969   at java.io.RandomAccessFile.open(Native Method)
2012-10-12_13:07:39.12976   at java.io.RandomAccessFile.(Unknown 
Source)
2012-10-12_13:07:39.12981   at 
org.apache.cassandra.io.util.RandomAccessReader.(RandomAccessReader.java:67)
2012-10-12_13:07:39.12990   at 
org.apache.cassandra.io.compress.CompressedRandomAccessReader.(CompressedRandomAccessReader.java:64)
2012-10-12_13:07:39.13003   at 
org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:46)
2012-10-12_13:07:39.13014   at 
org.apache.cassandra.io.sstable.SSTableReader.openDataReader(SSTableReader.java:1007)
2012-10-12_13:07:39.13024   at 
org.apache.cassandra.io.sstable.SSTableScanner.(SSTableScanner.java:56)
2012-10-12_13:07:39.13032   at 
org.apache.cassandra.io.sstable.SSTableBoundedScanner.(SSTableBoundedScanner.java:41)
2012-10-12_13:07:39.13043   at 
org.apache.cassandra.io.sstable.SSTableReader.getDirectScanner(SSTableReader.java:869)
2012-10-12_13:07:39.13053   at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:247)
2012-10-12_13:07:39.13066   at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:240)
// More stack recursion goes here
2012-10-12_13:07:39.25061   at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:240)
{code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4644) Compaction error with Cassandra 1.1.4 and LCS

2012-09-14 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4644:
-

@Jonathan: compaction strategy was changed to LCS after upgrading to 1.1.4 so 
it looks like even with CASSANDRA-4321/CASSANDRA-4411 fixes, sstables are 
wrongly leveled. The patch will temporarily work around the problem though.

{quote}
since you already scrubbed online which will catch out-of-order within an 
sstable
{quote}

Online scrub can't fix out-of-order sstables since they won't be loaded in the 
first place, unless it has somehow has escaped from IntervalTree's assumption 
that sstables are sorted. [1]


[1] 
https://issues.apache.org/jira/browse/CASSANDRA-4321?focusedCommentId=13295704&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-13295704

> Compaction error with Cassandra 1.1.4 and LCS 
> --
>
> Key: CASSANDRA-4644
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4644
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.4
> Environment: Cassandra 1.1.4, Ubuntu Lucid (2.6.32-346), Amazon EC2 
> m1.xlarge
>Reporter: Rudolf VanderLeeden
>Assignee: Jonathan Ellis
> Attachments: 4644.txt
>
>
> In our 1.1.4 testcluster of 3 nodes with RF=3, KS=1, and CF=5, we are getting 
> an asserting error when running 'nodetool compact highscores leaderboard'. 
> This stops compactions on the 'leaderboard' CF summing up to 11835 pending 
> compactions. This error is seen only one one node. 
> The SSTables have originally been created on a 1.1.2 cluster with STCS and 
> then copied to the testcluster also with 1.1.2. Repair, cleanup, compact were 
> OK with STCS. Next, we changed to LCS and did again repair, cleanup, compact 
> with success. 
> Then we started to use this LCS-based testcluster intensively and created 
> lots of data and also large keys with millions of columns. 
> The assertion error in system.log :
>  INFO [CompactionExecutor:8] 2012-09-11 14:20:45,043 
> CompactionController.java (line 172) Compacting large row 
> highscores/leaderboard:4c422d64626331353166372d363464612d343235342d396130322d6535616365343337373532332d676c6f62616c2d30
>  (72589650 bytes) incrementally
> ERROR [CompactionExecutor:8] 2012-09-11 14:20:50,336 
> AbstractCassandraDaemon.java (line 135) Exception in thread 
> Thread[CompactionExecutor:8,1,RMI Runtime]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:992)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:288)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-4417) invalid counter shard detected

2012-09-11 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4417 at 9/12/12 10:18 AM:
---

{quote}
A simple "workaround" is to use batch commit log, but that has a potentially 
important performance impact.
{quote}

I'm a bit confused why batch commit would solve the problem. If cassandra 
crashes before the batch is fsynced, the counter mutations in the batch which 
it was the leader for will still be lost although they might have been applied 
on other replicas. The difference would be that the mutations won't be 
acknowledged to the client, and since counters aren't idempotent, the client 
won't know weather to retry or not. Am I missing something?

  was (Author: omid):
{quote}
A simple "workaround" is to use batch commit log, but that has a potentially 
important performance impact.
{quote}

I'm a bit confused why batch commit would solve the problem. If cassandra 
crashes before the batch is fsynced, the counter mutations which it was the 
leader for will still be lost although they might have been applied on other 
replicas. The difference would be that the mutations won't be acknowledged to 
the client, and since counters aren't idempotent, the client won't know weather 
to retry or not. Am I missing something?
  
> invalid counter shard detected 
> ---
>
> Key: CASSANDRA-4417
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4417
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
> Environment: Amazon Linux
>Reporter: Senthilvel Rangaswamy
>
> Seeing errors like these:
> 2012-07-06_07:00:27.22662 ERROR 07:00:27,226 invalid counter shard detected; 
> (17bfd850-ac52-11e1--6ecd0b5b61e7, 1, 13) and 
> (17bfd850-ac52-11e1--6ecd0b5b61e7, 1, 1) differ only in count; will pick 
> highest to self-heal; this indicates a bug or corruption generated a bad 
> counter shard
> What does it mean ?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4417) invalid counter shard detected

2012-09-11 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4417:
-

{quote}
A simple "workaround" is to use batch commit log, but that has a potentially 
important performance impact.
{quote}

I'm a bit confused why batch commit would solve the problem. If cassandra 
crashes before the batch is fsynced, the counter mutations which it was the 
leader for will still be lost although they might have been applied on other 
replicas. The difference would be that the mutations won't be acknowledged to 
the client, and since counters aren't idempotent, the client won't know weather 
to retry or not. Am I missing something?

> invalid counter shard detected 
> ---
>
> Key: CASSANDRA-4417
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4417
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
> Environment: Amazon Linux
>Reporter: Senthilvel Rangaswamy
>
> Seeing errors like these:
> 2012-07-06_07:00:27.22662 ERROR 07:00:27,226 invalid counter shard detected; 
> (17bfd850-ac52-11e1--6ecd0b5b61e7, 1, 13) and 
> (17bfd850-ac52-11e1--6ecd0b5b61e7, 1, 1) differ only in count; will pick 
> highest to self-heal; this indicates a bug or corruption generated a bad 
> counter shard
> What does it mean ?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4644) Compaction error with Cassandra 1.1.4 and LCS

2012-09-11 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4644:
-

{quote}
The SSTables have originally been created on a 1.1.2 cluster with STCS and then 
copied to the testcluster also with 1.1.2. Repair, cleanup, compact were OK 
with STCS. _Next, we changed to LCS_ and did again repair, cleanup, compact 
with success.
{quote}

Just to double-check, as you mentioned in the mailing list, you upgraded to 
1.1.4 *before* changing to LCS, right?

> Compaction error with Cassandra 1.1.4 and LCS 
> --
>
> Key: CASSANDRA-4644
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4644
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.4
> Environment: Cassandra 1.1.4, Ubuntu Lucid (2.6.32-346), Amazon EC2 
> m1.xlarge
>Reporter: Rudolf VanderLeeden
>
> In our 1.1.4 testcluster of 3 nodes with RF=3, KS=1, and CF=5, we are getting 
> an asserting error when running 'nodetool compact highscores leaderboard'. 
> This stops compactions on the 'leaderboard' CF summing up to 11835 pending 
> compactions. This error is seen only one one node. 
> The SSTables have originally been created on a 1.1.2 cluster with STCS and 
> then copied to the testcluster also with 1.1.2. Repair, cleanup, compact were 
> OK with STCS. Next, we changed to LCS and did again repair, cleanup, compact 
> with success. 
> Then we started to use this LCS-based testcluster intensively and created 
> lots of data and also large keys with millions of columns. 
> The assertion error in system.log :
>  INFO [CompactionExecutor:8] 2012-09-11 14:20:45,043 
> CompactionController.java (line 172) Compacting large row 
> highscores/leaderboard:4c422d64626331353166372d363464612d343235342d396130322d6535616365343337373532332d676c6f62616c2d30
>  (72589650 bytes) incrementally
> ERROR [CompactionExecutor:8] 2012-09-11 14:20:50,336 
> AbstractCassandraDaemon.java (line 135) Exception in thread 
> Thread[CompactionExecutor:8,1,RMI Runtime]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:992)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:288)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-4644) Compaction error with Cassandra 1.1.4 and LCS

2012-09-11 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4644 at 9/12/12 2:26 AM:
--

{quote}
The SSTables have originally been created on a 1.1.2 cluster with STCS and then 
copied to the testcluster also with 1.1.2. Repair, cleanup, compact were OK 
with STCS. _Next, we changed to LCS_ and did again repair, cleanup, compact 
with success.
{quote}

Just to double-check, as you mentioned in the mailing list, you upgraded to 
1.1.4 before changing to LCS, right?

  was (Author: omid):
{quote}
The SSTables have originally been created on a 1.1.2 cluster with STCS and then 
copied to the testcluster also with 1.1.2. Repair, cleanup, compact were OK 
with STCS. _Next, we changed to LCS_ and did again repair, cleanup, compact 
with success.
{quote}

Just to double-check, as you mentioned in the mailing list, you upgraded to 
1.1.4 *before* changing to LCS, right?
  
> Compaction error with Cassandra 1.1.4 and LCS 
> --
>
> Key: CASSANDRA-4644
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4644
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.4
> Environment: Cassandra 1.1.4, Ubuntu Lucid (2.6.32-346), Amazon EC2 
> m1.xlarge
>Reporter: Rudolf VanderLeeden
>
> In our 1.1.4 testcluster of 3 nodes with RF=3, KS=1, and CF=5, we are getting 
> an asserting error when running 'nodetool compact highscores leaderboard'. 
> This stops compactions on the 'leaderboard' CF summing up to 11835 pending 
> compactions. This error is seen only one one node. 
> The SSTables have originally been created on a 1.1.2 cluster with STCS and 
> then copied to the testcluster also with 1.1.2. Repair, cleanup, compact were 
> OK with STCS. Next, we changed to LCS and did again repair, cleanup, compact 
> with success. 
> Then we started to use this LCS-based testcluster intensively and created 
> lots of data and also large keys with millions of columns. 
> The assertion error in system.log :
>  INFO [CompactionExecutor:8] 2012-09-11 14:20:45,043 
> CompactionController.java (line 172) Compacting large row 
> highscores/leaderboard:4c422d64626331353166372d363464612d343235342d396130322d6535616365343337373532332d676c6f62616c2d30
>  (72589650 bytes) incrementally
> ERROR [CompactionExecutor:8] 2012-09-11 14:20:50,336 
> AbstractCassandraDaemon.java (line 135) Exception in thread 
> Thread[CompactionExecutor:8,1,RMI Runtime]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:992)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:288)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
> at java.util.concurrent.FutureTask.run(FutureTask.java:138)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
> at java.lang.Thread.run(Thread.java:662)

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4639) Incorrect counter values

2012-09-11 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4639:
-

This looks the same as CASSANDRA-4417 which is currently in progress.

> Incorrect counter values
> 
>
> Key: CASSANDRA-4639
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4639
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
> Environment: We've got a production 21 node cluster with 3 virtual 
> data centers. Keyspace that contains counter column families has a 
> replication 3 on DC1 and 1 on DC3. DC1 is using SSD drives, DC3 spinning hard 
> drives. We are using Ubuntu Server as an OS. Machines have 24GB of RAM. 
>Reporter: Bartłomiej Romański
>Priority: Critical
>
> Since yesterday almost all counters are incorrect. Usually about 4-5 times 
> higher than expected. In logs we've got this message:
> ERROR [MutationStage:15] 2012-09-10 13:47:13,280 CounterContext.java (line 
> 381) invalid counter shard detected; (6de8e940-dd23-11e1--5233df6faaff, 
> 7, 242) and (6de8e940-dd23-11e1--5233df6faaff, 7, 392) differ only in 
> count; will pick highest to self-heal; this indicates a bug or corruption 
> generated a bad counter shard
> every couple of seconds.
> This cluster was running without any serious problems for at least 2 months.
> Any ideas?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Commented] (CASSANDRA-4602) Stack Size on Sun JVM 1.6.0_33 must be at least 160k

2012-09-06 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4602:
-

Apparently this is due to HotSpot fix 7059899 [1] on 1.6.0_34 that increased 
"StackShadowPages"'s default to 20, since a change in socketWrite's native 
implementation required more stack space. Increased StackShadowPages might 
require increased stack size (-Xss) [2] so that upon a call to a native method, 
there would be at least "StackShadowPages" stack space available.

[1] http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=7059899
[2] http://www.oracle.com/technetwork/java/javase/crashes-137240.html 

> Stack Size on Sun JVM 1.6.0_33 must be at least 160k
> 
>
> Key: CASSANDRA-4602
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4602
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Ubuntu 10.04 
> java version "1.6.0_35"
> Java(TM) SE Runtime Environment (build 1.6.0_35-b10)
> Java HotSpot(TM) 64-Bit Server VM (build 20.10-b01, mixed mode)
>Reporter: Aaron Morton
>Assignee: Jonathan Ellis
> Fix For: 1.0.12, 1.1.5
>
> Attachments: 4602.txt
>
>
> I started a fresh Cassandra 1.1.4 install with Sun JVM 1.6.35.
> On startup I got this in output.log
> {noformat}
> The stack size specified is too small, Specify at least 160k
> Cannot create Java VM
> Service exit with a return value of 1
> {noformat}
> Remembering CASSANDRA-4275 I monkeyed around and started the JVM with 
> -Xss160k the same as Java 7. I then got
> {code:java}
> ERROR [WRITE-/192.168.1.12] 2012-08-31 01:43:29,865 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[WRITE-/192.168.1.12,5,main]
> java.lang.StackOverflowError
>   at java.net.SocketOutputStream.socketWrite0(Native Method)
>   at java.net.SocketOutputStream.socketWrite(Unknown Source)
>   at java.net.SocketOutputStream.write(Unknown Source)
>   at java.io.BufferedOutputStream.flushBuffer(Unknown Source)
>   at java.io.BufferedOutputStream.flush(Unknown Source)
>   at java.io.DataOutputStream.flush(Unknown Source)
>   at 
> org.apache.cassandra.net.OutboundTcpConnection.writeConnected(OutboundTcpConnection.java:156)
>   at 
> org.apache.cassandra.net.OutboundTcpConnection.run(OutboundTcpConnection.java:126)
> {code}
> Same as CASSANDRA-4442
> At which point I dropped back to Java 6.33. 
> CASSANDRA-4457 bumped the stack size to 180 for java 7, should we also do 
> this for Java 6.33+ ?

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira


[jira] [Comment Edited] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-09 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4411 at 8/9/12 4:28 PM:
-

Thanks!

There are two more off-by-ones. 

One is the Scrubber to detect out-of-order keys (similar to the one already 
patched). 

The other one is in manifestCheck to send overlapping sstables back to L0 
(which has causes assertion errors in LeveledManifest::promote)

Patch is attached.

  was (Author: omid):
There are two more off-by-ones. 

One is the Scrubber to detect out-of-order keys (similar to the one already 
patched). 

The other one is in manifestCheck to send overlapping sstables back to L0 
(which has causes assertion errors in LeveledManifest::promote)
  
> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 
> 0001-Fix-off-by-one-for-out-of-order-and-overlapping-ssta.patch, 
> 4411-followup.txt, 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-09 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-4411:


Attachment: 0001-Fix-off-by-one-for-out-of-order-and-overlapping-ssta.patch

There are two more off-by-ones. 

One is the Scrubber to detect out-of-order keys (similar to the one already 
patched). 

The other one is in manifestCheck to send overlapping sstables back to L0 
(which has causes assertion errors in LeveledManifest::promote)

> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 
> 0001-Fix-off-by-one-for-out-of-order-and-overlapping-ssta.patch, 
> 4411-followup.txt, 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Comment Edited] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-08 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4411 at 8/8/12 7:03 PM:
-

Not sure, but from the fact that the two keys are identical and the "Last 
written key"-check checks for greater-or-equal, shouldn't the ">" be ">=" in:

https://github.com/apache/cassandra/blob/cassandra-1.1/src/java/org/apache/cassandra/db/compaction/Scrubber.java#L178

?

  was (Author: omid):
Not sure, but from the fact that the two keys are identical, shouldn't the 
">" be ">=" in:

https://github.com/apache/cassandra/blob/cassandra-1.1/src/java/org/apache/cassandra/db/compaction/Scrubber.java#L178

?
  
> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
> 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-08 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4411:
-

Not sure, but from the fact that the two keys are identical, shouldn't the ">" 
be ">=" in:

https://github.com/apache/cassandra/blob/cassandra-1.1/src/java/org/apache/cassandra/db/compaction/Scrubber.java#L178

?

> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
> 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-08 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4411:
-

I can confirm that the problem is still there. I offline-scrubbed using 1.1.3 
(sstables were generated by 1.1.0) , but the scrubber did not report any 
out-of-order sstables, but sent some sstables back to L0. On compaction though, 
I get the exception:

{quote}
2012-08-08_18:15:41.85260 java.lang.RuntimeException: Last written key 
DecoratedKey(135076574692378869287086649376333921820, SOME_KEY_1) >= current 
key DecoratedKey(135076574692378869287086649376333921820, SOME_KEY_1) writing 
into /var/lib/cassandra/abcd/data/KSP/CF1/KSP-CF1-tmp-he-178793-Data.db
2012-08-08_18:15:41.85303   at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
2012-08-08_18:15:41.85314   at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
2012-08-08_18:15:41.85326   at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
2012-08-08_18:15:41.85338   at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
2012-08-08_18:15:41.85351   at 
org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:288)
2012-08-08_18:15:41.85364   at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
2012-08-08_18:15:41.85375   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-08-08_18:15:41.85385   at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-08-08_18:15:41.85395   at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-08-08_18:15:41.85403   at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-08-08_18:15:41.85414   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-08-08_18:15:41.85424   at java.lang.Thread.run(Unknown Source)
{quote}



> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
> 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please co

[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-08-06 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4411:
-

@Mina were the sstables created after CASSANDRA-4321 patch? Otherwise 
offline-scrub with --manifest-check is unlikely to solve the problem (or at 
least I don't understand how) since there would still be out-of-order sstables 
existing.

> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
> 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-18 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4411:
-

Awesome. Thanks for the patch. Tested it and it works.

Sylvain, regarding your earlier comment on CASSANDRA-4321:

{quote} This is not really a new bug, but I believe that prior to 
CASSANDRA-4142, this had less consequences. {quote}

Does it mean LC-compacted SSTables created by 1.1.0 or earlier are as well 
affected and need to be scrubbed?

> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 4411-followup.txt, 
> 4411.txt, assertion-w-more-debugging-info-omid.log, 
> assertion.moreinfo.system.log, system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Comment Edited] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-15 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4411 at 7/15/12 1:52 PM:
--

Attached the log (assertion-w-more-debugging-info-omid.log) with more debugging 
info that leads to AssertionError on LeveledManifest.promote.

  was (Author: omid):
Log with more debugging that leads to AssertionError on 
LeveledManifest.promote.
  
> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 
> assertion-w-more-debugging-info-omid.log, assertion.moreinfo.system.log, 
> system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Updated] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-15 Thread Omid Aladini (JIRA)

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

Omid Aladini updated CASSANDRA-4411:


Attachment: assertion-w-more-debugging-info-omid.log

Log with more debugging that leads to AssertionError on LeveledManifest.promote.

> Assertion with LCS compaction
> -
>
> Key: CASSANDRA-4411
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4411
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.2
>Reporter: Anton Winter
> Fix For: 1.1.3
>
> Attachments: 0001-Add-debugging-info-for-LCS.txt, 
> assertion-w-more-debugging-info-omid.log, assertion.moreinfo.system.log, 
> system.log
>
>
> As instructed in CASSANDRA-4321 I have raised this issue as a continuation of 
> that issue as it appears the problem still exists.
> I have repeatedly run sstablescrub across all my nodes after the 1.1.2 
> upgrade until sstablescrub shows no errors.  The exceptions described in 
> CASSANDRA-4321 do not occur as frequently now but the integrity check still 
> throws exceptions on a number of nodes.  Once those exceptions occur 
> compactionstats shows a large number of pending tasks with no progression 
> afterwards.
> {code}
> ERROR [CompactionExecutor:150] 2012-07-05 04:26:15,570 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:150,1,main]
> java.lang.AssertionError
> at 
> org.apache.cassandra.db.compaction.LeveledManifest.promote(LeveledManifest.java:214)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionStrategy.handleNotification(LeveledCompactionStrategy.java:158)
> at 
> org.apache.cassandra.db.DataTracker.notifySSTablesChanged(DataTracker.java:531)
> at 
> org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:254)
> at 
> org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:978)
> at 
> org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:200)
> at 
> org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
> at 
> org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
> at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
> at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334)
> at java.util.concurrent.FutureTask.run(FutureTask.java:166)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
> at java.lang.Thread.run(Thread.java:636)
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira




[jira] [Commented] (CASSANDRA-4411) Assertion with LCS compaction

2012-07-11 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4411:
-

I could also reproduce it from the data I had:

{code}
DEBUG 12:47:37,353 adding /home/omid/data/KSP/CF/KSP-CF-hd-121136 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,354 adding /home/omid/data/KSP/CF/KSP-CF-hd-121137 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,354 adding /home/omid/data/KSP/CF/KSP-CF-hd-121138 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,354 adding /home/omid/data/KSP/CF/KSP-CF-hd-121139 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,354 adding /home/omid/data/KSP/CF/KSP-CF-hd-121140 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,354 adding /home/omid/data/KSP/CF/KSP-CF-hd-121141 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,354 adding /home/omid/data/KSP/CF/KSP-CF-hd-121142 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,355 adding /home/omid/data/KSP/CF/KSP-CF-hd-121143 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,355 adding /home/omid/data/KSP/CF/KSP-CF-hd-121144 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,355 adding /home/omid/data/KSP/CF/KSP-CF-hd-121145 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,355 adding /home/omid/data/KSP/CF/KSP-CF-hd-121146 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,355 adding /home/omid/data/KSP/CF/KSP-CF-hd-121147 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,355 adding /home/omid/data/KSP/CF/KSP-CF-hd-121148 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,355 adding /home/omid/data/KSP/CF/KSP-CF-hd-121149 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,356 adding /home/omid/data/KSP/CF/KSP-CF-hd-121150 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,356 adding /home/omid/data/KSP/CF/KSP-CF-hd-121151 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,356 adding /home/omid/data/KSP/CF/KSP-CF-hd-121152 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,356 adding /home/omid/data/KSP/CF/KSP-CF-hd-121153 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,356 adding /home/omid/data/KSP/CF/KSP-CF-hd-121154 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,356 adding /home/omid/data/KSP/CF/KSP-CF-hd-121155 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,356 adding /home/omid/data/KSP/CF/KSP-CF-hd-121156 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,357 adding /home/omid/data/KSP/CF/KSP-CF-hd-121157 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,357 adding /home/omid/data/KSP/CF/KSP-CF-hd-121158 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,357 adding /home/omid/data/KSP/CF/KSP-CF-hd-121159 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,357 adding /home/omid/data/KSP/CF/KSP-CF-hd-121160 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,357 adding /home/omid/data/KSP/CF/KSP-CF-hd-121161 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,357 GC for ParNew: 14 ms for 1 collections, 5438330152 used; max 
is 8506048512
DEBUG 12:47:37,357 adding /home/omid/data/KSP/CF/KSP-CF-hd-121162 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,358 adding /home/omid/data/KSP/CF/KSP-CF-hd-121163 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,358 adding /home/omid/data/KSP/CF/KSP-CF-hd-121164 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,358 adding /home/omid/data/KSP/CF/KSP-CF-hd-121165 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,358 adding /home/omid/data/KSP/CF/KSP-CF-hd-121166 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,358 adding /home/omid/data/KSP/CF/KSP-CF-hd-121167 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,358 adding /home/omid/data/KSP/CF/KSP-CF-hd-121168 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,358 adding /home/omid/data/KSP/CF/KSP-CF-hd-121169 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,359 adding /home/omid/data/KSP/CF/KSP-CF-hd-121170 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,359 adding /home/omid/data/KSP/CF/KSP-CF-hd-121171 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,359 adding /home/omid/data/KSP/CF/KSP-CF-hd-121172 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,359 adding /home/omid/data/KSP/CF/KSP-CF-hd-121173 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,359 adding /home/omid/data/KSP/CF/KSP-CF-hd-121174 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,359 adding /home/omid/data/KSP/CF/KSP-CF-hd-121175 to list of 
files tracked for KSP.CF
DEBUG 12:47:37,359 removing /home/omid/data/KSP/CF/KSP-CF-hd-121068 from list 
of files tracked for KSP.CF
DEBUG 12:47:37,360 Marking /home/omid/data/KSP/CF/KSP-CF-hd-121068-Data.db 
compacted
DEBUG 12:47:37,360 All segments have been unmapped successfully
DEBUG 12:47:37,360 removing /home/omid/data/KSP/CF/KSP-CF-hd-120844 from list 
of files tracked for KSP.CF
DEBUG 12:47:37,360 Marking /home/omid/data/KSP/CF/KSP-CF-hd-120844-Data.db 
compacted
DEBUG 12:47:37,

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-22 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

So I had offline-scrubbed the live Cassandra node and I had copied the sstables 
that participated in one of the failed compaction. Assuming the sstables had 
been offline-scrubbed, I had skipped step 2 above locally, so unfortunately I 
can't yet reproduce it locally with a limited set of data.

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v5.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v5.txt, 
> 0003-Create-standalone-scrub-v5.txt, ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.In

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-21 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

I experienced the same as Anton's. One observation is that the out-of-order key 
being wrongly iterated by CompactionIterable's MergeIterator which causes the 
exception, happen to be the start of an interval:

DEBUG 18:10:41,693 Creating IntervalNode from [... 
Interval(DecoratedKey(33736808147257072541807562080745136438, ... ), 

which leads me to suspect if it's due to the "Range" (vs. Bounds) used in 
LeveledCompactionStrategy::getScanners. Any ideas?

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v5.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v5.txt, 
> 0003-Create-standalone-scrub-v5.txt, ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNo

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-20 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

The above error is due to the %d in StandaloneScrubber.java:179's 
interpolation. Will fix and try again.

Jonathan: Sun Java 6, 1.6.0_26

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v5.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v5.txt, 
> 0003-Create-standalone-scrub-v5.txt, ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalTree.(IntervalTree.java:39)
> at 
> org.

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-20 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Got "java.lang.OutOfMemoryError: Java heap space" with -Xmx256M.

Tried with -Xmx512M and the scrub failed with:

{code}
Checking leveled manifest
d != org.apache.cassandra.io.sstable.SSTableReader
java.util.IllegalFormatConversionException: d != 
org.apache.cassandra.io.sstable.SSTableReader
at 
java.util.Formatter$FormatSpecifier.failConversion(Formatter.java:3999)
at java.util.Formatter$FormatSpecifier.printInteger(Formatter.java:2709)
at java.util.Formatter$FormatSpecifier.print(Formatter.java:2661)
at java.util.Formatter.format(Formatter.java:2433)
at java.util.Formatter.format(Formatter.java:2367)
at java.lang.String.format(String.java:2769)
at 
org.apache.cassandra.tools.StandaloneScrubber.checkManifest(StandaloneScrubber.java:179)
at 
org.apache.cassandra.tools.StandaloneScrubber.main(StandaloneScrubber.java:148)
{code}

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v5.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v5.txt, 
> 0003-Create-standalone-scrub-v5.txt, ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-20 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Will try it again. 
LeveledCompactionStrategyTest:testValidationMultipleSSTablePerLevel fails 
because of junit timeout when I run it together with all other suits, but 
passes when I only run LeveledCompactionStrategyTest suit. Is it related?
{code}
[junit] Testsuite: 
org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest
[junit] Tests run: 1, Failures: 0, Errors: 1, Time elapsed: 0 sec
[junit] 
[junit] Testcase: 
org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest:testValidationMultipleSSTablePerLevel:
   Caused an ERROR
[junit] Timeout occurred. Please note the time in the report does not 
reflect the time until the timeout.
[junit] junit.framework.AssertionFailedError: Timeout occurred. Please note 
the time in the report does not reflect the time until the timeout.
[junit] 
[junit] 
[junit] Test 
org.apache.cassandra.db.compaction.LeveledCompactionStrategyTest FAILED 
(timeout)
{code}

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v5.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v5.txt, 
> 0003-Create-standalone-scrub-v5.txt, ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apa

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-20 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

The exceptions happens not quickly afterwards but after some rounds of 
compaction on the CF. I had re-bootstrapped so there are tons of ~(1500) 
pending compaction tasks. If I restart the node and compact the problem happens 
again and I can reproduce it. I'll send you an email about the data.

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, 
> 0003-Create-standalone-scrub-v3.txt, 0003-Create-standalone-scrub-v4.txt, 
> ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> 

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-19 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Let me know if I can provide more data.

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, 
> 0003-Create-standalone-scrub-v3.txt, 0003-Create-standalone-scrub-v4.txt, 
> ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalTree.(IntervalTree.java:39)
> at 
> org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTrac

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-19 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Exactly.

- Applied v3
- Ran offline scrub and it failed because of tmp files.
- Started Cassandra and saw failures.
- Applied v4 to cassandra-1.1 branch.
- Ran offline scrub successfully.
- Started Cassandra successfully.
- Compaction failed because of above error.

All done on the same instance.

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, 
> 0003-Create-standalone-scrub-v3.txt, 0003-Create-standalone-scrub-v4.txt, 
> ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
>  

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-19 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Tried v4 patch and offline scrub went through completely. Cassandra started 
without any error but compaction halted again:

{code}
2012-06-19_14:01:03.47432  INFO 14:01:03,474 Compacting 
[SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-67792-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-65607-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-63279-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-65491-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-68332-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-64720-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-65322-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-66557-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-64504-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-68179-Data.db'),
 
SSTableReader(path='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-65005-Data.db')]
2012-06-19_14:01:08.73528 ERROR 14:01:08,733 Exception in thread 
Thread[CompactionExecutor:11,1,main]
2012-06-19_14:01:08.73538 java.lang.RuntimeException: Last written key 
DecoratedKey(42351003983459534782466386414991462257, 
313632303432347c3130303632313432) >= current key 
DecoratedKey(38276735926421753773204634663641518108, 
31343638373735327c3439343837333932) writing into 
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-tmp-hd-68399-Data.db
2012-06-19_14:01:08.73572   at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
2012-06-19_14:01:08.73581   at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
2012-06-19_14:01:08.73590   at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
2012-06-19_14:01:08.73600   at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
2012-06-19_14:01:08.73611   at 
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
2012-06-19_14:01:08.73622   at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
2012-06-19_14:01:08.73633   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-19_14:01:08.73642   at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-06-19_14:01:08.73650   at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-06-19_14:01:08.73657   at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-06-19_14:01:08.73665   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-06-19_14:01:08.73672   at java.lang.Thread.run(Unknown Source)
{code}

All SSTables that participated in compaction were new ones written by the 
offline scrub (according their timestamp and also id range.) although the first 
one didn't exist any more (already promoted before the exception?)

{quote}This is not really a new bug, but I believe that prior to 
CASSANDRA-4142, *this had less consequences*.{quote}

Sylvain, could you please elaborate on this? I'd like to know how pre-1.1.1 
data is affected by the Range-vs-Bounds bug. Only overlapping/duplicate 
sstables on the same level leading to slower reads caused by unneeded sstable 
lookups?


> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows-v3.txt, 
> 0003-Create-standalone-scrub-v3.txt, 0003-Create-standalone-scrub-v4.txt, 
> ooyala-hastur-stacktrace.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found

[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-18 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4321 at 6/18/12 9:52 PM:
--

Thanks for the patch. Offline scrub is indeed very useful.

Tried the v3 patches and the scrub didn't complete, possibly because of a 
different issue, with the following failed assertion:

{code}
Exception in thread "main" java.lang.AssertionError: Unexpected empty index 
file: 
RandomAccessReader(filePath='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-tmp-hd-33827-Index.db',
 skipIOCache=true)
at 
org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:221)
at 
org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:376)
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:203)
at 
org.apache.cassandra.io.sstable.SSTableReader.openNoValidation(SSTableReader.java:143)
at 
org.apache.cassandra.tools.StandaloneScrubber.main(StandaloneScrubber.java:79)
{code}

which consequently, encountered corrupt SSTables during start-up:

{code}
2012-06-18_20:36:19.89543  INFO 20:36:19,895 Opening 
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-24984 (1941993 bytes)
2012-06-18_20:36:19.90217 ERROR 20:36:19,900 Exception in thread 
Thread[SSTableBatchOpen:9,5,main]
2012-06-18_20:36:19.90222 java.lang.IllegalStateException: SSTable first key 
DecoratedKey(41255474878128469814942789647212295629, 
31303132393937357c3337313730333536) > last key 
DecoratedKey(41219536226656199861610796307350537953, 
31303234323538397c3331383436373338)
2012-06-18_20:36:19.90261   at 
org.apache.cassandra.io.sstable.SSTableReader.validate(SSTableReader.java:441)
2012-06-18_20:36:19.90275   at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:208)
2012-06-18_20:36:19.90291   at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:153)
2012-06-18_20:36:19.90309   at 
org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:245)
2012-06-18_20:36:19.90324   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-18_20:36:19.90389   at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-06-18_20:36:19.90391   at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-06-18_20:36:19.90391   at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-06-18_20:36:19.90392   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-06-18_20:36:19.90392   at java.lang.Thread.run(Unknown Source)
{code}

although didn't prevent Cassandra from starting up, but compaction failed 
subsequently:

{code}
2012-06-18_20:51:41.79122 ERROR 20:51:41,790 Exception in thread 
Thread[CompactionExecutor:81,1,main]
2012-06-18_20:51:41.79131 java.lang.RuntimeException: Last written key 
DecoratedKey(12341204629749023303706929560940823070, 33363037353338) >= current 
key DecoratedKey(12167298275958419273792070792442127650, 31363431343537) 
writing into 
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-tmp-hd-40992-Data.db
2012-06-18_20:51:41.79161   at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
2012-06-18_20:51:41.79169   at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
2012-06-18_20:51:41.79180   at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
2012-06-18_20:51:41.79189   at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
2012-06-18_20:51:41.79199   at 
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
2012-06-18_20:51:41.79210   at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
2012-06-18_20:51:41.79218   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-18_20:51:41.79227   at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-06-18_20:51:41.79235   at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-06-18_20:51:41.79242   at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-06-18_20:51:41.79250   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-06-18_20:51:41.79259   at java.lang.Thread.run(Unknown Source)
{code}

  was (Author: omid):
Thanks for the patch. Offline scrub is indeed very useful.

Tried the v3 patches and the scrub didn't complete, possibly because of a 
different issue, with the following failed assertion:

{code}
Exception in thread "main" java.lang.AssertionError: Unexpected empty index 
file: 
RandomAccessReader(filePath='/var/lib/cassandra/abcd/data/SOMEKSP/C

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-18 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Thanks for the patch. Offline scrub is indeed very useful.

Tried the v3 patches and the scrub didn't complete, possibly because of a 
different issue, with the following failed assertion:

{code}
Exception in thread "main" java.lang.AssertionError: Unexpected empty index 
file: 
RandomAccessReader(filePath='/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-tmp-hd-33827-Index.db',
 skipIOCache=true)
at 
org.apache.cassandra.io.sstable.SSTable.estimateRowsFromIndex(SSTable.java:221)
at 
org.apache.cassandra.io.sstable.SSTableReader.load(SSTableReader.java:376)
at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:203)
at 
org.apache.cassandra.io.sstable.SSTableReader.openNoValidation(SSTableReader.java:143)
at 
org.apache.cassandra.tools.StandaloneScrubber.main(StandaloneScrubber.java:79)
{code}

which consequently, it encountered a corrupt SSTable during start-up:

{code}
2012-06-18_20:36:19.89543  INFO 20:36:19,895 Opening 
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-24984 (1941993 bytes)
2012-06-18_20:36:19.90217 ERROR 20:36:19,900 Exception in thread 
Thread[SSTableBatchOpen:9,5,main]
2012-06-18_20:36:19.90222 java.lang.IllegalStateException: SSTable first key 
DecoratedKey(41255474878128469814942789647212295629, 
31303132393937357c3337313730333536) > last key 
DecoratedKey(41219536226656199861610796307350537953, 
31303234323538397c3331383436373338)
2012-06-18_20:36:19.90261   at 
org.apache.cassandra.io.sstable.SSTableReader.validate(SSTableReader.java:441)
2012-06-18_20:36:19.90275   at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:208)
2012-06-18_20:36:19.90291   at 
org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:153)
2012-06-18_20:36:19.90309   at 
org.apache.cassandra.io.sstable.SSTableReader$1.run(SSTableReader.java:245)
2012-06-18_20:36:19.90324   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-18_20:36:19.90389   at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-06-18_20:36:19.90391   at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-06-18_20:36:19.90391   at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-06-18_20:36:19.90392   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-06-18_20:36:19.90392   at java.lang.Thread.run(Unknown Source)
{code}

although didn't prevent Cassandra from starting up, but compaction failed 
subsequently:

{code}
2012-06-18_20:51:41.79122 ERROR 20:51:41,790 Exception in thread 
Thread[CompactionExecutor:81,1,main]
2012-06-18_20:51:41.79131 java.lang.RuntimeException: Last written key 
DecoratedKey(12341204629749023303706929560940823070, 33363037353338) >= current 
key DecoratedKey(12167298275958419273792070792442127650, 31363431343537) 
writing into 
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-tmp-hd-40992-Data.db
2012-06-18_20:51:41.79161   at 
org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:134)
2012-06-18_20:51:41.79169   at 
org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:153)
2012-06-18_20:51:41.79180   at 
org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:159)
2012-06-18_20:51:41.79189   at 
org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:50)
2012-06-18_20:51:41.79199   at 
org.apache.cassandra.db.compaction.CompactionManager$1.runMayThrow(CompactionManager.java:150)
2012-06-18_20:51:41.79210   at 
org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
2012-06-18_20:51:41.79218   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-18_20:51:41.79227   at 
java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
2012-06-18_20:51:41.79235   at java.util.concurrent.FutureTask.run(Unknown 
Source)
2012-06-18_20:51:41.79242   at 
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
2012-06-18_20:51:41.79250   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
2012-06-18_20:51:41.79259   at java.lang.Thread.run(Unknown Source)
{code}

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>A

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-14 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Scrubbed the column family on a node which had booted up with assertions `on` 
and there were still corrupt sstables.

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>Assignee: Sylvain Lebresne
> Fix For: 1.1.2
>
> Attachments: 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v2.txt, 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping-v3.txt, 
> 0001-Change-Range-Bounds-in-LeveledManifest.overlapping.txt, 
> 0002-Scrub-detects-and-repair-outOfOrder-rows.txt
>
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalTree.(Inte

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-12 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Tried the patch but the server still doesn't start. The StackOverFlow that gets 
thrown causes an already loaded column family to be loaded again:

Load CF1:

{code}
reading saved cache /var/lib/cassandra/abcd/saved_caches/SOMEKSP-CF1-KeyCache
2012-06-12_16:18:04.12387  INFO 16:18:04,123 Opening 
/var/lib/cassandra/abcd/data/SOMEKSP/CF1/SOMEKSP-CF1-hd-2248
...
{code}

Load CF3 which has the corrupted sstables
{code}
2012-06-12_15:31:20.56185  INFO 15:31:20,561 Opening 
/var/lib/cassandra/abcd/data/SOMEKSP/CF3/SOMEKSP-CF3-hd-7924 (2372910 bytes)
2012-06-12_15:31:20.81897 ERROR 15:31:20,811 Exception in thread 
Thread[OptionalTasks:1,5,main]
2012-06-12_15:31:20.81901 java.lang.StackOverflowError
2012-06-12_15:31:20.81901   at 
org.apache.cassandra.db.DecoratedKey.compareTo(DecoratedKey.java:90)
2012-06-12_15:31:20.81906   at 
org.apache.cassandra.db.DecoratedKey.compareTo(DecoratedKey.java:38)
2012-06-12_15:31:20.81918   at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81927   at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81934   at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81940   at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81946   at java.util.Arrays.mergeSort(Unknown Source)
2012-06-12_15:31:20.81954   at java.util.Arrays.sort(Unknown Source)
2012-06-12_15:31:20.81960   at java.util.Collections.sort(Unknown Source)
2012-06-12_15:31:20.81980   at 
org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
2012-06-12_15:31:20.81981   at 
org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
2012-06-12_15:31:20.81990   at 
org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)

// stacktrace goes on

2012-06-12_15:31:20.88633   at 
org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
2012-06-12_15:31:20.88643   at 
org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
2012-06-12_15:31:20.88654   at 
org.apache.cassandra.utils.IntervalTree.IntervalTree.(IntervalTree.java:39)
2012-06-12_15:31:20.88664   at 
org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:560)
2012-06-12_15:31:20.88673   at 
org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:617)
2012-06-12_15:31:20.88683   at 
org.apache.cassandra.db.DataTracker.replace(DataTracker.java:320)
2012-06-12_15:31:20.88692   at 
org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTracker.java:259)
2012-06-12_15:31:20.88702   at 
org.apache.cassandra.db.ColumnFamilyStore.(ColumnFamilyStore.java:234)
2012-06-12_15:31:20.88712   at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:331)
2012-06-12_15:31:20.88723   at 
org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:309)
2012-06-12_15:31:20.88734   at 
org.apache.cassandra.db.Table.initCf(Table.java:367)
2012-06-12_15:31:20.88742   at 
org.apache.cassandra.db.Table.(Table.java:299)
2012-06-12_15:31:20.88750   at 
org.apache.cassandra.db.Table.open(Table.java:114)
2012-06-12_15:31:20.88758   at 
org.apache.cassandra.db.Table.open(Table.java:97)
2012-06-12_15:31:20.88766   at 
org.apache.cassandra.db.Table$2.apply(Table.java:574)
2012-06-12_15:31:20.88773   at 
org.apache.cassandra.db.Table$2.apply(Table.java:571)
2012-06-12_15:31:20.88782   at 
com.google.common.collect.Iterators$8.next(Iterators.java:751)
2012-06-12_15:31:20.88790   at 
org.apache.cassandra.db.ColumnFamilyStore.all(ColumnFamilyStore.java:1625)
2012-06-12_15:31:20.88800   at 
org.apache.cassandra.db.MeteredFlusher.countFlushingBytes(MeteredFlusher.java:118)
2012-06-12_15:31:20.88810   at 
org.apache.cassandra.db.MeteredFlusher.run(MeteredFlusher.java:45)
2012-06-12_15:31:20.88818   at 
org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:79)
2012-06-12_15:31:20.88833   at 
java.util.concurrent.Executors$RunnableAdapter.call(Unknown Source)
2012-06-12_15:31:20.88842   at 
java.util.concurrent.FutureTask$Sync.innerRunAndReset(Unknown Source)
2012-06-12_15:31:20.88851   at 
java.util.concurrent.FutureTask.runAndReset(Unknown Source)
2012-06-12_15:31:20.88860   at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$101(Unknown
 Source)
2012-06-12_15:31:20.88870   at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.runPeriodic(Unknown
 Source)
2012-06-12_15:31:20.2   at 
java.util.concurrent.S

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-11 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

Jonathan, yes I use LeveledCompactionStrategy with non-default 
sstable_size_in_mb = 10

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalTree.(IntervalTree.java:39)
> at 
> org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:560)
> at 
> org.apache.cassandra.db.DataTracker$View.replace(DataTracker.java:617)
> at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:320)
> at 
> org.apache.cassandra.db.DataTracker.addInitialSSTables(DataTracker.java:259)
>   

[jira] [Comment Edited] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-08 Thread Omid Aladini (JIRA)

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

Omid Aladini edited comment on CASSANDRA-4321 at 6/8/12 9:45 PM:
-

We're seeing the same issue after upgrading from 1.0.9 to 1.1.1 on only a 
single node in a 16 node cluster. Wiping the data off and bootstrapping again 
didn't help. Compaction looks to be not progressing (according to 
compactionstats) and I can reproduce this on every "nodetool flush".

  was (Author: omid):
We're see the same issue after upgrading from 1.0.9 to 1.1.1 on only a 
single node in a 16 node cluster. Wiping the data off and bootstrapping again 
didn't help. Compaction looks to be not progressing (according to 
compactionstats) and I can reproduce this on every "nodetool flush".
  
> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.Interva

[jira] [Commented] (CASSANDRA-4321) stackoverflow building interval tree & possible sstable corruptions

2012-06-08 Thread Omid Aladini (JIRA)

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

Omid Aladini commented on CASSANDRA-4321:
-

We're see the same issue after upgrading from 1.0.9 to 1.1.1 on only a single 
node in a 16 node cluster. Wiping the data off and bootstrapping again didn't 
help. Compaction looks to be not progressing (according to compactionstats) and 
I can reproduce this on every "nodetool flush".

> stackoverflow building interval tree & possible sstable corruptions
> ---
>
> Key: CASSANDRA-4321
> URL: https://issues.apache.org/jira/browse/CASSANDRA-4321
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Affects Versions: 1.1.1
>Reporter: Anton Winter
>
> After upgrading to 1.1.1 (from 1.1.0) I have started experiencing 
> StackOverflowError's resulting in compaction backlog and failure to restart. 
> The ring currently consists of 6 DC's and 22 nodes using LCS & compression.  
> This issue was first noted on 2 nodes in one DC and then appears to have 
> spread to various other nodes in the other DC's.  
> When the first occurrence of this was found I restarted the instance but it 
> failed to start so I cleared its data and treated it as a replacement node 
> for the token it was previously responsible for.  This node successfully 
> streamed all the relevant data back but failed again a number of hours later 
> with the same StackOverflowError and again was unable to restart. 
> The initial stack overflow error on a running instance looks like this:
> ERROR [CompactionExecutor:314] 2012-06-07 09:59:43,017 
> AbstractCassandraDaemon.java (line 134) Exception in thread 
> Thread[CompactionExecutor:314,1,main]
> java.lang.StackOverflowError
> at java.util.Arrays.mergeSort(Arrays.java:1157)
> at java.util.Arrays.sort(Arrays.java:1092)
> at java.util.Collections.sort(Collections.java:134)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.findMinMedianMax(IntervalNode.java:114)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:49)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow.  Compactions stop from this point 
> onwards]
> I restarted this failing instance with DEBUG logging enabled and it throws 
> the following exception part way through startup:
> ERROR 11:37:51,046 Exception in thread Thread[OptionalTasks:1,5,main]
> java.lang.StackOverflowError
> at 
> org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:307)
> at 
> org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:276)
> at 
> org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:230)
> at 
> org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:124)
> at 
> org.slf4j.impl.Log4jLoggerAdapter.debug(Log4jLoggerAdapter.java:228)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:45)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> [snip - this repeats until stack overflow]
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:64)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalNode.(IntervalNode.java:62)
> at 
> org.apache.cassandra.utils.IntervalTree.IntervalTree.(IntervalTree.java:39)
> at 
> org.apache.cassandra.db.DataTracker.buildIntervalTree(DataTracker.java:560)
> at 
> org.apache.cassandra.db.DataTracker$View.replace(Dat