[jira] [Updated] (CASSANDRA-6558) Failure Detector takes 4-5x longer than it used to
[ https://issues.apache.org/jira/browse/CASSANDRA-6558?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-6558: Priority: Minor (was: Major) Failure Detector takes 4-5x longer than it used to -- Key: CASSANDRA-6558 URL: https://issues.apache.org/jira/browse/CASSANDRA-6558 Project: Cassandra Issue Type: Bug Reporter: Joaquin Casares Priority: Minor Labels: datastax_qa The Failure Detector appears to also be used by the java-driver (https://datastax-oss.atlassian.net/browse/JAVA-246) in determining if nodes are down or not. Because of the recent increase in time that it takes for Cassandra to noticed downed nodes, tests within the java-driver integration suite are currently failing. This should only impact driver usage minimally since it also relies on failed requests to find out if a node is down, but fixing the issue means we get the tests back online and the Failure Detector working as it previously had been. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6558) Failure Detector takes 4-5x longer than it used to
[ https://issues.apache.org/jira/browse/CASSANDRA-6558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865188#comment-13865188 ] Sylvain Lebresne commented on CASSANDRA-6558: - bq. We could add a private method to the FD to basically disable CASSANDRA-6385 that you could call. That said, using the FD on the client side smells funny to me. Just so there is no misunderstanding, the java driver does not use the FD code client side. It's just that there is whole family of tests in which you test behaviors that only apply when a dead node has been detected as such by other nodes, and so such tests will kill a node and then wait for the FD to kick in before continuing (testing for UnavailableException, testing native protocol notification (that are triggered by the FD), etc...). This does not affect just the java driver in particular, I strongly suspect many dtests are aso made a lot slower by this. Maybe we can just add some startup variable to set the initial FD interval (to basically allow disabling CASSANDRA-6385 for testing) the same way we allow tweaking ring_delay for tests? Failure Detector takes 4-5x longer than it used to -- Key: CASSANDRA-6558 URL: https://issues.apache.org/jira/browse/CASSANDRA-6558 Project: Cassandra Issue Type: Bug Reporter: Joaquin Casares Priority: Minor Labels: datastax_qa The Failure Detector appears to also be used by the java-driver (https://datastax-oss.atlassian.net/browse/JAVA-246) in determining if nodes are down or not. Because of the recent increase in time that it takes for Cassandra to noticed downed nodes, tests within the java-driver integration suite are currently failing. This should only impact driver usage minimally since it also relies on failed requests to find out if a node is down, but fixing the issue means we get the tests back online and the Failure Detector working as it previously had been. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6522) DroppableTombstoneRatio JMX value is 0.0 for all CFs
[ https://issues.apache.org/jira/browse/CASSANDRA-6522?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-6522: --- Attachment: 0001-account-for-range-and-row-tombstones-in-tombstone-dr.patch Patch that accounts for row and range tombstones in the histogram. Also prints the histogram in tools/bin/sstablemetadata. Note that it is very hard to cheaply know how many live columns a range tombstone shadows, so this metric is only the tombstone count / number of columns, which in this case would be pretty pointless. CQL deletes were not accounted for before this, so it should also help for tombstone-only compaction in those cases. DroppableTombstoneRatio JMX value is 0.0 for all CFs Key: CASSANDRA-6522 URL: https://issues.apache.org/jira/browse/CASSANDRA-6522 Project: Cassandra Issue Type: Bug Components: Core Environment: Ubuntu 12.04 LTS, Cassandra 1.2.8 Reporter: Daniel Kador Assignee: Marcus Eriksson Priority: Minor Fix For: 2.0.5 Attachments: 0001-account-for-range-and-row-tombstones-in-tombstone-dr.patch We're seeing that the JMX value for DroppableTombstoneRatio for all our CFs is 0.0. On the face of it that seems wrong since we've definitely issued a ton of deletes for row keys to expire some old data that we no longer need (and it definitely hasn't been reclaimed from disk yet). Am I misunderstanding what this means / how to use it? We're on 1.2.8 and using leveled compaction for all our CFs. gc_grace_seconds is set to 1 day and we've issued a series of deletes over a day ago, so gc_grace has elapsed. Cluster is 18 nodes. Two DCs, so 9 nodes in each DC. Each node has capacity for 1.5TB or so and is sitting with about 1TB under management. That's why we wanted to do deletes, obviously. Most of that 1TB is a single CF (called events) which represents intermediate state for us that we can delete. Happy to provide any more info, just let me know. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5357) Query cache / partition head cache
[ https://issues.apache.org/jira/browse/CASSANDRA-5357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865208#comment-13865208 ] Sylvain Lebresne commented on CASSANDRA-5357: - bq. Personally I'd lean towards (2) Personally, I'd lean towards both :). That is, I'd add a new rows_per_partition_to_cache table option that would either be a user set fixed value, or some auto (the default presumably) that we would determine automatically. Of course, in the interest of shipping sooner, the auto option could be added later on. But while I'm all for having smart automatic default options that most user don't have to ever change, it seems to me that for something as important as caching, there will always be cases where the user will know better than whatever heuristic we come up. bq. I think this also means we should go back to a separate cache per CF with its own size limit – if we have 1000 queries/s against CF X's cache, then we shouldn't throw those away when a query against CF Y comes in where we expect only 10/s It seems to me that this reasoning apply equally well to the current row cache. Is there something specific to this ticket that makes you say that, or is it just saying making the caches global was possibly a mistake we'd want to reconsider? For what is worth, when we made the caches global, that kind of objection was raised and the answer had been that you could disable caching for CF Y to avoid that and that if that was not enough we'd add optional per-CF quota on top of the global one later on. Overall, I do think we really should maintain a global limit on how much is cached, though I don't disagree that some finer per-CF quotas could be desirable. Query cache / partition head cache -- Key: CASSANDRA-5357 URL: https://issues.apache.org/jira/browse/CASSANDRA-5357 Project: Cassandra Issue Type: New Feature Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 I think that most people expect the row cache to act like a query cache, because that's a reasonable model. Caching the entire partition is, in retrospect, not really reasonable, so it's not surprising that it catches people off guard, especially given the confusion we've inflicted on ourselves as to what a row constitutes. I propose replacing it with a true query cache. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5357) Query cache / partition head cache
[ https://issues.apache.org/jira/browse/CASSANDRA-5357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865383#comment-13865383 ] Jonathan Ellis commented on CASSANDRA-5357: --- bq. It seems to me that this reasoning apply equally well to the current row cache. It would, but the current row cache is dangerous enough that I don't want to spend effort making it smarter. :) bq. I do think we really should maintain a global limit on how much is cached Agreed 100%; I'm just saying we should apportion that to the CFs based on our metrics. In any case, I'm getting ahead of what we should focus on for 2.1.0 here. Query cache / partition head cache -- Key: CASSANDRA-5357 URL: https://issues.apache.org/jira/browse/CASSANDRA-5357 Project: Cassandra Issue Type: New Feature Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 I think that most people expect the row cache to act like a query cache, because that's a reasonable model. Caching the entire partition is, in retrospect, not really reasonable, so it's not surprising that it catches people off guard, especially given the confusion we've inflicted on ourselves as to what a row constitutes. I propose replacing it with a true query cache. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6544) Reduce GC activity during compaction
[ https://issues.apache.org/jira/browse/CASSANDRA-6544?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865388#comment-13865388 ] Benedict commented on CASSANDRA-6544: - Good point. I was looking at the 2.0 branch I briefly had checked out. That said, it could still be beneficial, and would require an even smaller buffer so less downside as well. No real need to move the buffer off heap, but re-using a buffer could help avoid both young-gen churn and also, if the tables being merged are large and have long intervals where neither overlap, could help mitigate promotion from eden so help keep pauses low. That all said, though, for small rows the general object overhead of LCR, LCR.Reducer, ColumnStats etc are likely to outweigh any BB costs now. Reduce GC activity during compaction Key: CASSANDRA-6544 URL: https://issues.apache.org/jira/browse/CASSANDRA-6544 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Vijay Assignee: Vijay Fix For: 2.1 We are noticing increase in P99 while the compactions are running at full stream. Most of it is because of the increased GC activity (followed by full GC). The obvious solution/work around is to throttle the compactions, but with SSD's we can get more disk bandwidth for reads and compactions. It will be nice to move the compaction object allocations off heap. First thing to do might be create a Offheap Slab allocator with the size as the compaction in memory size and recycle it. Also we might want to make it configurable so folks can disable it when they don't have off-heap memory to reserve. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5357) Query cache / partition head cache
[ https://issues.apache.org/jira/browse/CASSANDRA-5357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865409#comment-13865409 ] Sylvain Lebresne commented on CASSANDRA-5357: - bq. In any case, I'm getting ahead of what we should focus on for 2.1.0 here. Absolutely :) Query cache / partition head cache -- Key: CASSANDRA-5357 URL: https://issues.apache.org/jira/browse/CASSANDRA-5357 Project: Cassandra Issue Type: New Feature Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 I think that most people expect the row cache to act like a query cache, because that's a reasonable model. Caching the entire partition is, in retrospect, not really reasonable, so it's not surprising that it catches people off guard, especially given the confusion we've inflicted on ourselves as to what a row constitutes. I propose replacing it with a true query cache. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
git commit: Secondary indexing of map keys
Updated Branches: refs/heads/trunk d63d07b92 - c2294aa21 Secondary indexing of map keys patch by slebresne; reviewed by iamaleksey for CASSANDRA-6383 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c2294aa2 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c2294aa2 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c2294aa2 Branch: refs/heads/trunk Commit: c2294aa21eb6310b6d5c05d6d9ff505f59b376c2 Parents: d63d07b Author: Sylvain Lebresne sylv...@datastax.com Authored: Thu Dec 19 14:52:13 2013 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Wed Jan 8 14:34:19 2014 +0100 -- CHANGES.txt | 2 +- src/java/org/apache/cassandra/cql3/Cql.g| 12 +- .../cql3/statements/CreateIndexStatement.java | 36 - .../cassandra/cql3/statements/IndexTarget.java | 42 .../db/index/composites/CompositesIndex.java| 11 - 5 files changed, 88 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2294aa2/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 24722b8..58a0906 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -15,7 +15,7 @@ * User-defined types for CQL3 (CASSANDRA-5590) * Use of o.a.c.metrics in nodetool (CASSANDRA-5871, 6406) * Batch read from OTC's queue and cleanup (CASSANDRA-1632) - * Secondary index support for collections (CASSANDRA-4511) + * Secondary index support for collections (CASSANDRA-4511, 6383) * SSTable metadata(Stats.db) format change (CASSANDRA-6356) * Push composites support in the storage engine (CASSANDRA-5417, CASSANDRA-6520) http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2294aa2/src/java/org/apache/cassandra/cql3/Cql.g -- diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g index f9f66df..11291b6 100644 --- a/src/java/org/apache/cassandra/cql3/Cql.g +++ b/src/java/org/apache/cassandra/cql3/Cql.g @@ -552,12 +552,18 @@ createIndexStatement returns [CreateIndexStatement expr] boolean ifNotExists = false; } : K_CREATE (K_CUSTOM { props.isCustom = true; })? K_INDEX (K_IF K_NOT K_EXISTS { ifNotExists = true; } )? -(idxName=IDENT)? K_ON cf=columnFamilyName '(' id=cident ')' +(idxName=IDENT)? K_ON cf=columnFamilyName '(' id=indexIdent ')' (K_USING cls=STRING_LITERAL { props.customClass = $cls.text; })? (K_WITH properties[props])? { $expr = new CreateIndexStatement(cf, $idxName.text, id, props, ifNotExists); } ; +indexIdent returns [IndexTarget id] +: c=cident{ $id = IndexTarget.of(c); } +| K_KEYS '(' c=cident ')' { $id = IndexTarget.keysOf(c); } +; + + /** * CREATE TRIGGER triggerName ON columnFamily USING 'triggerClass'; */ @@ -958,7 +964,7 @@ relation[ListRelation clauses] { $clauses.add(new Relation(name, Relation.Type.IN, marker)); } | name=cident K_IN { Relation rel = Relation.createInRelation($name.id); } '(' ( f1=term { rel.addInValue(f1); } (',' fN=term { rel.addInValue(fN); } )* )? ')' { $clauses.add(rel); } -| name=cident K_CONTAINS { Relation.Type rt = Relation.Type.CONTAINS; } /* (K_KEY { rt = Relation.Type.CONTAINS_KEY })? */ +| name=cident K_CONTAINS { Relation.Type rt = Relation.Type.CONTAINS; } (K_KEY { rt = Relation.Type.CONTAINS_KEY; })? t=term { $clauses.add(new Relation(name, rt, t)); } | '(' relation[$clauses] ')' ; @@ -1036,6 +1042,7 @@ unreserved_function_keyword returns [String str] basic_unreserved_keyword returns [String str] : k=( K_KEY +| K_KEYS | K_AS | K_CLUSTERING | K_COMPACT @@ -1070,6 +1077,7 @@ K_AS: A S; K_WHERE: W H E R E; K_AND: A N D; K_KEY: K E Y; +K_KEYS:K E Y S; K_INSERT: I N S E R T; K_UPDATE: U P D A T E; K_WITH:W I T H; http://git-wip-us.apache.org/repos/asf/cassandra/blob/c2294aa2/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java -- diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java index 56b465e..ca43d20 100644 --- a/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/CreateIndexStatement.java @@ -29,6 +29,8 @@ import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.config.ColumnDefinition; import
[jira] [Updated] (CASSANDRA-6157) Selectively Disable hinted handoff for a data center
[ https://issues.apache.org/jira/browse/CASSANDRA-6157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lyuben Todorov updated CASSANDRA-6157: -- Attachment: trunk-6157-v3.diff Patch looks good except for a few nits, attaching v3 with some minor fixes: - changed message for hinted_handoff_enabled in cassandra.yaml - kept globally enabled hints as the default setting ({{hinted_handoff_enabled: true}}) - Added space after {{if}} statement @ SProxy#shouldHint - lowercase .equals true / .equals false to save 2 checks. - Added {{.toLowerCase()}} in Config#initializeCustomConfig to save 2 more checks in the if statement - Renamed {{initializeCustomConfig}} to {{configHintedHandoff}} as custom config sounds too general. Selectively Disable hinted handoff for a data center Key: CASSANDRA-6157 URL: https://issues.apache.org/jira/browse/CASSANDRA-6157 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Fix For: 2.0.5 Attachments: trunk-6157-v2.diff, trunk-6157-v3.diff, trunk-6157.txt Cassandra supports disabling the hints or reducing the window for hints. It would be helpful to have a switch which stops hints to a down data center but continue hints to other DCs. This is helpful during data center fail over as hints will put more unnecessary pressure on the DC taking double traffic. Also since now Cassandra is under reduced reduncany, we don't want to disable hints within the DC. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5357) Query cache / partition head cache
[ https://issues.apache.org/jira/browse/CASSANDRA-5357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865495#comment-13865495 ] Marcus Eriksson commented on CASSANDRA-5357: So, trying to distill what we actually want here; * Make it possible to cache parts of the partitions (CASSANDRA-1956) (head, tail and all perhaps, for users who use it today) * Don't evict entire partitions on writes (CASSANDRA-2864) * Add heuristics to guess the number of columns to be cached (maybe push to later versions, configurable sizes could be fine for now) anything else? [~jbellis] [~slebresne] Query cache / partition head cache -- Key: CASSANDRA-5357 URL: https://issues.apache.org/jira/browse/CASSANDRA-5357 Project: Cassandra Issue Type: New Feature Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 I think that most people expect the row cache to act like a query cache, because that's a reasonable model. Caching the entire partition is, in retrospect, not really reasonable, so it's not surprising that it catches people off guard, especially given the confusion we've inflicted on ourselves as to what a row constitutes. I propose replacing it with a true query cache. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5357) Query cache / partition head cache
[ https://issues.apache.org/jira/browse/CASSANDRA-5357?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865515#comment-13865515 ] Jonathan Ellis commented on CASSANDRA-5357: --- Yup, good summary. I'm getting Time To Ship Religion now so for our purposes here I vote for head cache only and sylvain's rows_per_partition_to_cache table option. Let's reopen 2864 and fancy heuristics as a follow up. Query cache / partition head cache -- Key: CASSANDRA-5357 URL: https://issues.apache.org/jira/browse/CASSANDRA-5357 Project: Cassandra Issue Type: New Feature Reporter: Jonathan Ellis Assignee: Marcus Eriksson Fix For: 2.1 I think that most people expect the row cache to act like a query cache, because that's a reasonable model. Caching the entire partition is, in retrospect, not really reasonable, so it's not surprising that it catches people off guard, especially given the confusion we've inflicted on ourselves as to what a row constitutes. I propose replacing it with a true query cache. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6558) Failure Detector takes 4-5x longer than it used to
[ https://issues.apache.org/jira/browse/CASSANDRA-6558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865519#comment-13865519 ] Brandon Williams commented on CASSANDRA-6558: - bq. Just so there is no misunderstanding, the java driver does not use the FD code client side. It's just that there is whole family of tests in which you test behaviors that only apply when a dead node has been detected as such by other nodes, and so such tests will kill a node and then wait for the FD to kick in If the shutdown is clean, why isn't CASSANDRA-3936 saving us? Or does the shutdown need to be unclean to test some behavior? bq. Maybe we can just add some startup variable to set the initial FD interval That would be CASSANDRA-4375, I'll try to get to that today. Failure Detector takes 4-5x longer than it used to -- Key: CASSANDRA-6558 URL: https://issues.apache.org/jira/browse/CASSANDRA-6558 Project: Cassandra Issue Type: Bug Reporter: Joaquin Casares Priority: Minor Labels: datastax_qa The Failure Detector appears to also be used by the java-driver (https://datastax-oss.atlassian.net/browse/JAVA-246) in determining if nodes are down or not. Because of the recent increase in time that it takes for Cassandra to noticed downed nodes, tests within the java-driver integration suite are currently failing. This should only impact driver usage minimally since it also relies on failed requests to find out if a node is down, but fixing the issue means we get the tests back online and the Failure Detector working as it previously had been. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6440) Repair should allow repairing particular endpoints to reduce WAN usage.
[ https://issues.apache.org/jira/browse/CASSANDRA-6440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lyuben Todorov updated CASSANDRA-6440: -- Attachment: 6440_repair.log Say we have a 4 node cluster composed of 2 DCs. Node 1 4 are in DC1 and node 2 3 are in DC2, we carry out a repair on node DC1 (node 1 4, via {{./nodetool repair -hosts 10.0.0.1,10.0.0.4}} All goes well, except for the system_traces keyspace where the non-neighbour error gets thrown. I'll attach a log with a few System.out statements added to show the keyspace being repaired, the hosts supplied and neighbours. Repair should allow repairing particular endpoints to reduce WAN usage. Key: CASSANDRA-6440 URL: https://issues.apache.org/jira/browse/CASSANDRA-6440 Project: Cassandra Issue Type: New Feature Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Attachments: 6440_repair.log, JIRA-6440.diff The way we send out data that does not match over WAN can be improved. Example: Say there are four nodes(A,B,C,D) which are replica of a range we are repairing. A, B is in DC1 and C,D is in DC2. If A does not have the data which other replicas have, then we will have following streams 1) A to B and back 2) A to C and back(Goes over WAN) 3) A to D and back(Goes over WAN) One of the ways of doing it to reduce WAN traffic is this. 1) Repair A and B only with each other and C and D with each other starting at same time t. 2) Once these repairs have finished, A,B and C,D are in sync with respect to time t. 3) Now run a repair between A and C, the streams which are exchanged as a result of the diff will also be streamed to B and D via A and C(C and D behaves like a proxy to the streams). For a replication of DC1:2,DC2:2, the WAN traffic will get reduced by 50% and even more for higher replication factors. Another easy way to do this is to have repair command take nodes with which you want to repair with. Then we can do something like this. 1) Run repair between (A and B) and (C and D) 2) Run repair between (A and C) 3) Run repair between (A and B) and (C and D) But this will increase the traffic inside the DC as we wont be doing proxy. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6381) Refactor nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-6381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Clément Lardeur updated CASSANDRA-6381: --- Attachment: trunk-6381-v2.patch +Command specific comments+ : bq. INFO - prints the token at the end of output. The old implementation prints it first. Not sure if it's important It's printed at the end of output to keep the same behavior with or without -T option. bq. REBUILD_INDEX - The 3rd argument (comma separated list of indexes) will not be split, and will be used as is. The old behavior (csv format) was not really POSIX-like argument conventions. bq. REMOVETOKEN - will just print a warning and exit. The old implementation would call REMOVENODE after a warning. Not sure if it's important I don't know if it's important. The message is : _Warn: removetoken is deprecated, please use removenode instead_ bq. REPAIR - I believe !startToken.isEmpty() !endToken.isEmpty() condition is wrong, and doesn’t match the original. I ask myself if the old condition was good or not. Is it possible to have just the option --start-token without --end-token and vice versa? bq. SETCOMPACTIONTHRESHOLD - the combination of minthreshold == maxthreshold == 0 is no more valid. Not sure if it’s important It was not possible before, see the condition in NodeCmd : {code:java} if (minthreshold 2 maxthreshold != 0){ badUse(Min threshold must be at least 2); } {code} bq. STATUS - doesn’t contain changes from CASSANDRA-2238 ( https://github.com/apache/cassandra/commit/76ee9a155ea14304595bc2e9755accbfded04e62) Add the --resolve-ip option. *Done.* +Missing commands+ : * DISABLEHANDOFF - *Done.* * DISABLETHRIFT - *Done.* * DRAIN - *Done.* * TPSTATS - *Done.* * TRUNCATEHINTS - *Done.* +General comment+ : Move into {{NodeToolCmd}} the duplicated code and create the abstract method {{execute()}} implemented by each specific command. Refactor nodetool - Key: CASSANDRA-6381 URL: https://issues.apache.org/jira/browse/CASSANDRA-6381 Project: Cassandra Issue Type: Improvement Reporter: Yuki Morishita Assignee: Clément Lardeur Priority: Minor Labels: lhf, nodetool Attachments: trunk-6381-v2.patch, trunk-6381.patch We have way too many nodetool commands(more than 40) packed in one NodeCmd class. And we are trying to add more commands. https://github.com/airlift/airline could be a good fit to take out each command into sub command class. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6558) Failure Detector takes 4-5x longer than it used to
[ https://issues.apache.org/jira/browse/CASSANDRA-6558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865547#comment-13865547 ] Sylvain Lebresne commented on CASSANDRA-6558: - bq. If the shutdown is clean, why isn't CASSANDRA-3936 saving us? Or does the shutdown need to be unclean to test some behavior? The java driver tests does seem to be using a lot of unclean shutdown, so it's totally possible CASSANDRA-3936 would save us otherwise. Joaquim, can you check if changing the forceStop() to simple stop() fixes it? That being said, it does is useful to also test that unclean shutdown don't mess things up, if only for dtests if not in the java driver. bq. That would be CASSANDRA-4375 Great, let's close that one as duplicate then Failure Detector takes 4-5x longer than it used to -- Key: CASSANDRA-6558 URL: https://issues.apache.org/jira/browse/CASSANDRA-6558 Project: Cassandra Issue Type: Bug Reporter: Joaquin Casares Priority: Minor Labels: datastax_qa The Failure Detector appears to also be used by the java-driver (https://datastax-oss.atlassian.net/browse/JAVA-246) in determining if nodes are down or not. Because of the recent increase in time that it takes for Cassandra to noticed downed nodes, tests within the java-driver integration suite are currently failing. This should only impact driver usage minimally since it also relies on failed requests to find out if a node is down, but fixing the issue means we get the tests back online and the Failure Detector working as it previously had been. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Resolved] (CASSANDRA-6558) Failure Detector takes 4-5x longer than it used to
[ https://issues.apache.org/jira/browse/CASSANDRA-6558?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne resolved CASSANDRA-6558. - Resolution: Duplicate Failure Detector takes 4-5x longer than it used to -- Key: CASSANDRA-6558 URL: https://issues.apache.org/jira/browse/CASSANDRA-6558 Project: Cassandra Issue Type: Bug Reporter: Joaquin Casares Priority: Minor Labels: datastax_qa The Failure Detector appears to also be used by the java-driver (https://datastax-oss.atlassian.net/browse/JAVA-246) in determining if nodes are down or not. Because of the recent increase in time that it takes for Cassandra to noticed downed nodes, tests within the java-driver integration suite are currently failing. This should only impact driver usage minimally since it also relies on failed requests to find out if a node is down, but fixing the issue means we get the tests back online and the Failure Detector working as it previously had been. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-4375) FD incorrectly using RPC timeout to ignore gossip heartbeats
[ https://issues.apache.org/jira/browse/CASSANDRA-4375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-4375: Attachment: 4375.txt I've thought about this a bit, and still think it should default to ring delay, but not be coupled to it. However, in normal operation, I do think it makes send to couple the initial value we seed the FD with and the max interval we accept. I don't think most people should be tweaking these though, so I've made them system properties (as ring delay is.) Patch adds cassandra.fd_initial_value_ms to control the value the FD is seeded with, which the max interval will also default to, but also adds cassandra.fd_max_interval_ms if you really need them to be disjoint (most likely for testing like CASSANDRA-6558 where you want the seed ridiculously low, but the max interval reasonable.) I will note that I changed the max interval from a double to an int, because a double just didn't make any sense. FD incorrectly using RPC timeout to ignore gossip heartbeats Key: CASSANDRA-4375 URL: https://issues.apache.org/jira/browse/CASSANDRA-4375 Project: Cassandra Issue Type: Bug Components: Core Reporter: Peter Schuller Assignee: Brandon Williams Labels: gossip Fix For: 1.2.14 Attachments: 4375.txt Short version: You can't run a cluster with short RPC timeouts because nodes just constantly flap up/down. Long version: CASSANDRA-3273 tried to fix a problem resulting from the way the failure detector works, but did so by introducing a much more sever bug: With low RPC timeouts, that are lower than the typical gossip propagation time, a cluster will just constantly have all nodes flapping other nodes up and down. The cause is this: {code} +// in the event of a long partition, never record an interval longer than the rpc timeout, +// since if a host is regularly experiencing connectivity problems lasting this long we'd +// rather mark it down quickly instead of adapting +private final double MAX_INTERVAL_IN_MS = DatabaseDescriptor.getRpcTimeout(); {code} And then: {code} -tLast_ = value; -arrivalIntervals_.add(interArrivalTime); +if (interArrivalTime = MAX_INTERVAL_IN_MS) +arrivalIntervals_.add(interArrivalTime); +else +logger_.debug(Ignoring interval time of {}, interArrivalTime); {code} Using the RPC timeout to ignore unreasonably long intervals is not correct, as the RPC timeout is completely orthogonal to gossip propagation delay (see CASSANDRA-3927 for a quick description of how the FD works). In practice, the propagation delay ends up being in the 0-3 second range on a cluster with good local latency. With a low RPC timeout of say 200 ms, very few heartbeat updates come in fast enough that it doesn't get ignored by the failure detector. This in turn means that the FD records a completely skewed average heartbeat interval, which in turn means that nodes almost always get flapped on interpret() unless they happen to *just* have had their heartbeat updated. Then they flap back up whenever the next heartbeat comes in (since it gets brought up immediately). In our build, we are replacing the FD with an implementation that simply uses a fixed {{N}} second time to convict, because this is just one of many ways in which the current FD hurts, while we still haven't found a way it actually helps relative to the trivial fixed-second conviction policy. For upstream, assuming people won't agree on changing it to a fixed timeout, I suggest, at minimum, never using a value lower than something like 10 seconds or something, when determining whether to ignore. Slightly better is to make it a config option. (I should note that if propagation delays are significantly off from the expected level, other things than the FD already breaks - such as the whole concept of {{RING_DELAY}}, which assumes the propagation time is roughly constant with e.g. cluster size.) -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-4375) FD incorrectly using RPC timeout to ignore gossip heartbeats
[ https://issues.apache.org/jira/browse/CASSANDRA-4375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-4375: Attachment: (was: 4375.txt) FD incorrectly using RPC timeout to ignore gossip heartbeats Key: CASSANDRA-4375 URL: https://issues.apache.org/jira/browse/CASSANDRA-4375 Project: Cassandra Issue Type: Bug Components: Core Reporter: Peter Schuller Assignee: Brandon Williams Labels: gossip Fix For: 1.2.14 Attachments: 4375.txt Short version: You can't run a cluster with short RPC timeouts because nodes just constantly flap up/down. Long version: CASSANDRA-3273 tried to fix a problem resulting from the way the failure detector works, but did so by introducing a much more sever bug: With low RPC timeouts, that are lower than the typical gossip propagation time, a cluster will just constantly have all nodes flapping other nodes up and down. The cause is this: {code} +// in the event of a long partition, never record an interval longer than the rpc timeout, +// since if a host is regularly experiencing connectivity problems lasting this long we'd +// rather mark it down quickly instead of adapting +private final double MAX_INTERVAL_IN_MS = DatabaseDescriptor.getRpcTimeout(); {code} And then: {code} -tLast_ = value; -arrivalIntervals_.add(interArrivalTime); +if (interArrivalTime = MAX_INTERVAL_IN_MS) +arrivalIntervals_.add(interArrivalTime); +else +logger_.debug(Ignoring interval time of {}, interArrivalTime); {code} Using the RPC timeout to ignore unreasonably long intervals is not correct, as the RPC timeout is completely orthogonal to gossip propagation delay (see CASSANDRA-3927 for a quick description of how the FD works). In practice, the propagation delay ends up being in the 0-3 second range on a cluster with good local latency. With a low RPC timeout of say 200 ms, very few heartbeat updates come in fast enough that it doesn't get ignored by the failure detector. This in turn means that the FD records a completely skewed average heartbeat interval, which in turn means that nodes almost always get flapped on interpret() unless they happen to *just* have had their heartbeat updated. Then they flap back up whenever the next heartbeat comes in (since it gets brought up immediately). In our build, we are replacing the FD with an implementation that simply uses a fixed {{N}} second time to convict, because this is just one of many ways in which the current FD hurts, while we still haven't found a way it actually helps relative to the trivial fixed-second conviction policy. For upstream, assuming people won't agree on changing it to a fixed timeout, I suggest, at minimum, never using a value lower than something like 10 seconds or something, when determining whether to ignore. Slightly better is to make it a config option. (I should note that if propagation delays are significantly off from the expected level, other things than the FD already breaks - such as the whole concept of {{RING_DELAY}}, which assumes the propagation time is roughly constant with e.g. cluster size.) -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Created] (CASSANDRA-6559) cqlsh should warn about ALLOW FILTERING
Tupshin Harper created CASSANDRA-6559: - Summary: cqlsh should warn about ALLOW FILTERING Key: CASSANDRA-6559 URL: https://issues.apache.org/jira/browse/CASSANDRA-6559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Tupshin Harper Assignee: Aleksey Yeschenko Fix For: 2.0.5 ALLOW FILTERING can be a convenience for preliminary exploration of your data, and can be useful for batch jobs, but it is such an anti-pattern for regular production queries, that cqlsh should provie an explicit warn ingwhenever such a query is performed. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6530) Fix logback configuration in scripts and debian packaging for trunk/2.1
[ https://issues.apache.org/jira/browse/CASSANDRA-6530?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865583#comment-13865583 ] Brandon Williams commented on CASSANDRA-6530: - I was wrong, this doesn't suppress the warnings, they just don't repro 100% of the time. It seems that logback is ignoring us here, but as far as I can tell we have it configured correctly. Fix logback configuration in scripts and debian packaging for trunk/2.1 --- Key: CASSANDRA-6530 URL: https://issues.apache.org/jira/browse/CASSANDRA-6530 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Michael Shuler Assignee: Michael Shuler Priority: Minor Fix For: 2.1 Attachments: logback_configurations_final.patch -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6405) When making heavy use of counters, neighbor nodes occasionally enter spiral of constant memory consumpion
[ https://issues.apache.org/jira/browse/CASSANDRA-6405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865589#comment-13865589 ] Aleksey Yeschenko commented on CASSANDRA-6405: -- [~alienth] No built-in metrics that comes to mind. [~slebresne] any ideas? When making heavy use of counters, neighbor nodes occasionally enter spiral of constant memory consumpion - Key: CASSANDRA-6405 URL: https://issues.apache.org/jira/browse/CASSANDRA-6405 Project: Cassandra Issue Type: Bug Environment: RF of 3, 15 nodes. Sun Java 7 (also occurred in OpenJDK 6, and Sun Java 6). Xmx of 8G. No row cache. Reporter: Jason Harvey Attachments: threaddump.txt We're randomly running into an interesting issue on our ring. When making use of counters, we'll occasionally have 3 nodes (always neighbors) suddenly start immediately filling up memory, CMSing, fill up again, repeat. This pattern goes on for 5-20 minutes. Nearly all requests to the nodes time out during this period. Restarting one, two, or all three of the nodes does not resolve the spiral; after a restart the three nodes immediately start hogging up memory again and CMSing constantly. When the issue resolves itself, all 3 nodes immediately get better. Sometimes it reoccurs in bursts, where it will be trashed for 20 minutes, fine for 5, trashed for 20, and repeat that cycle a few times. There are no unusual logs provided by cassandra during this period of time, other than recording of the constant dropped read requests and the constant CMS runs. I have analyzed the log files prior to multiple distinct instances of this issue and have found no preceding events which are associated with this issue. I have verified that our apps are not performing any unusual number or type of requests during this time. This behaviour occurred on 1.0.12, 1.1.7, and now on 1.2.11. The way I've narrowed this down to counters is a bit naive. It started happening when we started making use of counter columns, went away after we rolled back use of counter columns. I've repeated this attempted rollout on each version now, and it consistently rears its head every time. I should note this incident does _seem_ to happen more rarely on 1.2.11 compared to the previous versions. This incident has been consistent across multiple different types of hardware, as well as major kernel version changes (2.6 all the way to 3.2). The OS is operating normally during the event. I managed to get an hprof dump when the issue was happening in the wild. Something notable in the class instance counts as reported by jhat. Here are the top 5 counts for this one node: {code} 5967846 instances of class org.apache.cassandra.db.CounterColumn 1247525 instances of class com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$WeightedValue 1247310 instances of class org.apache.cassandra.cache.KeyCacheKey 1246648 instances of class com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node 1237526 instances of class org.apache.cassandra.db.RowIndexEntry {code} Is it normal or expected for CounterColumn to have that number of instances? The data model for how we use counters is as follows: between 50-2 counter columns per key. We currently have around 3 million keys total, but this issue also replicated when we only had a few thousand keys total. Average column count is around 1k, and 90th is 18k. New columns are added regularly, and columns are incremented regularly. No column or key deletions occur. We probably have 1-5k hot keys at any given time, spread across the entire ring. R:W ratio is typically around 50:1. This is the only CF we're using counters on, at this time. CF details are as follows: {code} ColumnFamily: CommentTree Key Validation Class: org.apache.cassandra.db.marshal.AsciiType Default column value validator: org.apache.cassandra.db.marshal.CounterColumnType Cells sorted by: org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.LongType,org.apache.cassandra.db.marshal.LongType,org.apache.cassandra.db.marshal.LongType) GC grace seconds: 864000 Compaction min/max thresholds: 4/32 Read repair chance: 0.01 DC Local Read repair chance: 0.0 Populate IO Cache on flush: false Replicate on write: true Caching: KEYS_ONLY Bloom Filter FP chance: default Built indexes: [] Compaction Strategy: org.apache.cassandra.db.compaction.LeveledCompactionStrategy Compaction Strategy Options: sstable_size_in_mb: 160 Column Family: CommentTree SSTable count: 30
git commit: Fix execution of LOCAL_QUORUM queries with SimpleStrategy
Updated Branches: refs/heads/cassandra-1.2 2a7c20ea9 - 32d7cb506 Fix execution of LOCAL_QUORUM queries with SimpleStrategy patch by alexliu68; reviewed by slebresne for CASSANDRA-6545 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/32d7cb50 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/32d7cb50 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/32d7cb50 Branch: refs/heads/cassandra-1.2 Commit: 32d7cb5066050ef6123f50a25c6e9b4c9e180ea0 Parents: 2a7c20e Author: Sylvain Lebresne sylv...@datastax.com Authored: Wed Jan 8 17:14:58 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Wed Jan 8 17:15:20 2014 +0100 -- CHANGES.txt | 1 + .../apache/cassandra/db/ConsistencyLevel.java | 60 +--- .../locator/AbstractReplicationStrategy.java| 2 +- 3 files changed, 42 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/32d7cb50/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 5a85977..cba97d0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -8,6 +8,7 @@ * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) * Add ability to throttle batchlog replay (CASSANDRA-6550) + * Fix executing LOCAL_QUORUM with SimpleStrategy (CASSANDRA-6545) 1.2.13 http://git-wip-us.apache.org/repos/asf/cassandra/blob/32d7cb50/src/java/org/apache/cassandra/db/ConsistencyLevel.java -- diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java index 4d72767..3737c73 100644 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@ -88,9 +88,16 @@ public enum ConsistencyLevel return codeIdx[code]; } +private int quorumFor(Table table) +{ +return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; +} + private int localQuorumFor(Table table, String dc) { -return (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1; +return (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) + ? (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1 + : quorumFor(table); } public int blockFor(Table table) @@ -107,17 +114,24 @@ public enum ConsistencyLevel case THREE: return 3; case QUORUM: -return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; +return quorumFor(table); case ALL: return table.getReplicationStrategy().getReplicationFactor(); case LOCAL_QUORUM: return localQuorumFor(table, DatabaseDescriptor.getLocalDataCenter()); case EACH_QUORUM: -NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy(); -int n = 0; -for (String dc : strategy.getDatacenters()) -n += localQuorumFor(table, dc); -return n; +if (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) +{ +NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy(); +int n = 0; +for (String dc : strategy.getDatacenters()) +n += localQuorumFor(table, dc); +return n; +} +else +{ +return quorumFor(table); +} default: throw new UnsupportedOperationException(Invalid consistency level: + toString()); } @@ -208,16 +222,20 @@ public enum ConsistencyLevel // local hint is acceptable, and local node is always live return true; case LOCAL_ONE: -return countLocalEndpoints(liveEndpoints) = 1; +return countLocalEndpoints(liveEndpoints) = 1; case LOCAL_QUORUM: return countLocalEndpoints(liveEndpoints) = blockFor(table); case EACH_QUORUM: -for (Map.EntryString, Integer entry : countPerDCEndpoints(table, liveEndpoints).entrySet()) +if (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) { -if (entry.getValue()
[1/2] git commit: Fix execution of LOCAL_QUORUM queries with SimpleStrategy
Updated Branches: refs/heads/cassandra-2.0 5fa605510 - f8fd7db67 Fix execution of LOCAL_QUORUM queries with SimpleStrategy patch by alexliu68; reviewed by slebresne for CASSANDRA-6545 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/32d7cb50 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/32d7cb50 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/32d7cb50 Branch: refs/heads/cassandra-2.0 Commit: 32d7cb5066050ef6123f50a25c6e9b4c9e180ea0 Parents: 2a7c20e Author: Sylvain Lebresne sylv...@datastax.com Authored: Wed Jan 8 17:14:58 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Wed Jan 8 17:15:20 2014 +0100 -- CHANGES.txt | 1 + .../apache/cassandra/db/ConsistencyLevel.java | 60 +--- .../locator/AbstractReplicationStrategy.java| 2 +- 3 files changed, 42 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/32d7cb50/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 5a85977..cba97d0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -8,6 +8,7 @@ * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) * Add ability to throttle batchlog replay (CASSANDRA-6550) + * Fix executing LOCAL_QUORUM with SimpleStrategy (CASSANDRA-6545) 1.2.13 http://git-wip-us.apache.org/repos/asf/cassandra/blob/32d7cb50/src/java/org/apache/cassandra/db/ConsistencyLevel.java -- diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java index 4d72767..3737c73 100644 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@ -88,9 +88,16 @@ public enum ConsistencyLevel return codeIdx[code]; } +private int quorumFor(Table table) +{ +return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; +} + private int localQuorumFor(Table table, String dc) { -return (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1; +return (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) + ? (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1 + : quorumFor(table); } public int blockFor(Table table) @@ -107,17 +114,24 @@ public enum ConsistencyLevel case THREE: return 3; case QUORUM: -return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; +return quorumFor(table); case ALL: return table.getReplicationStrategy().getReplicationFactor(); case LOCAL_QUORUM: return localQuorumFor(table, DatabaseDescriptor.getLocalDataCenter()); case EACH_QUORUM: -NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy(); -int n = 0; -for (String dc : strategy.getDatacenters()) -n += localQuorumFor(table, dc); -return n; +if (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) +{ +NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy(); +int n = 0; +for (String dc : strategy.getDatacenters()) +n += localQuorumFor(table, dc); +return n; +} +else +{ +return quorumFor(table); +} default: throw new UnsupportedOperationException(Invalid consistency level: + toString()); } @@ -208,16 +222,20 @@ public enum ConsistencyLevel // local hint is acceptable, and local node is always live return true; case LOCAL_ONE: -return countLocalEndpoints(liveEndpoints) = 1; +return countLocalEndpoints(liveEndpoints) = 1; case LOCAL_QUORUM: return countLocalEndpoints(liveEndpoints) = blockFor(table); case EACH_QUORUM: -for (Map.EntryString, Integer entry : countPerDCEndpoints(table, liveEndpoints).entrySet()) +if (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) { -if (entry.getValue()
[2/2] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Conflicts: src/java/org/apache/cassandra/db/ConsistencyLevel.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f8fd7db6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f8fd7db6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f8fd7db6 Branch: refs/heads/cassandra-2.0 Commit: f8fd7db67e3ff7268bd2bc96bd1373d54664b7dd Parents: 5fa6055 32d7cb5 Author: Sylvain Lebresne sylv...@datastax.com Authored: Wed Jan 8 17:22:55 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Wed Jan 8 17:22:55 2014 +0100 -- CHANGES.txt | 1 + .../apache/cassandra/db/ConsistencyLevel.java | 59 +--- .../locator/AbstractReplicationStrategy.java| 2 +- 3 files changed, 41 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8fd7db6/CHANGES.txt -- diff --cc CHANGES.txt index faf52a4,cba97d0..d0b63a0 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -5,29 -8,10 +5,30 @@@ Merged from 1.2 * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) * Add ability to throttle batchlog replay (CASSANDRA-6550) + * Fix executing LOCAL_QUORUM with SimpleStrategy (CASSANDRA-6545) -1.2.13 +2.0.4 + * Allow removing snapshots of no-longer-existing CFs (CASSANDRA-6418) + * add StorageService.stopDaemon() (CASSANDRA-4268) + * add IRE for invalid CF supplied to get_count (CASSANDRA-5701) + * add client encryption support to sstableloader (CASSANDRA-6378) + * Fix accept() loop for SSL sockets post-shutdown (CASSANDRA-6468) + * Fix size-tiered compaction in LCS L0 (CASSANDRA-6496) + * Fix assertion failure in filterColdSSTables (CASSANDRA-6483) + * Fix row tombstones in larger-than-memory compactions (CASSANDRA-6008) + * Fix cleanup ClassCastException (CASSANDRA-6462) + * Reduce gossip memory use by interning VersionedValue strings (CASSANDRA-6410) + * Allow specifying datacenters to participate in a repair (CASSANDRA-6218) + * Fix divide-by-zero in PCI (CASSANDRA-6403) + * Fix setting last compacted key in the wrong level for LCS (CASSANDRA-6284) + * Add millisecond precision formats to the timestamp parser (CASSANDRA-6395) + * Expose a total memtable size metric for a CF (CASSANDRA-6391) + * cqlsh: handle symlinks properly (CASSANDRA-6425) + * Fix potential infinite loop when paging query with IN (CASSANDRA-6464) + * Fix assertion error in AbstractQueryPager.discardFirst (CASSANDRA-6447) + * Fix streaming older SSTable yields unnecessary tombstones (CASSANDRA-6527) +Merged from 1.2: * Improved error message on bad properties in DDL queries (CASSANDRA-6453) * Randomize batchlog candidates selection (CASSANDRA-6481) * Fix thundering herd on endpoint cache invalidation (CASSANDRA-6345, 6485) http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8fd7db6/src/java/org/apache/cassandra/db/ConsistencyLevel.java -- diff --cc src/java/org/apache/cassandra/db/ConsistencyLevel.java index 0f6aba7,3737c73..6d04314 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@@ -89,12 -88,19 +89,19 @@@ public enum ConsistencyLeve return codeIdx[code]; } -private int quorumFor(Table table) ++private int quorumFor(Keyspace keyspace) + { -return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; ++return (keyspace.getReplicationStrategy().getReplicationFactor() / 2) + 1; + } + -private int localQuorumFor(Table table, String dc) +private int localQuorumFor(Keyspace keyspace, String dc) { - return (((NetworkTopologyStrategy) keyspace.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1; -return (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) - ? (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1 - : quorumFor(table); ++return (keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy) ++ ? (((NetworkTopologyStrategy) keyspace.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1 ++ : quorumFor(keyspace); } -public int blockFor(Table table) +public int blockFor(Keyspace keyspace) { switch (this) { @@@ -108,17 -114,24 +115,24 @@@ case THREE: return 3; case QUORUM: - return
[jira] [Commented] (CASSANDRA-6405) When making heavy use of counters, neighbor nodes occasionally enter spiral of constant memory consumpion
[ https://issues.apache.org/jira/browse/CASSANDRA-6405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865639#comment-13865639 ] Sylvain Lebresne commented on CASSANDRA-6405: - Nothing coming to mind no, not by default at least. I suppose it wouldn't be too hard to add some instrumentation to count the number of times CounterColumn.reconcile() is called and see if the issue happening is linked to a sudden increase in those calls. That being said, that would still not tell us why there is a sudden increase of the calls... It's still mysterious to me why nodes would suddenly start allocating counters like crazy. When making heavy use of counters, neighbor nodes occasionally enter spiral of constant memory consumpion - Key: CASSANDRA-6405 URL: https://issues.apache.org/jira/browse/CASSANDRA-6405 Project: Cassandra Issue Type: Bug Environment: RF of 3, 15 nodes. Sun Java 7 (also occurred in OpenJDK 6, and Sun Java 6). Xmx of 8G. No row cache. Reporter: Jason Harvey Attachments: threaddump.txt We're randomly running into an interesting issue on our ring. When making use of counters, we'll occasionally have 3 nodes (always neighbors) suddenly start immediately filling up memory, CMSing, fill up again, repeat. This pattern goes on for 5-20 minutes. Nearly all requests to the nodes time out during this period. Restarting one, two, or all three of the nodes does not resolve the spiral; after a restart the three nodes immediately start hogging up memory again and CMSing constantly. When the issue resolves itself, all 3 nodes immediately get better. Sometimes it reoccurs in bursts, where it will be trashed for 20 minutes, fine for 5, trashed for 20, and repeat that cycle a few times. There are no unusual logs provided by cassandra during this period of time, other than recording of the constant dropped read requests and the constant CMS runs. I have analyzed the log files prior to multiple distinct instances of this issue and have found no preceding events which are associated with this issue. I have verified that our apps are not performing any unusual number or type of requests during this time. This behaviour occurred on 1.0.12, 1.1.7, and now on 1.2.11. The way I've narrowed this down to counters is a bit naive. It started happening when we started making use of counter columns, went away after we rolled back use of counter columns. I've repeated this attempted rollout on each version now, and it consistently rears its head every time. I should note this incident does _seem_ to happen more rarely on 1.2.11 compared to the previous versions. This incident has been consistent across multiple different types of hardware, as well as major kernel version changes (2.6 all the way to 3.2). The OS is operating normally during the event. I managed to get an hprof dump when the issue was happening in the wild. Something notable in the class instance counts as reported by jhat. Here are the top 5 counts for this one node: {code} 5967846 instances of class org.apache.cassandra.db.CounterColumn 1247525 instances of class com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$WeightedValue 1247310 instances of class org.apache.cassandra.cache.KeyCacheKey 1246648 instances of class com.googlecode.concurrentlinkedhashmap.ConcurrentLinkedHashMap$Node 1237526 instances of class org.apache.cassandra.db.RowIndexEntry {code} Is it normal or expected for CounterColumn to have that number of instances? The data model for how we use counters is as follows: between 50-2 counter columns per key. We currently have around 3 million keys total, but this issue also replicated when we only had a few thousand keys total. Average column count is around 1k, and 90th is 18k. New columns are added regularly, and columns are incremented regularly. No column or key deletions occur. We probably have 1-5k hot keys at any given time, spread across the entire ring. R:W ratio is typically around 50:1. This is the only CF we're using counters on, at this time. CF details are as follows: {code} ColumnFamily: CommentTree Key Validation Class: org.apache.cassandra.db.marshal.AsciiType Default column value validator: org.apache.cassandra.db.marshal.CounterColumnType Cells sorted by: org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.LongType,org.apache.cassandra.db.marshal.LongType,org.apache.cassandra.db.marshal.LongType) GC grace seconds: 864000 Compaction min/max thresholds: 4/32 Read repair chance: 0.01 DC Local Read repair chance: 0.0 Populate IO Cache on flush: false
[jira] [Commented] (CASSANDRA-6381) Refactor nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-6381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865771#comment-13865771 ] Mikhail Stepura commented on CASSANDRA-6381: h5. REBUILD_INDEX bq. The old behavior (csv format) was not really POSIX-like argument conventions. Your description for REBUILD_INDEX still describes CSV format: {{(IndexNameExample: Standard3.IdxName,Standard3.IdxName1)}}. So there is a disconnect between the behavior and its description/documentation. h5. SETCOMPACTIONTHRESHOLD: {quote}It was not possible before, see the condition in NodeCmd. {code} if (minthreshold 2 maxthreshold != 0){ badUse(Min threshold must be at least 2); } {code} {quote} It was. Remember, {{maxthreshold == 0}} so the condition is false h5. STATUS Why assign to null initially? {code:java|title=getOwnershipByDc} String dc = null; dc = epSnitchInfo.getDatacenter(ownership.ip); {code} h5. TRUNCATEHINTS You only have 1 parameter there. Why do you use a list to hold it? Refactor nodetool - Key: CASSANDRA-6381 URL: https://issues.apache.org/jira/browse/CASSANDRA-6381 Project: Cassandra Issue Type: Improvement Reporter: Yuki Morishita Assignee: Clément Lardeur Priority: Minor Labels: lhf, nodetool Attachments: trunk-6381-v2.patch, trunk-6381.patch We have way too many nodetool commands(more than 40) packed in one NodeCmd class. And we are trying to add more commands. https://github.com/airlift/airline could be a good fit to take out each command into sub command class. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6505) counters++ global shards 2.0 back port
[ https://issues.apache.org/jira/browse/CASSANDRA-6505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865788#comment-13865788 ] Sylvain Lebresne commented on CASSANDRA-6505: - A few small remarks on the patch: * currently, merge doesn't do any allocation because copyTo uses ByteBufferUtil.arrayCopy (and contexts are always array backed). The reuse of writeElement() for copyTo makes it allocate a BB (the duplicate()) for every shard. It's possible this wouldn't have noticeable impact in practice, but given things like CASSANDRA-6405, I'd rather be on the safe side and save the allocations by making writeElementAtOffset use ByteBufferUtil.arrayCopy instead of duplicating. This will even save some duplicate() calls on context creation compared to today as a bonus. * Let's leave the rewrite of hasCounterId() (to use a binary search) out of this backported patch. We should aim for minimal amount of changes to support global shards here and that patch play with the border of that notion enough without that. * Nit: Is there a reason to add GLOBAL_SHARD_INDEX_OFFSET rather than say -x-1 (à la binary search) for global shards? Not that it doesn't work as it is, just that negating would seem slightly more natural/simpler to me. * Nit: the CounterContext class javadoc should specify how it distinguishes a local shard from a global one. You currently have to read the code to figure it out. * Nit: we don't need createGlobal for this patch (even tests don't use it), so let's not add it to drive home the fact we don't create global shards. Outside of that, it might be a bit safer to wait on CASSANDRA-6504 review (at least a first pass) before committing this (I do plan on reviewing CASSANDRA-6504 asap, and it shouldn't be too hard to rebase CASSANDRA-6504 on trunk + this patch in the meantime). Even if it's very unlikely, it would suck to realize during the review of CASSANDRA-6504 that the global shard mechanism needs some minor adjustement but that we have already released this, and I doubt having this in 2.0.6 instead of 2.0.5 would make a huge difference. counters++ global shards 2.0 back port -- Key: CASSANDRA-6505 URL: https://issues.apache.org/jira/browse/CASSANDRA-6505 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.0.5 CASSANDRA-6504 introduces a new type of shard - 'global' - to 2.1. To enable live upgrade from 2.0 to 2.1, it's necessary that 2.0 nodes are able to understand the new 'global' shards in the counter contexts. 2.0 nodes will not produce 'global' shards, but must contain the merge logic. It isn't a trivial code change (non-trivial code in a non-trivial part of the code), hence this separate JIRA issue. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6505) counters++ global shards 2.0 back port
[ https://issues.apache.org/jira/browse/CASSANDRA-6505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865818#comment-13865818 ] Aleksey Yeschenko commented on CASSANDRA-6505: -- bq. Nit: Is there a reason to add GLOBAL_SHARD_INDEX_OFFSET rather than say -x-1 (à la binary search) for global shards? Not that it doesn't work as it is, just that negating would seem slightly more natural/simpler to me. I used to have just '-x', but that obviously doesn't work for 0. And 'MIN_VALUE + x' seemed cleaner than '-x - 1'. The rest makes sense to me, will do. counters++ global shards 2.0 back port -- Key: CASSANDRA-6505 URL: https://issues.apache.org/jira/browse/CASSANDRA-6505 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.0.5 CASSANDRA-6504 introduces a new type of shard - 'global' - to 2.1. To enable live upgrade from 2.0 to 2.1, it's necessary that 2.0 nodes are able to understand the new 'global' shards in the counter contexts. 2.0 nodes will not produce 'global' shards, but must contain the merge logic. It isn't a trivial code change (non-trivial code in a non-trivial part of the code), hence this separate JIRA issue. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6505) counters++ global shards 2.0 back port
[ https://issues.apache.org/jira/browse/CASSANDRA-6505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865846#comment-13865846 ] Sylvain Lebresne commented on CASSANDRA-6505: - bq. And 'MIN_VALUE + x' seemed cleaner than '-x - 1' It's just that -x - 1 felt more natural because that's the encoding Collections.binarySearch actually uses. Anyway, I guess I don't care a whole lot, though I do would maybe suggest inlining Short.MIN_VALUE if we keep it the way it is. Cause the code do rely on 'index + GLOBAL_SHARD_INDEX_OFFSET' being negative and I find that the indirection of having a global constant makes it a bit harder to validate that invariant mentally. counters++ global shards 2.0 back port -- Key: CASSANDRA-6505 URL: https://issues.apache.org/jira/browse/CASSANDRA-6505 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.0.5 CASSANDRA-6504 introduces a new type of shard - 'global' - to 2.1. To enable live upgrade from 2.0 to 2.1, it's necessary that 2.0 nodes are able to understand the new 'global' shards in the counter contexts. 2.0 nodes will not produce 'global' shards, but must contain the merge logic. It isn't a trivial code change (non-trivial code in a non-trivial part of the code), hence this separate JIRA issue. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6381) Refactor nodetool
[ https://issues.apache.org/jira/browse/CASSANDRA-6381?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865843#comment-13865843 ] Mikhail Stepura commented on CASSANDRA-6381: One more thing, new implementation doesn't save nodetool.history anymore. Right? Refactor nodetool - Key: CASSANDRA-6381 URL: https://issues.apache.org/jira/browse/CASSANDRA-6381 Project: Cassandra Issue Type: Improvement Reporter: Yuki Morishita Assignee: Clément Lardeur Priority: Minor Labels: lhf, nodetool Attachments: trunk-6381-v2.patch, trunk-6381.patch We have way too many nodetool commands(more than 40) packed in one NodeCmd class. And we are trying to add more commands. https://github.com/airlift/airline could be a good fit to take out each command into sub command class. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6505) counters++ global shards 2.0 back port
[ https://issues.apache.org/jira/browse/CASSANDRA-6505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865850#comment-13865850 ] Aleksey Yeschenko commented on CASSANDRA-6505: -- Will do one of those two (: counters++ global shards 2.0 back port -- Key: CASSANDRA-6505 URL: https://issues.apache.org/jira/browse/CASSANDRA-6505 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.0.5 CASSANDRA-6504 introduces a new type of shard - 'global' - to 2.1. To enable live upgrade from 2.0 to 2.1, it's necessary that 2.0 nodes are able to understand the new 'global' shards in the counter contexts. 2.0 nodes will not produce 'global' shards, but must contain the merge logic. It isn't a trivial code change (non-trivial code in a non-trivial part of the code), hence this separate JIRA issue. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6505) counters++ global shards 2.0 back port
[ https://issues.apache.org/jira/browse/CASSANDRA-6505?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865863#comment-13865863 ] Benedict commented on CASSANDRA-6505: - bq. I'd rather be on the safe side and save the allocations I wouldn't worry about this particular duplicate() call. It will absolutely be stack allocated, so won't affect the heap at all. ByteBufferUtil.arrayCopy devolves to a for loop of put/get if either is a DirectByteBuffer (though this could be optimised) so it might also result in suboptimal future behaviour, in a place we won't necessarily look to change. counters++ global shards 2.0 back port -- Key: CASSANDRA-6505 URL: https://issues.apache.org/jira/browse/CASSANDRA-6505 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Fix For: 2.0.5 CASSANDRA-6504 introduces a new type of shard - 'global' - to 2.1. To enable live upgrade from 2.0 to 2.1, it's necessary that 2.0 nodes are able to understand the new 'global' shards in the counter contexts. 2.0 nodes will not produce 'global' shards, but must contain the merge logic. It isn't a trivial code change (non-trivial code in a non-trivial part of the code), hence this separate JIRA issue. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5789) Data not fully replicated with 2 nodes and replication factor 2
[ https://issues.apache.org/jira/browse/CASSANDRA-5789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865901#comment-13865901 ] Russ Hatch commented on CASSANDRA-5789: --- I've not been able to consistently reproduce the reported issue on 1.2.6 Using the code sample provided earlier, I did see some NotFoundExceptions raised by pycassa but I was not 100% certain those were raised for rows that were successfully reported back as written (CL=1). I think there could be something unexpected happening in the sample code or pycassa. I created an additional python test with cassandra-dtest and cassandra-dbapi2, and was not able to reproduce any similar issue. I used two nodes local to my machine, with a replication factor of two. Writes and Reads were done with CL=1. I used up to 40 threads making writes across the two nodes, and up to 125k records created in each keyspace (so 40 threads, each creating 125k rows at the same time). In some cases I exhausted the resources on my machine enough that cassandra did not respond to some write requests, but reading back was 100% successful for the rows actually written (checked a random sampling of 100k rows spanning all 40 keyspaces). I will attach my python code to this ticket. Data not fully replicated with 2 nodes and replication factor 2 --- Key: CASSANDRA-5789 URL: https://issues.apache.org/jira/browse/CASSANDRA-5789 Project: Cassandra Issue Type: Bug Affects Versions: 1.2.2, 1.2.6 Environment: Official Datastax Cassandra 1.2.6, running on Linux RHEL 6.2. I've seen the same behavior with Cassandra 1.2.2. Sun Java 1.7.0_10-b18 64-bit Java heap settings: -Xms8192M -Xmx8192M -Xmn2048M Reporter: James Lee Assignee: Russ Hatch Attachments: CassBugRepro.py, CassTestData.py I'm seeing a problem with a 2-node Cassandra test deployment, where it seems that data isn't being replicated among the nodes as I would expect. The setup and test is as follows: - Two Cassandra nodes in the cluster (they each have themselves and the other node as seeds in cassandra.yaml). - Create 40 keyspaces, each with simple replication strategy and replication factor 2. - Populate 125,000 rows into each keyspace, using a pycassa client with a connection pool pointed at both nodes. These are populated with writes using consistency level of 1. - Wait until nodetool on each node reports that there are no hinted handoffs outstanding (see output below). - Do random reads of the rows in the keyspaces, again using a pycassa client with a connection pool pointed at both nodes. These are read using consistency level 1. I'm finding that the vast majority of reads are successful, but a small proportion (~0.1%) are returned as Not Found. If I manually try to look up those keys using cassandra-cli, I see that they are returned when querying one of the nodes, but not when querying the other. So it seems like some of the rows have simply not been replicated, even though the write for these rows was reported to the client as successful. If I reduce the rate at which the test tool initially writes data into the database then I don't see any failed reads, so this seems like a load-related issue. My understanding is that if all writes were successful and there are no pending hinted handoffs, then the data should be fully-replicated and reads should return it (even with read and write consistency of 1). Here's the output from notetool on the two nodes: comet-mvs01:/dsc-cassandra-1.2.6# ./bin/nodetool tpstats Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 2 0 0 RequestResponseStage 0 0 878494 0 0 MutationStage 0 02869107 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 2208 0 0 AntiEntropyStage 0 0 0 0 0 MigrationStage0 0994 0 0 MemtablePostFlusher 0 0 4399 0 0 FlushWriter 0 0 2264 0 556 MiscStage 0 0 0 0 0 commitlog_archiver0 0 0 0 0
[jira] [Comment Edited] (CASSANDRA-5789) Data not fully replicated with 2 nodes and replication factor 2
[ https://issues.apache.org/jira/browse/CASSANDRA-5789?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865901#comment-13865901 ] Russ Hatch edited comment on CASSANDRA-5789 at 1/8/14 9:22 PM: --- I've not been able to consistently reproduce the reported issue on 1.2.6 Using the code sample provided earlier, I did see some NotFoundExceptions raised by pycassa but I was not 100% certain those were raised for rows that were successfully reported back as written (CL=1). I think there could be something unexpected happening in the sample code or pycassa. I created an additional python test with cassandra-dtest and cassandra-dbapi2, and was not able to reproduce any similar issue. I used two nodes local to my machine, with a replication factor of two. Writes and Reads were done with CL=1. I used up to 40 threads making writes across the two nodes, and up to 125k records created in each keyspace (so 40 threads, each creating 125k rows at the same time). In some cases I exhausted the resources on my machine enough that cassandra did not respond to some write requests, but reading back was 100% successful for the rows actually written (checked a random sampling of 100k rows spanning all 40 keyspaces). I will attach my python code to this ticket. edit: I should note, this was on Ubuntu 13.10, java version 1.7.0_45 was (Author: rhatch): I've not been able to consistently reproduce the reported issue on 1.2.6 Using the code sample provided earlier, I did see some NotFoundExceptions raised by pycassa but I was not 100% certain those were raised for rows that were successfully reported back as written (CL=1). I think there could be something unexpected happening in the sample code or pycassa. I created an additional python test with cassandra-dtest and cassandra-dbapi2, and was not able to reproduce any similar issue. I used two nodes local to my machine, with a replication factor of two. Writes and Reads were done with CL=1. I used up to 40 threads making writes across the two nodes, and up to 125k records created in each keyspace (so 40 threads, each creating 125k rows at the same time). In some cases I exhausted the resources on my machine enough that cassandra did not respond to some write requests, but reading back was 100% successful for the rows actually written (checked a random sampling of 100k rows spanning all 40 keyspaces). I will attach my python code to this ticket. Data not fully replicated with 2 nodes and replication factor 2 --- Key: CASSANDRA-5789 URL: https://issues.apache.org/jira/browse/CASSANDRA-5789 Project: Cassandra Issue Type: Bug Affects Versions: 1.2.2, 1.2.6 Environment: Official Datastax Cassandra 1.2.6, running on Linux RHEL 6.2. I've seen the same behavior with Cassandra 1.2.2. Sun Java 1.7.0_10-b18 64-bit Java heap settings: -Xms8192M -Xmx8192M -Xmn2048M Reporter: James Lee Assignee: Russ Hatch Attachments: CassBugRepro.py, CassTestData.py I'm seeing a problem with a 2-node Cassandra test deployment, where it seems that data isn't being replicated among the nodes as I would expect. The setup and test is as follows: - Two Cassandra nodes in the cluster (they each have themselves and the other node as seeds in cassandra.yaml). - Create 40 keyspaces, each with simple replication strategy and replication factor 2. - Populate 125,000 rows into each keyspace, using a pycassa client with a connection pool pointed at both nodes. These are populated with writes using consistency level of 1. - Wait until nodetool on each node reports that there are no hinted handoffs outstanding (see output below). - Do random reads of the rows in the keyspaces, again using a pycassa client with a connection pool pointed at both nodes. These are read using consistency level 1. I'm finding that the vast majority of reads are successful, but a small proportion (~0.1%) are returned as Not Found. If I manually try to look up those keys using cassandra-cli, I see that they are returned when querying one of the nodes, but not when querying the other. So it seems like some of the rows have simply not been replicated, even though the write for these rows was reported to the client as successful. If I reduce the rate at which the test tool initially writes data into the database then I don't see any failed reads, so this seems like a load-related issue. My understanding is that if all writes were successful and there are no pending hinted handoffs, then the data should be fully-replicated and reads should return it (even with read and write consistency of 1). Here's the output from notetool on the two nodes: comet-mvs01:/dsc-cassandra-1.2.6# ./bin/nodetool
[jira] [Resolved] (CASSANDRA-5789) Data not fully replicated with 2 nodes and replication factor 2
[ https://issues.apache.org/jira/browse/CASSANDRA-5789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-5789. --- Resolution: Cannot Reproduce Thanks, Russ. Data not fully replicated with 2 nodes and replication factor 2 --- Key: CASSANDRA-5789 URL: https://issues.apache.org/jira/browse/CASSANDRA-5789 Project: Cassandra Issue Type: Bug Affects Versions: 1.2.2, 1.2.6 Environment: Official Datastax Cassandra 1.2.6, running on Linux RHEL 6.2. I've seen the same behavior with Cassandra 1.2.2. Sun Java 1.7.0_10-b18 64-bit Java heap settings: -Xms8192M -Xmx8192M -Xmn2048M Reporter: James Lee Assignee: Russ Hatch Attachments: 5789.py, CassBugRepro.py, CassTestData.py I'm seeing a problem with a 2-node Cassandra test deployment, where it seems that data isn't being replicated among the nodes as I would expect. The setup and test is as follows: - Two Cassandra nodes in the cluster (they each have themselves and the other node as seeds in cassandra.yaml). - Create 40 keyspaces, each with simple replication strategy and replication factor 2. - Populate 125,000 rows into each keyspace, using a pycassa client with a connection pool pointed at both nodes. These are populated with writes using consistency level of 1. - Wait until nodetool on each node reports that there are no hinted handoffs outstanding (see output below). - Do random reads of the rows in the keyspaces, again using a pycassa client with a connection pool pointed at both nodes. These are read using consistency level 1. I'm finding that the vast majority of reads are successful, but a small proportion (~0.1%) are returned as Not Found. If I manually try to look up those keys using cassandra-cli, I see that they are returned when querying one of the nodes, but not when querying the other. So it seems like some of the rows have simply not been replicated, even though the write for these rows was reported to the client as successful. If I reduce the rate at which the test tool initially writes data into the database then I don't see any failed reads, so this seems like a load-related issue. My understanding is that if all writes were successful and there are no pending hinted handoffs, then the data should be fully-replicated and reads should return it (even with read and write consistency of 1). Here's the output from notetool on the two nodes: comet-mvs01:/dsc-cassandra-1.2.6# ./bin/nodetool tpstats Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 2 0 0 RequestResponseStage 0 0 878494 0 0 MutationStage 0 02869107 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 2208 0 0 AntiEntropyStage 0 0 0 0 0 MigrationStage0 0994 0 0 MemtablePostFlusher 0 0 4399 0 0 FlushWriter 0 0 2264 0 556 MiscStage 0 0 0 0 0 commitlog_archiver0 0 0 0 0 InternalResponseStage 0 0153 0 0 HintedHandoff 0 0 2 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 BINARY 0 READ 0 MUTATION 87655 _TRACE 0 REQUEST_RESPONSE 0 comet-mvs02:/dsc-cassandra-1.2.6# ./bin/nodetool tpstats Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0868 0 0 RequestResponseStage 0 03919665 0 0 MutationStage 0 08177325 0 0 ReadRepairStage 0 0113 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage
[jira] [Updated] (CASSANDRA-5789) Data not fully replicated with 2 nodes and replication factor 2
[ https://issues.apache.org/jira/browse/CASSANDRA-5789?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russ Hatch updated CASSANDRA-5789: -- Attachment: 5789.py Data not fully replicated with 2 nodes and replication factor 2 --- Key: CASSANDRA-5789 URL: https://issues.apache.org/jira/browse/CASSANDRA-5789 Project: Cassandra Issue Type: Bug Affects Versions: 1.2.2, 1.2.6 Environment: Official Datastax Cassandra 1.2.6, running on Linux RHEL 6.2. I've seen the same behavior with Cassandra 1.2.2. Sun Java 1.7.0_10-b18 64-bit Java heap settings: -Xms8192M -Xmx8192M -Xmn2048M Reporter: James Lee Assignee: Russ Hatch Attachments: 5789.py, CassBugRepro.py, CassTestData.py I'm seeing a problem with a 2-node Cassandra test deployment, where it seems that data isn't being replicated among the nodes as I would expect. The setup and test is as follows: - Two Cassandra nodes in the cluster (they each have themselves and the other node as seeds in cassandra.yaml). - Create 40 keyspaces, each with simple replication strategy and replication factor 2. - Populate 125,000 rows into each keyspace, using a pycassa client with a connection pool pointed at both nodes. These are populated with writes using consistency level of 1. - Wait until nodetool on each node reports that there are no hinted handoffs outstanding (see output below). - Do random reads of the rows in the keyspaces, again using a pycassa client with a connection pool pointed at both nodes. These are read using consistency level 1. I'm finding that the vast majority of reads are successful, but a small proportion (~0.1%) are returned as Not Found. If I manually try to look up those keys using cassandra-cli, I see that they are returned when querying one of the nodes, but not when querying the other. So it seems like some of the rows have simply not been replicated, even though the write for these rows was reported to the client as successful. If I reduce the rate at which the test tool initially writes data into the database then I don't see any failed reads, so this seems like a load-related issue. My understanding is that if all writes were successful and there are no pending hinted handoffs, then the data should be fully-replicated and reads should return it (even with read and write consistency of 1). Here's the output from notetool on the two nodes: comet-mvs01:/dsc-cassandra-1.2.6# ./bin/nodetool tpstats Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0 2 0 0 RequestResponseStage 0 0 878494 0 0 MutationStage 0 02869107 0 0 ReadRepairStage 0 0 0 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 2208 0 0 AntiEntropyStage 0 0 0 0 0 MigrationStage0 0994 0 0 MemtablePostFlusher 0 0 4399 0 0 FlushWriter 0 0 2264 0 556 MiscStage 0 0 0 0 0 commitlog_archiver0 0 0 0 0 InternalResponseStage 0 0153 0 0 HintedHandoff 0 0 2 0 0 Message type Dropped RANGE_SLICE 0 READ_REPAIR 0 BINARY 0 READ 0 MUTATION 87655 _TRACE 0 REQUEST_RESPONSE 0 comet-mvs02:/dsc-cassandra-1.2.6# ./bin/nodetool tpstats Pool NameActive Pending Completed Blocked All time blocked ReadStage 0 0868 0 0 RequestResponseStage 0 03919665 0 0 MutationStage 0 08177325 0 0 ReadRepairStage 0 0113 0 0 ReplicateOnWriteStage 0 0 0 0 0 GossipStage 0 0 9624
[jira] [Reopened] (CASSANDRA-5202) CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name
[ https://issues.apache.org/jira/browse/CASSANDRA-5202?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita reopened CASSANDRA-5202: --- Some unit test failures revealed I have to work on this a little bit more. With committed version, secondary indexes can get wrong directory. Will post fix here. CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name Key: CASSANDRA-5202 URL: https://issues.apache.org/jira/browse/CASSANDRA-5202 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.9 Environment: OS: Windows 7, Server: Cassandra 1.1.9 release drop Client: astyanax 1.56.21, JVM: Sun/Oracle JVM 64 bit (jdk1.6.0_27) Reporter: Marat Bedretdinov Assignee: Yuki Morishita Labels: test Fix For: 2.1 Attachments: 5202.txt, astyanax-stress-driver.zip Attached is a driver that sequentially: 1. Drops keyspace 2. Creates keyspace 4. Creates 2 column families 5. Seeds 1M rows with 100 columns 6. Queries these 2 column families The above steps are repeated 1000 times. The following exception is observed at random (race - SEDA?): ERROR [ReadStage:55] 2013-01-29 19:24:52,676 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:55,5,main] java.lang.AssertionError: DecoratedKey(-1, ) != DecoratedKey(62819832764241410631599989027761269388, 313a31) in C:\var\lib\cassandra\data\user_role_reverse_index\business_entity_role\user_role_reverse_index-business_entity_role-hf-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1367) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1229) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1164) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:822) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1271) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) This exception appears in the server at the time of client submitting a query request (row slice) and not at the time data is seeded. The client times out and this data can no longer be queried as the same exception would always occur from there on. Also on iteration 201, it appears that dropping column families failed and as a result their recreation failed with unique column family name violation (see exception below). Note that the data files are actually gone, so it appears that the server runtime responsible for creating column family was out of sync with the piece that dropped them: Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5105 ms Iteration: 200; Total running time for 1000 queries is 232; Average running time of 1000 queries is 0 ms Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5361 ms Iteration: 201; Total running time for 1000 queries is 222; Average running time of 1000 queries is 0 ms Starting dropping column families Starting creating column families Exception in thread main com.netflix.astyanax.connectionpool.exceptions.BadRequestException: BadRequestException: [host=127.0.0.1(127.0.0.1):9160, latency=2468(2469), attempts=1]InvalidRequestException(why:Keyspace names must be case-insensitively unique (user_role_reverse_index conflicts with user_role_reverse_index)) at
[jira] [Commented] (CASSANDRA-5202) CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name
[ https://issues.apache.org/jira/browse/CASSANDRA-5202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865922#comment-13865922 ] Aleksey Yeschenko commented on CASSANDRA-5202: -- [~yukim] any way you could alter auth setup code (creation of auth tables) to use the old deterministic cfIds explicitly, while at it? CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name Key: CASSANDRA-5202 URL: https://issues.apache.org/jira/browse/CASSANDRA-5202 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.9 Environment: OS: Windows 7, Server: Cassandra 1.1.9 release drop Client: astyanax 1.56.21, JVM: Sun/Oracle JVM 64 bit (jdk1.6.0_27) Reporter: Marat Bedretdinov Assignee: Yuki Morishita Labels: test Fix For: 2.1 Attachments: 5202.txt, astyanax-stress-driver.zip Attached is a driver that sequentially: 1. Drops keyspace 2. Creates keyspace 4. Creates 2 column families 5. Seeds 1M rows with 100 columns 6. Queries these 2 column families The above steps are repeated 1000 times. The following exception is observed at random (race - SEDA?): ERROR [ReadStage:55] 2013-01-29 19:24:52,676 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:55,5,main] java.lang.AssertionError: DecoratedKey(-1, ) != DecoratedKey(62819832764241410631599989027761269388, 313a31) in C:\var\lib\cassandra\data\user_role_reverse_index\business_entity_role\user_role_reverse_index-business_entity_role-hf-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1367) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1229) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1164) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:822) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1271) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) This exception appears in the server at the time of client submitting a query request (row slice) and not at the time data is seeded. The client times out and this data can no longer be queried as the same exception would always occur from there on. Also on iteration 201, it appears that dropping column families failed and as a result their recreation failed with unique column family name violation (see exception below). Note that the data files are actually gone, so it appears that the server runtime responsible for creating column family was out of sync with the piece that dropped them: Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5105 ms Iteration: 200; Total running time for 1000 queries is 232; Average running time of 1000 queries is 0 ms Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5361 ms Iteration: 201; Total running time for 1000 queries is 222; Average running time of 1000 queries is 0 ms Starting dropping column families Starting creating column families Exception in thread main com.netflix.astyanax.connectionpool.exceptions.BadRequestException: BadRequestException: [host=127.0.0.1(127.0.0.1):9160, latency=2468(2469), attempts=1]InvalidRequestException(why:Keyspace names must be case-insensitively unique (user_role_reverse_index conflicts with
[jira] [Commented] (CASSANDRA-6157) Selectively Disable hinted handoff for a data center
[ https://issues.apache.org/jira/browse/CASSANDRA-6157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865920#comment-13865920 ] sankalp kohli commented on CASSANDRA-6157: -- Looks good except that I did not see this change in v3 patch Renamed initializeCustomConfig to configHintedHandoff as custom config sounds too general. Selectively Disable hinted handoff for a data center Key: CASSANDRA-6157 URL: https://issues.apache.org/jira/browse/CASSANDRA-6157 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Fix For: 2.0.5 Attachments: trunk-6157-v2.diff, trunk-6157-v3.diff, trunk-6157.txt Cassandra supports disabling the hints or reducing the window for hints. It would be helpful to have a switch which stops hints to a down data center but continue hints to other DCs. This is helpful during data center fail over as hints will put more unnecessary pressure on the DC taking double traffic. Also since now Cassandra is under reduced reduncany, we don't want to disable hints within the DC. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-5202) CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name
[ https://issues.apache.org/jira/browse/CASSANDRA-5202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865947#comment-13865947 ] Yuki Morishita commented on CASSANDRA-5202: --- [~iamaleksey] sure. Is it just enough to special case tables under 'system_auth' keyspace? CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name Key: CASSANDRA-5202 URL: https://issues.apache.org/jira/browse/CASSANDRA-5202 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.9 Environment: OS: Windows 7, Server: Cassandra 1.1.9 release drop Client: astyanax 1.56.21, JVM: Sun/Oracle JVM 64 bit (jdk1.6.0_27) Reporter: Marat Bedretdinov Assignee: Yuki Morishita Labels: test Fix For: 2.1 Attachments: 5202.txt, astyanax-stress-driver.zip Attached is a driver that sequentially: 1. Drops keyspace 2. Creates keyspace 4. Creates 2 column families 5. Seeds 1M rows with 100 columns 6. Queries these 2 column families The above steps are repeated 1000 times. The following exception is observed at random (race - SEDA?): ERROR [ReadStage:55] 2013-01-29 19:24:52,676 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:55,5,main] java.lang.AssertionError: DecoratedKey(-1, ) != DecoratedKey(62819832764241410631599989027761269388, 313a31) in C:\var\lib\cassandra\data\user_role_reverse_index\business_entity_role\user_role_reverse_index-business_entity_role-hf-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1367) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1229) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1164) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:822) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1271) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) This exception appears in the server at the time of client submitting a query request (row slice) and not at the time data is seeded. The client times out and this data can no longer be queried as the same exception would always occur from there on. Also on iteration 201, it appears that dropping column families failed and as a result their recreation failed with unique column family name violation (see exception below). Note that the data files are actually gone, so it appears that the server runtime responsible for creating column family was out of sync with the piece that dropped them: Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5105 ms Iteration: 200; Total running time for 1000 queries is 232; Average running time of 1000 queries is 0 ms Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5361 ms Iteration: 201; Total running time for 1000 queries is 222; Average running time of 1000 queries is 0 ms Starting dropping column families Starting creating column families Exception in thread main com.netflix.astyanax.connectionpool.exceptions.BadRequestException: BadRequestException: [host=127.0.0.1(127.0.0.1):9160, latency=2468(2469), attempts=1]InvalidRequestException(why:Keyspace names must be case-insensitively unique (user_role_reverse_index conflicts with user_role_reverse_index)) at
[jira] [Commented] (CASSANDRA-5202) CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name
[ https://issues.apache.org/jira/browse/CASSANDRA-5202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865948#comment-13865948 ] Pavel Yaskevich commented on CASSANDRA-5202: [~yukim] Sorry, Yuki, I completely forgot about secondary indexes, tested standard scenarios with different setups with stress tho. Will be happy to review v2. CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name Key: CASSANDRA-5202 URL: https://issues.apache.org/jira/browse/CASSANDRA-5202 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.9 Environment: OS: Windows 7, Server: Cassandra 1.1.9 release drop Client: astyanax 1.56.21, JVM: Sun/Oracle JVM 64 bit (jdk1.6.0_27) Reporter: Marat Bedretdinov Assignee: Yuki Morishita Labels: test Fix For: 2.1 Attachments: 5202.txt, astyanax-stress-driver.zip Attached is a driver that sequentially: 1. Drops keyspace 2. Creates keyspace 4. Creates 2 column families 5. Seeds 1M rows with 100 columns 6. Queries these 2 column families The above steps are repeated 1000 times. The following exception is observed at random (race - SEDA?): ERROR [ReadStage:55] 2013-01-29 19:24:52,676 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:55,5,main] java.lang.AssertionError: DecoratedKey(-1, ) != DecoratedKey(62819832764241410631599989027761269388, 313a31) in C:\var\lib\cassandra\data\user_role_reverse_index\business_entity_role\user_role_reverse_index-business_entity_role-hf-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1367) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1229) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1164) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:822) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1271) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) This exception appears in the server at the time of client submitting a query request (row slice) and not at the time data is seeded. The client times out and this data can no longer be queried as the same exception would always occur from there on. Also on iteration 201, it appears that dropping column families failed and as a result their recreation failed with unique column family name violation (see exception below). Note that the data files are actually gone, so it appears that the server runtime responsible for creating column family was out of sync with the piece that dropped them: Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5105 ms Iteration: 200; Total running time for 1000 queries is 232; Average running time of 1000 queries is 0 ms Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5361 ms Iteration: 201; Total running time for 1000 queries is 222; Average running time of 1000 queries is 0 ms Starting dropping column families Starting creating column families Exception in thread main com.netflix.astyanax.connectionpool.exceptions.BadRequestException: BadRequestException: [host=127.0.0.1(127.0.0.1):9160, latency=2468(2469), attempts=1]InvalidRequestException(why:Keyspace names must be case-insensitively unique (user_role_reverse_index conflicts with
[jira] [Comment Edited] (CASSANDRA-5202) CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name
[ https://issues.apache.org/jira/browse/CASSANDRA-5202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865955#comment-13865955 ] Aleksey Yeschenko edited comment on CASSANDRA-5202 at 1/8/14 10:01 PM: --- [~yukim] I was thinking about announcing the migration for all the auth tables with cf_id preset to what 2.0 would've generated. No special cases anywhere else. was (Author: iamaleksey): @yukim I was thinking about announcing the migration for all the auth tables with cf_id preset to what 2.0 would've generated. No special cases anywhere else. CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name Key: CASSANDRA-5202 URL: https://issues.apache.org/jira/browse/CASSANDRA-5202 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.9 Environment: OS: Windows 7, Server: Cassandra 1.1.9 release drop Client: astyanax 1.56.21, JVM: Sun/Oracle JVM 64 bit (jdk1.6.0_27) Reporter: Marat Bedretdinov Assignee: Yuki Morishita Labels: test Fix For: 2.1 Attachments: 5202.txt, astyanax-stress-driver.zip Attached is a driver that sequentially: 1. Drops keyspace 2. Creates keyspace 4. Creates 2 column families 5. Seeds 1M rows with 100 columns 6. Queries these 2 column families The above steps are repeated 1000 times. The following exception is observed at random (race - SEDA?): ERROR [ReadStage:55] 2013-01-29 19:24:52,676 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:55,5,main] java.lang.AssertionError: DecoratedKey(-1, ) != DecoratedKey(62819832764241410631599989027761269388, 313a31) in C:\var\lib\cassandra\data\user_role_reverse_index\business_entity_role\user_role_reverse_index-business_entity_role-hf-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1367) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1229) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1164) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:822) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1271) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) This exception appears in the server at the time of client submitting a query request (row slice) and not at the time data is seeded. The client times out and this data can no longer be queried as the same exception would always occur from there on. Also on iteration 201, it appears that dropping column families failed and as a result their recreation failed with unique column family name violation (see exception below). Note that the data files are actually gone, so it appears that the server runtime responsible for creating column family was out of sync with the piece that dropped them: Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5105 ms Iteration: 200; Total running time for 1000 queries is 232; Average running time of 1000 queries is 0 ms Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5361 ms Iteration: 201; Total running time for 1000 queries is 222; Average running time of 1000 queries is 0 ms Starting dropping column families Starting creating column families Exception in thread main
[jira] [Commented] (CASSANDRA-5202) CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name
[ https://issues.apache.org/jira/browse/CASSANDRA-5202?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13865955#comment-13865955 ] Aleksey Yeschenko commented on CASSANDRA-5202: -- @yukim I was thinking about announcing the migration for all the auth tables with cf_id preset to what 2.0 would've generated. No special cases anywhere else. CFs should have globally and temporally unique CF IDs to prevent reusing data from earlier incarnation of same CF name Key: CASSANDRA-5202 URL: https://issues.apache.org/jira/browse/CASSANDRA-5202 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.9 Environment: OS: Windows 7, Server: Cassandra 1.1.9 release drop Client: astyanax 1.56.21, JVM: Sun/Oracle JVM 64 bit (jdk1.6.0_27) Reporter: Marat Bedretdinov Assignee: Yuki Morishita Labels: test Fix For: 2.1 Attachments: 5202.txt, astyanax-stress-driver.zip Attached is a driver that sequentially: 1. Drops keyspace 2. Creates keyspace 4. Creates 2 column families 5. Seeds 1M rows with 100 columns 6. Queries these 2 column families The above steps are repeated 1000 times. The following exception is observed at random (race - SEDA?): ERROR [ReadStage:55] 2013-01-29 19:24:52,676 AbstractCassandraDaemon.java (line 135) Exception in thread Thread[ReadStage:55,5,main] java.lang.AssertionError: DecoratedKey(-1, ) != DecoratedKey(62819832764241410631599989027761269388, 313a31) in C:\var\lib\cassandra\data\user_role_reverse_index\business_entity_role\user_role_reverse_index-business_entity_role-hf-1-Data.db at org.apache.cassandra.db.columniterator.SSTableSliceIterator.init(SSTableSliceIterator.java:60) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:67) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:256) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1367) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1229) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1164) at org.apache.cassandra.db.Table.getRow(Table.java:378) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:69) at org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:822) at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1271) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:662) This exception appears in the server at the time of client submitting a query request (row slice) and not at the time data is seeded. The client times out and this data can no longer be queried as the same exception would always occur from there on. Also on iteration 201, it appears that dropping column families failed and as a result their recreation failed with unique column family name violation (see exception below). Note that the data files are actually gone, so it appears that the server runtime responsible for creating column family was out of sync with the piece that dropped them: Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5105 ms Iteration: 200; Total running time for 1000 queries is 232; Average running time of 1000 queries is 0 ms Starting dropping column families Dropped column families Starting dropping keyspace Dropped keyspace Starting creating column families Created column families Starting seeding data Total rows inserted: 100 in 5361 ms Iteration: 201; Total running time for 1000 queries is 222; Average running time of 1000 queries is 0 ms Starting dropping column families Starting creating column families Exception in thread main com.netflix.astyanax.connectionpool.exceptions.BadRequestException: BadRequestException: [host=127.0.0.1(127.0.0.1):9160, latency=2468(2469), attempts=1]InvalidRequestException(why:Keyspace names must be case-insensitively unique (user_role_reverse_index conflicts with
[jira] [Commented] (CASSANDRA-4288) prevent thrift server from starting before gossip has settled
[ https://issues.apache.org/jira/browse/CASSANDRA-4288?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866010#comment-13866010 ] Jonathan Ellis commented on CASSANDRA-4288: --- bq. The WARN line is only triggered if it has taken more than polls than expected. That ought not to be the normal case. Looks to me like it logs warn whenever there is any active or pending task while it loops. That doesn't strike me as an unusual scenario. prevent thrift server from starting before gossip has settled - Key: CASSANDRA-4288 URL: https://issues.apache.org/jira/browse/CASSANDRA-4288 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Peter Schuller Assignee: Chris Burroughs Fix For: 2.0.5 Attachments: CASSANDRA-4288-trunk.txt, j4288-1.2-v1-txt, j4288-1.2-v2-txt, j4288-1.2-v3.txt A serious problem is that there is no co-ordination whatsoever between gossip and the consumers of gossip. In particular, on a large cluster with hundreds of nodes, it takes several seconds for gossip to settle because the gossip stage is CPU bound. This leads to a node starting up and accessing thrift traffic long before it has any clue of what up and down. This leads to client-visible timeouts (for nodes that are down but not identified as such) and UnavailableException (for nodes that are up but not yet identified as such). This is really bad in general, but in particular for clients doing non-idempotent writes (counter increments). I was going to fix this as part of more significant re-writing in other tickets having to do with gossip/topology/etc, but that's not going to happen. So, the attached patch is roughly what we're running with in production now to make restarts bearable. The minimum wait time is both for ensuring that gossip has time to start becoming CPU bound if it will be, and the reason it's large is to allow for down nodes to be identified as such in most typical cases with a default phi conviction threshold (untested, we actually ran with a smaller number of 5 seconds minimum, but from past experience I believe 15 seconds is enough). The patch is tested on our 1.1 branch. It applies on trunk, and the diff is against trunk, but I have not tested it against trunk. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-4288) prevent thrift server from starting before gossip has settled
[ https://issues.apache.org/jira/browse/CASSANDRA-4288?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866049#comment-13866049 ] Chris Burroughs commented on CASSANDRA-4288: I guess I don't have enough data to answer that, just the anecdotes on my clusters. I was assuming that gossip mostly works for most clusters at this point and this is a fail safe for largish clusters. Since there is no WARN_BUT_COULD_JUST_BE_UNLUCKY_IN_A_LOOP I'm fine with making it INFO or adding another check to make it WARN only on more than one hit. prevent thrift server from starting before gossip has settled - Key: CASSANDRA-4288 URL: https://issues.apache.org/jira/browse/CASSANDRA-4288 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Peter Schuller Assignee: Chris Burroughs Fix For: 2.0.5 Attachments: CASSANDRA-4288-trunk.txt, j4288-1.2-v1-txt, j4288-1.2-v2-txt, j4288-1.2-v3.txt A serious problem is that there is no co-ordination whatsoever between gossip and the consumers of gossip. In particular, on a large cluster with hundreds of nodes, it takes several seconds for gossip to settle because the gossip stage is CPU bound. This leads to a node starting up and accessing thrift traffic long before it has any clue of what up and down. This leads to client-visible timeouts (for nodes that are down but not identified as such) and UnavailableException (for nodes that are up but not yet identified as such). This is really bad in general, but in particular for clients doing non-idempotent writes (counter increments). I was going to fix this as part of more significant re-writing in other tickets having to do with gossip/topology/etc, but that's not going to happen. So, the attached patch is roughly what we're running with in production now to make restarts bearable. The minimum wait time is both for ensuring that gossip has time to start becoming CPU bound if it will be, and the reason it's large is to allow for down nodes to be identified as such in most typical cases with a default phi conviction threshold (untested, we actually ran with a smaller number of 5 seconds minimum, but from past experience I believe 15 seconds is enough). The patch is tested on our 1.1 branch. It applies on trunk, and the diff is against trunk, but I have not tested it against trunk. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6558) Failure Detector takes 4-5x longer than it used to
[ https://issues.apache.org/jira/browse/CASSANDRA-6558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866102#comment-13866102 ] Joaquin Casares commented on CASSANDRA-6558: [~slebresne] I gave that whirl on a few random lines of code and they started spitting errors. I remember while coding, I ended up going with forceStop() since it allowed for faster shutdown times, especially when trying to catch/force issues. If I were to switch all of them to stops() it wouldn't hit the same expected cases. Let's see if CASSANDRA-4375 fixes this, if not we can revisit and see if we can get a disable FD command/switch/config. Failure Detector takes 4-5x longer than it used to -- Key: CASSANDRA-6558 URL: https://issues.apache.org/jira/browse/CASSANDRA-6558 Project: Cassandra Issue Type: Bug Reporter: Joaquin Casares Priority: Minor Labels: datastax_qa The Failure Detector appears to also be used by the java-driver (https://datastax-oss.atlassian.net/browse/JAVA-246) in determining if nodes are down or not. Because of the recent increase in time that it takes for Cassandra to noticed downed nodes, tests within the java-driver integration suite are currently failing. This should only impact driver usage minimally since it also relies on failed requests to find out if a node is down, but fixing the issue means we get the tests back online and the Failure Detector working as it previously had been. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQLSH hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866117#comment-13866117 ] Nikolai Grigoriev commented on CASSANDRA-6407: -- Some additional details. I can confirm that the problem is not limited to CQLSH, it can be reproduced via CQL/Thrift. Which does not surprise me, I was assuming that's what CQLSH is using today. One of my coworkers has pointed out that he did not observe this problem in his small single-node cluster, even with larger amounts of data in one response. I was curious enough to try it so I have configured a single-node Cassandra 2.0.4 cluster on a spare Linux machine, loaded my schema there and generated the problematic test data set. I could not reproduce the problem, i.e. I was getting back much larger result set than in my larger cluster. After that I took my production cassandra.yaml, changed the cluster name to a dummy one, reinitialized that single-node cluster with new config, reloaded the data and I could immediately reproduce the problem. To keep long story short, I was comparing the parameters I changed in my config with the defaults and finally found THE parameter that is clearly responsible for this issue: rpc_server_type. If set to sync, then I can query larger data set. If set to hsha - I can only query up to ~256Kb of data and then the connection gets stuck forever. Anything obvious that I am missing about the limitations of hsha? CQLSH hangs forever when querying more than certain amount of data -- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Tools Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikolai Grigoriev updated CASSANDRA-6407: - Reproduced In: 2.0.4 CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikolai Grigoriev updated CASSANDRA-6407: - Component/s: (was: Tools) Core CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikolai Grigoriev updated CASSANDRA-6407: - Summary: CQL/Thrift request hangs forever when querying more than certain amount of data (was: CQLSH hangs forever when querying more than certain amount of data) CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866133#comment-13866133 ] Nikolai Grigoriev commented on CASSANDRA-6407: -- It sounds somewhat related to: CASSANDRA-4573 CASSANDRA-6373 CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866164#comment-13866164 ] Jonathan Ellis commented on CASSANDRA-6407: --- /cc [~xedin] CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6157) Selectively Disable hinted handoff for a data center
[ https://issues.apache.org/jira/browse/CASSANDRA-6157?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Lyuben Todorov updated CASSANDRA-6157: -- Attachment: trunk-6157-v4.diff Ah, attached before renaming, I've attached v4 with the rename. Thanks for the catch! Selectively Disable hinted handoff for a data center Key: CASSANDRA-6157 URL: https://issues.apache.org/jira/browse/CASSANDRA-6157 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Fix For: 2.0.5 Attachments: trunk-6157-v2.diff, trunk-6157-v3.diff, trunk-6157-v4.diff, trunk-6157.txt Cassandra supports disabling the hints or reducing the window for hints. It would be helpful to have a switch which stops hints to a down data center but continue hints to other DCs. This is helpful during data center fail over as hints will put more unnecessary pressure on the DC taking double traffic. Also since now Cassandra is under reduced reduncany, we don't want to disable hints within the DC. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866206#comment-13866206 ] Pavel Yaskevich commented on CASSANDRA-6407: [~ngrigoriev] Is it possible to you to give us at least part of the data that you have been testing with? It sounds like it could be a bug in hsha implementation of the thrift server. CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6558) Failure Detector takes 4-5x longer than it used to
[ https://issues.apache.org/jira/browse/CASSANDRA-6558?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866225#comment-13866225 ] Brandon Williams commented on CASSANDRA-6558: - Well, CASSANDRA-4375 is going to 'fix' it by basically allowing you to set the initial interval and max interval to whatever you want via system properties :) Failure Detector takes 4-5x longer than it used to -- Key: CASSANDRA-6558 URL: https://issues.apache.org/jira/browse/CASSANDRA-6558 Project: Cassandra Issue Type: Bug Reporter: Joaquin Casares Priority: Minor Labels: datastax_qa The Failure Detector appears to also be used by the java-driver (https://datastax-oss.atlassian.net/browse/JAVA-246) in determining if nodes are down or not. Because of the recent increase in time that it takes for Cassandra to noticed downed nodes, tests within the java-driver integration suite are currently failing. This should only impact driver usage minimally since it also relies on failed requests to find out if a node is down, but fixing the issue means we get the tests back online and the Failure Detector working as it previously had been. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[3/5] git commit: Wait for gossip to settle before accepting client connections patch by Chris Burroughs; reviewed by Tyler Hobbs and jbellis for CASSANDRA-4288
Wait for gossip to settle before accepting client connections patch by Chris Burroughs; reviewed by Tyler Hobbs and jbellis for CASSANDRA-4288 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/de19f963 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/de19f963 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/de19f963 Branch: refs/heads/trunk Commit: de19f963aeed2752374d2f84c1b230f6cab253f1 Parents: f8fd7db Author: Jonathan Ellis jbel...@apache.org Authored: Wed Jan 8 20:53:46 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Wed Jan 8 20:53:46 2014 -0600 -- CHANGES.txt | 1 + .../cassandra/service/CassandraDaemon.java | 50 2 files changed, 51 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/de19f963/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d0b63a0..e96a8e0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.0.5 + * Wait for gossip to settle before accepting client connections (CASSANDRA-4288) * Delete unfinished compaction incrementally (CASSANDRA-6086) * Allow specifying custom secondary index options in CQL3 (CASSANDRA-6480) Merged from 1.2: http://git-wip-us.apache.org/repos/asf/cassandra/blob/de19f963/src/java/org/apache/cassandra/service/CassandraDaemon.java -- diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index d36b0db..d497a38 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -34,10 +34,14 @@ import javax.management.StandardMBean; import com.addthis.metrics.reporter.config.ReporterConfig; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.Uninterruptibles; import org.apache.log4j.PropertyConfigurator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.Schema; @@ -365,6 +369,8 @@ public class CassandraDaemon } } +waitForGossipToSettle(); + // Thift InetAddress rpcAddr = DatabaseDescriptor.getRpcAddress(); int rpcPort = DatabaseDescriptor.getRpcPort(); @@ -489,6 +495,50 @@ public class CassandraDaemon destroy(); } + +private void waitForGossipToSettle() +{ +int forceAfter = Integer.getInteger(cassandra.skip_wait_for_gossip_to_settle, -1); +if (forceAfter == 0) +{ +return; +} +final int GOSSIP_SETTLE_MIN_WAIT_MS = 5000; +final int GOSSIP_SETTLE_POLL_INTERVAL_MS = 1000; +final int GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED = 3; + +logger.info(waiting for gossip to settle before accepting client requests...); +Uninterruptibles.sleepUninterruptibly(GOSSIP_SETTLE_MIN_WAIT_MS, TimeUnit.MILLISECONDS); +int totalPolls = 0; +int numOkay = 0; +JMXEnabledThreadPoolExecutor gossipStage = (JMXEnabledThreadPoolExecutor)StageManager.getStage(Stage.GOSSIP); +while (numOkay GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED) +{ + Uninterruptibles.sleepUninterruptibly(GOSSIP_SETTLE_POLL_INTERVAL_MS, TimeUnit.MILLISECONDS); +long completed = gossipStage.getCompletedTasks(); +long active = gossipStage.getActiveCount(); +long pending = gossipStage.getPendingTasks(); +totalPolls++; +if (active == 0 pending == 0) +{ +logger.debug(gossip looks settled. CompletedTasks: {}, completed); +numOkay++; +} +else +{ +logger.info(gossip not settled after {} polls. Gossip Stage active/pending/completed: {}/{}/{}, totalPolls, active, pending, completed); +numOkay = 0; +} +if (forceAfter 0 totalPolls forceAfter) +{ +logger.warn(Gossip not settled but startup forced by cassandra.skip_wait_for_gossip_to_settle. Gossip Stage active/pending/completed: {}/{}/{}, +totalPolls, active, pending, completed); +break; +} +} +logger.info(gossip settled after {} extra polls; proceeding, totalPolls -
[4/5] git commit: Wait for gossip to settle before accepting client connections patch by Chris Burroughs; reviewed by Tyler Hobbs and jbellis for CASSANDRA-4288
Wait for gossip to settle before accepting client connections patch by Chris Burroughs; reviewed by Tyler Hobbs and jbellis for CASSANDRA-4288 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/de19f963 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/de19f963 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/de19f963 Branch: refs/heads/cassandra-2.0 Commit: de19f963aeed2752374d2f84c1b230f6cab253f1 Parents: f8fd7db Author: Jonathan Ellis jbel...@apache.org Authored: Wed Jan 8 20:53:46 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Wed Jan 8 20:53:46 2014 -0600 -- CHANGES.txt | 1 + .../cassandra/service/CassandraDaemon.java | 50 2 files changed, 51 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/de19f963/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d0b63a0..e96a8e0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.0.5 + * Wait for gossip to settle before accepting client connections (CASSANDRA-4288) * Delete unfinished compaction incrementally (CASSANDRA-6086) * Allow specifying custom secondary index options in CQL3 (CASSANDRA-6480) Merged from 1.2: http://git-wip-us.apache.org/repos/asf/cassandra/blob/de19f963/src/java/org/apache/cassandra/service/CassandraDaemon.java -- diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java index d36b0db..d497a38 100644 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@ -34,10 +34,14 @@ import javax.management.StandardMBean; import com.addthis.metrics.reporter.config.ReporterConfig; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.Uninterruptibles; import org.apache.log4j.PropertyConfigurator; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; +import org.apache.cassandra.concurrent.Stage; +import org.apache.cassandra.concurrent.StageManager; import org.apache.cassandra.config.CFMetaData; import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.Schema; @@ -365,6 +369,8 @@ public class CassandraDaemon } } +waitForGossipToSettle(); + // Thift InetAddress rpcAddr = DatabaseDescriptor.getRpcAddress(); int rpcPort = DatabaseDescriptor.getRpcPort(); @@ -489,6 +495,50 @@ public class CassandraDaemon destroy(); } + +private void waitForGossipToSettle() +{ +int forceAfter = Integer.getInteger(cassandra.skip_wait_for_gossip_to_settle, -1); +if (forceAfter == 0) +{ +return; +} +final int GOSSIP_SETTLE_MIN_WAIT_MS = 5000; +final int GOSSIP_SETTLE_POLL_INTERVAL_MS = 1000; +final int GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED = 3; + +logger.info(waiting for gossip to settle before accepting client requests...); +Uninterruptibles.sleepUninterruptibly(GOSSIP_SETTLE_MIN_WAIT_MS, TimeUnit.MILLISECONDS); +int totalPolls = 0; +int numOkay = 0; +JMXEnabledThreadPoolExecutor gossipStage = (JMXEnabledThreadPoolExecutor)StageManager.getStage(Stage.GOSSIP); +while (numOkay GOSSIP_SETTLE_POLL_SUCCESSES_REQUIRED) +{ + Uninterruptibles.sleepUninterruptibly(GOSSIP_SETTLE_POLL_INTERVAL_MS, TimeUnit.MILLISECONDS); +long completed = gossipStage.getCompletedTasks(); +long active = gossipStage.getActiveCount(); +long pending = gossipStage.getPendingTasks(); +totalPolls++; +if (active == 0 pending == 0) +{ +logger.debug(gossip looks settled. CompletedTasks: {}, completed); +numOkay++; +} +else +{ +logger.info(gossip not settled after {} polls. Gossip Stage active/pending/completed: {}/{}/{}, totalPolls, active, pending, completed); +numOkay = 0; +} +if (forceAfter 0 totalPolls forceAfter) +{ +logger.warn(Gossip not settled but startup forced by cassandra.skip_wait_for_gossip_to_settle. Gossip Stage active/pending/completed: {}/{}/{}, +totalPolls, active, pending, completed); +break; +} +} +logger.info(gossip settled after {} extra polls; proceeding, totalPolls -
[1/5] git commit: Fix execution of LOCAL_QUORUM queries with SimpleStrategy
Updated Branches: refs/heads/cassandra-2.0 f8fd7db67 - de19f963a refs/heads/trunk c2294aa21 - db07b20ed Fix execution of LOCAL_QUORUM queries with SimpleStrategy patch by alexliu68; reviewed by slebresne for CASSANDRA-6545 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/32d7cb50 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/32d7cb50 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/32d7cb50 Branch: refs/heads/trunk Commit: 32d7cb5066050ef6123f50a25c6e9b4c9e180ea0 Parents: 2a7c20e Author: Sylvain Lebresne sylv...@datastax.com Authored: Wed Jan 8 17:14:58 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Wed Jan 8 17:15:20 2014 +0100 -- CHANGES.txt | 1 + .../apache/cassandra/db/ConsistencyLevel.java | 60 +--- .../locator/AbstractReplicationStrategy.java| 2 +- 3 files changed, 42 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/32d7cb50/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 5a85977..cba97d0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -8,6 +8,7 @@ * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) * Add ability to throttle batchlog replay (CASSANDRA-6550) + * Fix executing LOCAL_QUORUM with SimpleStrategy (CASSANDRA-6545) 1.2.13 http://git-wip-us.apache.org/repos/asf/cassandra/blob/32d7cb50/src/java/org/apache/cassandra/db/ConsistencyLevel.java -- diff --git a/src/java/org/apache/cassandra/db/ConsistencyLevel.java b/src/java/org/apache/cassandra/db/ConsistencyLevel.java index 4d72767..3737c73 100644 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@ -88,9 +88,16 @@ public enum ConsistencyLevel return codeIdx[code]; } +private int quorumFor(Table table) +{ +return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; +} + private int localQuorumFor(Table table, String dc) { -return (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1; +return (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) + ? (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1 + : quorumFor(table); } public int blockFor(Table table) @@ -107,17 +114,24 @@ public enum ConsistencyLevel case THREE: return 3; case QUORUM: -return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; +return quorumFor(table); case ALL: return table.getReplicationStrategy().getReplicationFactor(); case LOCAL_QUORUM: return localQuorumFor(table, DatabaseDescriptor.getLocalDataCenter()); case EACH_QUORUM: -NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy(); -int n = 0; -for (String dc : strategy.getDatacenters()) -n += localQuorumFor(table, dc); -return n; +if (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) +{ +NetworkTopologyStrategy strategy = (NetworkTopologyStrategy) table.getReplicationStrategy(); +int n = 0; +for (String dc : strategy.getDatacenters()) +n += localQuorumFor(table, dc); +return n; +} +else +{ +return quorumFor(table); +} default: throw new UnsupportedOperationException(Invalid consistency level: + toString()); } @@ -208,16 +222,20 @@ public enum ConsistencyLevel // local hint is acceptable, and local node is always live return true; case LOCAL_ONE: -return countLocalEndpoints(liveEndpoints) = 1; +return countLocalEndpoints(liveEndpoints) = 1; case LOCAL_QUORUM: return countLocalEndpoints(liveEndpoints) = blockFor(table); case EACH_QUORUM: -for (Map.EntryString, Integer entry : countPerDCEndpoints(table, liveEndpoints).entrySet()) +if (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) { -
[5/5] git commit: merge from 2.0
merge from 2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/db07b20e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/db07b20e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/db07b20e Branch: refs/heads/trunk Commit: db07b20edbcd2a23b0669e64e466cd13ce47e2f3 Parents: c2294aa de19f96 Author: Jonathan Ellis jbel...@apache.org Authored: Wed Jan 8 20:54:51 2014 -0600 Committer: Jonathan Ellis jbel...@apache.org Committed: Wed Jan 8 20:54:51 2014 -0600 -- CHANGES.txt | 2 + .../apache/cassandra/db/ConsistencyLevel.java | 59 +--- .../locator/AbstractReplicationStrategy.java| 2 +- .../cassandra/service/CassandraDaemon.java | 51 - 4 files changed, 92 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/db07b20e/CHANGES.txt -- diff --cc CHANGES.txt index 58a0906,e96a8e0..0a8b9b9 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,31 -1,5 +1,32 @@@ +2.1 + * Introduce AtomicBTreeColumns (CASSANDRA-6271) + * Multithreaded commitlog (CASSANDRA-3578) + * allocate fixed index summary memory pool and resample cold index summaries + to use less memory (CASSANDRA-5519) + * Removed multithreaded compaction (CASSANDRA-6142) + * Parallelize fetching rows for low-cardinality indexes (CASSANDRA-1337) + * change logging from log4j to logback (CASSANDRA-5883) + * switch to LZ4 compression for internode communication (CASSANDRA-5887) + * Stop using Thrift-generated Index* classes internally (CASSANDRA-5971) + * Remove 1.2 network compatibility code (CASSANDRA-5960) + * Remove leveled json manifest migration code (CASSANDRA-5996) + * Remove CFDefinition (CASSANDRA-6253) + * Use AtomicIntegerFieldUpdater in RefCountedMemory (CASSANDRA-6278) + * User-defined types for CQL3 (CASSANDRA-5590) + * Use of o.a.c.metrics in nodetool (CASSANDRA-5871, 6406) + * Batch read from OTC's queue and cleanup (CASSANDRA-1632) + * Secondary index support for collections (CASSANDRA-4511, 6383) + * SSTable metadata(Stats.db) format change (CASSANDRA-6356) + * Push composites support in the storage engine + (CASSANDRA-5417, CASSANDRA-6520) + * Add snapshot space used to cfstats (CASSANDRA-6231) + * Add cardinality estimator for key count estimation (CASSANDRA-5906) + * CF id is changed to be non-deterministic. Data dir/key cache are created + uniquely for CF id (CASSANDRA-5202) + + 2.0.5 + * Wait for gossip to settle before accepting client connections (CASSANDRA-4288) * Delete unfinished compaction incrementally (CASSANDRA-6086) * Allow specifying custom secondary index options in CQL3 (CASSANDRA-6480) Merged from 1.2: http://git-wip-us.apache.org/repos/asf/cassandra/blob/db07b20e/src/java/org/apache/cassandra/service/CassandraDaemon.java -- diff --cc src/java/org/apache/cassandra/service/CassandraDaemon.java index ccabad5,d497a38..260dcb2 --- a/src/java/org/apache/cassandra/service/CassandraDaemon.java +++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java @@@ -30,8 -32,10 +30,8 @@@ import javax.management.ObjectName import javax.management.StandardMBean; import com.addthis.metrics.reporter.config.ReporterConfig; -- import com.google.common.collect.Iterables; + import com.google.common.util.concurrent.Uninterruptibles; -import org.apache.log4j.PropertyConfigurator; import org.slf4j.Logger; import org.slf4j.LoggerFactory;
[2/5] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Conflicts: src/java/org/apache/cassandra/db/ConsistencyLevel.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/f8fd7db6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/f8fd7db6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/f8fd7db6 Branch: refs/heads/trunk Commit: f8fd7db67e3ff7268bd2bc96bd1373d54664b7dd Parents: 5fa6055 32d7cb5 Author: Sylvain Lebresne sylv...@datastax.com Authored: Wed Jan 8 17:22:55 2014 +0100 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Wed Jan 8 17:22:55 2014 +0100 -- CHANGES.txt | 1 + .../apache/cassandra/db/ConsistencyLevel.java | 59 +--- .../locator/AbstractReplicationStrategy.java| 2 +- 3 files changed, 41 insertions(+), 21 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8fd7db6/CHANGES.txt -- diff --cc CHANGES.txt index faf52a4,cba97d0..d0b63a0 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -5,29 -8,10 +5,30 @@@ Merged from 1.2 * fsync compression metadata (CASSANDRA-6531) * Validate CF existence on execution for prepared statement (CASSANDRA-6535) * Add ability to throttle batchlog replay (CASSANDRA-6550) + * Fix executing LOCAL_QUORUM with SimpleStrategy (CASSANDRA-6545) -1.2.13 +2.0.4 + * Allow removing snapshots of no-longer-existing CFs (CASSANDRA-6418) + * add StorageService.stopDaemon() (CASSANDRA-4268) + * add IRE for invalid CF supplied to get_count (CASSANDRA-5701) + * add client encryption support to sstableloader (CASSANDRA-6378) + * Fix accept() loop for SSL sockets post-shutdown (CASSANDRA-6468) + * Fix size-tiered compaction in LCS L0 (CASSANDRA-6496) + * Fix assertion failure in filterColdSSTables (CASSANDRA-6483) + * Fix row tombstones in larger-than-memory compactions (CASSANDRA-6008) + * Fix cleanup ClassCastException (CASSANDRA-6462) + * Reduce gossip memory use by interning VersionedValue strings (CASSANDRA-6410) + * Allow specifying datacenters to participate in a repair (CASSANDRA-6218) + * Fix divide-by-zero in PCI (CASSANDRA-6403) + * Fix setting last compacted key in the wrong level for LCS (CASSANDRA-6284) + * Add millisecond precision formats to the timestamp parser (CASSANDRA-6395) + * Expose a total memtable size metric for a CF (CASSANDRA-6391) + * cqlsh: handle symlinks properly (CASSANDRA-6425) + * Fix potential infinite loop when paging query with IN (CASSANDRA-6464) + * Fix assertion error in AbstractQueryPager.discardFirst (CASSANDRA-6447) + * Fix streaming older SSTable yields unnecessary tombstones (CASSANDRA-6527) +Merged from 1.2: * Improved error message on bad properties in DDL queries (CASSANDRA-6453) * Randomize batchlog candidates selection (CASSANDRA-6481) * Fix thundering herd on endpoint cache invalidation (CASSANDRA-6345, 6485) http://git-wip-us.apache.org/repos/asf/cassandra/blob/f8fd7db6/src/java/org/apache/cassandra/db/ConsistencyLevel.java -- diff --cc src/java/org/apache/cassandra/db/ConsistencyLevel.java index 0f6aba7,3737c73..6d04314 --- a/src/java/org/apache/cassandra/db/ConsistencyLevel.java +++ b/src/java/org/apache/cassandra/db/ConsistencyLevel.java @@@ -89,12 -88,19 +89,19 @@@ public enum ConsistencyLeve return codeIdx[code]; } -private int quorumFor(Table table) ++private int quorumFor(Keyspace keyspace) + { -return (table.getReplicationStrategy().getReplicationFactor() / 2) + 1; ++return (keyspace.getReplicationStrategy().getReplicationFactor() / 2) + 1; + } + -private int localQuorumFor(Table table, String dc) +private int localQuorumFor(Keyspace keyspace, String dc) { - return (((NetworkTopologyStrategy) keyspace.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1; -return (table.getReplicationStrategy() instanceof NetworkTopologyStrategy) - ? (((NetworkTopologyStrategy) table.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1 - : quorumFor(table); ++return (keyspace.getReplicationStrategy() instanceof NetworkTopologyStrategy) ++ ? (((NetworkTopologyStrategy) keyspace.getReplicationStrategy()).getReplicationFactor(dc) / 2) + 1 ++ : quorumFor(keyspace); } -public int blockFor(Table table) +public int blockFor(Keyspace keyspace) { switch (this) { @@@ -108,17 -114,24 +115,24 @@@ case THREE: return 3; case QUORUM: - return
[jira] [Resolved] (CASSANDRA-4288) prevent thrift server from starting before gossip has settled
[ https://issues.apache.org/jira/browse/CASSANDRA-4288?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-4288. --- Resolution: Fixed changed to info and committed. prevent thrift server from starting before gossip has settled - Key: CASSANDRA-4288 URL: https://issues.apache.org/jira/browse/CASSANDRA-4288 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Peter Schuller Assignee: Chris Burroughs Fix For: 2.0.5 Attachments: CASSANDRA-4288-trunk.txt, j4288-1.2-v1-txt, j4288-1.2-v2-txt, j4288-1.2-v3.txt A serious problem is that there is no co-ordination whatsoever between gossip and the consumers of gossip. In particular, on a large cluster with hundreds of nodes, it takes several seconds for gossip to settle because the gossip stage is CPU bound. This leads to a node starting up and accessing thrift traffic long before it has any clue of what up and down. This leads to client-visible timeouts (for nodes that are down but not identified as such) and UnavailableException (for nodes that are up but not yet identified as such). This is really bad in general, but in particular for clients doing non-idempotent writes (counter increments). I was going to fix this as part of more significant re-writing in other tickets having to do with gossip/topology/etc, but that's not going to happen. So, the attached patch is roughly what we're running with in production now to make restarts bearable. The minimum wait time is both for ensuring that gossip has time to start becoming CPU bound if it will be, and the reason it's large is to allow for down nodes to be identified as such in most typical cases with a default phi conviction threshold (untested, we actually ran with a smaller number of 5 seconds minimum, but from past experience I believe 15 seconds is enough). The patch is tested on our 1.1 branch. It applies on trunk, and the diff is against trunk, but I have not tested it against trunk. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikolai Grigoriev updated CASSANDRA-6407: - Attachment: cassandra.yaml CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikolai Grigoriev updated CASSANDRA-6407: - Attachment: cassandra6407test.cql.gz CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6157) Selectively Disable hinted handoff for a data center
[ https://issues.apache.org/jira/browse/CASSANDRA-6157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866244#comment-13866244 ] Jonathan Ellis commented on CASSANDRA-6157: --- So... not to bikeshed myself, but what format should we use here? I suggested {{ {'dc1', 'dc2'} }} (python set literals, admittedly somewhat obscure), other options could be {{ ['dc1', 'dc2'] }} (json list) or {{ dc1,dc2 }} (CDL). What we have here is {{ {dc1,dc2} }} which is invalid just about everything. :) Suggest just switching to CDL? (I don't think we can have a YAML construct that can be either a string, or a yaml list but I'm not an expert so I may be wrong on that point.) P.S. We typically just use Config as a raw values dump and do extra parsing in DatabaseDescriptor if necessary. Selectively Disable hinted handoff for a data center Key: CASSANDRA-6157 URL: https://issues.apache.org/jira/browse/CASSANDRA-6157 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Fix For: 2.0.5 Attachments: trunk-6157-v2.diff, trunk-6157-v3.diff, trunk-6157-v4.diff, trunk-6157.txt Cassandra supports disabling the hints or reducing the window for hints. It would be helpful to have a switch which stops hints to a down data center but continue hints to other DCs. This is helpful during data center fail over as hints will put more unnecessary pressure on the DC taking double traffic. Also since now Cassandra is under reduced reduncany, we don't want to disable hints within the DC. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Comment Edited] (CASSANDRA-6157) Selectively Disable hinted handoff for a data center
[ https://issues.apache.org/jira/browse/CASSANDRA-6157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866244#comment-13866244 ] Jonathan Ellis edited comment on CASSANDRA-6157 at 1/9/14 3:10 AM: --- So... not to bikeshed myself, but what format should we use here? I suggested {code} {'dc1', 'dc2'} {code} (python set literals, admittedly somewhat obscure), other options could be {code} ['dc1', 'dc2'] {code} (json list) or {code} dc1,dc2 {code} (CDL). What we have here is {code} {dc1,dc2} {code} which is invalid just about everything. :) Suggest just switching to CDL? (I don't think we can have a YAML construct that can be either a string, or a yaml list but I'm not an expert so I may be wrong on that point.) P.S. We typically just use Config as a raw values dump and do extra parsing in DatabaseDescriptor if necessary. was (Author: jbellis): So... not to bikeshed myself, but what format should we use here? I suggested {{ {'dc1', 'dc2'} }} (python set literals, admittedly somewhat obscure), other options could be {{ ['dc1', 'dc2'] }} (json list) or {{ dc1,dc2 }} (CDL). What we have here is {{ {dc1,dc2} }} which is invalid just about everything. :) Suggest just switching to CDL? (I don't think we can have a YAML construct that can be either a string, or a yaml list but I'm not an expert so I may be wrong on that point.) P.S. We typically just use Config as a raw values dump and do extra parsing in DatabaseDescriptor if necessary. Selectively Disable hinted handoff for a data center Key: CASSANDRA-6157 URL: https://issues.apache.org/jira/browse/CASSANDRA-6157 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Fix For: 2.0.5 Attachments: trunk-6157-v2.diff, trunk-6157-v3.diff, trunk-6157-v4.diff, trunk-6157.txt Cassandra supports disabling the hints or reducing the window for hints. It would be helpful to have a switch which stops hints to a down data center but continue hints to other DCs. This is helpful during data center fail over as hints will put more unnecessary pressure on the DC taking double traffic. Also since now Cassandra is under reduced reduncany, we don't want to disable hints within the DC. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866245#comment-13866245 ] Nikolai Grigoriev commented on CASSANDRA-6407: -- [~xedin] I have prepared a simple test that does demonstrate the problem even in a small single-node cluster. Interestingly enough, with this test and such a small cluster with no load at all sometimes it actually works. So, here is how I use it: 1. Set the RPC server type to hsha 2. Load the attached CQL ile 3. Use CQLSH use cassandra6407test ; select * from my_test_table ; In most of the cases this SELECT gets stuck forever. Sometimes if you interrupt it (after a while) and do it again it actually returns all the data on the second attempt. Sometimes it does not. If you restart CQLSH and do it again - it will get stuck again. Specifying a LIMIT above 24-25 demonstrates similar behavior. If you switch RPC server type to sync and restart, then select * from my_test_table ; works all the time. It almost feels like some sort of race condition or a timing issue somewhere between the part that produces the query result and the part that streams it back to the client. CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6157) Selectively Disable hinted handoff for a data center
[ https://issues.apache.org/jira/browse/CASSANDRA-6157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866257#comment-13866257 ] Jonathan Ellis commented on CASSANDRA-6157: --- FWIW this is a thing, but probably overkill here unless we are worried about people having commas or spaces in their datacenter names: https://github.com/FasterXML/jackson-dataformat-csv Selectively Disable hinted handoff for a data center Key: CASSANDRA-6157 URL: https://issues.apache.org/jira/browse/CASSANDRA-6157 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Fix For: 2.0.5 Attachments: trunk-6157-v2.diff, trunk-6157-v3.diff, trunk-6157-v4.diff, trunk-6157.txt Cassandra supports disabling the hints or reducing the window for hints. It would be helpful to have a switch which stops hints to a down data center but continue hints to other DCs. This is helpful during data center fail over as hints will put more unnecessary pressure on the DC taking double traffic. Also since now Cassandra is under reduced reduncany, we don't want to disable hints within the DC. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Nikolai Grigoriev updated CASSANDRA-6407: - Attachment: system.log.gz this is the DEBUG log - I have tried that select * request 3 times after restarting the server with RPC server type set to hsha. CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz, system.log.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Comment Edited] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866245#comment-13866245 ] Nikolai Grigoriev edited comment on CASSANDRA-6407 at 1/9/14 3:21 AM: -- [~xedin] I have prepared a simple test that does demonstrate the problem even in a small single-node cluster. Interestingly enough, with this test and such a small cluster with no load at all sometimes it actually works. So, here is how I use it: 1. Set the RPC server type to hsha 2. Load the attached CQL ile 3. Use CQLSH use cassandra6407test ; select * from my_test_table ; In most of the cases this SELECT gets stuck forever. Sometimes if you interrupt it (after a while) and do it again it actually returns all the data on the second attempt. Sometimes it does not. If you restart CQLSH and do it again - it will get stuck again. Specifying a LIMIT above 24-25 demonstrates similar behavior. If you switch RPC server type to sync and restart, then select * from my_test_table ; works all the time. It almost feels like some sort of race condition or a timing issue somewhere between the part that produces the query result and the part that streams it back to the client. The server config I have attached is simplified, I have disabled JNA, JEMalloc etc to have a configuration that is as close as possible to the default installation. was (Author: ngrigoriev): [~xedin] I have prepared a simple test that does demonstrate the problem even in a small single-node cluster. Interestingly enough, with this test and such a small cluster with no load at all sometimes it actually works. So, here is how I use it: 1. Set the RPC server type to hsha 2. Load the attached CQL ile 3. Use CQLSH use cassandra6407test ; select * from my_test_table ; In most of the cases this SELECT gets stuck forever. Sometimes if you interrupt it (after a while) and do it again it actually returns all the data on the second attempt. Sometimes it does not. If you restart CQLSH and do it again - it will get stuck again. Specifying a LIMIT above 24-25 demonstrates similar behavior. If you switch RPC server type to sync and restart, then select * from my_test_table ; works all the time. It almost feels like some sort of race condition or a timing issue somewhere between the part that produces the query result and the part that streams it back to the client. CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz, system.log.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866286#comment-13866286 ] Pavel Yaskevich commented on CASSANDRA-6407: Thank you, [~ngrigoriev]! I will start working on this asap. CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz, system.log.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866354#comment-13866354 ] Nick Bailey commented on CASSANDRA-6407: This definitely sounds like the same thing as CASSANDRA-6373 given that it happens with vnode clusters which would make describe_ring return a fairly large response. CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz, system.log.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866359#comment-13866359 ] Pavel Yaskevich commented on CASSANDRA-6407: Yeah, they look similar but at lease we can reproduce, can you try one more thing please while you are on it - disable vnodes and try hsha with your data? CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz, system.log.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866362#comment-13866362 ] Nick Bailey commented on CASSANDRA-6407: I'm not sure if you are asking me to try that in CASSANDRA-6373. I've verified it does not happen with vnodes disabled. The steps I described there should also let you reproduce the issue in a unit test. CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz, system.log.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6440) Repair should allow repairing particular endpoints to reduce WAN usage.
[ https://issues.apache.org/jira/browse/CASSANDRA-6440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] sankalp kohli updated CASSANDRA-6440: - Attachment: JIRA-6440-v2.diff The problem is that I am checking that all hosts passed as param during repair need to be neighbors. If someone is repairing two ranges, the neighbors are different for each range and hence it will fail. If you try to specify -pr and specify the right neighbors, then it will work. So I am removing this check. Now it will ignore any hosts provided which are not neighbors. Attaching the v2 patch with all changes Repair should allow repairing particular endpoints to reduce WAN usage. Key: CASSANDRA-6440 URL: https://issues.apache.org/jira/browse/CASSANDRA-6440 Project: Cassandra Issue Type: New Feature Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Attachments: 6440_repair.log, JIRA-6440-v2.diff, JIRA-6440.diff The way we send out data that does not match over WAN can be improved. Example: Say there are four nodes(A,B,C,D) which are replica of a range we are repairing. A, B is in DC1 and C,D is in DC2. If A does not have the data which other replicas have, then we will have following streams 1) A to B and back 2) A to C and back(Goes over WAN) 3) A to D and back(Goes over WAN) One of the ways of doing it to reduce WAN traffic is this. 1) Repair A and B only with each other and C and D with each other starting at same time t. 2) Once these repairs have finished, A,B and C,D are in sync with respect to time t. 3) Now run a repair between A and C, the streams which are exchanged as a result of the diff will also be streamed to B and D via A and C(C and D behaves like a proxy to the streams). For a replication of DC1:2,DC2:2, the WAN traffic will get reduced by 50% and even more for higher replication factors. Another easy way to do this is to have repair command take nodes with which you want to repair with. Then we can do something like this. 1) Run repair between (A and B) and (C and D) 2) Run repair between (A and C) 3) Run repair between (A and B) and (C and D) But this will increase the traffic inside the DC as we wont be doing proxy. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6440) Repair should allow repairing particular endpoints to reduce WAN usage.
[ https://issues.apache.org/jira/browse/CASSANDRA-6440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] sankalp kohli updated CASSANDRA-6440: - Attachment: (was: JIRA-6440-v2.diff) Repair should allow repairing particular endpoints to reduce WAN usage. Key: CASSANDRA-6440 URL: https://issues.apache.org/jira/browse/CASSANDRA-6440 Project: Cassandra Issue Type: New Feature Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Attachments: 6440_repair.log, JIRA-6440-v2.diff, JIRA-6440.diff The way we send out data that does not match over WAN can be improved. Example: Say there are four nodes(A,B,C,D) which are replica of a range we are repairing. A, B is in DC1 and C,D is in DC2. If A does not have the data which other replicas have, then we will have following streams 1) A to B and back 2) A to C and back(Goes over WAN) 3) A to D and back(Goes over WAN) One of the ways of doing it to reduce WAN traffic is this. 1) Repair A and B only with each other and C and D with each other starting at same time t. 2) Once these repairs have finished, A,B and C,D are in sync with respect to time t. 3) Now run a repair between A and C, the streams which are exchanged as a result of the diff will also be streamed to B and D via A and C(C and D behaves like a proxy to the streams). For a replication of DC1:2,DC2:2, the WAN traffic will get reduced by 50% and even more for higher replication factors. Another easy way to do this is to have repair command take nodes with which you want to repair with. Then we can do something like this. 1) Run repair between (A and B) and (C and D) 2) Run repair between (A and C) 3) Run repair between (A and B) and (C and D) But this will increase the traffic inside the DC as we wont be doing proxy. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Updated] (CASSANDRA-6440) Repair should allow repairing particular endpoints to reduce WAN usage.
[ https://issues.apache.org/jira/browse/CASSANDRA-6440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] sankalp kohli updated CASSANDRA-6440: - Attachment: JIRA-6440-v2.diff Repair should allow repairing particular endpoints to reduce WAN usage. Key: CASSANDRA-6440 URL: https://issues.apache.org/jira/browse/CASSANDRA-6440 Project: Cassandra Issue Type: New Feature Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Attachments: 6440_repair.log, JIRA-6440-v2.diff, JIRA-6440.diff The way we send out data that does not match over WAN can be improved. Example: Say there are four nodes(A,B,C,D) which are replica of a range we are repairing. A, B is in DC1 and C,D is in DC2. If A does not have the data which other replicas have, then we will have following streams 1) A to B and back 2) A to C and back(Goes over WAN) 3) A to D and back(Goes over WAN) One of the ways of doing it to reduce WAN traffic is this. 1) Repair A and B only with each other and C and D with each other starting at same time t. 2) Once these repairs have finished, A,B and C,D are in sync with respect to time t. 3) Now run a repair between A and C, the streams which are exchanged as a result of the diff will also be streamed to B and D via A and C(C and D behaves like a proxy to the streams). For a replication of DC1:2,DC2:2, the WAN traffic will get reduced by 50% and even more for higher replication factors. Another easy way to do this is to have repair command take nodes with which you want to repair with. Then we can do something like this. 1) Run repair between (A and B) and (C and D) 2) Run repair between (A and C) 3) Run repair between (A and B) and (C and D) But this will increase the traffic inside the DC as we wont be doing proxy. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6157) Selectively Disable hinted handoff for a data center
[ https://issues.apache.org/jira/browse/CASSANDRA-6157?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866406#comment-13866406 ] sankalp kohli commented on CASSANDRA-6157: -- Here are the problems and why i chose what we have. {'dc1', 'dc2'} and ['dc1', 'dc2'] - If you specify this, snake yaml does not accept it as a string. It needs to be wrapped in like {'dc1','dc2'} dc1,dc2 or dc1- This can be done. In Java, this will come as a string dc1,dc2 or dc1(without quotes) . One of the problems with this is that if someone has a typo in true or false, it will treat is as a data center name. I have to do something like if (hinted_handoff_enabled.toLowerCase().equals(true) || hinted_handoff_enabled.toLowerCase().equals(false)) { hinted_handoff_enabled_default = Boolean.parseBoolean(hinted_handoff_enabled); } else { split it by , and get the dc names. } So if someone types it as true1, we will treat it as a data center name. If this is fine, I can do the changes. Selectively Disable hinted handoff for a data center Key: CASSANDRA-6157 URL: https://issues.apache.org/jira/browse/CASSANDRA-6157 Project: Cassandra Issue Type: Improvement Components: Core Reporter: sankalp kohli Assignee: sankalp kohli Priority: Minor Fix For: 2.0.5 Attachments: trunk-6157-v2.diff, trunk-6157-v3.diff, trunk-6157-v4.diff, trunk-6157.txt Cassandra supports disabling the hints or reducing the window for hints. It would be helpful to have a switch which stops hints to a down data center but continue hints to other DCs. This is helpful during data center fail over as hints will put more unnecessary pressure on the DC taking double traffic. Also since now Cassandra is under reduced reduncany, we don't want to disable hints within the DC. -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6407) CQL/Thrift request hangs forever when querying more than certain amount of data
[ https://issues.apache.org/jira/browse/CASSANDRA-6407?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13866409#comment-13866409 ] Pavel Yaskevich commented on CASSANDRA-6407: I was asking you to test your queries with hsha and vnodes disabled on a single node, but you mentioned right now that it works without vnodes so we are good, I will take it from here, thanks! CQL/Thrift request hangs forever when querying more than certain amount of data --- Key: CASSANDRA-6407 URL: https://issues.apache.org/jira/browse/CASSANDRA-6407 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle Linux 6.4, JDK 1.7.0_25-b15, Cassandra 2.0.2 Reporter: Nikolai Grigoriev Attachments: cassandra.jstack.gz, cassandra.yaml, cassandra6407test.cql.gz, system.log.gz I have a table like this (slightly simplified for clarity): {code} CREATE TABLE my_test_table ( uid uuid, d_id uuid, a_id uuid, c_idtext, i_idblob, datatext, PRIMARY KEY ((uid, d_id, a_id), c_id, i_id) ); {code} I have created about over a hundred (117 to be specific) of sample entities with the same row key and different clustering keys. Each has a blob of approximately 4Kb. I have tried to fetch all of them with a query like this via CQLSH: {code} select * from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} This query simply hangs in CQLSH, it does not return at all until I abort it. Then I started playing with LIMIT clause and found that this query returns instantly (with good data) when I use LIMIT 55 but hangs forever when I use LIMIT 56. Then I tried to just query all i_id values like this: {code} select i_id from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' {code} And this query returns instantly with the complete set of 117 values. So I started thinking that it must be something about the total size of the response, not the number of results or the number of columns to be fetches in slices. And I have tried another test: {code} select cdata from my_test_table where uid=44338526-7aac-4640-bcde-0f4663c07572 and a_id=--4000--0002 and d_id=--1e64--0001 and c_id='list-2' LIMIT 63 {code} This query returns instantly but if I change the limit to 64 it hangs forever. Since my blob is about 4Kb for each entity it *seems* like the query hangs when the total size of the response exceeds 252..256Kb. Looks quite suspicious especially because 256Kb is such a particular number. I am wondering if this has something to do with the result paging. I did not test if the issue is reproducible outside of CQLSH but I do recall that I observed somewhat similar behavior when fetching relatively large data sets. I can consistently reproduce this problem on my cluster. I am also attaching the jstack output that I have captured when CQLSH was hanging on one of these queries. -- This message was sent by Atlassian JIRA (v6.1.5#6160)