[jira] [Updated] (CASSANDRA-6497) Iterable CqlPagingRecordReader
[ https://issues.apache.org/jira/browse/CASSANDRA-6497?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Luca Rosellini updated CASSANDRA-6497: -- Attachment: IterableCqlPagingRecordReader_v2.diff Uploaded a new patch version following cassandra code conventions. Iterable CqlPagingRecordReader -- Key: CASSANDRA-6497 URL: https://issues.apache.org/jira/browse/CASSANDRA-6497 Project: Cassandra Issue Type: Improvement Components: Hadoop Reporter: Luca Rosellini Fix For: 2.1 Attachments: IterableCqlPagingRecordReader_v2.diff, iterable-CqlPagingRecordReader.diff The current CqlPagingRecordReader implementation provides a non-standard way of iterating over the underlying {{rowIterator}}. It would be nice to have an Iterable CqlPagingRecordReader like the one proposed in the attached diff. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6082) 1.1.12 -- 1.2.x upgrade may result inconsistent ring
[ https://issues.apache.org/jira/browse/CASSANDRA-6082?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882318#comment-13882318 ] graham sanderson commented on CASSANDRA-6082: - We have seen the same thing with 2.0.3: {code} ERROR [GossipStage:1] 2014-01-26 02:46:10,048 CassandraDaemon.java (line 187) Exception in thread Thread[GossipStage:1,5,main] java.lang.ArrayIndexOutOfBoundsException: 2 at org.apache.cassandra.service.StorageService.extractExpireTime(StorageService.java:1674) at org.apache.cassandra.service.StorageService.handleStateRemoving(StorageService.java:1629) at org.apache.cassandra.service.StorageService.onChange(StorageService.java:1247) at org.apache.cassandra.service.StorageService.onJoin(StorageService.java:1848) at org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:888) at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:966) at org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:49) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) {code} I'm not sure of the exact repro steps, however we had a node go down (hardware failure) during nodetool repair, so we had to nodetool removenode it.. this was the result. Cluster is fine, but had to do the assassinate to make nodetool describecluster not include it 1.1.12 -- 1.2.x upgrade may result inconsistent ring - Key: CASSANDRA-6082 URL: https://issues.apache.org/jira/browse/CASSANDRA-6082 Project: Cassandra Issue Type: Bug Components: Core Environment: 1.1.12 -- 1.2.9 Reporter: Chris Burroughs Priority: Minor Attachments: c-gossipinfo, c-status This happened to me once, and since I don't have any more 1.1.x clusters I won't be testing again. I hope the attached files are enough for someone to connect the dots. I did a rolling restart to upgrade from 1.1.12 -- 1.2.9. About a week later I discovered that one node was in an inconsistent state in the ring. It was either: * up * host-id=null * missing Depending on which node I ran nodetool status from. I *think* I just missed this during the upgrade but can not rule out the possibility that it just happened for no reason some time after the upgrade. It was detected when running repair in such a ring caused all sorts of terrible data duplication and performance tanked. Restarting the seeds + bad node caused the ring to be consistent again. Two possibly suspicious things are a ArrayIndexOutOfBoundsException on startup: {noformat} ERROR [GossipStage:1] 2013-09-06 10:45:35,213 CassandraDaemon.java (line 194) Exception in thread Thread[GossipStage:1,5,main] java.lang.ArrayIndexOutOfBoundsException: 2 at org.apache.cassandra.service.StorageService.extractExpireTime(StorageService.java:1660) at org.apache.cassandra.service.StorageService.handleStateRemoving(StorageService.java:1607) at org.apache.cassandra.service.StorageService.onChange(StorageService.java:1230) at org.apache.cassandra.service.StorageService.onJoin(StorageService.java:1958) at org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:841) at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:919) at org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:50) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {noformat} and problems to hint delivery to multiple node. {noformat} ERROR [MutationStage:11] 2013-09-06 13:59:19,604 CassandraDaemon.java (line 194) Exception in thread Thread[MutationStage:11,5,main] java.lang.AssertionError: Missing host ID for 10.20.2.45 at org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:583) at org.apache.cassandra.service.StorageProxy$5.runMayThrow(StorageProxy.java:552) at org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1658) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441) at
git commit: Add iterable CqlPagingRecordReader. Patch by Luca Rosellini, reviewed by brandonwilliams for CASSANDRA-6497
Updated Branches: refs/heads/trunk f5d542c5a - 4d13d0998 Add iterable CqlPagingRecordReader. Patch by Luca Rosellini, reviewed by brandonwilliams for CASSANDRA-6497 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/4d13d099 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/4d13d099 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/4d13d099 Branch: refs/heads/trunk Commit: 4d13d09985722e1a06e45f1e96dff7d771959068 Parents: f5d542c Author: Brandon Williams brandonwilli...@apache.org Authored: Sun Jan 26 09:02:29 2014 -0600 Committer: Brandon Williams brandonwilli...@apache.org Committed: Sun Jan 26 09:03:31 2014 -0600 -- .../hadoop/cql3/CqlPagingRecordReader.java | 4 +- .../cql3/IterableCqlPagingRecordReader.java | 75 2 files changed, 77 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/4d13d099/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java -- diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java index d9b9a39..002992f 100644 --- a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java +++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java @@ -65,7 +65,7 @@ public class CqlPagingRecordReader extends RecordReaderMapString, ByteBuffer, public static final int DEFAULT_CQL_PAGE_LIMIT = 1000; // TODO: find the number large enough but not OOM private ColumnFamilySplit split; -private RowIterator rowIterator; +protected RowIterator rowIterator; private PairMapString, ByteBuffer, MapString, ByteBuffer currentRow; private int totalRowCount; // total number of rows to fetch @@ -282,7 +282,7 @@ public class CqlPagingRecordReader extends RecordReaderMapString, ByteBuffer, } /** CQL row iterator */ -private class RowIterator extends AbstractIteratorPairMapString, ByteBuffer, MapString, ByteBuffer +protected class RowIterator extends AbstractIteratorPairMapString, ByteBuffer, MapString, ByteBuffer { protected int totalRead = 0; // total number of cf rows read protected IteratorCqlRow rows; http://git-wip-us.apache.org/repos/asf/cassandra/blob/4d13d099/src/java/org/apache/cassandra/hadoop/cql3/IterableCqlPagingRecordReader.java -- diff --git a/src/java/org/apache/cassandra/hadoop/cql3/IterableCqlPagingRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/IterableCqlPagingRecordReader.java new file mode 100644 index 000..85beed9 --- /dev/null +++ b/src/java/org/apache/cassandra/hadoop/cql3/IterableCqlPagingRecordReader.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.cassandra.hadoop.cql3; + +import java.io.Closeable; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.Map; + +import org.apache.commons.lang.NotImplementedException; + +import org.apache.cassandra.utils.Pair; + +/** + * Implements an iterable-friendly {@link CqlPagingRecordReader}. + */ +public class IterableCqlPagingRecordReader extends CqlPagingRecordReader + implements IterablePairMapString, ByteBuffer, MapString, ByteBuffer, Closeable +{ +public IteratorPairMapString, ByteBuffer, MapString, ByteBuffer iterator() +{ +return new IteratorPairMapString, ByteBuffer, MapString, ByteBuffer() +{ +public boolean hasNext() +{ +return rowIterator.hasNext(); +} + +public PairMapString, ByteBuffer, MapString, ByteBuffer next() +{ +return rowIterator.next(); +} + +public void remove() +{ +throw new NotImplementedException(Cannot remove an element on
[jira] [Resolved] (CASSANDRA-6082) 1.1.12 -- 1.2.x upgrade may result inconsistent ring
[ https://issues.apache.org/jira/browse/CASSANDRA-6082?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams resolved CASSANDRA-6082. - Resolution: Duplicate 1.1.12 -- 1.2.x upgrade may result inconsistent ring - Key: CASSANDRA-6082 URL: https://issues.apache.org/jira/browse/CASSANDRA-6082 Project: Cassandra Issue Type: Bug Components: Core Environment: 1.1.12 -- 1.2.9 Reporter: Chris Burroughs Priority: Minor Attachments: c-gossipinfo, c-status This happened to me once, and since I don't have any more 1.1.x clusters I won't be testing again. I hope the attached files are enough for someone to connect the dots. I did a rolling restart to upgrade from 1.1.12 -- 1.2.9. About a week later I discovered that one node was in an inconsistent state in the ring. It was either: * up * host-id=null * missing Depending on which node I ran nodetool status from. I *think* I just missed this during the upgrade but can not rule out the possibility that it just happened for no reason some time after the upgrade. It was detected when running repair in such a ring caused all sorts of terrible data duplication and performance tanked. Restarting the seeds + bad node caused the ring to be consistent again. Two possibly suspicious things are a ArrayIndexOutOfBoundsException on startup: {noformat} ERROR [GossipStage:1] 2013-09-06 10:45:35,213 CassandraDaemon.java (line 194) Exception in thread Thread[GossipStage:1,5,main] java.lang.ArrayIndexOutOfBoundsException: 2 at org.apache.cassandra.service.StorageService.extractExpireTime(StorageService.java:1660) at org.apache.cassandra.service.StorageService.handleStateRemoving(StorageService.java:1607) at org.apache.cassandra.service.StorageService.onChange(StorageService.java:1230) at org.apache.cassandra.service.StorageService.onJoin(StorageService.java:1958) at org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:841) at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:919) at org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:50) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {noformat} and problems to hint delivery to multiple node. {noformat} ERROR [MutationStage:11] 2013-09-06 13:59:19,604 CassandraDaemon.java (line 194) Exception in thread Thread[MutationStage:11,5,main] java.lang.AssertionError: Missing host ID for 10.20.2.45 at org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:583) at org.apache.cassandra.service.StorageProxy$5.runMayThrow(StorageProxy.java:552) at org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1658) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {noformat} Not however that while there were delivery problems to multiple nodes during the rolling upgrade, only one node was in a funky state a week later. Attached are the results of running gossipinfo and status on every node. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Reopened] (CASSANDRA-6082) 1.1.12 -- 1.2.x upgrade may result inconsistent ring
[ https://issues.apache.org/jira/browse/CASSANDRA-6082?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams reopened CASSANDRA-6082: - 1.1.12 -- 1.2.x upgrade may result inconsistent ring - Key: CASSANDRA-6082 URL: https://issues.apache.org/jira/browse/CASSANDRA-6082 Project: Cassandra Issue Type: Bug Components: Core Environment: 1.1.12 -- 1.2.9 Reporter: Chris Burroughs Priority: Minor Attachments: c-gossipinfo, c-status This happened to me once, and since I don't have any more 1.1.x clusters I won't be testing again. I hope the attached files are enough for someone to connect the dots. I did a rolling restart to upgrade from 1.1.12 -- 1.2.9. About a week later I discovered that one node was in an inconsistent state in the ring. It was either: * up * host-id=null * missing Depending on which node I ran nodetool status from. I *think* I just missed this during the upgrade but can not rule out the possibility that it just happened for no reason some time after the upgrade. It was detected when running repair in such a ring caused all sorts of terrible data duplication and performance tanked. Restarting the seeds + bad node caused the ring to be consistent again. Two possibly suspicious things are a ArrayIndexOutOfBoundsException on startup: {noformat} ERROR [GossipStage:1] 2013-09-06 10:45:35,213 CassandraDaemon.java (line 194) Exception in thread Thread[GossipStage:1,5,main] java.lang.ArrayIndexOutOfBoundsException: 2 at org.apache.cassandra.service.StorageService.extractExpireTime(StorageService.java:1660) at org.apache.cassandra.service.StorageService.handleStateRemoving(StorageService.java:1607) at org.apache.cassandra.service.StorageService.onChange(StorageService.java:1230) at org.apache.cassandra.service.StorageService.onJoin(StorageService.java:1958) at org.apache.cassandra.gms.Gossiper.handleMajorStateChange(Gossiper.java:841) at org.apache.cassandra.gms.Gossiper.applyStateLocally(Gossiper.java:919) at org.apache.cassandra.gms.GossipDigestAck2VerbHandler.doVerb(GossipDigestAck2VerbHandler.java:50) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:56) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {noformat} and problems to hint delivery to multiple node. {noformat} ERROR [MutationStage:11] 2013-09-06 13:59:19,604 CassandraDaemon.java (line 194) Exception in thread Thread[MutationStage:11,5,main] java.lang.AssertionError: Missing host ID for 10.20.2.45 at org.apache.cassandra.service.StorageProxy.writeHintForMutation(StorageProxy.java:583) at org.apache.cassandra.service.StorageProxy$5.runMayThrow(StorageProxy.java:552) at org.apache.cassandra.service.StorageProxy$HintRunnable.run(StorageProxy.java:1658) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303) at java.util.concurrent.FutureTask.run(FutureTask.java:138) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) {noformat} Not however that while there were delivery problems to multiple nodes during the rolling upgrade, only one node was in a funky state a week later. Attached are the results of running gossipinfo and status on every node. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882330#comment-13882330 ] Edward Capriolo commented on CASSANDRA-6620: Dtests may not be revealing the problem for two reasons. 1) You doing 100 operations not 1 for n in xrange(100, 200): insert_c1c2(cursor1, n, QUORUM) query_c1c2(cursor2, n, QUORUM) 2) Every test in the suite has wait_other_notice=True # shutdown another node and test we get unavailabe exception node2.stop(wait_other_notice=True) My test is not waiting for others to notice. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882332#comment-13882332 ] Edward Capriolo commented on CASSANDRA-6620: Right yes. I did not correlate FD with failure detector in your message. I would guess that is the problem. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Resolved] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams resolved CASSANDRA-6620. - Resolution: Not A Problem Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882334#comment-13882334 ] Edward Capriolo commented on CASSANDRA-6620: Why resolve this? It is a problem I think. Cassandra should not time out in this scenario? Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (CASSANDRA-6621) STCS fallback is probably not optimal in some scenarios
Bartłomiej Romański created CASSANDRA-6621: -- Summary: STCS fallback is probably not optimal in some scenarios Key: CASSANDRA-6621 URL: https://issues.apache.org/jira/browse/CASSANDRA-6621 Project: Cassandra Issue Type: Improvement Reporter: Bartłomiej Romański The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my last comment here... After streaming (e.g. during boostrap) Cassandra places all sstables at L0. At the end of the process we end up with huge number of sstables at the lowest level. Currently, Cassandra falls back to STCS until the number of sstables at L0 reaches the reasonable level (32 or something). I'm not sure if falling back to STCS is the best way to handle this particular situation. I've read the comment in the code and I'm aware why it is a good thing to do if we have to many sstables at L0 as a result of too many random inserts. We have a lot of sstables, each of them covers the whole ring, there's simply no better option. However, after the bootstrap situation looks a bit different. The loaded sstables already have very small ranges! We just have to tidy up a bit and everything should be OK. STCS ignores that completely and after a while we have a bit less sstables but each of them covers the whole ring instead of just a small part. I believe that in that case letting LCS do the job is a better option that allowing STCS mix everything up before. Is there a way to disable STCS fallback? I'd like to test that scenario in practice during the our next bootstrap... Does Cassandra really have to put streamed sstables at L0? The only thing we have to assure is that sstables at any given level do not overlap. If we stream different regions from different nodes how can we get any overlaps? -- 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=13882336#comment-13882336 ] Bartłomiej Romański commented on CASSANDRA-5371: I've just created CASSANDRA-6621 describing the issue from the last comments. 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] [Updated] (CASSANDRA-6621) STCS fallback is probably not optimal in some scenarios
[ https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Bartłomiej Romański updated CASSANDRA-6621: --- Description: The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my last comment here... After streaming (e.g. during boostrap) Cassandra places all sstables at L0. At the end of the process we end up with huge number of sstables at the lowest level. Currently, Cassandra falls back to STCS until the number of sstables at L0 reaches the reasonable level (32 or something). I'm not sure if falling back to STCS is the best way to handle this particular situation. I've read the comment in the code and I'm aware why it is a good thing to do if we have to many sstables at L0 as a result of too many random inserts. We have a lot of sstables, each of them covers the whole ring, there's simply no better option. However, after the bootstrap situation looks a bit different. The loaded sstables already have very small ranges! We just have to tidy up a bit and everything should be OK. STCS ignores that completely and after a while we have a bit less sstables but each of them covers the whole ring instead of just a small part. I believe that in that case letting LCS do the job is a better option that allowing STCS mix everything up before. Is there a way to disable STCS fallback? I'd like to test that scenario in practice during our next bootstrap... Does Cassandra really have to put streamed sstables at L0? The only thing we have to assure is that sstables at any given level do not overlap. If we stream different regions from different nodes how can we get any overlaps? was: The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my last comment here... After streaming (e.g. during boostrap) Cassandra places all sstables at L0. At the end of the process we end up with huge number of sstables at the lowest level. Currently, Cassandra falls back to STCS until the number of sstables at L0 reaches the reasonable level (32 or something). I'm not sure if falling back to STCS is the best way to handle this particular situation. I've read the comment in the code and I'm aware why it is a good thing to do if we have to many sstables at L0 as a result of too many random inserts. We have a lot of sstables, each of them covers the whole ring, there's simply no better option. However, after the bootstrap situation looks a bit different. The loaded sstables already have very small ranges! We just have to tidy up a bit and everything should be OK. STCS ignores that completely and after a while we have a bit less sstables but each of them covers the whole ring instead of just a small part. I believe that in that case letting LCS do the job is a better option that allowing STCS mix everything up before. Is there a way to disable STCS fallback? I'd like to test that scenario in practice during the our next bootstrap... Does Cassandra really have to put streamed sstables at L0? The only thing we have to assure is that sstables at any given level do not overlap. If we stream different regions from different nodes how can we get any overlaps? STCS fallback is probably not optimal in some scenarios --- Key: CASSANDRA-6621 URL: https://issues.apache.org/jira/browse/CASSANDRA-6621 Project: Cassandra Issue Type: Improvement Reporter: Bartłomiej Romański The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my last comment here... After streaming (e.g. during boostrap) Cassandra places all sstables at L0. At the end of the process we end up with huge number of sstables at the lowest level. Currently, Cassandra falls back to STCS until the number of sstables at L0 reaches the reasonable level (32 or something). I'm not sure if falling back to STCS is the best way to handle this particular situation. I've read the comment in the code and I'm aware why it is a good thing to do if we have to many sstables at L0 as a result of too many random inserts. We have a lot of sstables, each of them covers the whole ring, there's simply no better option. However, after the bootstrap situation looks a bit different. The loaded sstables already have very small ranges! We just have to tidy up a bit and everything should be OK. STCS ignores that completely and after a while we have a bit less sstables but each of them covers the whole ring instead of just a small part. I believe that in that case letting LCS do the job is a better option that allowing STCS mix everything up before. Is there a way to disable STCS fallback? I'd like to test that scenario in practice during our next bootstrap... Does Cassandra really have to put streamed sstables at L0? The only thing we
[jira] [Reopened] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edward Capriolo reopened CASSANDRA-6620: It is a problem. Writes at ONE should not timeout with RF=2 cluster=3 and only one node down. The user should not have to re-try this write operation. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882342#comment-13882342 ] Brandon Williams commented on CASSANDRA-6620: - That depends on the consistency level. Can you point to where your code is setting that? This should be easily reproducible from the cli/cqlsh, in any case. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882344#comment-13882344 ] Edward Capriolo commented on CASSANDRA-6620: According to the CQL docs the default consistency level is ONE. Look at the exception: {quote} TimedOutException(acknowledged_by:0) {quote} acknoledged_by:0 shows that of the two replicas, one was down, the other should have responded. It is not a socket time out, so a GC does not explain. It is an application level time out. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Comment Edited] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882344#comment-13882344 ] Edward Capriolo edited comment on CASSANDRA-6620 at 1/26/14 4:41 PM: - According to the CQL docs the default consistency level is ONE. Look at the exception: {quote} TimedOutException(acknowledged_by:0) {quote} acknoledged_by:0 shows that of the natural endpoints for the key it was writing to increment, one was down, the other should have responded. It is not a socket time out, so a GC does not explain. It is an application level time out. was (Author: appodictic): According to the CQL docs the default consistency level is ONE. Look at the exception: {quote} TimedOutException(acknowledged_by:0) {quote} acknoledged_by:0 shows that of the two replicas, one was down, the other should have responded. It is not a socket time out, so a GC does not explain. It is an application level time out. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882347#comment-13882347 ] Brandon Williams commented on CASSANDRA-6620: - Can you show me where the CF is created in 'theeNodeTest' here? https://github.com/edwardcapriolo/farsandra/blob/master/src/test/java/io/teknek/farsandra/TestFarsandra.java#L12 Maybe I'm missing something, but it looks like it mostly just sleeps for 10s after it has started. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882353#comment-13882353 ] Edward Capriolo commented on CASSANDRA-6620: My bad. I pushed latest. https://github.com/edwardcapriolo/farsandra/blob/master/src/test/java/io/teknek/farsandra/TestFarsandra.java#L102 Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882354#comment-13882354 ] Edward Capriolo commented on CASSANDRA-6620: I have set it to ALL. Just for argument sake. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882356#comment-13882356 ] Edward Capriolo commented on CASSANDRA-6620: I guess at ALL you could see a TimedOutException or an UnavailableException. At ONE this should always work. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882357#comment-13882357 ] Brandon Williams commented on CASSANDRA-6620: - bq. According to the CQL docs the default consistency level is ONE Not if you're passing it over thrift, which obviously requires the parameter passed outside of the query. bq. I have set it to ALL Why? This is certainly going to fail with either TOE before the FD sees it, or UE after it does. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Resolved] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Edward Capriolo resolved CASSANDRA-6620. Resolution: Fixed My bad Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882361#comment-13882361 ] Edward Capriolo commented on CASSANDRA-6620: Right that makes sense. The strange thing about ALL is you end up in situations with counts of like 11785 before you get a timeout exception. When you would assume, but I guess that is just the instance you killed taking a while to die. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882364#comment-13882364 ] Edward Capriolo commented on CASSANDRA-6620: Question. Do system_* methods ignore the consistency level? Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6590) Gossip does not heal after a temporary partition at startup
[ https://issues.apache.org/jira/browse/CASSANDRA-6590?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Vijay updated CASSANDRA-6590: - Attachment: 0001-logging-for-6590.patch Hi Brandon, Looks like the realMarkAlive is called multiple times and hence the issue i removed the localState.markDead() and it works fine for now (Attached patch). Let me know... Gossip does not heal after a temporary partition at startup --- Key: CASSANDRA-6590 URL: https://issues.apache.org/jira/browse/CASSANDRA-6590 Project: Cassandra Issue Type: Bug Components: Core Reporter: Brandon Williams Assignee: Vijay Fix For: 2.0.5 Attachments: 0001-CASSANDRA-6590.patch, 0001-logging-for-6590.patch, 6590_disable_echo.txt See CASSANDRA-6571 for background. If a node is partitioned on startup when the echo command is sent, but then the partition heals, the halves of the partition will never mark each other up despite being able to communicate. This stems from CASSANDRA-3533. -- 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] [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=13882495#comment-13882495 ] Brandon Williams commented on CASSANDRA-6622: - The failure detector shouldn't be tracking any nodes with a dead state (since they're already dead.) It sounds like you're doing a replace on the same IP address, before the failure detector has marked the original node being replaced down. 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-5549) Remove Table.switchLock
[ https://issues.apache.org/jira/browse/CASSANDRA-5549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882570#comment-13882570 ] Jonathan Ellis commented on CASSANDRA-5549: --- Okay, went with plan B of unifying MT/Pool and MO/Allocator. Pretty happy with how this worked out: https://github.com/jbellis/cassandra/commits/5549-3 In my mind, off heap stuff can now be added by creating an appropriate Allocator class, and a Pool implementation that owns two sub-pools. (I'm seeing a bunch of test failures but I don't think it's from this refactor.) Remove Table.switchLock --- Key: CASSANDRA-5549 URL: https://issues.apache.org/jira/browse/CASSANDRA-5549 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Benedict Labels: performance Fix For: 2.1 Attachments: 5549-removed-switchlock.png, 5549-sunnyvale.png As discussed in CASSANDRA-5422, Table.switchLock is a bottleneck on the write path. ReentrantReadWriteLock is not lightweight, even if there is no contention per se between readers and writers of the lock (in Cassandra, memtable updates and switches). -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5549) Remove Table.switchLock
[ https://issues.apache.org/jira/browse/CASSANDRA-5549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882591#comment-13882591 ] Benedict commented on CASSANDRA-5549: - I'm not totally convinced this is better, but I think I can make it worth with the off-heap stuff. I think it will get a little ugly, as we'll need two allocators as well - we'll need some method in the AbstractAllocator interface to getOffHeapPartner() and getOnHeapPartner(), or something, so that Memtable can avoid special casing the OffHeapAllocator. But the OnHeapPartner won't actually support allocating BBs, it will just be for querying and updating amounts of memory. Not relishing it, but don't mind pushing it off until we have to address that problem. Remove Table.switchLock --- Key: CASSANDRA-5549 URL: https://issues.apache.org/jira/browse/CASSANDRA-5549 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Benedict Labels: performance Fix For: 2.1 Attachments: 5549-removed-switchlock.png, 5549-sunnyvale.png As discussed in CASSANDRA-5422, Table.switchLock is a bottleneck on the write path. ReentrantReadWriteLock is not lightweight, even if there is no contention per se between readers and writers of the lock (in Cassandra, memtable updates and switches). -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5549) Remove Table.switchLock
[ https://issues.apache.org/jira/browse/CASSANDRA-5549?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882598#comment-13882598 ] Jonathan Ellis commented on CASSANDRA-5549: --- Fair enough. Remove Table.switchLock --- Key: CASSANDRA-5549 URL: https://issues.apache.org/jira/browse/CASSANDRA-5549 Project: Cassandra Issue Type: Bug Reporter: Jonathan Ellis Assignee: Benedict Labels: performance Fix For: 2.1 Attachments: 5549-removed-switchlock.png, 5549-sunnyvale.png As discussed in CASSANDRA-5422, Table.switchLock is a bottleneck on the write path. ReentrantReadWriteLock is not lightweight, even if there is no contention per se between readers and writers of the lock (in Cassandra, memtable updates and switches). -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6620) Collateral damage from killing a node
[ https://issues.apache.org/jira/browse/CASSANDRA-6620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13882613#comment-13882613 ] Jonathan Ellis commented on CASSANDRA-6620: --- If you mean system_ tables, then they do effectively ignore CL because they are stored with LocalStrategy, i.e. each node stores its own copy with no replication. Collateral damage from killing a node - Key: CASSANDRA-6620 URL: https://issues.apache.org/jira/browse/CASSANDRA-6620 Project: Cassandra Issue Type: Bug Reporter: Edward Capriolo Attachments: nodelogs.txt I have designed a new scenario with farsandra: 3 nodes with Replication factor = 2 a Counter column family. I perform 10,000 inserts to node 1. I kill off node 2, do 1 more inserts. restart node 2. Sometimes I made it completely though this test. However sometimes I do not. I have seen the client throw time out exceptions. it seems like the death of node 2 greatly upsets node 1 and it times out a request. Since the default is ready 1 should this be happening? -- This message was sent by Atlassian JIRA (v6.1.5#6160)