[jira] [Commented] (CASSANDRA-6913) Compaction of system keyspaces during startup can cause early loading of non-system keyspaces
[ https://issues.apache.org/jira/browse/CASSANDRA-6913?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13960220#comment-13960220 ] Ravi Prasad commented on CASSANDRA-6913: we were noticing occasional FilenotFoundException due to compaction leftovers at startup on restart, after upgrading to cassandra-2.0 (CASSANDRA-5151). I think this fixes that issue. Would it make sense to change the changes.txt to 'Avoid early loading of non-system keyspaces before compaction-leftovers cleanup at startup' instead of https://github.com/apache/cassandra/blob/56d84a7c028c0498158efb1a3cadea149ab7c1cd/CHANGES.txt#L2 ? Compaction of system keyspaces during startup can cause early loading of non-system keyspaces - Key: CASSANDRA-6913 URL: https://issues.apache.org/jira/browse/CASSANDRA-6913 Project: Cassandra Issue Type: Bug Reporter: Benedict Assignee: Benedict Priority: Minor Fix For: 2.0.7, 2.1 beta2 Attachments: 6913.txt This then can result in an inconsistent CFS state, as cleanup of e.g. compaction leftovers does not get reflected in DataTracker. It happens because StorageService.getLoad() iterates over and opens all CFS, and this is called by Compaction. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6815) Decided if we want to bring back thrift HSHA in 2.0.7
[ https://issues.apache.org/jira/browse/CASSANDRA-6815?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13955630#comment-13955630 ] Ravi Prasad commented on CASSANDRA-6815: Are the disruptor changes suggested by [~kvaster] making into 2.0.7 release? Decided if we want to bring back thrift HSHA in 2.0.7 - Key: CASSANDRA-6815 URL: https://issues.apache.org/jira/browse/CASSANDRA-6815 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Pavel Yaskevich This is the followup of CASSANDRA-6285, to decide what we want to do regarding thrift servers moving forward. My reading of CASSANDRA-6285 suggests that the possible options includes: # bring back the old HSHA implementation from 1.2 as hsha and make the disruptor implementation be disruptor_hsha. # use the new TThreadedSelectorServer from thrift as hsha, making the disruptor implementation disruptor_hsha as above # just wait for Pavel to fix the disruptor implementation for off-heap buffers to switch back to that, keeping on-heap buffer until then. # keep on-heap buffer for the disruptor implementation and do nothing particular. I could be missing some options and we can probably do some mix of those. I don't have a particular opinion to offer on the matter. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6716) nodetool scrub constantly fails with RuntimeException (Tried to hard link to file that does not exist)
[ https://issues.apache.org/jira/browse/CASSANDRA-6716?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13903783#comment-13903783 ] Ravi Prasad commented on CASSANDRA-6716: Do you see the missing file(s) compacted before in the logs ? I've been seeing ocassional FileNotFoundException during compaction running 2.0.5, but not able to reproduce consistently. in our case, the missing file was compacted away, but for some reason they weren't cleaned up, and after restart, looked like the sstable was loaded opening a reference before getting cleaned up by cleanupCompactionleftovers. the issue goes away after another restart for me. Could be related to CASSANDRA-5151 and CASSANDRA-6086. nodetool scrub constantly fails with RuntimeException (Tried to hard link to file that does not exist) -- Key: CASSANDRA-6716 URL: https://issues.apache.org/jira/browse/CASSANDRA-6716 Project: Cassandra Issue Type: Bug Components: Core Environment: Cassandra 2.0.5 (built from source), Linux, 6 nodes, JDK 1.7 Reporter: Nikolai Grigoriev Attachments: system.log.gz It seems that since recently I have started getting a number of exceptions like File not found on all Cassandra nodes. Currently I am getting an exception like this every couple of seconds on each node, for different keyspaces and CFs. I have tried to restart the nodes, tried to scrub them. No luck so far. It seems that scrub cannot complete on any of these nodes, at some point it fails because of the file that it can't find. One one of the nodes currently the nodetool scrub command fails instantly and consistently with this exception: {code} # /opt/cassandra/bin/nodetool scrub Exception in thread main java.lang.RuntimeException: Tried to hard link to file that does not exist /mnt/disk5/cassandra/data/mykeyspace_jmeter/test_contacts/mykeyspace_jmeter-test_contacts-jb-28049-Data.db at org.apache.cassandra.io.util.FileUtils.createHardLink(FileUtils.java:75) at org.apache.cassandra.io.sstable.SSTableReader.createLinks(SSTableReader.java:1215) at org.apache.cassandra.db.ColumnFamilyStore.snapshotWithoutFlush(ColumnFamilyStore.java:1826) at org.apache.cassandra.db.ColumnFamilyStore.scrub(ColumnFamilyStore.java:1122) at org.apache.cassandra.service.StorageService.scrub(StorageService.java:2159) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:75) at sun.reflect.GeneratedMethodAccessor15.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:279) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1487) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:97) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1328) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1420) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:848) at sun.reflect.GeneratedMethodAccessor38.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at
[jira] [Commented] (CASSANDRA-6673) Compaction assertion error on beforeAppend test due to decommision - add node and/or overlapping sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-6673?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13898255#comment-13898255 ] Ravi Prasad commented on CASSANDRA-6673: I think this can be marked as dup of https://issues.apache.org/jira/browse/CASSANDRA-6688. Compaction assertion error on beforeAppend test due to decommision - add node and/or overlapping sstables - Key: CASSANDRA-6673 URL: https://issues.apache.org/jira/browse/CASSANDRA-6673 Project: Cassandra Issue Type: Bug Components: Core Environment: linux, replication factor 1, 4 node, LCS, 3 compaction executors Reporter: Ignace Desimpel Priority: Minor Fix For: 2.0.6 Attachments: beforeAppend.txt Compaction assertion on test for LastWrittenKey compared to current key to write during compaction. This happens after a decommission of a node, and then the node is added again immediately (short time after decommssion). This makes the node shutdown. Restart of node a couple of time finally finishes the bootstrapping. During restarts to finish the bootstrapping there is also a warning message about overlapping sstables -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6622) Streaming session failures during node replace of same address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13895714#comment-13895714 ] Ravi Prasad commented on CASSANDRA-6622: bq. You mean rise above it, so the node is still being convicted? Can you add new logs? Maybe now it actually is the restart event, so trying that patch with 6658 might work. Tried with 6658 patch and 0001-don-t-signal-restart-of-dead-states.txt applied on cassandra-2.0.5 tag. Still see the same thing, where FD convicts the streaming session. I'm attaching the logs(6622_logs.tgz). This should be easily reproducible when replacing a dead node in a cluster with same ip address. the issue is, the peer nodes could take 1-3 seconds to see the previously down node (now replacing) to be up(to reset the PHI score of the down node). Since, the streaming request arrives before this reset happens, they could be convicted leading to stream close. So, i think a couple of seconds sleep time for gossip to settle, before the bootstrap/streaming starts is what is needed? 1.) node x.x.x.72 was dead 2.) node x.x.x.80's FD keeps notifying its listener to convict as PHI for .72 threshold, every minute. 3.) node x.x.x.72 is restarted with replace_address=x.x.x.72 at 18:56:27,806 4.) node x.x.x.72 : Gossip thread started at 18:56:33,308 after shadow gossip round 5.) node x.xx.72: Starts stream request at 18:56:35,443 INFO [main] 2014-02-08 18:56:35,405 StorageService.java (line 947) JOINING: Starting to bootstrap... INFO [main] 2014-02-08 18:56:35,443 StreamResultFuture.java (line 82) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653] Executing streaming plan for Bootstrap 6.) node x.x.x.80 : still hasn't seen the gossip from .72 with new generation at 18:56:35,031 TRACE [GossipTasks:1] 2014-02-08 18:56:35,031 FailureDetector.java (line 229) PHI for /x.x.x.72 : 36700.042810594234 TRACE [GossipTasks:1] 2014-02-08 18:56:35,032 FailureDetector.java (line 233) notifying listeners that /x.x.x.72 is down 7.) node x.x.x.80 : got the stream request at 18:56:35,450 INFO [STREAM-INIT-/x.x.x.72:47408] 2014-02-08 18:56:35,450 StreamResultFuture.java (line 116) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653] Received streaming plan for Bootstrap 8.) node x.x.x.80: at 18:56:36,090, still hasn't reset the interval times for .72 TRACE [GossipTasks:1] 2014-02-08 18:56:36,090 FailureDetector.java (line 229) PHI for /x.x.x.72 : 36700.87918907657 TRACE [GossipTasks:1] 2014-02-08 18:56:36,090 FailureDetector.java (line 233) notifying listeners that /x.x.x.72 is down 9.) node x.x.x.80: closes the stream session due to convict() notification: INFO [GossipTasks:1] 2014-02-08 18:56:36,090 StreamResultFuture.java (line 181) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653] Session with /x.x.x.72 is complete WARN [GossipTasks:1] 2014-02-08 18:56:36,091 StreamResultFuture.java (line 210) [Stream #bb897500-90f2-11e3-9d67-d5d417af8653] Stream failed 10.) node x.x.x.80: at 18:56:36,097, Gossiper thread on x.x.x.80 clears the interval times for .72, thereby resetting the PHI. DEBUG [GossipStage:1] 2014-02-08 18:56:36,097 Gossiper.java (line 790) Clearing interval times for /x.x.x.72 due to generation change TRACE [GossipStage:1] 2014-02-08 18:56:36,097 FailureDetector.java (line 203) reporting /x.x.x.72 11.) node x.x.x.80: PHI score for .72 at 18:56:37,094 TRACE [GossipTasks:1] 2014-02-08 18:56:37,094 FailureDetector.java (line 229) PHI for /x.x.x.72 : 0.06483452387313912 Streaming session failures during node replace of same address -- Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java
[jira] [Updated] (CASSANDRA-6622) Streaming session failures during node replace of same address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ravi Prasad updated CASSANDRA-6622: --- Attachment: 6622_logs.tgz Streaming session failures during node replace of same address -- Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, 6622_logs.tgz, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6285) LCS compaction failing with Exception
[ https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13893660#comment-13893660 ] Ravi Prasad commented on CASSANDRA-6285: cc [~xedin] we were also seeing such random out of place partitions/rows in sstables (rows not hashing to the node) while using disruptor based hsha thrift server, causing compaction to fail with out of order keys. this used to happen on freshly flushed sstables in L0. We also used to see thrift validation failing on some columns while reading back. We don't see these after switching back to sync server. LCS compaction failing with Exception - Key: CASSANDRA-6285 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285 Project: Cassandra Issue Type: Bug Components: Core Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2 Reporter: David Sauer Assignee: Tyler Hobbs Fix For: 2.0.6 Attachments: compaction_test.py After altering everything to LCS the table OpsCenter.rollups60 amd one other none OpsCenter-Table got stuck with everything hanging around in L0. The compaction started and ran until the logs showed this: ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime] java.lang.RuntimeException: Last written key DecoratedKey(1326283851463420237, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564) = current key DecoratedKey(954210699457429663, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f) writing into /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164) at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160) at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60) at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) at org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 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:724) Moving back to STC worked to keep the compactions running. Especialy my own Table i would like to move to LCS. After a major compaction with STC the move to LCS fails with the same Exception. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6285) LCS compaction failing with Exception
[ https://issues.apache.org/jira/browse/CASSANDRA-6285?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13894144#comment-13894144 ] Ravi Prasad commented on CASSANDRA-6285: Also, one more factor with disruptor based hsha is direct memory/Unsafe versus heap-based message buffers. When we encountered this issue, we were running with jna, hence was using direct memory buffers. I didn't test with heap-based message buffers. LCS compaction failing with Exception - Key: CASSANDRA-6285 URL: https://issues.apache.org/jira/browse/CASSANDRA-6285 Project: Cassandra Issue Type: Bug Components: Core Environment: 4 nodes, shortly updated from 1.2.11 to 2.0.2 Reporter: David Sauer Assignee: Tyler Hobbs Fix For: 2.0.6 Attachments: compaction_test.py After altering everything to LCS the table OpsCenter.rollups60 amd one other none OpsCenter-Table got stuck with everything hanging around in L0. The compaction started and ran until the logs showed this: ERROR [CompactionExecutor:111] 2013-11-01 19:14:53,865 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:111,1,RMI Runtime] java.lang.RuntimeException: Last written key DecoratedKey(1326283851463420237, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574426c6f6f6d46696c746572537061636555736564) = current key DecoratedKey(954210699457429663, 37382e34362e3132382e3139382d6a7576616c69735f6e6f72785f696e6465785f323031335f31305f30382d63616368655f646f63756d656e74736c6f6f6b75702d676574546f74616c4469736b5370616365557365640b0f) writing into /var/lib/cassandra/data/OpsCenter/rollups60/OpsCenter-rollups60-tmp-jb-58656-Data.db at org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:141) at org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:164) at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160) at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:60) at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) at org.apache.cassandra.db.compaction.CompactionManager$6.runMayThrow(CompactionManager.java:296) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) 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:724) Moving back to STC worked to keep the compactions running. Especialy my own Table i would like to move to LCS. After a major compaction with STC the move to LCS fails with the same Exception. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5921) Don't return empty list when the L0 compaction candidates could cause overlap in L1
[ https://issues.apache.org/jira/browse/CASSANDRA-5921?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13894154#comment-13894154 ] Ravi Prasad commented on CASSANDRA-5921: we're seeing overlaps in L1 in cassandra-2.0.4, multithreaded_compaction:false, concurrent_compactors:default(num_cores). Also reported by other user in ilist: http://qnalist.com/questions/4702288/exception-during-add-node-due-to-test-beforeappend-on-sstablewriter reverting the changes here to pre-2.0/CASSANDRA-5907, don't see any overlaps. Don't return empty list when the L0 compaction candidates could cause overlap in L1 --- Key: CASSANDRA-5921 URL: https://issues.apache.org/jira/browse/CASSANDRA-5921 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Assignee: Marcus Eriksson Priority: Minor Fix For: 2.0.1 Attachments: 0001-instead-of-doing-no-compaction-if-we-have-sstables-t.patch, 0001-instead-of-doing-no-compaction-if-we-have-sstables-t.patch, 5921-v3.txt Followup to CASSANDRA-5907 - instead of returning empty list when the compaction candidates could cause overlap in L1, remove sstables that would cause the overlap from the candidates. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13894191#comment-13894191 ] Ravi Prasad commented on CASSANDRA-6622: bq. Can you try the patch from CASSANDRA-6658? Didn't help. What i'm seeing is, the other nodes in the ring take around 2-3 seconds for PHI on the replacing node to drop below convict threshold. But, they also receive the stream plan from the replacing node with in 2 seconds of starting of replacing node. I think this would affect normal bootstrap also, but there we sleep for RING_DELAY already. Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Comment Edited] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13894191#comment-13894191 ] Ravi Prasad edited comment on CASSANDRA-6622 at 2/7/14 5:23 AM: bq. Can you try the patch from CASSANDRA-6658? Didn't help. What i'm seeing is, the other nodes in the ring take around 2-3 seconds for PHI on the replacing node to drop below convict threshold. But, they also receive the stream plan from the replacing node with in 2 seconds of starting of replacing node. was (Author: ravilr): bq. Can you try the patch from CASSANDRA-6658? Didn't help. What i'm seeing is, the other nodes in the ring take around 2-3 seconds for PHI on the replacing node to drop below convict threshold. But, they also receive the stream plan from the replacing node with in 2 seconds of starting of replacing node. I think this would affect normal bootstrap also, but there we sleep for RING_DELAY already. Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6622) Streaming session failures during node replace of same address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ravi Prasad updated CASSANDRA-6622: --- Summary: Streaming session failures during node replace of same address (was: Streaming session failures during node replace using replace_address) Streaming session failures during node replace of same address -- Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13892506#comment-13892506 ] Ravi Prasad commented on CASSANDRA-6622: I'm seeing FailureDetector notifying listeners every second invoked through GossiperTask's doStatusCheck(). Tested sleeping for RING_DELAY (instead of BROADCAST_INTERVAL) before bootstrap, works without any stream session closure. Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ravi Prasad updated CASSANDRA-6622: --- Attachment: logs.tgz Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13889856#comment-13889856 ] Ravi Prasad commented on CASSANDRA-6622: In attached logs, .72 was the replacing node, .73 is where the streaming session failed. I had trace logging turned on in .73 for org.apache.cassandra.gms. Looks like, it is FailureDetector is convicting. I have to mention that this was with '0001-don-t-signal-restart-of-dead-states.txt' applied on cassandra-2.0.4. Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt, logs.tgz When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13889242#comment-13889242 ] Ravi Prasad commented on CASSANDRA-6622: bq. Maybe our best bet here is to not call onRestart for dead states. Seeing the same error in original description, with the above patch on cassandra-2.0. Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Assignee: Brandon Williams Attachments: 0001-don-t-signal-restart-of-dead-states.txt, 6622-2.0.txt When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): {noformat} INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed {noformat} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
Ravi Prasad created CASSANDRA-6622: -- Summary: Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ravi Prasad updated CASSANDRA-6622: --- Attachment: 6622-2.0.txt Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Attachments: 6622-2.0.txt When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6622) Streaming session failures during node replace using replace_address
[ https://issues.apache.org/jira/browse/CASSANDRA-6622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882538#comment-13882538 ] Ravi Prasad commented on CASSANDRA-6622: yes, i was replacing the node with same ip address, which was dead before. Despite being dead before, since we set the state to hibernate and due to the generation change of the replacing node at startup, it gets marked down again. StorageService thread already sleeps for broadcast_interval, if the replacing address is not same as broadcast address. the attached patch sleeps for same address also. Streaming session failures during node replace using replace_address Key: CASSANDRA-6622 URL: https://issues.apache.org/jira/browse/CASSANDRA-6622 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Attachments: 6622-2.0.txt When using replace_address, Gossiper ApplicationState is set to hibernate, which is a down state. We are seeing that the peer nodes are seeing streaming plan request even before the Gossiper on them marks the replacing node as dead. As a result, streaming on peer nodes convicts the replacing node by closing the stream handler. I think, making the StorageService thread on the replacing node, sleep for BROADCAST_INTERVAL before bootstrapping, would avoid this scenario. Relevant logs from peer node (see that the Gossiper on peer node mark the replacing node as down, 2 secs after the streaming init request): INFO [STREAM-INIT-/x.x.x.x:46436] 2014-01-26 20:42:24,388 StreamResultFuture.java (line 116) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Received streaming plan for Bootstrap INFO [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [GossipTasks:1] 2014-01-26 20:42:25,240 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed INFO [GossipStage:1] 2014-01-26 20:42:25,242 Gossiper.java (line 850) InetAddress /x.x.x.x is now DOWN ERROR [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,766 StreamSession.java (line 410) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:175) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:722) INFO [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 181) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Session with /x.x.x.x is complete WARN [STREAM-IN-/x.x.x.x] 2014-01-26 20:42:25,768 StreamResultFuture.java (line 210) [Stream #5c6cd940-86ca-11e3-90a0-411b913c0e88] Stream failed -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6614) 2 hours loop flushing+compacting system/{schema_keyspaces,schema_columnfamilies,schema_columns} when upgrading
[ https://issues.apache.org/jira/browse/CASSANDRA-6614?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881372#comment-13881372 ] Ravi Prasad commented on CASSANDRA-6614: Seeing this too, while upgrading from 1.2.9 to 2.0.4. As Jason mentioned, found this subsides, once all the nodes in the cluster are upgraded or on same schema. 2 hours loop flushing+compacting system/{schema_keyspaces,schema_columnfamilies,schema_columns} when upgrading -- Key: CASSANDRA-6614 URL: https://issues.apache.org/jira/browse/CASSANDRA-6614 Project: Cassandra Issue Type: Bug Components: Core Environment: ubuntu 12.04 Reporter: Cyril Scetbon It happens when we upgrade one node to 1.2.13 on a 1.2.2 cluster see http://pastebin.com/YZKUQLXz If I grep for only InternalResponseStage logs I get http://pastebin.com/htnXZCiT which always displays same account of ops and serialized/live bytes per column family. When I upgrade one node from 1.2.2 to 1.2.13, for 2h I get the previous messages with a raise of CPU (as it flushes and compacts continually) on all nodes http://picpaste.com/pics/Screen_Shot_2014-01-24_at_09.18.50-ggcCDVqd.1390587562.png After that, everything is fine and I can upgrade other nodes without any raise of cpus load. when I start the upgrade, the more nodes I upgrade at the same time (at the beginning), the higher the cpu load is http://picpaste.com/pics/Screen_Shot_2014-01-23_at_17.45.56-I3fdEQ2T.1390587597.png -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (CASSANDRA-6618) NullPointerException while stopping/draining if native transport wasn't started
Ravi Prasad created CASSANDRA-6618: -- Summary: NullPointerException while stopping/draining if native transport wasn't started Key: CASSANDRA-6618 URL: https://issues.apache.org/jira/browse/CASSANDRA-6618 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Priority: Minor if using a custom Authenticator, native transport server wouldn't be started. ERROR [main] 2014-01-24 03:04:40,876 Server.java (line 131) Not starting native transport as the configured IAuthenticator is not capable of SASL authentication But, while stopping cassandra/'nodetool drain', this results in NullpointerException being thrown currently: Exception in thread main java.lang.NullPointerException at org.apache.cassandra.transport.Server.close(Server.java:177) at org.apache.cassandra.transport.Server.stop(Server.java:116) at org.apache.cassandra.service.StorageService.stopNativeTransport(StorageService.java:349) at org.apache.cassandra.service.StorageService.shutdownClientServers(StorageService.java:364) at org.apache.cassandra.service.StorageService.drain(StorageService.java:3288) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:111) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:45) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:235) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:792) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1486) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:96) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1327) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1419) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:847) at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:553) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:808) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:667) 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:722) -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6618) NullPointerException while stopping/draining if native transport wasn't started
[ https://issues.apache.org/jira/browse/CASSANDRA-6618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881452#comment-13881452 ] Ravi Prasad commented on CASSANDRA-6618: Brandon, there still seems to be a logic error, which could wrongly set nativeServer.isRunning to true, even when it wasn't started. Can you please check. NullPointerException while stopping/draining if native transport wasn't started --- Key: CASSANDRA-6618 URL: https://issues.apache.org/jira/browse/CASSANDRA-6618 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Priority: Minor Attachments: 6618-2.0.txt if using a custom Authenticator, native transport server wouldn't be started. ERROR [main] 2014-01-24 03:04:40,876 Server.java (line 131) Not starting native transport as the configured IAuthenticator is not capable of SASL authentication But, while stopping cassandra/'nodetool drain', this results in NullpointerException being thrown currently: Exception in thread main java.lang.NullPointerException at org.apache.cassandra.transport.Server.close(Server.java:177) at org.apache.cassandra.transport.Server.stop(Server.java:116) at org.apache.cassandra.service.StorageService.stopNativeTransport(StorageService.java:349) at org.apache.cassandra.service.StorageService.shutdownClientServers(StorageService.java:364) at org.apache.cassandra.service.StorageService.drain(StorageService.java:3288) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:111) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:45) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:235) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:792) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1486) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:96) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1327) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1419) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:847) at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:553) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:808) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:667) 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:722) -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6618) NullPointerException while stopping/draining if native transport wasn't started
[ https://issues.apache.org/jira/browse/CASSANDRA-6618?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ravi Prasad updated CASSANDRA-6618: --- Attachment: 6618-2.0.txt nativeServer.isRunning could be set to true, even when it wasn't started due to custom authenticator not supporting sasl. NullPointerException while stopping/draining if native transport wasn't started --- Key: CASSANDRA-6618 URL: https://issues.apache.org/jira/browse/CASSANDRA-6618 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Priority: Minor Attachments: 6618-2.0.txt if using a custom Authenticator, native transport server wouldn't be started. ERROR [main] 2014-01-24 03:04:40,876 Server.java (line 131) Not starting native transport as the configured IAuthenticator is not capable of SASL authentication But, while stopping cassandra/'nodetool drain', this results in NullpointerException being thrown currently: Exception in thread main java.lang.NullPointerException at org.apache.cassandra.transport.Server.close(Server.java:177) at org.apache.cassandra.transport.Server.stop(Server.java:116) at org.apache.cassandra.service.StorageService.stopNativeTransport(StorageService.java:349) at org.apache.cassandra.service.StorageService.shutdownClientServers(StorageService.java:364) at org.apache.cassandra.service.StorageService.drain(StorageService.java:3288) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:111) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:45) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:235) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:792) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1486) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:96) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1327) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1419) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:847) at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:553) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:808) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:667) 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:722) -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6618) NullPointerException while stopping/draining if native transport wasn't started
[ https://issues.apache.org/jira/browse/CASSANDRA-6618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881517#comment-13881517 ] Ravi Prasad commented on CASSANDRA-6618: if the authenticator is not Saslaware, the run() returns and we stills set isRunning to true. https://github.com/apache/cassandra/blob/cassandra-2.0/src/java/org/apache/cassandra/transport/Server.java#L133 NullPointerException while stopping/draining if native transport wasn't started --- Key: CASSANDRA-6618 URL: https://issues.apache.org/jira/browse/CASSANDRA-6618 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Priority: Minor Attachments: 6618-2.0.txt if using a custom Authenticator, native transport server wouldn't be started. ERROR [main] 2014-01-24 03:04:40,876 Server.java (line 131) Not starting native transport as the configured IAuthenticator is not capable of SASL authentication But, while stopping cassandra/'nodetool drain', this results in NullpointerException being thrown currently: Exception in thread main java.lang.NullPointerException at org.apache.cassandra.transport.Server.close(Server.java:177) at org.apache.cassandra.transport.Server.stop(Server.java:116) at org.apache.cassandra.service.StorageService.stopNativeTransport(StorageService.java:349) at org.apache.cassandra.service.StorageService.shutdownClientServers(StorageService.java:364) at org.apache.cassandra.service.StorageService.drain(StorageService.java:3288) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:111) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:45) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:235) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:792) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1486) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:96) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1327) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1419) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:847) at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:553) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:808) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:667) 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:722) -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Comment Edited] (CASSANDRA-6618) NullPointerException while stopping/draining if native transport wasn't started
[ https://issues.apache.org/jira/browse/CASSANDRA-6618?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881517#comment-13881517 ] Ravi Prasad edited comment on CASSANDRA-6618 at 1/24/14 11:04 PM: -- if the authenticator is not Saslaware, the run() returns without starting the nativeServer and we still set isRunning to true, in that case: https://github.com/apache/cassandra/blob/cassandra-2.0/src/java/org/apache/cassandra/transport/Server.java#L133 was (Author: ravilr): if the authenticator is not Saslaware, the run() returns and we stills set isRunning to true. https://github.com/apache/cassandra/blob/cassandra-2.0/src/java/org/apache/cassandra/transport/Server.java#L133 NullPointerException while stopping/draining if native transport wasn't started --- Key: CASSANDRA-6618 URL: https://issues.apache.org/jira/browse/CASSANDRA-6618 Project: Cassandra Issue Type: Bug Environment: RHEL6, cassandra-2.0.4 Reporter: Ravi Prasad Priority: Minor Attachments: 6618-2.0.txt if using a custom Authenticator, native transport server wouldn't be started. ERROR [main] 2014-01-24 03:04:40,876 Server.java (line 131) Not starting native transport as the configured IAuthenticator is not capable of SASL authentication But, while stopping cassandra/'nodetool drain', this results in NullpointerException being thrown currently: Exception in thread main java.lang.NullPointerException at org.apache.cassandra.transport.Server.close(Server.java:177) at org.apache.cassandra.transport.Server.stop(Server.java:116) at org.apache.cassandra.service.StorageService.stopNativeTransport(StorageService.java:349) at org.apache.cassandra.service.StorageService.shutdownClientServers(StorageService.java:364) at org.apache.cassandra.service.StorageService.drain(StorageService.java:3288) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:111) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:45) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:235) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:792) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1486) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:96) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1327) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1419) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:847) at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:601) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:553) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:808) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:667) 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:722) -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5371) Perform size-tiered compactions in L0 (hybrid compaction)
[ https://issues.apache.org/jira/browse/CASSANDRA-5371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13881634#comment-13881634 ] Ravi Prasad commented on CASSANDRA-5371: +1 on [~br1985 ] comment. Even during dead node replace (using replace_address), streaming puts all sstables in L0. 2.0.x switches to STCS, in doing so, also creates larger sstables, which means more free disk space to be left, in order for them to be compacted later into higher levels. LCS is known to lower the amount of free disk space (headroom) needed for compaction. this is no more true with LCS in above scenarios. Is there a way to disable STCS fallback, please? Perform size-tiered compactions in L0 (hybrid compaction) --- Key: CASSANDRA-5371 URL: https://issues.apache.org/jira/browse/CASSANDRA-5371 Project: Cassandra Issue Type: Bug Components: Core Reporter: Jonathan Ellis Assignee: Jonathan Ellis Fix For: 2.0 beta 1 Attachments: HybridCompactionStrategy.java If LCS gets behind, read performance deteriorates as we have to check bloom filters on man sstables in L0. For wide rows, this can mean having to seek for each one since the BF doesn't help us reject much. Performing size-tiered compaction in L0 will mitigate this until we can catch up on merging it into higher levels. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5916) gossip and tokenMetadata get hostId out of sync on failed replace_node with the same IP address
[ https://issues.apache.org/jira/browse/CASSANDRA-5916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13788570#comment-13788570 ] Ravi Prasad commented on CASSANDRA-5916: That is true regardless of shadow mode though, since hibernate is a dead state and the node doesn't go live to reset the hint timer until the replace has completed. my understanding is due to the generation change of the replacing node, gossiper.handleMajorStateChange marks the node as dead, as hibernate is one of the DEAD_STATES. So, the other nodes marks the replacing node as dead before the token bootstrap starts, hence should be storing hints to the replacing node from that point. gossip and tokenMetadata get hostId out of sync on failed replace_node with the same IP address --- Key: CASSANDRA-5916 URL: https://issues.apache.org/jira/browse/CASSANDRA-5916 Project: Cassandra Issue Type: Bug Reporter: Brandon Williams Assignee: Brandon Williams Fix For: 1.2.11 Attachments: 5916.txt If you try to replace_node an existing, live hostId, it will error out. However if you're using an existing IP to do this (as in, you chose the wrong uuid to replace on accident) then the newly generated hostId wipes out the old one in TMD, and when you do try to replace it replace_node will complain it does not exist. Examination of gossipinfo still shows the old hostId, however now you can't replace it either. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Comment Edited] (CASSANDRA-5916) gossip and tokenMetadata get hostId out of sync on failed replace_node with the same IP address
[ https://issues.apache.org/jira/browse/CASSANDRA-5916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13788570#comment-13788570 ] Ravi Prasad edited comment on CASSANDRA-5916 at 10/7/13 9:39 PM: - bq. That is true regardless of shadow mode though, since hibernate is a dead state and the node doesn't go live to reset the hint timer until the replace has completed. my understanding is, due to the generation change of the replacing node, gossiper.handleMajorStateChange marks the node as dead, as hibernate is one of the DEAD_STATES. So, the other nodes marks the replacing node as dead before the token bootstrap starts, hence should be storing hints to the replacing node from that point. Am i reading it wrong? was (Author: ravilr): That is true regardless of shadow mode though, since hibernate is a dead state and the node doesn't go live to reset the hint timer until the replace has completed. my understanding is due to the generation change of the replacing node, gossiper.handleMajorStateChange marks the node as dead, as hibernate is one of the DEAD_STATES. So, the other nodes marks the replacing node as dead before the token bootstrap starts, hence should be storing hints to the replacing node from that point. gossip and tokenMetadata get hostId out of sync on failed replace_node with the same IP address --- Key: CASSANDRA-5916 URL: https://issues.apache.org/jira/browse/CASSANDRA-5916 Project: Cassandra Issue Type: Bug Reporter: Brandon Williams Assignee: Brandon Williams Fix For: 1.2.11 Attachments: 5916.txt If you try to replace_node an existing, live hostId, it will error out. However if you're using an existing IP to do this (as in, you chose the wrong uuid to replace on accident) then the newly generated hostId wipes out the old one in TMD, and when you do try to replace it replace_node will complain it does not exist. Examination of gossipinfo still shows the old hostId, however now you can't replace it either. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-5916) gossip and tokenMetadata get hostId out of sync on failed replace_node with the same IP address
[ https://issues.apache.org/jira/browse/CASSANDRA-5916?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13788709#comment-13788709 ] Ravi Prasad commented on CASSANDRA-5916: bq. once the bootstrap has started the node should be receiving any incoming writes for the range it owns, so 'new' hints shouldn't matter in the common case where it succeeds. Is this true for node bootstrapping in hibernate state? From what i have observed, writes to hibernate'd node during its bootstrap are not sent to it, as gossip marks that node down right. gossip and tokenMetadata get hostId out of sync on failed replace_node with the same IP address --- Key: CASSANDRA-5916 URL: https://issues.apache.org/jira/browse/CASSANDRA-5916 Project: Cassandra Issue Type: Bug Reporter: Brandon Williams Assignee: Brandon Williams Fix For: 1.2.11 Attachments: 5916.txt If you try to replace_node an existing, live hostId, it will error out. However if you're using an existing IP to do this (as in, you chose the wrong uuid to replace on accident) then the newly generated hostId wipes out the old one in TMD, and when you do try to replace it replace_node will complain it does not exist. Examination of gossipinfo still shows the old hostId, however now you can't replace it either. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-5947) Sampling bug in metrics-core-2.0.3.jar used by Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-5947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13763535#comment-13763535 ] Ravi Prasad commented on CASSANDRA-5947: Is it just me or anyone else seeing this issue: all mbean objectNames of org.apache.cassandra.metrics* have double quotes surrounding them after upgrading to metrics-core-2.2.0.jar. like org.apache.cassandra.metrics:type=DroppedMessage,scope=READ,name=Dropped ? Sampling bug in metrics-core-2.0.3.jar used by Cassandra Key: CASSANDRA-5947 URL: https://issues.apache.org/jira/browse/CASSANDRA-5947 Project: Cassandra Issue Type: Bug Components: Tools Reporter: J.B. Langston Assignee: Jonathan Ellis Priority: Minor Fix For: 1.2.10, 2.0.1 There is a sampling bug in the version of the metrics library we're using in Cassandra. See https://github.com/codahale/metrics/issues/421. ExponentiallyDecayingSample is used by the Timer's histogram that is used in stress tool, and according to [~brandon.williams] it is also in a few other places like the dynamic snitch. The statistical theory involved in this bug goes over my head so i'm not sure if this would bug would meaningfully affect its usage by Cassandra. One of the comments on the bug mentions that it affects slow sampling rates (10 samples/min was the example given). We're currently distributing metrics-core-2.0.3.jar and according to the release nodes, this bug is fixed in 2.1.3: http://metrics.codahale.com/about/release-notes/#v2-1-3-aug-06-2012 -- 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-5947) Sampling bug in metrics-core-2.0.3.jar used by Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-5947?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ravi Prasad updated CASSANDRA-5947: --- Attachment: Screen Shot 2013-09-10 at 2.23.11 PM.png Screenshot of jconsole showing double quoted object names only for org.apache.cassandra.metrics yammer metrics in cassandra-1.2.10(latest cassandra-1.2 branch) with metrics-core-2.2.0.jar Sampling bug in metrics-core-2.0.3.jar used by Cassandra Key: CASSANDRA-5947 URL: https://issues.apache.org/jira/browse/CASSANDRA-5947 Project: Cassandra Issue Type: Bug Components: Tools Reporter: J.B. Langston Assignee: Jonathan Ellis Priority: Minor Fix For: 1.2.10, 2.0.1 Attachments: Screen Shot 2013-09-10 at 2.23.11 PM.png There is a sampling bug in the version of the metrics library we're using in Cassandra. See https://github.com/codahale/metrics/issues/421. ExponentiallyDecayingSample is used by the Timer's histogram that is used in stress tool, and according to [~brandon.williams] it is also in a few other places like the dynamic snitch. The statistical theory involved in this bug goes over my head so i'm not sure if this would bug would meaningfully affect its usage by Cassandra. One of the comments on the bug mentions that it affects slow sampling rates (10 samples/min was the example given). We're currently distributing metrics-core-2.0.3.jar and according to the release nodes, this bug is fixed in 2.1.3: http://metrics.codahale.com/about/release-notes/#v2-1-3-aug-06-2012 -- 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-5947) Sampling bug in metrics-core-2.0.3.jar used by Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-5947?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13763765#comment-13763765 ] Ravi Prasad commented on CASSANDRA-5947: Sorry, missed Chris's comment above. If this is going to be the case, can we have the http://wiki.apache.org/cassandra/Metrics and NEWS.txt updated on this change. Sampling bug in metrics-core-2.0.3.jar used by Cassandra Key: CASSANDRA-5947 URL: https://issues.apache.org/jira/browse/CASSANDRA-5947 Project: Cassandra Issue Type: Bug Components: Tools Reporter: J.B. Langston Assignee: Jonathan Ellis Priority: Minor Fix For: 1.2.10, 2.0.1 Attachments: Screen Shot 2013-09-10 at 2.23.11 PM.png There is a sampling bug in the version of the metrics library we're using in Cassandra. See https://github.com/codahale/metrics/issues/421. ExponentiallyDecayingSample is used by the Timer's histogram that is used in stress tool, and according to [~brandon.williams] it is also in a few other places like the dynamic snitch. The statistical theory involved in this bug goes over my head so i'm not sure if this would bug would meaningfully affect its usage by Cassandra. One of the comments on the bug mentions that it affects slow sampling rates (10 samples/min was the example given). We're currently distributing metrics-core-2.0.3.jar and according to the release nodes, this bug is fixed in 2.1.3: http://metrics.codahale.com/about/release-notes/#v2-1-3-aug-06-2012 -- 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-5979) HintedHandOffManager task triggering java.util.NoSuchElementException
Ravi Prasad created CASSANDRA-5979: -- Summary: HintedHandOffManager task triggering java.util.NoSuchElementException Key: CASSANDRA-5979 URL: https://issues.apache.org/jira/browse/CASSANDRA-5979 Project: Cassandra Issue Type: Bug Components: Core Reporter: Ravi Prasad Seeing this exception every 10 minutes, whenever scheduled HintedHandoff task runs: ERROR [OptionalTasks:1] 2013-09-05 20:48:00,334 CassandraDaemon.java (line 192) Exception in thread Thread[OptionalTasks:1,5,main] java.util.NoSuchElementException at com.google.common.collect.EmptyImmutableSortedSet.last(EmptyImmutableSortedSet.java:100) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.readSimpleColumns(SSTableNamesIterator.java:186) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:156) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.init(SSTableNamesIterator.java:83) at org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:86) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:75) at org.apache.cassandra.io.sstable.SSTableScanner$FilteredKeyScanningIterator$1.create(SSTableScanner.java:248) at org.apache.cassandra.db.columniterator.LazyColumnIterator.getSubIterator(LazyColumnIterator.java:75) at org.apache.cassandra.db.columniterator.LazyColumnIterator.getColumnFamily(LazyColumnIterator.java:87) at org.apache.cassandra.db.RowIteratorFactory$2.reduce(RowIteratorFactory.java:95) at org.apache.cassandra.db.RowIteratorFactory$2.reduce(RowIteratorFactory.java:79) at org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:111) at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:97) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) at org.apache.cassandra.db.ColumnFamilyStore$6.computeNext(ColumnFamilyStore.java:1433) at org.apache.cassandra.db.ColumnFamilyStore$6.computeNext(ColumnFamilyStore.java:1429) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) at org.apache.cassandra.db.ColumnFamilyStore.filter(ColumnFamilyStore.java:1500) at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1477) at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1472) at org.apache.cassandra.db.HintedHandOffManager.scheduleAllDeliveries(HintedHandOffManager.java:464) at org.apache.cassandra.db.HintedHandOffManager.access$000(HintedHandOffManager.java:90) at org.apache.cassandra.db.HintedHandOffManager$1.run(HintedHandOffManager.java:133) at org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run(DebuggableScheduledThreadPoolExecutor.java:75) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) at java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:351) at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:178) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:178) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293) 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:722) this seems to be a regression from https://issues.apache.org/jira/browse/CASSANDRA-5966. -- 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-5966) Average name query performance much worse for wide rows
[ https://issues.apache.org/jira/browse/CASSANDRA-5966?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13759531#comment-13759531 ] Ravi Prasad commented on CASSANDRA-5966: This seems to be breaking HintedHandOffManager scheduled task, which does a empty names column filter: https://issues.apache.org/jira/browse/CASSANDRA-5979 Average name query performance much worse for wide rows --- Key: CASSANDRA-5966 URL: https://issues.apache.org/jira/browse/CASSANDRA-5966 Project: Cassandra Issue Type: Bug Reporter: Rick Branson Assignee: Rick Branson Fix For: 1.2.10, 2.0.1 Attachments: 5966-trunk.txt, 5966.txt, 5966-v2.txt The average performance for by-name queries degrades heavily on wide rows. This is because in the wide row case SSTableNamesIterator deserializes every column in the entire row chunk (64KB by default), where-as in the narrow row case, it stops deserializing as soon as it's found the columns it's looking for. -- 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