[jira] [Commented] (CASSANDRA-10052) Bringing one node down, makes the whole cluster go down for a second
[ https://issues.apache.org/jira/browse/CASSANDRA-10052?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726830#comment-14726830 ] Stefania commented on CASSANDRA-10052: -- Thanks for your suggestion [~thobbs]. So what I did is to suppress the status change (UP/DOWN) and topology change (MOVE/JOIN/LEAVE) notifications if the endpoint broadcast rpc address is the same as our {{broadcast_rpc_address}} and yet the endpoint is not the local node. There is no need to check {{rpc_address}} as well because the broadcast rpc address is set to the rpc address when the user does not specify it and it is the broadcast rpc address that is sent over Gossip. Relevant code in {{SS.getRpcAddress()}}. The code looks something like this: {code} if (!endpoint.equals(FBUtilities.getBroadcastAddress()) && event.nodeAddress().equals(DatabaseDescriptor.getBroadcastRpcAddress())) return; {code} where {{event.nodeAddress()}} is the rpc broadcast address of the endpoint, as returned by {{Server.getRpcAddress(endpoint)}}, which is unchanged. CI results for 2.1. will appear here: http://cassci.datastax.com/job/stef1927-10052-2.1-testall/ http://cassci.datastax.com/job/stef1927-10052-2.1-dtest/ Do you have time to review or would you like to suggest someone else? If this solution is OK I will then merge into the 2.2+ branches to run CI there. > Bringing one node down, makes the whole cluster go down for a second > > > Key: CASSANDRA-10052 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10052 > Project: Cassandra > Issue Type: Bug >Reporter: Sharvanath Pathak >Assignee: Stefania > Labels: client-impacting > Fix For: 2.1.x, 2.2.x > > > When a node goes down, the other nodes learn that through the gossip. > And I do see the log from (Gossiper.java): > {code} > private void markDead(InetAddress addr, EndpointState localState) >{ >if (logger.isTraceEnabled()) >logger.trace("marking as down {}", addr); >localState.markDead(); >liveEndpoints.remove(addr); >unreachableEndpoints.put(addr, System.nanoTime()); >logger.info("InetAddress {} is now DOWN", addr); >for (IEndpointStateChangeSubscriber subscriber : subscribers) >subscriber.onDead(addr, localState); >if (logger.isTraceEnabled()) >logger.trace("Notified " + subscribers); >} > {code} > Saying: "InetAddress 192.168.101.1 is now Down", in the Cassandra's system > log. > Now on all the other nodes the client side (java driver) says, " Cannot > connect to any host, scheduling retry in 1000 milliseconds". They eventually > do reconnect but some queries fail during this intermediate period. > To me it seems like when the server pushes the nodeDown event, it call the > getRpcAddress(endpoint), and thus sends localhost as the argument in the > nodeDown event. > As in org.apache.cassandra.transport.Server.java > {code} > public void onDown(InetAddress endpoint) >{ > > server.connectionTracker.send(Event.StatusChange.nodeDown(getRpcAddress(endpoint), > server.socket.getPort())); >} > {code} > the getRpcAddress returns localhost for any endpoint if the cassandra.yaml is > using localhost as the configuration for rpc_address (which by the way is the > default). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[3/4] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5f02f202 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5f02f202 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5f02f202 Branch: refs/heads/trunk Commit: 5f02f202f85a6e65ded436c9471c4001d71927af Parents: ae669b0 45dd5d1 Author: Jason Brown Authored: Tue Sep 1 22:23:34 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:23:34 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) --
[1/4] cassandra git commit: ninja-fix to make an EnumSet a static and final field, not an instance field.
Repository: cassandra Updated Branches: refs/heads/trunk 97b57cbdf -> 2dbf029f0 ninja-fix to make an EnumSet a static and final field, not an instance field. Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c457dce6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c457dce6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c457dce6 Branch: refs/heads/trunk Commit: c457dce647c1b1b16ab95a920ab12aa36a55cb7e Parents: 2267d79 Author: Jason Brown Authored: Tue Sep 1 22:19:25 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:19:25 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c457dce6/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 06caf94..a16f53f 100644 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@ -78,7 +78,7 @@ public class MessageDeliveryTask implements Runnable Gossiper.instance.setLastProcessedMessageAt(constructionTime); } -EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, +private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, MessagingService.Verb.GOSSIP_DIGEST_ACK2, MessagingService.Verb.GOSSIP_DIGEST_SYN); }
[2/4] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Conflicts: src/java/org/apache/cassandra/net/MessageDeliveryTask.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/45dd5d17 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/45dd5d17 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/45dd5d17 Branch: refs/heads/trunk Commit: 45dd5d178f0c817c2ef1908c8b8c1563a2bf3cf1 Parents: 02e5478 c457dce Author: Jason Brown Authored: Tue Sep 1 22:23:04 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:23:04 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/45dd5d17/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --cc src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 7f30797,a16f53f..0ff0319 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@@ -85,17 -78,7 +85,17 @@@ public class MessageDeliveryTask implem Gossiper.instance.setLastProcessedMessageAt(constructionTime); } +private void handleFailure(Throwable t) +{ +if (message.doCallbackOnFailure()) +{ +MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE) + .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE); +MessagingService.instance().sendReply(response, id, message.from); +} +} + - EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, - MessagingService.Verb.GOSSIP_DIGEST_ACK2, - MessagingService.Verb.GOSSIP_DIGEST_SYN); + private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, - MessagingService.Verb.GOSSIP_DIGEST_ACK2, - MessagingService.Verb.GOSSIP_DIGEST_SYN); ++ MessagingService.Verb.GOSSIP_DIGEST_ACK2, ++ MessagingService.Verb.GOSSIP_DIGEST_SYN); }
[4/4] cassandra git commit: Merge branch 'cassandra-3.0' into trunk
Merge branch 'cassandra-3.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2dbf029f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2dbf029f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2dbf029f Branch: refs/heads/trunk Commit: 2dbf029f098a403acf9c7c65a645814fac700208 Parents: 97b57cb 5f02f20 Author: Jason Brown Authored: Tue Sep 1 22:24:09 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:24:09 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) --
[1/2] cassandra git commit: ninja-fix to make an EnumSet a static and final field, not an instance field.
Repository: cassandra Updated Branches: refs/heads/cassandra-2.2 02e5478f0 -> 45dd5d178 ninja-fix to make an EnumSet a static and final field, not an instance field. Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c457dce6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c457dce6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c457dce6 Branch: refs/heads/cassandra-2.2 Commit: c457dce647c1b1b16ab95a920ab12aa36a55cb7e Parents: 2267d79 Author: Jason Brown Authored: Tue Sep 1 22:19:25 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:19:25 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c457dce6/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 06caf94..a16f53f 100644 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@ -78,7 +78,7 @@ public class MessageDeliveryTask implements Runnable Gossiper.instance.setLastProcessedMessageAt(constructionTime); } -EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, +private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, MessagingService.Verb.GOSSIP_DIGEST_ACK2, MessagingService.Verb.GOSSIP_DIGEST_SYN); }
[2/3] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Conflicts: src/java/org/apache/cassandra/net/MessageDeliveryTask.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/45dd5d17 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/45dd5d17 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/45dd5d17 Branch: refs/heads/cassandra-3.0 Commit: 45dd5d178f0c817c2ef1908c8b8c1563a2bf3cf1 Parents: 02e5478 c457dce Author: Jason Brown Authored: Tue Sep 1 22:23:04 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:23:04 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/45dd5d17/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --cc src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 7f30797,a16f53f..0ff0319 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@@ -85,17 -78,7 +85,17 @@@ public class MessageDeliveryTask implem Gossiper.instance.setLastProcessedMessageAt(constructionTime); } +private void handleFailure(Throwable t) +{ +if (message.doCallbackOnFailure()) +{ +MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE) + .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE); +MessagingService.instance().sendReply(response, id, message.from); +} +} + - EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, - MessagingService.Verb.GOSSIP_DIGEST_ACK2, - MessagingService.Verb.GOSSIP_DIGEST_SYN); + private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, - MessagingService.Verb.GOSSIP_DIGEST_ACK2, - MessagingService.Verb.GOSSIP_DIGEST_SYN); ++ MessagingService.Verb.GOSSIP_DIGEST_ACK2, ++ MessagingService.Verb.GOSSIP_DIGEST_SYN); }
[3/3] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/5f02f202 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5f02f202 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5f02f202 Branch: refs/heads/cassandra-3.0 Commit: 5f02f202f85a6e65ded436c9471c4001d71927af Parents: ae669b0 45dd5d1 Author: Jason Brown Authored: Tue Sep 1 22:23:34 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:23:34 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) --
[1/3] cassandra git commit: ninja-fix to make an EnumSet a static and final field, not an instance field.
Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 ae669b0b3 -> 5f02f202f ninja-fix to make an EnumSet a static and final field, not an instance field. Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c457dce6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c457dce6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c457dce6 Branch: refs/heads/cassandra-3.0 Commit: c457dce647c1b1b16ab95a920ab12aa36a55cb7e Parents: 2267d79 Author: Jason Brown Authored: Tue Sep 1 22:19:25 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:19:25 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c457dce6/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 06caf94..a16f53f 100644 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@ -78,7 +78,7 @@ public class MessageDeliveryTask implements Runnable Gossiper.instance.setLastProcessedMessageAt(constructionTime); } -EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, +private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, MessagingService.Verb.GOSSIP_DIGEST_ACK2, MessagingService.Verb.GOSSIP_DIGEST_SYN); }
[2/2] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Conflicts: src/java/org/apache/cassandra/net/MessageDeliveryTask.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/45dd5d17 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/45dd5d17 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/45dd5d17 Branch: refs/heads/cassandra-2.2 Commit: 45dd5d178f0c817c2ef1908c8b8c1563a2bf3cf1 Parents: 02e5478 c457dce Author: Jason Brown Authored: Tue Sep 1 22:23:04 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:23:04 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/45dd5d17/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --cc src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 7f30797,a16f53f..0ff0319 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@@ -85,17 -78,7 +85,17 @@@ public class MessageDeliveryTask implem Gossiper.instance.setLastProcessedMessageAt(constructionTime); } +private void handleFailure(Throwable t) +{ +if (message.doCallbackOnFailure()) +{ +MessageOut response = new MessageOut(MessagingService.Verb.INTERNAL_RESPONSE) + .withParameter(MessagingService.FAILURE_RESPONSE_PARAM, MessagingService.ONE_BYTE); +MessagingService.instance().sendReply(response, id, message.from); +} +} + - EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, - MessagingService.Verb.GOSSIP_DIGEST_ACK2, - MessagingService.Verb.GOSSIP_DIGEST_SYN); + private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, - MessagingService.Verb.GOSSIP_DIGEST_ACK2, - MessagingService.Verb.GOSSIP_DIGEST_SYN); ++ MessagingService.Verb.GOSSIP_DIGEST_ACK2, ++ MessagingService.Verb.GOSSIP_DIGEST_SYN); }
cassandra git commit: ninja-fix to make an EnumSet a static and final field, not an instance field.
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 2267d7918 -> c457dce64 ninja-fix to make an EnumSet a static and final field, not an instance field. Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c457dce6 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c457dce6 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c457dce6 Branch: refs/heads/cassandra-2.1 Commit: c457dce647c1b1b16ab95a920ab12aa36a55cb7e Parents: 2267d79 Author: Jason Brown Authored: Tue Sep 1 22:19:25 2015 -0700 Committer: Jason Brown Committed: Tue Sep 1 22:19:25 2015 -0700 -- src/java/org/apache/cassandra/net/MessageDeliveryTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c457dce6/src/java/org/apache/cassandra/net/MessageDeliveryTask.java -- diff --git a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java index 06caf94..a16f53f 100644 --- a/src/java/org/apache/cassandra/net/MessageDeliveryTask.java +++ b/src/java/org/apache/cassandra/net/MessageDeliveryTask.java @@ -78,7 +78,7 @@ public class MessageDeliveryTask implements Runnable Gossiper.instance.setLastProcessedMessageAt(constructionTime); } -EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, +private static final EnumSet GOSSIP_VERBS = EnumSet.of(MessagingService.Verb.GOSSIP_DIGEST_ACK, MessagingService.Verb.GOSSIP_DIGEST_ACK2, MessagingService.Verb.GOSSIP_DIGEST_SYN); }
[jira] [Commented] (CASSANDRA-10244) Replace heartbeats with locally recorded metrics for failure detection
[ https://issues.apache.org/jira/browse/CASSANDRA-10244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726735#comment-14726735 ] Jason Brown commented on CASSANDRA-10244: - bq. If two machines A and B don't communicate a lot, are they going to detect that other one is down on the next read/write? A node will detect the peer is *unreachable* (the peer may be down, there may be a partition, and so on) when a response is not received within the timeout window on the first set of requests to the peer. At that point, A can choose to mark B down after some amount of failures have occurred; of course, we'll need some alg to determine when to try the peer again. So, yes, A will need to send requests to B to determine if it is "available", and not rely solely upon transitive availability via heartbeat propagation. bq. If any machine detects that a machine is down, will be propagate that information? No, because if node A cannot contact node B, how should that affect node C (who is current having successful communication with B)? Furthermore, how would A inform the rest of the cluster about it's unavailability to reach B? Gossip? By the time the message from A percolated throughout the cluster, the partition between A and B could be healed, thus making the "I think node B is DOWN" no longer correct/relevant, and potentially misleading other nodes in the cluster. We should avoid being too clever and simply allow each node to make it's own local decision about the availability of each of the other nodes in the cluster. bq. We should not confuse the FD if someone is writing a large batch which is slow I'm not quite understanding how that pertains here, but let me try to respond. If anything, the inputs to the FD would be something like the response times/rates from peers to which we have sent requests. If we fail to get responses, that is a red flag (the peer is being unavailable); but slower responses would just cause that peer to be pushed down in the list of "fastest peers" via the dsnitch. (If I haven't answered your question, can you please restate it?) > Replace heartbeats with locally recorded metrics for failure detection > -- > > Key: CASSANDRA-10244 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10244 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jason Brown >Assignee: Jason Brown > > In the current implementation, the primary purpose of sending gossip messages > is for delivering the updated heartbeat values of each node in a cluster. The > other data that is passed in gossip (node metadata such as status, dc, rack, > tokens, and so on) changes very infrequently (or rarely), such that the > eventual delivery of that data is entirely reasonable. Heartbeats, however, > are quite different. A continuous and nearly consistent delivery time of > updated heartbeats is critical for the stability of a cluster. It is through > the receipt of the updated heartbeat that a node determines the reachability > (UP/DOWN status) of all peers in the cluster. The current implementation of > FailureDetector measures the time differences between the heartbeat updates > received about a peer (Note: I said about a peer, not from the peer directly, > as those values are disseminated via gossip). Without a consistent time > delivery of those updates, the FD, via it's use of the PHI-accrual > algorigthm, will mark the peer as DOWN (unreachable). The two nodes could be > sending all other traffic without problem, but if the heartbeats are not > propagated correctly, each of the nodes will mark the other as DOWN, which is > clearly suboptimal to cluster health. Further, heartbeat updates are the only > mechanism we use to determine reachability (UP/DOWN) of a peer; dynamic > snitch measurements, for example, are not included in the determination. > To illustrate this, in the current implementation, assume a cluster of nodes: > A, B, and C. A partition starts between nodes A and C (no communication > succeeds), but both nodes can communicate with B. As B will get the updated > heartbeats from both A and C, it will, via gossip, send those over to the > other node. Thus, A thinks C is UP, and C thinks A is UP. Unfortunately, due > to the partition between them, all communication between A and C will fail, > yet neither node will mark the other as down because each is receiving, > transitively via B, the updated heartbeat about the other. While it's true > that the other node is alive, only having transitive knowledge about a peer, > and allowing that to be the sole determinant of UP/DOWN reachability status, > is not sufficient for a correct and effieicently operating cluster. > This transitive availa
[jira] [Updated] (CASSANDRA-10250) Executing lots of schema alters concurrently can lead to dropped alters
[ https://issues.apache.org/jira/browse/CASSANDRA-10250?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Hust updated CASSANDRA-10250: Description: A recently added [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] has been flapping on cassci and has exposed an issue with running lots of schema alterations concurrently. The failures occur on healthy clusters but seem to occur at higher rates when 1 node is down during the alters. The test executes the following – 440 total commands: - Create 20 new tables - Drop 7 columns one at time across 20 tables - Add 7 columns one at time across 20 tables - Add one column index on each of the 7 columns on 20 tables Outcome is random. Majority of the failures are dropped columns still being present, but new columns and indexes have been observed to be incorrect. The logs are don’t have exceptions and the columns/indexes that are incorrect don’t seem to follow a pattern. Running a {{nodetool describecluster}} on each node shows the same schema id on all nodes. Attached is a python script extracted from the dtest. Running against a local 3 node cluster will reproduce the issue (with enough runs – fails ~20% on my machine). Also attached is the node logs from a run with when a dropped column (alter_me_7 table, column s1) is still present. Checking the system_schema tables for this case shows the s1 column in both the columns and drop_columns tables. This has been flapping on cassci on versions 2+ and doesn’t seem to be related to changes in 3.0. More testing needs to be done though. //cc [~enigmacurry] was: A recently added [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] has been flapping on cassci and has exposed an issue with running lots of schema alterations concurrently. The failures occur on healthy clusters but seem to occur at higher rates when 1 node is down during the alters. The test executes the following – 440 total commands: - Create 20 new tables - Drop 7 columns one at time across 20 tables - Add 7 columns on at time across 20 tables - Add one column index on each of the 7 columns on 20 tables Outcome is random. Majority of the failures are dropped columns still being present, but new columns and indexes have been observed to be incorrect. The logs are don’t have exceptions and the columns/indexes that are incorrect don’t seem to follow a pattern. Running a {{nodetool describecluster}} on each node shows the same schema id on all nodes. Attached is a python script extracted from the dtest. Running against a local 3 node cluster will reproduce the issue (with enough runs – fails ~20% on my machine). Also attached is the node logs from a run with when a dropped column (alter_me_7 table, column s1) is still present. Checking the system_schema tables for this case shows the s1 column in both the columns and drop_columns tables. This has been flapping on cassci on versions 2+ and doesn’t seem to be related to changes in 3.0. More testing needs to be done though. //cc [~enigmacurry] > Executing lots of schema alters concurrently can lead to dropped alters > --- > > Key: CASSANDRA-10250 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10250 > Project: Cassandra > Issue Type: Bug >Reporter: Andrew Hust > Attachments: concurrent_schema_changes.py, node1.log, node2.log, > node3.log > > > A recently added > [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] > has been flapping on cassci and has exposed an issue with running lots of > schema alterations concurrently. The failures occur on healthy clusters but > seem to occur at higher rates when 1 node is down during the alters. > The test executes the following – 440 total commands: > - Create 20 new tables > - Drop 7 columns one at time across 20 tables > - Add 7 columns one at time across 20 tables > - Add one column index on each of the 7 columns on 20 tables > Outcome is random. Majority of the failures are dropped columns still being > present, but new columns and indexes have been observed to be incorrect. The > logs are don’t have exceptions and the columns/indexes that are incorrect > don’t seem to follow a pattern. Running a {{nodetool describecluster}} on > each node shows the same schema id on all nodes. > Attached is a python script ex
[jira] [Updated] (CASSANDRA-10250) Executing lots of schema alters concurrently can lead to dropped alters
[ https://issues.apache.org/jira/browse/CASSANDRA-10250?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Hust updated CASSANDRA-10250: Description: A recently added [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] has been flapping on cassci and has exposed an issue with running lots of schema alterations concurrently. The failures occur on healthy clusters but seem to occur at higher rates when 1 node is down during the alters. The test executes the following – 440 total commands: - Create 20 new tables - Drop 7 columns one at time across 20 tables - Add 7 columns on at time across 20 tables - Add one column index on each of the 7 columns on 20 tables Outcome is random. Majority of the failures are dropped columns still being present, but new columns and indexes have been observed to be incorrect. The logs are don’t have exceptions and the columns/indexes that are incorrect don’t seem to follow a pattern. Running a {{nodetool describecluster}} on each node shows the same schema id on all nodes. Attached is a python script extracted from the dtest. Running against a local 3 node cluster will reproduce the issue (with enough runs – fails ~20% on my machine). Also attached is the node logs from a run with when a dropped column (alter_me_7 table, column s1) is still present. Checking the system_schema tables for this case shows the s1 column in both the columns and drop_columns tables. This has been flapping on cassci on versions 2+ and doesn’t seem to be related to changes in 3.0. More testing needs to be done though. //cc [~enigmacurry] was: A recently added [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] has been flapping on cassci and has exposed an issue with running lots of schema alterations concurrently. The failures occur on healthy clusters but seem to occur at higher rates when 1 node is down during the alters. The test executes the following – 440 total commands: - Create 20 new tables - Drop 7 columns one at time across 20 tables - Add 7 columns on at time across 20 tables - Add one column index on each of the 7 columns on 20 tables Outcome is random. Majority of the failures are dropped columns still being present, but new columns and indexes have been observed to be incorrect. The logs are don’t have exceptions and the columns/indexes that are incorrect don’t seem to follow a pattern. Running a {{nodetool describecluster}} on each node shows the same schema id on all nodes. Attached is a python script extracted from the dtest. Running against a local 3 node cluster will reproduce the issue (with enough runs – fails ~20% on my machine). Also attached is the node logs from a run with when a dropped column (alter_me_7 table, column s1) is still present. Checking the system_schema tables for this case shows the s1 column in both the columns and drop_columns tables. This has been flapping on cassci on versions 2+ and doesn’t seem to be related to changes in 3.0. More testing needs to be done though. > Executing lots of schema alters concurrently can lead to dropped alters > --- > > Key: CASSANDRA-10250 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10250 > Project: Cassandra > Issue Type: Bug >Reporter: Andrew Hust > Attachments: concurrent_schema_changes.py, node1.log, node2.log, > node3.log > > > A recently added > [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] > has been flapping on cassci and has exposed an issue with running lots of > schema alterations concurrently. The failures occur on healthy clusters but > seem to occur at higher rates when 1 node is down during the alters. > The test executes the following – 440 total commands: > - Create 20 new tables > - Drop 7 columns one at time across 20 tables > - Add 7 columns on at time across 20 tables > - Add one column index on each of the 7 columns on 20 tables > Outcome is random. Majority of the failures are dropped columns still being > present, but new columns and indexes have been observed to be incorrect. The > logs are don’t have exceptions and the columns/indexes that are incorrect > don’t seem to follow a pattern. Running a {{nodetool describecluster}} on > each node shows the same schema id on all nodes. > Attached is a python script extracted from the dtest
[jira] [Commented] (CASSANDRA-10250) Executing lots of schema alters concurrently can lead to dropped alters
[ https://issues.apache.org/jira/browse/CASSANDRA-10250?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726721#comment-14726721 ] Andrew Hust commented on CASSANDRA-10250: - example script run -- missing c4 column {code} ❯ python concurrent_schema_changes.py creating base tables to be added/altered executing creation of tables, add/drop column and index creation sleeping 20 to make sure things are settled verifing schema status Errors found: alter_me_8 expected c1 -> c7, id, got: [u'c1', u'c2', u'c3', u'c5', u'c6', u'c7', u'id'] {code} > Executing lots of schema alters concurrently can lead to dropped alters > --- > > Key: CASSANDRA-10250 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10250 > Project: Cassandra > Issue Type: Bug >Reporter: Andrew Hust > Attachments: concurrent_schema_changes.py, node1.log, node2.log, > node3.log > > > A recently added > [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] > has been flapping on cassci and has exposed an issue with running lots of > schema alterations concurrently. The failures occur on healthy clusters but > seem to occur at higher rates when 1 node is down during the alters. > The test executes the following – 440 total commands: > - Create 20 new tables > - Drop 7 columns one at time across 20 tables > - Add 7 columns on at time across 20 tables > - Add one column index on each of the 7 columns on 20 tables > Outcome is random. Majority of the failures are dropped columns still being > present, but new columns and indexes have been observed to be incorrect. The > logs are don’t have exceptions and the columns/indexes that are incorrect > don’t seem to follow a pattern. Running a {{nodetool describecluster}} on > each node shows the same schema id on all nodes. > Attached is a python script extracted from the dtest. Running against a > local 3 node cluster will reproduce the issue (with enough runs – fails ~20% > on my machine). > Also attached is the node logs from a run with when a dropped column > (alter_me_7 table, column s1) is still present. Checking the system_schema > tables for this case shows the s1 column in both the columns and drop_columns > tables. > This has been flapping on cassci on versions 2+ and doesn’t seem to be > related to changes in 3.0. More testing needs to be done though. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10250) Executing lots of schema alters concurrently can lead to dropped alters
[ https://issues.apache.org/jira/browse/CASSANDRA-10250?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrew Hust updated CASSANDRA-10250: Attachment: node3.log node2.log node1.log concurrent_schema_changes.py > Executing lots of schema alters concurrently can lead to dropped alters > --- > > Key: CASSANDRA-10250 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10250 > Project: Cassandra > Issue Type: Bug >Reporter: Andrew Hust > Attachments: concurrent_schema_changes.py, node1.log, node2.log, > node3.log > > > A recently added > [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] > has been flapping on cassci and has exposed an issue with running lots of > schema alterations concurrently. The failures occur on healthy clusters but > seem to occur at higher rates when 1 node is down during the alters. > The test executes the following – 440 total commands: > - Create 20 new tables > - Drop 7 columns one at time across 20 tables > - Add 7 columns on at time across 20 tables > - Add one column index on each of the 7 columns on 20 tables > Outcome is random. Majority of the failures are dropped columns still being > present, but new columns and indexes have been observed to be incorrect. The > logs are don’t have exceptions and the columns/indexes that are incorrect > don’t seem to follow a pattern. Running a {{nodetool describecluster}} on > each node shows the same schema id on all nodes. > Attached is a python script extracted from the dtest. Running against a > local 3 node cluster will reproduce the issue (with enough runs – fails ~20% > on my machine). > Also attached is the node logs from a run with when a dropped column > (alter_me_7 table, column s1) is still present. Checking the system_schema > tables for this case shows the s1 column in both the columns and drop_columns > tables. > This has been flapping on cassci on versions 2+ and doesn’t seem to be > related to changes in 3.0. More testing needs to be done though. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10250) Executing lots of schema alters concurrently can lead to dropped alters
Andrew Hust created CASSANDRA-10250: --- Summary: Executing lots of schema alters concurrently can lead to dropped alters Key: CASSANDRA-10250 URL: https://issues.apache.org/jira/browse/CASSANDRA-10250 Project: Cassandra Issue Type: Bug Reporter: Andrew Hust A recently added [dtest|http://cassci.datastax.com/view/cassandra-3.0/job/cassandra-3.0_dtest/132/testReport/junit/concurrent_schema_changes_test/TestConcurrentSchemaChanges/create_lots_of_schema_churn_test/] has been flapping on cassci and has exposed an issue with running lots of schema alterations concurrently. The failures occur on healthy clusters but seem to occur at higher rates when 1 node is down during the alters. The test executes the following – 440 total commands: - Create 20 new tables - Drop 7 columns one at time across 20 tables - Add 7 columns on at time across 20 tables - Add one column index on each of the 7 columns on 20 tables Outcome is random. Majority of the failures are dropped columns still being present, but new columns and indexes have been observed to be incorrect. The logs are don’t have exceptions and the columns/indexes that are incorrect don’t seem to follow a pattern. Running a {{nodetool describecluster}} on each node shows the same schema id on all nodes. Attached is a python script extracted from the dtest. Running against a local 3 node cluster will reproduce the issue (with enough runs – fails ~20% on my machine). Also attached is the node logs from a run with when a dropped column (alter_me_7 table, column s1) is still present. Checking the system_schema tables for this case shows the s1 column in both the columns and drop_columns tables. This has been flapping on cassci on versions 2+ and doesn’t seem to be related to changes in 3.0. More testing needs to be done though. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8630) Faster sequential IO (on compaction, streaming, etc)
[ https://issues.apache.org/jira/browse/CASSANDRA-8630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726651#comment-14726651 ] Stefania commented on CASSANDRA-8630: - The latest cperf test has completed. These are the read results of the last two runs: http://cstar.datastax.com/graph?stats=25b50e7c-5090-11e5-a17a-42010af0688f&metric=op_rate&operation=2_read&smoothing=1&show_aggregates=true&xmin=0&xmax=145.42&ymin=0&ymax=157588.2 http://cstar.datastax.com/graph?stats=ebe0-510f-11e5-a17a-42010af0688f&metric=op_rate&operation=2_read&smoothing=1&show_aggregates=true&xmin=0&xmax=140.91&ymin=0&ymax=155889.8 In the second run, performance is definitely the same as 3.0. Node that 8630 before suggestions is not rebased. > Faster sequential IO (on compaction, streaming, etc) > > > Key: CASSANDRA-8630 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8630 > Project: Cassandra > Issue Type: Improvement > Components: Core, Tools >Reporter: Oleg Anastasyev >Assignee: Stefania > Labels: compaction, performance > Fix For: 3.x > > Attachments: 8630-FasterSequencialReadsAndWrites.txt, cpu_load.png, > flight_recorder_001_files.tar.gz, flight_recorder_002_files.tar.gz, > mmaped_uncomp_hotspot.png > > > When node is doing a lot of sequencial IO (streaming, compacting, etc) a lot > of CPU is lost in calls to RAF's int read() and DataOutputStream's write(int). > This is because default implementations of readShort,readLong, etc as well as > their matching write* are implemented with numerous calls of byte by byte > read and write. > This makes a lot of syscalls as well. > A quick microbench shows than just reimplementation of these methods in > either way gives 8x speed increase. > A patch attached implements RandomAccessReader.read and > SequencialWriter.write methods in more efficient way. > I also eliminated some extra byte copies in CompositeType.split and > ColumnNameHelper.maxComponents, which were on my profiler's hotspot method > list during tests. > A stress tests on my laptop show that this patch makes compaction 25-30% > faster on uncompressed sstables and 15% faster for compressed ones. > A deployment to production shows much less CPU load for compaction. > (I attached a cpu load graph from one of our production, orange is niced CPU > load - i.e. compaction; yellow is user - i.e. not compaction related tasks) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10244) Replace heartbeats with locally recorded metrics for failure detection
[ https://issues.apache.org/jira/browse/CASSANDRA-10244?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726641#comment-14726641 ] sankalp kohli commented on CASSANDRA-10244: --- I like the idea. I have following questions on this 1) If two machines A and B don't communicate a lot, are they going to detect that other one is down on the next read/write? If any machine detects that a machine is down, will be propagate that information? 2) We should not confuse the FD if someone is writing a large batch which is slow. > Replace heartbeats with locally recorded metrics for failure detection > -- > > Key: CASSANDRA-10244 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10244 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Jason Brown >Assignee: Jason Brown > > In the current implementation, the primary purpose of sending gossip messages > is for delivering the updated heartbeat values of each node in a cluster. The > other data that is passed in gossip (node metadata such as status, dc, rack, > tokens, and so on) changes very infrequently (or rarely), such that the > eventual delivery of that data is entirely reasonable. Heartbeats, however, > are quite different. A continuous and nearly consistent delivery time of > updated heartbeats is critical for the stability of a cluster. It is through > the receipt of the updated heartbeat that a node determines the reachability > (UP/DOWN status) of all peers in the cluster. The current implementation of > FailureDetector measures the time differences between the heartbeat updates > received about a peer (Note: I said about a peer, not from the peer directly, > as those values are disseminated via gossip). Without a consistent time > delivery of those updates, the FD, via it's use of the PHI-accrual > algorigthm, will mark the peer as DOWN (unreachable). The two nodes could be > sending all other traffic without problem, but if the heartbeats are not > propagated correctly, each of the nodes will mark the other as DOWN, which is > clearly suboptimal to cluster health. Further, heartbeat updates are the only > mechanism we use to determine reachability (UP/DOWN) of a peer; dynamic > snitch measurements, for example, are not included in the determination. > To illustrate this, in the current implementation, assume a cluster of nodes: > A, B, and C. A partition starts between nodes A and C (no communication > succeeds), but both nodes can communicate with B. As B will get the updated > heartbeats from both A and C, it will, via gossip, send those over to the > other node. Thus, A thinks C is UP, and C thinks A is UP. Unfortunately, due > to the partition between them, all communication between A and C will fail, > yet neither node will mark the other as down because each is receiving, > transitively via B, the updated heartbeat about the other. While it's true > that the other node is alive, only having transitive knowledge about a peer, > and allowing that to be the sole determinant of UP/DOWN reachability status, > is not sufficient for a correct and effieicently operating cluster. > This transitive availability is suboptimal, and I propose we drop the > heartbeat concept altogether. Instead, the dynamic snitch should become more > intelligent, and it's measurements ultimately become the input for > determining the reachability status of each peer(as fed into a revamped FD). > As we already capture latencies in the dsntich, we can reasonably extend it > to include timeouts/missed responses, and make that the basis for the UP/DOWN > decisioning. Thus we will have more accurate and relevant peer statueses that > is tailored to the local node. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-10222) Periodically attempt to delete failed snapshot deletions on Windows
[ https://issues.apache.org/jira/browse/CASSANDRA-10222?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726616#comment-14726616 ] Stefania edited comment on CASSANDRA-10222 at 9/2/15 2:24 AM: -- Looks much better now. Just one comment: * When we catch the exception in {{Directories.clearSnapshot()}} did you mean to retry immediately by calling {{run()}}? I think not because the constructor will also submit the task. It was a nice idea to add the call to {{CompactionExecutor.afterExecute}}, all compaction operations go through this executor right? I noticed you haven't run CI yet. I suppose it makes more sense to run it on Windows than on Linux? was (Author: stefania): Looks much better now. Just one comment: * When we catch the exception in {{Directories.clearSnapshot()}} did you mean to retry immediately by calling {{run()}}? I think not because the constructor will also submit the task. It was a nice idea to add the call to {{CompactionExecutor.afterExecute}}, all compaction operations go though this executor right? I noticed you haven't run CI yet. I suppose it makes more sense to run it on Windows than on Linux? > Periodically attempt to delete failed snapshot deletions on Windows > --- > > Key: CASSANDRA-10222 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10222 > Project: Cassandra > Issue Type: Improvement >Reporter: Joshua McKenzie >Assignee: Joshua McKenzie > Labels: Windows > Fix For: 2.2.2 > > > The changes in CASSANDRA-9658 leave us in a position where a node on Windows > will have to be restarted to clear out snapshots that cannot be deleted at > request time due to sstables still being mapped, thus preventing deletions of > hard links. A simple periodic task to categorize failed snapshot deletions > and retry them would help prevent node disk utilization from growing > unbounded by snapshots as compaction will eventually make these snapshot > files deletable. > Given that hard links to files in NTFS don't take up any extra space on disk > so long as the original file still exists, the only limitation for users from > this approach will be the inability to 'move' a snapshot file to another > drive share. They will be copyable, however, so it's a minor platform > difference. > This goes directly against the goals of CASSANDRA-8271 and will likely be > built on top of that code. Until such time as we get buffered performance > in-line with memory-mapped, this is an interim necessity for production > roll-outs. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10222) Periodically attempt to delete failed snapshot deletions on Windows
[ https://issues.apache.org/jira/browse/CASSANDRA-10222?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726616#comment-14726616 ] Stefania commented on CASSANDRA-10222: -- Looks much better now. Just one comment: * When we catch the exception in {{Directories.clearSnapshot()}} did you mean to retry immediately by calling {{run()}}? I think not because the constructor will also submit the task. It was a nice idea to add the call to {{CompactionExecutor.afterExecute}}, all compaction operations go though this executor right? I noticed you haven't run CI yet. I suppose it makes more sense to run it on Windows than on Linux? > Periodically attempt to delete failed snapshot deletions on Windows > --- > > Key: CASSANDRA-10222 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10222 > Project: Cassandra > Issue Type: Improvement >Reporter: Joshua McKenzie >Assignee: Joshua McKenzie > Labels: Windows > Fix For: 2.2.2 > > > The changes in CASSANDRA-9658 leave us in a position where a node on Windows > will have to be restarted to clear out snapshots that cannot be deleted at > request time due to sstables still being mapped, thus preventing deletions of > hard links. A simple periodic task to categorize failed snapshot deletions > and retry them would help prevent node disk utilization from growing > unbounded by snapshots as compaction will eventually make these snapshot > files deletable. > Given that hard links to files in NTFS don't take up any extra space on disk > so long as the original file still exists, the only limitation for users from > this approach will be the inability to 'move' a snapshot file to another > drive share. They will be copyable, however, so it's a minor platform > difference. > This goes directly against the goals of CASSANDRA-8271 and will likely be > built on top of that code. Until such time as we get buffered performance > in-line with memory-mapped, this is an interim necessity for production > roll-outs. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10249) Reduce over-read for standard disk io by 16x
[ https://issues.apache.org/jira/browse/CASSANDRA-10249?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Albert P Tobey updated CASSANDRA-10249: --- Attachment: stock-2.1.9-dstat-lvn10.png > Reduce over-read for standard disk io by 16x > > > Key: CASSANDRA-10249 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10249 > Project: Cassandra > Issue Type: Improvement >Reporter: Albert P Tobey > Fix For: 2.1.x > > Attachments: stock-2.1.9-dstat-lvn10.png, yourkit-screenshot.png > > > On read workloads, Cassandra 2.1 reads drastically more data than it emits > over the network. This causes problems throughput the system by wasting disk > IO and causing unnecessary GC. > I have reproduce the issue on clusters and locally with a single instance. > The only requirement to reproduce the issue is enough data to blow through > the page cache. The default schema and data size with cassandra-stress is > sufficient for exposing the issue. > With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 > disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was > doing 300-500MB/s of disk reads, saturating the drive. > After applying this patch for standard IO mode > https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around > 100:1 on my local test rig. Latency improved considerably and GC became a lot > less frequent. > I tested with 512 byte reads as well, but got the same performance, which > makes sense since all HDD and SSD made in the last few years have a 4K block > size (many of them lie and say 512). > I'm re-running the numbers now and will post them tomorrow. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9673) Improve batchlog write path
[ https://issues.apache.org/jira/browse/CASSANDRA-9673?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726587#comment-14726587 ] Stefania edited comment on CASSANDRA-9673 at 9/2/15 1:58 AM: - bq. I meant checking for table emptiness at the beginning of startup migration, and logging once if not empty. Right, makes more sense now. bq. Also, the logic in conversion for != 1 uuids seems a bit weird. It will never be the case that we'll get a mutation The replay unit tests were still checking for 1.2 mutations. bq. There was a bug in SP::syncWriteToBatchlog that is as old as is batchlog itself. We are using CL.ONE unconditionally, even when we have two endpoints. And both legacy/modern writes should be sharing the same callback, so I switched back to using WriteResponseHandler for both. Thanks for fixing this. bq. I think I broke some replay tests. I removed the 1.2 mutations from the replay tests. They are fine now. Rebased again and force pushed, if the latest CI is good then I'm +1 as well: http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9673-3.0-dtest/ http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9673-3.0-testall/ Thanks! was (Author: stefania): bq. I meant checking for table emptiness at the beginning of startup migration, and logging once if not empty. Right, makes more sense now. bq. Also, the logic in conversion for != 1 uuids seems a bit weird. It will never be the case that we'll get a mutation The replay unit tests were still checking for 1.2 mutations. bq. There was a bug in SP::syncWriteToBatchlog that is as old as is batchlog itself. We are using CL.ONE unconditionally, even when we have two endpoints. And both legacy/modern writes should be sharing the same callback, so I switched back to using WriteResponseHandler for both. Thanks for fixing this. bq. I think I broke some replay tests. I removed the 1.2 mutations from the replay tests. They are fine now. Rebased again and force pushed, if the latest CI is good then I'm +1 as well: http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9673-3.0-dtest/ http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9673-3.0-testall/ Thanks! > Improve batchlog write path > --- > > Key: CASSANDRA-9673 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9673 > Project: Cassandra > Issue Type: Improvement >Reporter: Aleksey Yeschenko >Assignee: Stefania > Labels: performance > Fix For: 3.0 beta 2 > > Attachments: 9673_001.tar.gz, 9673_004.tar.gz, > gc_times_first_node_patched_004.png, gc_times_first_node_trunk_004.png > > > Currently we allocate an on-heap {{ByteBuffer}} to serialize the batched > mutations into, before sending it to a distant node, generating unnecessary > garbage (potentially a lot of it). > With materialized views using the batchlog, it would be nice to optimise the > write path: > - introduce a new verb ({{Batch}}) > - introduce a new message ({{BatchMessage}}) that would encapsulate the > mutations, expiration, and creation time (similar to {{HintMessage}} in > CASSANDRA-6230) > - have MS serialize it directly instead of relying on an intermediate buffer > To avoid merely shifting the temp buffer to the receiving side(s) we should > change the structure of the batchlog table to use a list or a map of > individual mutations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9673) Improve batchlog write path
[ https://issues.apache.org/jira/browse/CASSANDRA-9673?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726587#comment-14726587 ] Stefania commented on CASSANDRA-9673: - bq. I meant checking for table emptiness at the beginning of startup migration, and logging once if not empty. Right, makes more sense now. bq. Also, the logic in conversion for != 1 uuids seems a bit weird. It will never be the case that we'll get a mutation The replay unit tests were still checking for 1.2 mutations. bq. There was a bug in SP::syncWriteToBatchlog that is as old as is batchlog itself. We are using CL.ONE unconditionally, even when we have two endpoints. And both legacy/modern writes should be sharing the same callback, so I switched back to using WriteResponseHandler for both. Thanks for fixing this. bq. I think I broke some replay tests. I removed the 1.2 mutations from the replay tests. They are fine now. Rebased again and force pushed, if the latest CI is good then I'm +1 as well: http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9673-3.0-dtest/ http://cassci.datastax.com/view/Dev/view/stef1927/job/stef1927-9673-3.0-testall/ Thanks! > Improve batchlog write path > --- > > Key: CASSANDRA-9673 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9673 > Project: Cassandra > Issue Type: Improvement >Reporter: Aleksey Yeschenko >Assignee: Stefania > Labels: performance > Fix For: 3.0 beta 2 > > Attachments: 9673_001.tar.gz, 9673_004.tar.gz, > gc_times_first_node_patched_004.png, gc_times_first_node_trunk_004.png > > > Currently we allocate an on-heap {{ByteBuffer}} to serialize the batched > mutations into, before sending it to a distant node, generating unnecessary > garbage (potentially a lot of it). > With materialized views using the batchlog, it would be nice to optimise the > write path: > - introduce a new verb ({{Batch}}) > - introduce a new message ({{BatchMessage}}) that would encapsulate the > mutations, expiration, and creation time (similar to {{HintMessage}} in > CASSANDRA-6230) > - have MS serialize it directly instead of relying on an intermediate buffer > To avoid merely shifting the temp buffer to the receiving side(s) we should > change the structure of the batchlog table to use a list or a map of > individual mutations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10249) Reduce over-read for standard disk io by 16x
[ https://issues.apache.org/jira/browse/CASSANDRA-10249?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Albert P Tobey updated CASSANDRA-10249: --- Attachment: patched-2.1.9-dstat-lvn10.png > Reduce over-read for standard disk io by 16x > > > Key: CASSANDRA-10249 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10249 > Project: Cassandra > Issue Type: Improvement >Reporter: Albert P Tobey > Fix For: 2.1.x > > Attachments: patched-2.1.9-dstat-lvn10.png, > stock-2.1.9-dstat-lvn10.png, yourkit-screenshot.png > > > On read workloads, Cassandra 2.1 reads drastically more data than it emits > over the network. This causes problems throughput the system by wasting disk > IO and causing unnecessary GC. > I have reproduce the issue on clusters and locally with a single instance. > The only requirement to reproduce the issue is enough data to blow through > the page cache. The default schema and data size with cassandra-stress is > sufficient for exposing the issue. > With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 > disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was > doing 300-500MB/s of disk reads, saturating the drive. > After applying this patch for standard IO mode > https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around > 100:1 on my local test rig. Latency improved considerably and GC became a lot > less frequent. > I tested with 512 byte reads as well, but got the same performance, which > makes sense since all HDD and SSD made in the last few years have a 4K block > size (many of them lie and say 512). > I'm re-running the numbers now and will post them tomorrow. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10249) Reduce over-read for standard disk io by 16x
[ https://issues.apache.org/jira/browse/CASSANDRA-10249?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Albert P Tobey updated CASSANDRA-10249: --- Attachment: yourkit-screenshot.png > Reduce over-read for standard disk io by 16x > > > Key: CASSANDRA-10249 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10249 > Project: Cassandra > Issue Type: Improvement >Reporter: Albert P Tobey > Fix For: 2.1.x > > Attachments: yourkit-screenshot.png > > > On read workloads, Cassandra 2.1 reads drastically more data than it emits > over the network. This causes problems throughput the system by wasting disk > IO and causing unnecessary GC. > I have reproduce the issue on clusters and locally with a single instance. > The only requirement to reproduce the issue is enough data to blow through > the page cache. The default schema and data size with cassandra-stress is > sufficient for exposing the issue. > With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 > disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was > doing 300-500MB/s of disk reads, saturating the drive. > After applying this patch for standard IO mode > https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around > 100:1 on my local test rig. Latency improved considerably and GC became a lot > less frequent. > I tested with 512 byte reads as well, but got the same performance, which > makes sense since all HDD and SSD made in the last few years have a 4K block > size (many of them lie and say 512). > I'm re-running the numbers now and will post them tomorrow. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10249) Reduce over-read for standard disk io by 16x
[ https://issues.apache.org/jira/browse/CASSANDRA-10249?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Albert P Tobey updated CASSANDRA-10249: --- Description: On read workloads, Cassandra 2.1 reads drastically more data than it emits over the network. This causes problems throughput the system by wasting disk IO and causing unnecessary GC. I have reproduce the issue on clusters and locally with a single instance. The only requirement to reproduce the issue is enough data to blow through the page cache. The default schema and data size with cassandra-stress is sufficient for exposing the issue. With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was doing 300-500MB/s of disk reads, saturating the drive. After applying this patch for standard IO mode https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around 100:1 on my local test rig. Latency improved considerably and GC became a lot less frequent. I tested with 512 byte reads as well, but got the same performance, which makes sense since all HDD and SSD made in the last few years have a 4K block size (many of them lie and say 512). I'm re-running the numbers now and will post them tomorrow. was: On read workloads, Cassandra 2.1 reads drastically more data than it emits over the network. This causes problems throughput the system by wasting disk IO and causing unnecessary GC. I have reproduce the issue on clusters and locally with a single instance. The only requirement to reproduce the issue is enough data to blow through the page cache. The default schema and data size with cassandra-stress is sufficient for exposing the issue. With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was doing 300-500MB/s of disk reads, saturating the drive. After applying this patch for standard IO mode https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around 100:1 on my local test rig. I tested with 512 byte reads as well, but got the same performance, which makes sense since all HDD and SSD made in the last few years have a 4K block size (many of them lie and say 512). Ideally, the reads in > Reduce over-read for standard disk io by 16x > > > Key: CASSANDRA-10249 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10249 > Project: Cassandra > Issue Type: Improvement >Reporter: Albert P Tobey > Fix For: 2.1.x > > > On read workloads, Cassandra 2.1 reads drastically more data than it emits > over the network. This causes problems throughput the system by wasting disk > IO and causing unnecessary GC. > I have reproduce the issue on clusters and locally with a single instance. > The only requirement to reproduce the issue is enough data to blow through > the page cache. The default schema and data size with cassandra-stress is > sufficient for exposing the issue. > With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 > disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was > doing 300-500MB/s of disk reads, saturating the drive. > After applying this patch for standard IO mode > https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around > 100:1 on my local test rig. Latency improved considerably and GC became a lot > less frequent. > I tested with 512 byte reads as well, but got the same performance, which > makes sense since all HDD and SSD made in the last few years have a 4K block > size (many of them lie and say 512). > I'm re-running the numbers now and will post them tomorrow. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10249) Reduce over-read for standard disk io by 16x
[ https://issues.apache.org/jira/browse/CASSANDRA-10249?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Albert P Tobey updated CASSANDRA-10249: --- Description: On read workloads, Cassandra 2.1 reads drastically more data than it emits over the network. This causes problems throughput the system by wasting disk IO and causing unnecessary GC. I have reproduce the issue on clusters and locally with a single instance. The only requirement to reproduce the issue is enough data to blow through the page cache. The default schema and data size with cassandra-stress is sufficient for exposing the issue. With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was doing 300-500MB/s of disk reads, saturating the drive. After applying this patch for standard IO mode https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around 100:1 on my local test rig. I tested with 512 byte reads as well, but got the same performance, which makes sense since all HDD and SSD made in the last few years have a 4K block size (many of them lie and say 512). Ideally, the reads in was: On read workloads, Cassandra 2.1 reads drastically more data than it emits over the network. This causes problems throughput the system by wasting disk IO and causing unnecessary GC. I have reproduce the issue on clusters and locally with a single instance. The only requirement to reproduce the issue is enough data to blow through the page cache. The default schema and data size with cassandra-stress is sufficient for exposing the issue. With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was doing 300-500MB/s of disk reads, saturating the drive. After applying this patch https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around 100:1 on my local test rig. I tested with 512 byte reads as well, but got the same performance, which makes sense since all HDD and SSD made in the last few years have a 4K block size (many of them lie and say 512). > Reduce over-read for standard disk io by 16x > > > Key: CASSANDRA-10249 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10249 > Project: Cassandra > Issue Type: Improvement >Reporter: Albert P Tobey > Fix For: 2.1.x > > > On read workloads, Cassandra 2.1 reads drastically more data than it emits > over the network. This causes problems throughput the system by wasting disk > IO and causing unnecessary GC. > I have reproduce the issue on clusters and locally with a single instance. > The only requirement to reproduce the issue is enough data to blow through > the page cache. The default schema and data size with cassandra-stress is > sufficient for exposing the issue. > With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 > disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was > doing 300-500MB/s of disk reads, saturating the drive. > After applying this patch for standard IO mode > https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around > 100:1 on my local test rig. > I tested with 512 byte reads as well, but got the same performance, which > makes sense since all HDD and SSD made in the last few years have a 4K block > size (many of them lie and say 512). Ideally, the reads in -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10249) Reduce over-read for standard disk io by 16x
Albert P Tobey created CASSANDRA-10249: -- Summary: Reduce over-read for standard disk io by 16x Key: CASSANDRA-10249 URL: https://issues.apache.org/jira/browse/CASSANDRA-10249 Project: Cassandra Issue Type: Improvement Reporter: Albert P Tobey Fix For: 2.1.x On read workloads, Cassandra 2.1 reads drastically more data than it emits over the network. This causes problems throughput the system by wasting disk IO and causing unnecessary GC. I have reproduce the issue on clusters and locally with a single instance. The only requirement to reproduce the issue is enough data to blow through the page cache. The default schema and data size with cassandra-stress is sufficient for exposing the issue. With stock 2.1.9 I regularly observed anywhere from 300:1 to 500 disk:network ratio. That is to say, for 1MB/s of network IO, Cassandra was doing 300-500MB/s of disk reads, saturating the drive. After applying this patch https://gist.github.com/tobert/10c307cf3709a585a7cf the ratio fell to around 100:1 on my local test rig. I tested with 512 byte reads as well, but got the same performance, which makes sense since all HDD and SSD made in the last few years have a 4K block size (many of them lie and say 512). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10241) Keep a separate production debug log for troubleshooting
[ https://issues.apache.org/jira/browse/CASSANDRA-10241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726556#comment-14726556 ] sankalp kohli commented on CASSANDRA-10241: --- Will Gossip not be doing lot of logging for large clusters? > Keep a separate production debug log for troubleshooting > > > Key: CASSANDRA-10241 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10241 > Project: Cassandra > Issue Type: Bug > Components: Config >Reporter: Jonathan Ellis >Assignee: Brandon Williams > Fix For: 2.1.x > > > [~aweisberg] had the suggestion to keep a separate debug log for aid in > troubleshooting, not intended for regular human consumption but where we can > log things that might help if something goes wrong. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10109) Windows dtest 3.0: ttl_test.py failures
[ https://issues.apache.org/jira/browse/CASSANDRA-10109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726549#comment-14726549 ] Stefania commented on CASSANDRA-10109: -- [~philipthompson] would you be able to set up CI on Windows for this branch? > Windows dtest 3.0: ttl_test.py failures > --- > > Key: CASSANDRA-10109 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10109 > Project: Cassandra > Issue Type: Sub-task >Reporter: Joshua McKenzie >Assignee: Stefania > Labels: Windows > Fix For: 3.0.0 rc1 > > > ttl_test.py:TestTTL.update_column_ttl_with_default_ttl_test2 > ttl_test.py:TestTTL.update_multiple_columns_ttl_test > ttl_test.py:TestTTL.update_single_column_ttl_test > Errors locally are different than CI from yesterday. Yesterday on CI we have > timeouts and general node hangs. Today on all 3 tests when run locally I see: > {noformat} > Traceback (most recent call last): > File "c:\src\cassandra-dtest\dtest.py", line 532, in tearDown > raise AssertionError('Unexpected error in %s node log: %s' % (node.name, > errors)) > AssertionError: Unexpected error in node1 node log: ['ERROR [main] 2015-08-17 > 16:53:43,120 NoSpamLogger.java:97 - This platform does not support atomic > directory streams (SecureDirectoryStream); race conditions when loading > sstable files could occurr'] > {noformat} > This traces back to the commit for CASSANDRA-7066 today by [~Stefania] and > [~benedict]. Stefania - care to take this ticket and also look further into > whether or not we're going to have issues with 7066 on Windows? That error > message certainly *sounds* like it's not a good thing. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8630) Faster sequential IO (on compaction, streaming, etc)
[ https://issues.apache.org/jira/browse/CASSANDRA-8630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726545#comment-14726545 ] Stefania commented on CASSANDRA-8630: - Fixed a small problem that was causing CI issues and now CI is stable again: http://cassci.datastax.com/job/stef1927-8630-3.0-testall/lastBuild/testReport/ http://cassci.datastax.com/job/stef1927-8630-3.0-dtest/lastBuild/testReport/ Relaunched cperf test with this fix in, see [here|http://cstar.datastax.com/tests/id/ebe0-510f-11e5-a17a-42010af0688f]. The test I launched yesterday shows only a small difference in read ops, not sure if related. My request on IRC to set-up Windows CI must have gone unnoticed; [~philipthompson] would you be able to set up CI on Windows for this branch? > Faster sequential IO (on compaction, streaming, etc) > > > Key: CASSANDRA-8630 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8630 > Project: Cassandra > Issue Type: Improvement > Components: Core, Tools >Reporter: Oleg Anastasyev >Assignee: Stefania > Labels: compaction, performance > Fix For: 3.x > > Attachments: 8630-FasterSequencialReadsAndWrites.txt, cpu_load.png, > flight_recorder_001_files.tar.gz, flight_recorder_002_files.tar.gz, > mmaped_uncomp_hotspot.png > > > When node is doing a lot of sequencial IO (streaming, compacting, etc) a lot > of CPU is lost in calls to RAF's int read() and DataOutputStream's write(int). > This is because default implementations of readShort,readLong, etc as well as > their matching write* are implemented with numerous calls of byte by byte > read and write. > This makes a lot of syscalls as well. > A quick microbench shows than just reimplementation of these methods in > either way gives 8x speed increase. > A patch attached implements RandomAccessReader.read and > SequencialWriter.write methods in more efficient way. > I also eliminated some extra byte copies in CompositeType.split and > ColumnNameHelper.maxComponents, which were on my profiler's hotspot method > list during tests. > A stress tests on my laptop show that this patch makes compaction 25-30% > faster on uncompressed sstables and 15% faster for compressed ones. > A deployment to production shows much less CPU load for compaction. > (I attached a cpu load graph from one of our production, orange is niced CPU > load - i.e. compaction; yellow is user - i.e. not compaction related tasks) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8894) Our default buffer size for (uncompressed) buffered reads should be smaller, and based on the expected record size
[ https://issues.apache.org/jira/browse/CASSANDRA-8894?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726544#comment-14726544 ] Albert P Tobey commented on CASSANDRA-8894: --- Sorry I'm late to the thread, but the recently added patch seems very overcomplicated for little benefit. I noticed the massive over-read in 2.1 and tracked it down independently by profiling with disk_access_mode: standard. I then ran a build with the DEFAULT_BUFFER_SIZE at 4K and saw an instant 4x increase in TXN/s on a simple -stress test with a 60% reduction in wasted disk IO. This over-read is causing performance problems on every Cassandra 2.1 cluster that isn't 100% writes. It doesn't always show up because of the massive amount of RAM a lot of folks are running, but under low memory situations it is killing even very fast SSDs. Patch for 2.1: https://gist.github.com/tobert/10c307cf3709a585a7cf In reading through the history, I think this is being overthought. If anything, the readahead and buffering in the read path should be *removed* and instead issue precise reads wherever it's possible. For now, the change to a 4K buffer size should be added to 2.1 in order to significantly speed up read workloads. > Our default buffer size for (uncompressed) buffered reads should be smaller, > and based on the expected record size > -- > > Key: CASSANDRA-8894 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8894 > Project: Cassandra > Issue Type: Improvement > Components: Core >Reporter: Benedict >Assignee: Stefania > Labels: benedict-to-commit > Fix For: 3.0 alpha 1 > > Attachments: 8894_25pct.yaml, 8894_5pct.yaml, 8894_tiny.yaml > > > A large contributor to slower buffered reads than mmapped is likely that we > read a full 64Kb at once, when average record sizes may be as low as 140 > bytes on our stress tests. The TLB has only 128 entries on a modern core, and > each read will touch 32 of these, meaning we are unlikely to almost ever be > hitting the TLB, and will be incurring at least 30 unnecessary misses each > time (as well as the other costs of larger than necessary accesses). When > working with an SSD there is little to no benefit reading more than 4Kb at > once, and in either case reading more data than we need is wasteful. So, I > propose selecting a buffer size that is the next larger power of 2 than our > average record size (with a minimum of 4Kb), so that we expect to read in one > operation. I also propose that we create a pool of these buffers up-front, > and that we ensure they are all exactly aligned to a virtual page, so that > the source and target operations each touch exactly one virtual page per 4Kb > of expected record size. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10248) Document compatibilities between native specs and Cassandra versions
Thibault Charbonnier created CASSANDRA-10248: Summary: Document compatibilities between native specs and Cassandra versions Key: CASSANDRA-10248 URL: https://issues.apache.org/jira/browse/CASSANDRA-10248 Project: Cassandra Issue Type: Improvement Components: Documentation & website Reporter: Thibault Charbonnier Priority: Minor Nowhere in the native specs is specified for which Cassandra version it is compatible with. This has been confusing to me when implementing a given protocol in a Lua driver, and has apparently been confusing other people [1]. I remember seeing a table specifying which specs were compatible with which Cassandra version somewhere in the Python driver documentation but I am currently unable to find it. Proposed solution: maybe include a small table in each specification file describing the compatibilities between Cassandra and the current (and eventually older) specs. [1] http://mail-archives.apache.org/mod_mbox/cassandra-dev/201504.mbox/%3ca87729c9-fa6a-4b34-bb7b-b324e154c...@datastax.com%3E -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10114) Allow count(*) and count(1) to be use as normal aggregation
[ https://issues.apache.org/jira/browse/CASSANDRA-10114?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Shuler updated CASSANDRA-10114: --- Description: For the following query: {code} SELECT count(*), max(timestamp), min(timestamp) FROM myData WHERE id = ? {code} Cassandra will throw a {{InvalidSyntaxException}}. We should allow {{count(\*)}} and {{count(1)}} to be queried with other aggregations or columns was: For the following query: {code} SELECT count(*), max(timestamp), min(timestamp) FROM myData WHERE id = ? {code} Cassandra will throw a {{InvalidSyntaxException}}. We should allow count(*) and count(1) to be queried with other aggregations or columns > Allow count(*) and count(1) to be use as normal aggregation > --- > > Key: CASSANDRA-10114 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10114 > Project: Cassandra > Issue Type: Improvement >Reporter: Benjamin Lerer >Assignee: Benjamin Lerer >Priority: Minor > Fix For: 2.2.1, 3.0 beta 1 > > Attachments: 10114-2.2.txt > > > For the following query: > {code} > SELECT count(*), max(timestamp), min(timestamp) FROM myData WHERE id = ? > {code} > Cassandra will throw a {{InvalidSyntaxException}}. > We should allow {{count(\*)}} and {{count(1)}} to be queried with other > aggregations or columns -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9921) Combine MV schema definition with MV table definition
[ https://issues.apache.org/jira/browse/CASSANDRA-9921?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726360#comment-14726360 ] Aleksey Yeschenko commented on CASSANDRA-9921: -- Looks all right on a very high level. Will take a deeper look tomorrow. Timing-wise, though, this might end up going to rc1, because of a necessary round-trip with the drivers team once we are done here. > Combine MV schema definition with MV table definition > - > > Key: CASSANDRA-9921 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9921 > Project: Cassandra > Issue Type: Improvement >Reporter: Carl Yeksigian >Assignee: Carl Yeksigian > Labels: client-impacting, materializedviews > Fix For: 3.0 beta 2 > > > Prevent MV from reusing {{system_schema.tables}} and instead move those > properties into the {{system_schema.materializedviews}} table to keep them > separate entities. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9839) Move crc_check_chance out of compressions options
[ https://issues.apache.org/jira/browse/CASSANDRA-9839?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-9839: - Assignee: Paulo Motta (was: Benjamin Lerer) Reviewer: Benjamin Lerer > Move crc_check_chance out of compressions options > - > > Key: CASSANDRA-9839 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9839 > Project: Cassandra > Issue Type: Bug >Reporter: Aleksey Yeschenko >Assignee: Paulo Motta >Priority: Minor > Labels: client-impacting, docs-impacting > Fix For: 3.0.0 rc1 > > > Follow up to CASSANDRA-8384. The option doesn't belong to compression params > - it doesn't affect compression, itself, and isn't passed to compressors upon > initialization. > While it's true that it is (currently) only being honored when reading > compressed sstables, it still doesn't belong to compression params (and is > causing CASSANDRA-7978 -like issues). > [~tjake] suggested we should make it an option of its own, and I think we > should. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9839) Move crc_check_chance out of compressions options
[ https://issues.apache.org/jira/browse/CASSANDRA-9839?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-9839: - Assignee: Benjamin Lerer (was: Paulo Motta) > Move crc_check_chance out of compressions options > - > > Key: CASSANDRA-9839 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9839 > Project: Cassandra > Issue Type: Bug >Reporter: Aleksey Yeschenko >Assignee: Benjamin Lerer >Priority: Minor > Labels: client-impacting, docs-impacting > Fix For: 3.0.0 rc1 > > > Follow up to CASSANDRA-8384. The option doesn't belong to compression params > - it doesn't affect compression, itself, and isn't passed to compressors upon > initialization. > While it's true that it is (currently) only being honored when reading > compressed sstables, it still doesn't belong to compression params (and is > causing CASSANDRA-7978 -like issues). > [~tjake] suggested we should make it an option of its own, and I think we > should. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10241) Keep a separate production debug log for troubleshooting
[ https://issues.apache.org/jira/browse/CASSANDRA-10241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726314#comment-14726314 ] Brandon Williams commented on CASSANDRA-10241: -- {quote} I suggest these packages: * config * gms * repair * service * streaming {quote} That list lgtm. > Keep a separate production debug log for troubleshooting > > > Key: CASSANDRA-10241 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10241 > Project: Cassandra > Issue Type: Bug > Components: Config >Reporter: Jonathan Ellis >Assignee: Brandon Williams > Fix For: 2.1.x > > > [~aweisberg] had the suggestion to keep a separate debug log for aid in > troubleshooting, not intended for regular human consumption but where we can > log things that might help if something goes wrong. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10169) Test regression for consistency_test.TestAccuracy
[ https://issues.apache.org/jira/browse/CASSANDRA-10169?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-10169: -- Assignee: Blake Eggleston (was: Benjamin Lerer) > Test regression for consistency_test.TestAccuracy > - > > Key: CASSANDRA-10169 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10169 > Project: Cassandra > Issue Type: Sub-task >Reporter: Sylvain Lebresne >Assignee: Blake Eggleston > Fix For: 3.0 beta 2 > > > We have 2 tests failures: > {noformat} > consistency_test.TestAccuracy.test_network_topology_strategy_users > consistency_test.TestAccuracy.test_simple_strategy_users > {noformat} > and looking at [the > history|http://cassci.datastax.com/job/cassandra-3.0_dtest/90/testReport/junit/consistency_test/TestAccuracy/test_simple_strategy_users/history/?start=50], > CASSANDRA-9913 appears to be the culprit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9964) Document post-2.1 caching table options syntax
[ https://issues.apache.org/jira/browse/CASSANDRA-9964?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-9964: - Assignee: Paulo Motta (was: Benjamin Lerer) > Document post-2.1 caching table options syntax > -- > > Key: CASSANDRA-9964 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9964 > Project: Cassandra > Issue Type: Bug >Reporter: Aleksey Yeschenko >Assignee: Paulo Motta >Priority: Minor > Fix For: 2.1.x, 2.2.x, 3.0.0 rc1 > > > CASSANDRA-5357 added new syntax for caching for {{CREATE TABLE}} in 2.1, but > [documentation|https://github.com/apache/cassandra/blob/trunk/doc/cql3/CQL.textile] > hasn't been updated accordingly. > Now that the old options are gone completely after CASSANDRA-9712, we *must* > update the CQL spec. > [DS > Docs|http://docs.datastax.com/en/cassandra/2.1/cassandra/operations/ops_set_caching_t.html] > have been up to date for a while. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9673) Improve batchlog write path
[ https://issues.apache.org/jira/browse/CASSANDRA-9673?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726299#comment-14726299 ] Aleksey Yeschenko commented on CASSANDRA-9673: -- I meant checking for table emptiness at the beginning of startup migration, and logging once if not empty. Logging on each conversion is obviously an overkill. Also, the logic in conversion for != 1 uuids seems a bit weird. It will never be the case that we'll get a mutation on upgrade with a random uuid. Made it just use the counter unconditionally at all times. There was a bug in {{SP::syncWriteToBatchlog}} that is as old as is batchlog itself. We are using CL.ONE unconditionally, even when we have two endpoints. And both legacy/modern writes should be sharing the same callback, so I switched back to using {{WriteResponseHandler}} for both. Rebased against most recent cassandra-3.0 and force-pushed. I think I broke some replay tests. Could you have a look and fix, if necessary? And, if you are overall +1, I'll commit. Thank you. > Improve batchlog write path > --- > > Key: CASSANDRA-9673 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9673 > Project: Cassandra > Issue Type: Improvement >Reporter: Aleksey Yeschenko >Assignee: Stefania > Labels: performance > Fix For: 3.0 beta 2 > > Attachments: 9673_001.tar.gz, 9673_004.tar.gz, > gc_times_first_node_patched_004.png, gc_times_first_node_trunk_004.png > > > Currently we allocate an on-heap {{ByteBuffer}} to serialize the batched > mutations into, before sending it to a distant node, generating unnecessary > garbage (potentially a lot of it). > With materialized views using the batchlog, it would be nice to optimise the > write path: > - introduce a new verb ({{Batch}}) > - introduce a new message ({{BatchMessage}}) that would encapsulate the > mutations, expiration, and creation time (similar to {{HintMessage}} in > CASSANDRA-6230) > - have MS serialize it directly instead of relying on an intermediate buffer > To avoid merely shifting the temp buffer to the receiving side(s) we should > change the structure of the batchlog table to use a list or a map of > individual mutations. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[04/10] cassandra git commit: Fix handling of streaming EOF
Fix handling of streaming EOF patch by Alexey Burylov; reviewed by yukim for CASSANDRA-10206 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2267d791 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2267d791 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2267d791 Branch: refs/heads/trunk Commit: 2267d79188c4388674ea40268a12f9c19f0fb9a7 Parents: 542bb1f Author: Alexey Burylov Authored: Tue Sep 1 12:18:17 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 12:18:17 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 15 ++- 2 files changed, 11 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7841f9b..681a40b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.10 + * Fix handling of streaming EOF (CASSANDRA-10206) * Only check KeyCache when it is enabled * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611) * (cqlsh) update list of CQL keywords (CASSANDRA-9232) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java -- diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java index 360b59e..9724bf1 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java @@ -18,9 +18,9 @@ package org.apache.cassandra.streaming.messages; import java.io.IOException; +import java.net.SocketException; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; import org.apache.cassandra.io.util.DataOutputStreamAndChannel; import org.apache.cassandra.streaming.StreamSession; @@ -48,18 +48,23 @@ public abstract class StreamMessage public static StreamMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException { ByteBuffer buff = ByteBuffer.allocate(1); -if (in.read(buff) > 0) +int readBytes = in.read(buff); +if (readBytes > 0) { buff.flip(); Type type = Type.get(buff.get()); return type.inSerializer.deserialize(in, version, session); } -else +else if (readBytes == 0) { -// when socket gets closed, there is a chance that buff is empty -// in that case, just return null +// input socket buffer was not filled yet return null; } +else +{ +// possibly socket gets closed +throw new SocketException("End-of-stream reached"); +} } /** StreamMessage serializer */
[02/10] cassandra git commit: Fix handling of streaming EOF
Fix handling of streaming EOF patch by Alexey Burylov; reviewed by yukim for CASSANDRA-10206 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2267d791 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2267d791 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2267d791 Branch: refs/heads/cassandra-2.2 Commit: 2267d79188c4388674ea40268a12f9c19f0fb9a7 Parents: 542bb1f Author: Alexey Burylov Authored: Tue Sep 1 12:18:17 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 12:18:17 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 15 ++- 2 files changed, 11 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7841f9b..681a40b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.10 + * Fix handling of streaming EOF (CASSANDRA-10206) * Only check KeyCache when it is enabled * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611) * (cqlsh) update list of CQL keywords (CASSANDRA-9232) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java -- diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java index 360b59e..9724bf1 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java @@ -18,9 +18,9 @@ package org.apache.cassandra.streaming.messages; import java.io.IOException; +import java.net.SocketException; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; import org.apache.cassandra.io.util.DataOutputStreamAndChannel; import org.apache.cassandra.streaming.StreamSession; @@ -48,18 +48,23 @@ public abstract class StreamMessage public static StreamMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException { ByteBuffer buff = ByteBuffer.allocate(1); -if (in.read(buff) > 0) +int readBytes = in.read(buff); +if (readBytes > 0) { buff.flip(); Type type = Type.get(buff.get()); return type.inSerializer.deserialize(in, version, session); } -else +else if (readBytes == 0) { -// when socket gets closed, there is a chance that buff is empty -// in that case, just return null +// input socket buffer was not filled yet return null; } +else +{ +// possibly socket gets closed +throw new SocketException("End-of-stream reached"); +} } /** StreamMessage serializer */
[10/10] cassandra git commit: Merge branch 'cassandra-3.0' into trunk
Merge branch 'cassandra-3.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/97b57cbd Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/97b57cbd Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/97b57cbd Branch: refs/heads/trunk Commit: 97b57cbdf790274934bcedfe435a5daad78839fc Parents: 0d2603c ae669b0 Author: Yuki Morishita Authored: Tue Sep 1 16:58:27 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 16:58:27 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 14 ++ 2 files changed, 11 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/97b57cbd/CHANGES.txt --
[06/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/02e5478f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/02e5478f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/02e5478f Branch: refs/heads/cassandra-2.2 Commit: 02e5478f05178cbc563605dcaa65de60066e046e Parents: 3effec5 2267d79 Author: Yuki Morishita Authored: Tue Sep 1 16:58:13 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 16:58:13 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 14 ++ 2 files changed, 11 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/02e5478f/CHANGES.txt -- diff --cc CHANGES.txt index 102,681a40b..91ae36c --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,8 -1,5 +1,9 @@@ -2.1.10 +2.2.2 + * Fix repair hang when snapshot failed (CASSANDRA-10057) + * Fall back to 1/4 commitlog volume for commitlog_total_space on small disks + (CASSANDRA-10199) +Merged from 2.1: + * Fix handling of streaming EOF (CASSANDRA-10206) * Only check KeyCache when it is enabled * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611) * (cqlsh) update list of CQL keywords (CASSANDRA-9232) http://git-wip-us.apache.org/repos/asf/cassandra/blob/02e5478f/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java --
[08/10] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ae669b0b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ae669b0b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ae669b0b Branch: refs/heads/cassandra-3.0 Commit: ae669b0b3b11798afe0162838e4d6659fcabb678 Parents: 45eba1a 02e5478 Author: Yuki Morishita Authored: Tue Sep 1 16:58:21 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 16:58:21 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 14 ++ 2 files changed, 11 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ae669b0b/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ae669b0b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java --
[07/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/02e5478f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/02e5478f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/02e5478f Branch: refs/heads/trunk Commit: 02e5478f05178cbc563605dcaa65de60066e046e Parents: 3effec5 2267d79 Author: Yuki Morishita Authored: Tue Sep 1 16:58:13 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 16:58:13 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 14 ++ 2 files changed, 11 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/02e5478f/CHANGES.txt -- diff --cc CHANGES.txt index 102,681a40b..91ae36c --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,8 -1,5 +1,9 @@@ -2.1.10 +2.2.2 + * Fix repair hang when snapshot failed (CASSANDRA-10057) + * Fall back to 1/4 commitlog volume for commitlog_total_space on small disks + (CASSANDRA-10199) +Merged from 2.1: + * Fix handling of streaming EOF (CASSANDRA-10206) * Only check KeyCache when it is enabled * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611) * (cqlsh) update list of CQL keywords (CASSANDRA-9232) http://git-wip-us.apache.org/repos/asf/cassandra/blob/02e5478f/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java --
[01/10] cassandra git commit: Fix handling of streaming EOF
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 542bb1fa4 -> 2267d7918 refs/heads/cassandra-2.2 3effec53e -> 02e5478f0 refs/heads/cassandra-3.0 45eba1adb -> ae669b0b3 refs/heads/trunk 0d2603cf9 -> 97b57cbdf Fix handling of streaming EOF patch by Alexey Burylov; reviewed by yukim for CASSANDRA-10206 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2267d791 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2267d791 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2267d791 Branch: refs/heads/cassandra-2.1 Commit: 2267d79188c4388674ea40268a12f9c19f0fb9a7 Parents: 542bb1f Author: Alexey Burylov Authored: Tue Sep 1 12:18:17 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 12:18:17 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 15 ++- 2 files changed, 11 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7841f9b..681a40b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.10 + * Fix handling of streaming EOF (CASSANDRA-10206) * Only check KeyCache when it is enabled * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611) * (cqlsh) update list of CQL keywords (CASSANDRA-9232) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java -- diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java index 360b59e..9724bf1 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java @@ -18,9 +18,9 @@ package org.apache.cassandra.streaming.messages; import java.io.IOException; +import java.net.SocketException; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; import org.apache.cassandra.io.util.DataOutputStreamAndChannel; import org.apache.cassandra.streaming.StreamSession; @@ -48,18 +48,23 @@ public abstract class StreamMessage public static StreamMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException { ByteBuffer buff = ByteBuffer.allocate(1); -if (in.read(buff) > 0) +int readBytes = in.read(buff); +if (readBytes > 0) { buff.flip(); Type type = Type.get(buff.get()); return type.inSerializer.deserialize(in, version, session); } -else +else if (readBytes == 0) { -// when socket gets closed, there is a chance that buff is empty -// in that case, just return null +// input socket buffer was not filled yet return null; } +else +{ +// possibly socket gets closed +throw new SocketException("End-of-stream reached"); +} } /** StreamMessage serializer */
[09/10] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ae669b0b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ae669b0b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ae669b0b Branch: refs/heads/trunk Commit: ae669b0b3b11798afe0162838e4d6659fcabb678 Parents: 45eba1a 02e5478 Author: Yuki Morishita Authored: Tue Sep 1 16:58:21 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 16:58:21 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 14 ++ 2 files changed, 11 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ae669b0b/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ae669b0b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java --
[03/10] cassandra git commit: Fix handling of streaming EOF
Fix handling of streaming EOF patch by Alexey Burylov; reviewed by yukim for CASSANDRA-10206 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2267d791 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2267d791 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2267d791 Branch: refs/heads/cassandra-3.0 Commit: 2267d79188c4388674ea40268a12f9c19f0fb9a7 Parents: 542bb1f Author: Alexey Burylov Authored: Tue Sep 1 12:18:17 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 12:18:17 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 15 ++- 2 files changed, 11 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 7841f9b..681a40b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.10 + * Fix handling of streaming EOF (CASSANDRA-10206) * Only check KeyCache when it is enabled * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611) * (cqlsh) update list of CQL keywords (CASSANDRA-9232) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2267d791/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java -- diff --git a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java index 360b59e..9724bf1 100644 --- a/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java +++ b/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java @@ -18,9 +18,9 @@ package org.apache.cassandra.streaming.messages; import java.io.IOException; +import java.net.SocketException; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; -import java.nio.channels.WritableByteChannel; import org.apache.cassandra.io.util.DataOutputStreamAndChannel; import org.apache.cassandra.streaming.StreamSession; @@ -48,18 +48,23 @@ public abstract class StreamMessage public static StreamMessage deserialize(ReadableByteChannel in, int version, StreamSession session) throws IOException { ByteBuffer buff = ByteBuffer.allocate(1); -if (in.read(buff) > 0) +int readBytes = in.read(buff); +if (readBytes > 0) { buff.flip(); Type type = Type.get(buff.get()); return type.inSerializer.deserialize(in, version, session); } -else +else if (readBytes == 0) { -// when socket gets closed, there is a chance that buff is empty -// in that case, just return null +// input socket buffer was not filled yet return null; } +else +{ +// possibly socket gets closed +throw new SocketException("End-of-stream reached"); +} } /** StreamMessage serializer */
[05/10] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2
Merge branch 'cassandra-2.1' into cassandra-2.2 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/02e5478f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/02e5478f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/02e5478f Branch: refs/heads/cassandra-3.0 Commit: 02e5478f05178cbc563605dcaa65de60066e046e Parents: 3effec5 2267d79 Author: Yuki Morishita Authored: Tue Sep 1 16:58:13 2015 -0500 Committer: Yuki Morishita Committed: Tue Sep 1 16:58:13 2015 -0500 -- CHANGES.txt | 1 + .../cassandra/streaming/messages/StreamMessage.java | 14 ++ 2 files changed, 11 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/02e5478f/CHANGES.txt -- diff --cc CHANGES.txt index 102,681a40b..91ae36c --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,8 -1,5 +1,9 @@@ -2.1.10 +2.2.2 + * Fix repair hang when snapshot failed (CASSANDRA-10057) + * Fall back to 1/4 commitlog volume for commitlog_total_space on small disks + (CASSANDRA-10199) +Merged from 2.1: + * Fix handling of streaming EOF (CASSANDRA-10206) * Only check KeyCache when it is enabled * Change streaming_socket_timeout_in_ms default to 1 hour (CASSANDRA-8611) * (cqlsh) update list of CQL keywords (CASSANDRA-9232) http://git-wip-us.apache.org/repos/asf/cassandra/blob/02e5478f/src/java/org/apache/cassandra/streaming/messages/StreamMessage.java --
[jira] [Updated] (CASSANDRA-10246) Names values don't work with batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10246?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Penick updated CASSANDRA-10246: --- Description: This is broken at the protocol-level and in the implementation. At the protocol-level the {{}} component of the batch comes after the queries. That means the protocol parser would need to read ahead (and back track) to determine the values encoding and correctly read the values from the query entries. Also, a batch-level setting for named values forces all queries to use the same encoding. Should batches force a single, homogenous query value encoding? (This is confusing) In the implementation, values are indiscriminately read using {{CBUtil.readValueList()}}, and the batch flags are never checked (for {{(Flag.NAMES_FOR_VALUES}}) to see if {{CBUtil.readNameAndValueList()}} should be called instead: https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 Proposed solution: CASSANDRA-10247 was: This is broken at the protocol-level and in the implementation. At the protocol-level the {{}} component of the batch comes after the queries. That means the protocol parser would need to read ahead (and back track) to determine the values encoding and correctly read the values from the query entries. Also, a batch-level setting for named values forces all queries to use the same encoding. Should batches force a single, homogenous query value encoding? (This is confusing) In the implementation, values are indiscriminately read using {{CBUtil.readValueList()}}, and the batch flags are never checked (for {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should be called instead: https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 Proposed solution: CASSANDRA-10247 > Names values don't work with batches > > > Key: CASSANDRA-10246 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10246 > Project: Cassandra > Issue Type: Bug > Components: API >Reporter: Michael Penick > > This is broken at the protocol-level and in the implementation. > At the protocol-level the {{}} component of the batch comes after the > queries. That means the protocol parser would need to read ahead (and back > track) to determine the values encoding and correctly read the values from > the query entries. Also, a batch-level setting for named values forces all > queries to use the same encoding. Should batches force a single, homogenous > query value encoding? (This is confusing) > In the implementation, values are indiscriminately read using > {{CBUtil.readValueList()}}, and the batch flags are never checked (for > {{(Flag.NAMES_FOR_VALUES}}) to see if {{CBUtil.readNameAndValueList()}} > should be called instead: > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 > Proposed solution: CASSANDRA-10247 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10246) Names values don't work with batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10246?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Penick updated CASSANDRA-10246: --- Description: This is broken at the protocol-level and in the implementation. At the protocol-level the {{}} component of the batch comes after the queries. That means the protocol parser would need to read ahead (and back track) to determine the values encoding and correctly read the values from the query entries. Also, a batch-level setting for named values forces all queries to use the same encoding. Should batches force a single, homogenous query value encoding? (This is confusing) In the implementation, values are indiscriminately read using {{CBUtil.readValueList()}}, and the batch flags are never checked (for {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should be called instead: https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 Proposed solution: CASSANDRA-10247 was: This is broken at the protocol-level and in the implementation. At the protocol-level the {{}} component of the batch comes after the queries. That means the protocol parser would need to read ahead (and back track) to correctly read the values from the queries entries. Also, a batch-level setting for named values forces all queries to use named values. Should batches force homogenous query value types? (This is confusing) In the implementation, values are indiscriminately read using {{CBUtil.readValueList()}}, and the batch flags are never checked (for {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should be called instead: https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 Proposed solution: CASSANDRA-10247 > Names values don't work with batches > > > Key: CASSANDRA-10246 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10246 > Project: Cassandra > Issue Type: Bug > Components: API >Reporter: Michael Penick > > This is broken at the protocol-level and in the implementation. > At the protocol-level the {{}} component of the batch comes after the > queries. That means the protocol parser would need to read ahead (and back > track) to determine the values encoding and correctly read the values from > the query entries. Also, a batch-level setting for named values forces all > queries to use the same encoding. Should batches force a single, homogenous > query value encoding? (This is confusing) > In the implementation, values are indiscriminately read using > {{CBUtil.readValueList()}}, and the batch flags are never checked (for > {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should > be called instead: > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 > Proposed solution: CASSANDRA-10247 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10246) Names values don't work with batches
[ https://issues.apache.org/jira/browse/CASSANDRA-10246?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Penick updated CASSANDRA-10246: --- Description: This is broken at the protocol-level and in the implementation. At the protocol-level the {{}} component of the batch comes after the queries. That means the protocol parser would need to read ahead (and back track) to correctly read the values from the queries entries. Also, a batch-level setting for named values forces all queries to use named values. Should batches force homogenous query value types? (This is confusing) In the implementation, values are indiscriminately read using {{CBUtil.readValueList()}}, and the batch flags are never checked (for {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should be called instead: https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 Proposed solution: CASSANDRA-10247 was: This is broken at the protocol-level and in the implementation. At the protocol-level the {{}} component of the batch comes after the queries. That means the protocol parser would need to read ahead (and back track) to correctly read the values from the queries entries. Also, a batch-level setting for named values forces all queries to use named values. Should batches force homogenous query value types? (This is confusing) In the implementation, values are indiscriminately read using {{CBUtil.readValueList()}}, and the batch flags are never checked (for {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should be called instead: https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 > Names values don't work with batches > > > Key: CASSANDRA-10246 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10246 > Project: Cassandra > Issue Type: Bug > Components: API >Reporter: Michael Penick > > This is broken at the protocol-level and in the implementation. > At the protocol-level the {{}} component of the batch comes after the > queries. That means the protocol parser would need to read ahead (and back > track) to correctly read the values from the queries entries. Also, a > batch-level setting for named values forces all queries to use named values. > Should batches force homogenous query value types? (This is confusing) > In the implementation, values are indiscriminately read using > {{CBUtil.readValueList()}}, and the batch flags are never checked (for > {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should > be called instead: > https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 > Proposed solution: CASSANDRA-10247 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10247) Allow both named values and regular values in a single batch
[ https://issues.apache.org/jira/browse/CASSANDRA-10247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Penick updated CASSANDRA-10247: --- Description: Currently, there's a single {{}} to determine whether a batch uses named values or not. This forces all query entries in a batch to use the same value encoding. There's also the fact that the current design is broken. See: CASSANDRA-10246 I propose that this information be encoded using the batch entries' {{}} component and remove "0x40: With names for values" as an option from {{}}. In the current design the {{}} component, a {{[byte]}}, only uses two values 0 and 1 for non-prepared and prepared, respectively. The proposed solution would add two more values 2 and 3 for non-prepared w/ names values and prepared w/ named values. This has a couple benefits: 1) It allows for heterogeneous value encodings in a single batch 2) It eliminates the need for reading ahead to determine the value encodings (See: CASSANDRA-10246) was: Currently, there's a single {{}} to determine whether a batch uses named values or not. This forces all query entries in a batch to use the same value encoding. There's also the fact that the current design is broken. See: CASSANDRA-10246 I propose that this information be encoded using the batch entries {{}} component and remove "0x40: With names for values" as an option from {{}}. In the current design the {{}}, a {{[byte]}}, only uses two values 0 and 1 for non-prepared and prepared, respectively. The proposed solution would add two more values 2 and 3 for non-prepared w/ names values and prepared w/ named values. This has a couple benefits: 1) It allows for heterogeneous value encodings in a single batch 2) It eliminates the need for reading ahead to determine the value encodings (See: CASSANDRA-10246) > Allow both named values and regular values in a single batch > > > Key: CASSANDRA-10247 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10247 > Project: Cassandra > Issue Type: Improvement >Reporter: Michael Penick > > Currently, there's a single {{}} to determine whether a batch uses > named values or not. This forces all query entries in a batch to use the same > value encoding. There's also the fact that the current design is broken. See: > CASSANDRA-10246 > I propose that this information be encoded using the batch entries' > {{}} component and remove "0x40: With names for values" as an option > from {{}}. In the current design the {{}} component, a > {{[byte]}}, only uses two values 0 and 1 for non-prepared and prepared, > respectively. The proposed solution would add two more values 2 and 3 for > non-prepared w/ names values and prepared w/ named values. This has a couple > benefits: > 1) It allows for heterogeneous value encodings in a single batch > 2) It eliminates the need for reading ahead to determine the value encodings > (See: CASSANDRA-10246) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10245) Provide after the fact visibility into the reliability of the environment C* operates in
[ https://issues.apache.org/jira/browse/CASSANDRA-10245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726267#comment-14726267 ] Ariel Weisberg commented on CASSANDRA-10245: For reading and writing to files I think we should add plumbing that times every single one, and then above a threshold record the event. Bridging the gap between IO and mmap is a hurdle. If it doesn't exceed the threshold maybe log a count just so we can represent the # that didn't hit an outlier. There is a commitment here of a few % of CPU to nanoTime(). We could try measuring with flight recorder on/off. I'll bet it's not noticeable. In C* it may be that since there are a lot of threads and there aren't really any special ones that must never block then we don't need this kind of visibility. Maybe we just want to monitor block device statistics via /proc so we can correlate with higher level metrics like read and write latency. > Provide after the fact visibility into the reliability of the environment C* > operates in > > > Key: CASSANDRA-10245 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10245 > Project: Cassandra > Issue Type: New Feature > Components: Core >Reporter: Ariel Weisberg > Fix For: 3.x > > > I think that by default databases should not be completely dependent on > operator provided tools for monitoring node and network health. > The database should be able to detect and report on several dimensions of > performance in its environment, and more specifically report on deviations > from acceptable performance. > * Node wide pauses > * JVM wide pauses > * Latency, and roundtrip time to all endpoints > * Block device IO latency > If flight recorder were available for use in production I would say as a > start just turn that on, add jHiccup (inside and outside the server process), > and a daemon inside the server to measure network performance between > endpoints. > FR is not available (requires a license in production) so instead focus on > adding instrumentation for the most useful facets of flight recorder in > diagnosing performance issues. I think we can get pretty far because what we > need to do is not quite as undirected as the exploration FR and JMC > facilitate. > Until we dial in how we measure and how to signal without false positives I > would expect this kind of logging to be in the background for post-hoc > analysis. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10247) Allow both named values and regular values in a single batch
[ https://issues.apache.org/jira/browse/CASSANDRA-10247?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Michael Penick updated CASSANDRA-10247: --- Summary: Allow both named values and regular values in a single batch (was: Allow both named values and regular values to simultaneously work in batches) > Allow both named values and regular values in a single batch > > > Key: CASSANDRA-10247 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10247 > Project: Cassandra > Issue Type: Improvement >Reporter: Michael Penick > > Currently, there's a single {{}} to determine whether a batch uses > named values or not. This forces all query entries in a batch to use the same > value encoding. There's also the fact that the current design is broken. See: > CASSANDRA-10246 > I propose that this information be encoded using the batch entries {{}} > component and remove "0x40: With names for values" as an option from > {{}}. In the current design the {{}}, a {{[byte]}}, only uses > two values 0 and 1 for non-prepared and prepared, respectively. The proposed > solution would add two more values 2 and 3 for non-prepared w/ names values > and prepared w/ named values. This has a couple benefits: > 1) It allows for heterogeneous value encodings in a single batch > 2) It eliminates the need for reading ahead to determine the value encodings > (See: CASSANDRA-10246) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10247) Allow both named values and regular values to simultaneously work in batches
Michael Penick created CASSANDRA-10247: -- Summary: Allow both named values and regular values to simultaneously work in batches Key: CASSANDRA-10247 URL: https://issues.apache.org/jira/browse/CASSANDRA-10247 Project: Cassandra Issue Type: Improvement Reporter: Michael Penick Currently, there's a single {{}} to determine whether a batch uses named values or not. This forces all query entries in a batch to use the same value encoding. There's also the fact that the current design is broken. See: CASSANDRA-10246 I propose that this information be encoded using the batch entries {{}} component and remove "0x40: With names for values" as an option from {{}}. In the current design the {{}}, a {{[byte]}}, only uses two values 0 and 1 for non-prepared and prepared, respectively. The proposed solution would add two more values 2 and 3 for non-prepared w/ names values and prepared w/ named values. This has a couple benefits: 1) It allows for heterogeneous value encodings in a single batch 2) It eliminates the need for reading ahead to determine the value encodings (See: CASSANDRA-10246) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10245) Provide after the fact visibility into the reliability of the environment C* operates in
[ https://issues.apache.org/jira/browse/CASSANDRA-10245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726246#comment-14726246 ] Ariel Weisberg commented on CASSANDRA-10245: For measuring network performance. Every 5 milliseconds (or whatever) send a message to every other node in the cluster, or some subset (do cover all nodes eventually). In the heartbeat place the wall clock time the message was sent. The thread waking up periodically to send messages should keep a histogram of how far off from it's target wakeup it is off. Also track the delta between when remote heartbeats claim to be sent and when they are received as well as the delta between the expected amount of time since the last heartbeat was received and the actual amount of time it took. Combining these facts across nodes will give you visibility into the difference between node wide pauses and network related pauses. You can also look at clock skew. If a node reliably delivers it's messages on the expected interval, but the timestamp is not as expected you can guess that there is some clock skew. You can set thresholds for when to be chatty about conditions and start dumping histograms, percentiles, or whatever to a human readable log. There is overlap between this and jHiccup, but we need to run something out of process anyways to track JVM pauses. jHiccup also comes with some reporting/visualization. > Provide after the fact visibility into the reliability of the environment C* > operates in > > > Key: CASSANDRA-10245 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10245 > Project: Cassandra > Issue Type: New Feature > Components: Core >Reporter: Ariel Weisberg > Fix For: 3.x > > > I think that by default databases should not be completely dependent on > operator provided tools for monitoring node and network health. > The database should be able to detect and report on several dimensions of > performance in its environment, and more specifically report on deviations > from acceptable performance. > * Node wide pauses > * JVM wide pauses > * Latency, and roundtrip time to all endpoints > * Block device IO latency > If flight recorder were available for use in production I would say as a > start just turn that on, add jHiccup (inside and outside the server process), > and a daemon inside the server to measure network performance between > endpoints. > FR is not available (requires a license in production) so instead focus on > adding instrumentation for the most useful facets of flight recorder in > diagnosing performance issues. I think we can get pretty far because what we > need to do is not quite as undirected as the exploration FR and JMC > facilitate. > Until we dial in how we measure and how to signal without false positives I > would expect this kind of logging to be in the background for post-hoc > analysis. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10246) Names values don't work with batches
Michael Penick created CASSANDRA-10246: -- Summary: Names values don't work with batches Key: CASSANDRA-10246 URL: https://issues.apache.org/jira/browse/CASSANDRA-10246 Project: Cassandra Issue Type: Bug Components: API Reporter: Michael Penick This is broken at the protocol-level and in the implementation. At the protocol-level the {{}} component of the batch comes after the queries. That means the protocol parser would need to read ahead (and back track) to correctly read the values from the queries entries. Also, a batch-level setting for named values forces all queries to use named values. Should batches force homogenous query value types? (This is confusing) In the implementation, values are indiscriminately read using {{CBUtil.readValueList()}}, and the batch flags are never checked (for {{(Flag.NAMES_FOR_VALUES}} to see if {{CBUtil.readNameAndValueList()}} should be called instead: https://github.com/apache/cassandra/blob/cassandra-2.1/src/java/org/apache/cassandra/transport/messages/BatchMessage.java#L64 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10245) Provide after the fact visibility into the reliability of the environment C* operates in
Ariel Weisberg created CASSANDRA-10245: -- Summary: Provide after the fact visibility into the reliability of the environment C* operates in Key: CASSANDRA-10245 URL: https://issues.apache.org/jira/browse/CASSANDRA-10245 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Ariel Weisberg Fix For: 3.x I think that by default databases should not be completely dependent on operator provided tools for monitoring node and network health. The database should be able to detect and report on several dimensions of performance in its environment, and more specifically report on deviations from acceptable performance. * Node wide pauses * JVM wide pauses * Latency, and roundtrip time to all endpoints * Block device IO latency If flight recorder were available for use in production I would say as a start just turn that on, add jHiccup (inside and outside the server process), and a daemon inside the server to measure network performance between endpoints. FR is not available (requires a license in production) so instead focus on adding instrumentation for the most useful facets of flight recorder in diagnosing performance issues. I think we can get pretty far because what we need to do is not quite as undirected as the exploration FR and JMC facilitate. Until we dial in how we measure and how to signal without false positives I would expect this kind of logging to be in the background for post-hoc analysis. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10239) Failure to launch on Windows with spaces in directory.
[ https://issues.apache.org/jira/browse/CASSANDRA-10239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-10239: Assignee: Joshua McKenzie (was: Philip Thompson) Fix Version/s: 2.2.x I was able to reproduce this. > Failure to launch on Windows with spaces in directory. > -- > > Key: CASSANDRA-10239 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10239 > Project: Cassandra > Issue Type: Bug > Components: Packaging > Environment: Windows Server 2012 > Oracle JDK 1.7.79 >Reporter: vyacheslav zaslavskiy >Assignee: Joshua McKenzie > Fix For: 2.2.x > > > Repro: > Download cassandra 2.2.0 > Extra to C:\Program Files > Execute with cassandra, cassandra.bat, powershell /file cassandra.ps1 > The only way I was able to get this to work was by setting contrast_home to > Progra~1. I have seen multiple resolved tickets for similar issues, but that > doesn't appear to be the case. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10241) Keep a separate production debug log for troubleshooting
[ https://issues.apache.org/jira/browse/CASSANDRA-10241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726197#comment-14726197 ] Ariel Weisberg commented on CASSANDRA-10241: My goal with this suggestion is to eliminate the conflicting priorities of user visible human readable logging and more verbose less readable debug/trace logging. We can't eliminate the overhead of trace logging so we can't log everything. Statements that execute one per operation or per record and statements that dump large data structures aren't good candidates. Once you can be chatty in the log you can start logging state transitions that you almost never care about, but are helpful when the unexpected happens. Over time you can start to accrue more useful logging as you work through real failures. We also really need go/no go measurements for what is logged via this mechanism so we know what it is costing us. Maybe a JMX metric that reports on the log throughput, statements/second, bytes/second with some historical window so we can diagnose issues related to poorly designed logging. Or we can just be careful. bq. Related: is it time to switch to AsyncAppender? One thing you can do with AsyncAppending is have error and warn log synchronously so you get a stronger guarantee errors are visible (such as before voluntarily terminating). That somewhat mitigates the risk of missing important messages. I think that switching to async appending is less critical in C* because it doesn't have a thread per core design. If there are places we log while holding locks, well that's a different story. As long as the thread doing the logging is the only one impacted and we don't have a design where for whatever reason we have some special threads everyone may end up waiting on then it is fine. Put another way it's 4th or 5th nine issue. > Keep a separate production debug log for troubleshooting > > > Key: CASSANDRA-10241 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10241 > Project: Cassandra > Issue Type: Bug > Components: Config >Reporter: Jonathan Ellis >Assignee: Brandon Williams > Fix For: 2.1.x > > > [~aweisberg] had the suggestion to keep a separate debug log for aid in > troubleshooting, not intended for regular human consumption but where we can > log things that might help if something goes wrong. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Assigned] (CASSANDRA-10239) Failure to launch on Windows with spaces in directory.
[ https://issues.apache.org/jira/browse/CASSANDRA-10239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson reassigned CASSANDRA-10239: --- Assignee: Philip Thompson > Failure to launch on Windows with spaces in directory. > -- > > Key: CASSANDRA-10239 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10239 > Project: Cassandra > Issue Type: Bug > Components: Packaging > Environment: Windows Server 2012 > Oracle JDK 1.7.79 >Reporter: vyacheslav zaslavskiy >Assignee: Philip Thompson > > Repro: > Download cassandra 2.2.0 > Extra to C:\Program Files > Execute with cassandra, cassandra.bat, powershell /file cassandra.ps1 > The only way I was able to get this to work was by setting contrast_home to > Progra~1. I have seen multiple resolved tickets for similar issues, but that > doesn't appear to be the case. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9893) Fix upgrade tests from #9704 that are still failing
[ https://issues.apache.org/jira/browse/CASSANDRA-9893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14704060#comment-14704060 ] Blake Eggleston edited comment on CASSANDRA-9893 at 9/1/15 8:53 PM: I've merged Tyler's 8099-backwards-compat branch with the current dtest master and run the tests in the upgrade_tests folder locally against the tip of cassandra-3.0 (CASSANDRA_DIR) and cassandra-2.1 (OLD_CASSANDRA_DIR) with Stupp's fix for CASSANDRA-8220 on osx. I'm getting 7 failures, and 9 other tests which are repeatably throwing various timeout themed errors. These are the tests that are failing: -cql_tests:TestCQL.collection_indexing_test- -cql_tests:TestCQL.composite_index_collections_test- -cql_tests:TestCQL.edge_2i_on_complex_pk_test- -cql_tests:TestCQL.indexed_with_eq_test- -*cql_tests:TestCQL.large_count_test*- (CASSANDRA-9857) -cql_tests:TestCQL.conditional_delete_test- -*cql_tests:TestCQL.static_columns_with_distinct_test*- (CASSANDRA-9857) -paging_test:TestPagingWithDeletions.test_failure_threshold_deletions- -cql_tests:TestCQL.static_columns_cas_test- -*paging_test:TestPagingSize.test_with_equal_results_to_page_size*- (CASSANDRA-9857) -*cql_tests:TestCQL.select_distinct_with_deletions_test*- (CASSANDRA-9857) -*paging_test:TestPagingData.basic_compound_paging_test*- (CASSANDRA-9857) -*paging_test:TestPagingSize.test_undefined_page_size_default*- (CASSANDRA-9857) these are the ones with timeout/no host errors: -*cql_tests:TestCQL.batch_test*- (CASSANDRA-9673) -*cql_tests:TestCQL.noncomposite_static_cf_test*- (CASSANDRA-9673) -*cql_tests:TestCQL.static_cf_test*- (CASSANDRA-9673) -*cql_tests:TestCQL.test_v2_protocol_IN_with_tuples*- (CASSANDRA-10146) -paging_test:TestPagingDatasetChanges.test_row_TTL_expiry_during_paging- and these had Truncate related timeouts: -paging_test:TestPagingData.test_paging_a_single_wide_row- -paging_test:TestPagingData.test_paging_across_multi_wide_rows- -paging_test:TestPagingDatasetChanges.test_cell_TTL_expiry_during_paging- -paging_test:TestPagingSize.test_undefined_page_size_default- with some more info about the failures here: https://gist.github.com/bdeggleston/cb29e277468a5861d33e https://gist.github.com/bdeggleston/ead704e3f62d55adf5f0 This is mainly a checklist for myself, since 8099-backwards-compat support on cassci isn't there yet. I'm going to focus on the failures first, then take a closer look at the errors. was (Author: bdeggleston): I've merged Tyler's 8099-backwards-compat branch with the current dtest master and run the tests in the upgrade_tests folder locally against the tip of cassandra-3.0 (CASSANDRA_DIR) and cassandra-2.1 (OLD_CASSANDRA_DIR) with Stupp's fix for CASSANDRA-8220 on osx. I'm getting 7 failures, and 9 other tests which are repeatably throwing various timeout themed errors. These are the tests that are failing: -cql_tests:TestCQL.collection_indexing_test- -cql_tests:TestCQL.composite_index_collections_test- -cql_tests:TestCQL.edge_2i_on_complex_pk_test- -cql_tests:TestCQL.indexed_with_eq_test- -*cql_tests:TestCQL.large_count_test*- (CASSANDRA-9857) -cql_tests:TestCQL.conditional_delete_test- -*cql_tests:TestCQL.static_columns_with_distinct_test*- (CASSANDRA-9857) -paging_test:TestPagingWithDeletions.test_failure_threshold_deletions- cql_tests:TestCQL.static_columns_cas_test paging_test:TestPagingSize.test_with_equal_results_to_page_size these are the ones with timeout/no host errors: -*cql_tests:TestCQL.batch_test*- (CASSANDRA-9673) -*cql_tests:TestCQL.noncomposite_static_cf_test*- (CASSANDRA-9673) -*cql_tests:TestCQL.static_cf_test*- (CASSANDRA-9673) -*cql_tests:TestCQL.test_v2_protocol_IN_with_tuples*- (CASSANDRA-10146) -paging_test:TestPagingDatasetChanges.test_row_TTL_expiry_during_paging- and these had Truncate related timeouts: -paging_test:TestPagingData.test_paging_a_single_wide_row- -paging_test:TestPagingData.test_paging_across_multi_wide_rows- -paging_test:TestPagingDatasetChanges.test_cell_TTL_expiry_during_paging- -paging_test:TestPagingSize.test_undefined_page_size_default- with some more info about the failures here: https://gist.github.com/bdeggleston/cb29e277468a5861d33e https://gist.github.com/bdeggleston/ead704e3f62d55adf5f0 This is mainly a checklist for myself, since 8099-backwards-compat support on cassci isn't there yet. I'm going to focus on the failures first, then take a closer look at the errors. > Fix upgrade tests from #9704 that are still failing > --- > > Key: CASSANDRA-9893 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9893 > Project: Cassandra > Issue Type: Bug >Reporter: Sylvain Lebresne >Assignee: Blake Eggleston > Fix For: 3.0 beta 2 > > > The first things to do on this ticket would be to commit Tyler's branch > (ht
[jira] [Commented] (CASSANDRA-9893) Fix upgrade tests from #9704 that are still failing
[ https://issues.apache.org/jira/browse/CASSANDRA-9893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726164#comment-14726164 ] Blake Eggleston commented on CASSANDRA-9893: here are the various bugfixes rebased against the latest cassandra-3.0: https://github.com/bdeggleston/cassandra/tree/9893-3 {{cql_tests:TestCQL.static_columns_cas_test}} was failing because calls to {{UnfilteredRowIterators.noRowsIterator}} discard the static Columns data, which causes problems deserializing the static row. The call to noRowsIterator from {{SelectNoSlices.makeSliceIterator}} was also discarding the static row entirely. This wasn't strictly a problem with upgrades either, 3.0-3.0 queries were also failing. {{paging_test:TestPagingSize.test_with_equal_results_to_page_size}}, {{cql_tests:TestCQL.select_distinct_with_deletions_test}}, {{paging_test:TestPagingData.basic_compound_paging_test}}, and {{paging_test:TestPagingSize.test_undefined_page_size_default}} failures are also caused by CASSANDRA-9857 > Fix upgrade tests from #9704 that are still failing > --- > > Key: CASSANDRA-9893 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9893 > Project: Cassandra > Issue Type: Bug >Reporter: Sylvain Lebresne >Assignee: Blake Eggleston > Fix For: 3.0 beta 2 > > > The first things to do on this ticket would be to commit Tyler's branch > (https://github.com/thobbs/cassandra-dtest/tree/8099-backwards-compat) to the > dtests so cassci run them. I've had to do a few minor modifications to have > them run locally so someone which access to cassci should do it and make sure > it runs properly. > Once we have that, we should fix any test that isn't passing. I've ran the > tests locally and I had 8 failures. for 2 of them, it sounds plausible that > they'll get fixed by the patch of CASSANDRA-9775, though that's just a guess. > The rest where test that timeouted without a particular error in the log, > and running some of them individually, they passed. So we'll have to see if > it's just my machine being overly slow when running them all. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10243) Warn or fail when changing cluster topology live
[ https://issues.apache.org/jira/browse/CASSANDRA-10243?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726159#comment-14726159 ] Jonathan Ellis commented on CASSANDRA-10243: Are we confident that this is *always* the wrong thing? If so we could fail changing the snitch topology when it would change a live node's location. But this is dangerous since different nodes could have different views of cluster states, resulting in some loading the new config and others rejecting it. Nor does it help when a node is legitimately down temporarily but comes back up to find its rack has been changed out from under it. Perhaps it is simplest to just warn loudly in all these cases. > Warn or fail when changing cluster topology live > > > Key: CASSANDRA-10243 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10243 > Project: Cassandra > Issue Type: Improvement > Components: Tools >Reporter: Jonathan Ellis > Fix For: 2.1.x > > > Moving a node from one rack to another in the snitch, while it is alive, is > almost always the wrong thing to do. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10244) Replace heartbeats with locally recorded metrics for failure detection
Jason Brown created CASSANDRA-10244: --- Summary: Replace heartbeats with locally recorded metrics for failure detection Key: CASSANDRA-10244 URL: https://issues.apache.org/jira/browse/CASSANDRA-10244 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jason Brown Assignee: Jason Brown In the current implementation, the primary purpose of sending gossip messages is for delivering the updated heartbeat values of each node in a cluster. The other data that is passed in gossip (node metadata such as status, dc, rack, tokens, and so on) changes very infrequently (or rarely), such that the eventual delivery of that data is entirely reasonable. Heartbeats, however, are quite different. A continuous and nearly consistent delivery time of updated heartbeats is critical for the stability of a cluster. It is through the receipt of the updated heartbeat that a node determines the reachability (UP/DOWN status) of all peers in the cluster. The current implementation of FailureDetector measures the time differences between the heartbeat updates received about a peer (Note: I said about a peer, not from the peer directly, as those values are disseminated via gossip). Without a consistent time delivery of those updates, the FD, via it's use of the PHI-accrual algorigthm, will mark the peer as DOWN (unreachable). The two nodes could be sending all other traffic without problem, but if the heartbeats are not propagated correctly, each of the nodes will mark the other as DOWN, which is clearly suboptimal to cluster health. Further, heartbeat updates are the only mechanism we use to determine reachability (UP/DOWN) of a peer; dynamic snitch measurements, for example, are not included in the determination. To illustrate this, in the current implementation, assume a cluster of nodes: A, B, and C. A partition starts between nodes A and C (no communication succeeds), but both nodes can communicate with B. As B will get the updated heartbeats from both A and C, it will, via gossip, send those over to the other node. Thus, A thinks C is UP, and C thinks A is UP. Unfortunately, due to the partition between them, all communication between A and C will fail, yet neither node will mark the other as down because each is receiving, transitively via B, the updated heartbeat about the other. While it's true that the other node is alive, only having transitive knowledge about a peer, and allowing that to be the sole determinant of UP/DOWN reachability status, is not sufficient for a correct and effieicently operating cluster. This transitive availability is suboptimal, and I propose we drop the heartbeat concept altogether. Instead, the dynamic snitch should become more intelligent, and it's measurements ultimately become the input for determining the reachability status of each peer(as fed into a revamped FD). As we already capture latencies in the dsntich, we can reasonably extend it to include timeouts/missed responses, and make that the basis for the UP/DOWN decisioning. Thus we will have more accurate and relevant peer statueses that is tailored to the local node. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10243) Warn or fail when changing cluster topology live
Jonathan Ellis created CASSANDRA-10243: -- Summary: Warn or fail when changing cluster topology live Key: CASSANDRA-10243 URL: https://issues.apache.org/jira/browse/CASSANDRA-10243 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Jonathan Ellis Fix For: 2.1.x Moving a node from one rack to another in the snitch, while it is alive, is almost always the wrong thing to do. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10242) Validate rack information on startup
Jonathan Ellis created CASSANDRA-10242: -- Summary: Validate rack information on startup Key: CASSANDRA-10242 URL: https://issues.apache.org/jira/browse/CASSANDRA-10242 Project: Cassandra Issue Type: Improvement Reporter: Jonathan Ellis Fix For: 2.1.x Moving to a new rack means that different data should be stored on a node. We already persist rack information in a system table; we should fail startup if this doesn't match what the snitch thinks it should be. (Either the snitch is wrong, and needs to be fixed, or the machine has been moved and needs to be rebootstrapped.) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10241) Keep a separate production debug log for troubleshooting
[ https://issues.apache.org/jira/browse/CASSANDRA-10241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726138#comment-14726138 ] Jonathan Ellis commented on CASSANDRA-10241: Related: is it time to switch to AsyncAppender? > Keep a separate production debug log for troubleshooting > > > Key: CASSANDRA-10241 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10241 > Project: Cassandra > Issue Type: Bug > Components: Config >Reporter: Jonathan Ellis >Assignee: Brandon Williams > Fix For: 2.1.x > > > [~aweisberg] had the suggestion to keep a separate debug log for aid in > troubleshooting, not intended for regular human consumption but where we can > log things that might help if something goes wrong. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9738) Migrate key-cache to be fully off-heap
[ https://issues.apache.org/jira/browse/CASSANDRA-9738?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726135#comment-14726135 ] Robert Stupp commented on CASSANDRA-9738: - Pushed some more commits. Includes serialization of IndexInfo offsets. Adds tests for RowIndexEntry class and legacy sstables based on CASSANDRA-10236 (latter not working yet due to CASSANDRA-10237). Open todo: benchmarking/comparison against "old" implementation, verification of compatibility with legacy sstables. > Migrate key-cache to be fully off-heap > -- > > Key: CASSANDRA-9738 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9738 > Project: Cassandra > Issue Type: Sub-task >Reporter: Robert Stupp >Assignee: Robert Stupp > Fix For: 3.0 beta 2 > > > Key cache still uses a concurrent map on-heap. This could go to off-heap and > feels doable now after CASSANDRA-8099. > Evaluation should be done in advance based on a POC to prove that pure > off-heap counter cache buys a performance and/or gc-pressure improvement. > In theory, elimination of on-heap management of the map should buy us some > benefit. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10241) Keep a separate production debug log for troubleshooting
[ https://issues.apache.org/jira/browse/CASSANDRA-10241?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726132#comment-14726132 ] Jonathan Ellis commented on CASSANDRA-10241: We can't enable debug on the client request path for performance reasons, but we could enable it on the rest of the server, which is still very useful. I suggest these packages: * config * gms * repair * service * streaming WDYT [~brandon.williams] ? > Keep a separate production debug log for troubleshooting > > > Key: CASSANDRA-10241 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10241 > Project: Cassandra > Issue Type: Bug > Components: Config >Reporter: Jonathan Ellis >Assignee: Brandon Williams > Fix For: 2.1.x > > > [~aweisberg] had the suggestion to keep a separate debug log for aid in > troubleshooting, not intended for regular human consumption but where we can > log things that might help if something goes wrong. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10241) Keep a separate production debug log for troubleshooting
Jonathan Ellis created CASSANDRA-10241: -- Summary: Keep a separate production debug log for troubleshooting Key: CASSANDRA-10241 URL: https://issues.apache.org/jira/browse/CASSANDRA-10241 Project: Cassandra Issue Type: Bug Components: Config Reporter: Jonathan Ellis Assignee: Brandon Williams Fix For: 2.1.x [~aweisberg] had the suggestion to keep a separate debug log for aid in troubleshooting, not intended for regular human consumption but where we can log things that might help if something goes wrong. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/3] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Conflicts: conf/cassandra-env.ps1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/45eba1ad Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/45eba1ad Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/45eba1ad Branch: refs/heads/trunk Commit: 45eba1adbb9f2ec84c05b540bd20d044d046cbc6 Parents: b586868 3effec5 Author: Joshua McKenzie Authored: Tue Sep 1 16:29:39 2015 -0400 Committer: Joshua McKenzie Committed: Tue Sep 1 16:29:39 2015 -0400 -- conf/cassandra-env.ps1 | 4 1 file changed, 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/45eba1ad/conf/cassandra-env.ps1 -- diff --cc conf/cassandra-env.ps1 index 67767e7,fc0f6fd..f6bc6a7 --- a/conf/cassandra-env.ps1 +++ b/conf/cassandra-env.ps1 @@@ -347,8 -372,9 +347,6 @@@ Function SetCassandraEnvironmen # store in env to check if it's avail in verification $env:JMX_PORT=$JMX_PORT - $env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.defaultInitOverride=true" -# some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 -$env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled" -- # enable thread priorities, primarily so we can give periodic tasks # a lower priority to avoid interfering with client workload $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
[1/3] cassandra git commit: ninja: remove vestigial log4j from cassandra-env.ps1
Repository: cassandra Updated Branches: refs/heads/trunk 17f8788ef -> 0d2603cf9 ninja: remove vestigial log4j from cassandra-env.ps1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3effec53 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3effec53 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3effec53 Branch: refs/heads/trunk Commit: 3effec53e3d99e886266cf508ae00db1c869b29e Parents: 0a4ce44 Author: Joshua McKenzie Authored: Tue Sep 1 16:29:03 2015 -0400 Committer: Joshua McKenzie Committed: Tue Sep 1 16:29:03 2015 -0400 -- conf/cassandra-env.ps1 | 4 1 file changed, 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3effec53/conf/cassandra-env.ps1 -- diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1 index f6cd6bc..fc0f6fd 100644 --- a/conf/cassandra-env.ps1 +++ b/conf/cassandra-env.ps1 @@ -372,8 +372,6 @@ Function SetCassandraEnvironment # store in env to check if it's avail in verification $env:JMX_PORT=$JMX_PORT -$env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.defaultInitOverride=true" - # some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled" @@ -471,7 +469,5 @@ Function SetCassandraEnvironment $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS" -$env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.configuration=log4j-server.properties" - #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder" }
[3/3] cassandra git commit: Merge branch 'cassandra-3.0' into trunk
Merge branch 'cassandra-3.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0d2603cf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0d2603cf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0d2603cf Branch: refs/heads/trunk Commit: 0d2603cf99012171a3f8a0ee3219de88458db5ea Parents: 17f8788 45eba1a Author: Joshua McKenzie Authored: Tue Sep 1 16:29:56 2015 -0400 Committer: Joshua McKenzie Committed: Tue Sep 1 16:29:56 2015 -0400 -- conf/cassandra-env.ps1 | 4 1 file changed, 4 deletions(-) --
cassandra git commit: ninja: remove vestigial log4j from cassandra-env.ps1
Repository: cassandra Updated Branches: refs/heads/cassandra-2.2 0a4ce4449 -> 3effec53e ninja: remove vestigial log4j from cassandra-env.ps1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3effec53 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3effec53 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3effec53 Branch: refs/heads/cassandra-2.2 Commit: 3effec53e3d99e886266cf508ae00db1c869b29e Parents: 0a4ce44 Author: Joshua McKenzie Authored: Tue Sep 1 16:29:03 2015 -0400 Committer: Joshua McKenzie Committed: Tue Sep 1 16:29:03 2015 -0400 -- conf/cassandra-env.ps1 | 4 1 file changed, 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3effec53/conf/cassandra-env.ps1 -- diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1 index f6cd6bc..fc0f6fd 100644 --- a/conf/cassandra-env.ps1 +++ b/conf/cassandra-env.ps1 @@ -372,8 +372,6 @@ Function SetCassandraEnvironment # store in env to check if it's avail in verification $env:JMX_PORT=$JMX_PORT -$env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.defaultInitOverride=true" - # some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled" @@ -471,7 +469,5 @@ Function SetCassandraEnvironment $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS" -$env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.configuration=log4j-server.properties" - #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder" }
[2/2] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0
Merge branch 'cassandra-2.2' into cassandra-3.0 Conflicts: conf/cassandra-env.ps1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/45eba1ad Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/45eba1ad Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/45eba1ad Branch: refs/heads/cassandra-3.0 Commit: 45eba1adbb9f2ec84c05b540bd20d044d046cbc6 Parents: b586868 3effec5 Author: Joshua McKenzie Authored: Tue Sep 1 16:29:39 2015 -0400 Committer: Joshua McKenzie Committed: Tue Sep 1 16:29:39 2015 -0400 -- conf/cassandra-env.ps1 | 4 1 file changed, 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/45eba1ad/conf/cassandra-env.ps1 -- diff --cc conf/cassandra-env.ps1 index 67767e7,fc0f6fd..f6bc6a7 --- a/conf/cassandra-env.ps1 +++ b/conf/cassandra-env.ps1 @@@ -347,8 -372,9 +347,6 @@@ Function SetCassandraEnvironmen # store in env to check if it's avail in verification $env:JMX_PORT=$JMX_PORT - $env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.defaultInitOverride=true" -# some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 -$env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled" -- # enable thread priorities, primarily so we can give periodic tasks # a lower priority to avoid interfering with client workload $env:JVM_OPTS="$env:JVM_OPTS -XX:+UseThreadPriorities"
[1/2] cassandra git commit: ninja: remove vestigial log4j from cassandra-env.ps1
Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 b58686858 -> 45eba1adb ninja: remove vestigial log4j from cassandra-env.ps1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3effec53 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3effec53 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3effec53 Branch: refs/heads/cassandra-3.0 Commit: 3effec53e3d99e886266cf508ae00db1c869b29e Parents: 0a4ce44 Author: Joshua McKenzie Authored: Tue Sep 1 16:29:03 2015 -0400 Committer: Joshua McKenzie Committed: Tue Sep 1 16:29:03 2015 -0400 -- conf/cassandra-env.ps1 | 4 1 file changed, 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3effec53/conf/cassandra-env.ps1 -- diff --git a/conf/cassandra-env.ps1 b/conf/cassandra-env.ps1 index f6cd6bc..fc0f6fd 100644 --- a/conf/cassandra-env.ps1 +++ b/conf/cassandra-env.ps1 @@ -372,8 +372,6 @@ Function SetCassandraEnvironment # store in env to check if it's avail in verification $env:JMX_PORT=$JMX_PORT -$env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.defaultInitOverride=true" - # some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541 $env:JVM_OPTS="$env:JVM_OPTS -XX:+CMSClassUnloadingEnabled" @@ -471,7 +469,5 @@ Function SetCassandraEnvironment $env:JVM_OPTS="$env:JVM_OPTS $env:JVM_EXTRA_OPTS" -$env:JVM_OPTS = "$env:JVM_OPTS -Dlog4j.configuration=log4j-server.properties" - #$env:JVM_OPTS="$env:JVM_OPTS -XX:+UnlockCommercialFeatures -XX:+FlightRecorder" }
[jira] [Comment Edited] (CASSANDRA-10155) 2i key cache load fails
[ https://issues.apache.org/jira/browse/CASSANDRA-10155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726103#comment-14726103 ] Ariel Weisberg edited comment on CASSANDRA-10155 at 9/1/15 8:20 PM: [Have a 3.0 change for this.|https://github.com/apache/cassandra/compare/trunk...aweisberg:C-10155-3.0?expand=1] I need to make sure existing tests run, and then make sure all the new code is tested. My code for consuming reads during cache load is probably uncovered and there are dropped table/keyspace cases that need coverage. I also need to rebase it to 2.1 and then figure out the right way to merge forward. * There is now only one saved cache file for each kind of key (row, key, counter) * This means that loading of each kind of cache is single threaded. Previous code loaded caches synchronously at startup so it was probably single threaded anyways. * Loading is async in its own thread for each cache so key and row cache can load at the same time * Caches could load concurrently with other startup activities, but there didn't seem to be a desire to have that happen so startup blocks until they are loaded * Cache keys now contain the keyspace and CF name instead of UUID. There is no footprint change for on heap caches because they are POJOs pointing to a canonical object. Off heap will potentially see an increase in size and time since it will have to store and decode two strings instead of a UUID. * Schema now has a function that will look up a CF, even if it is an index, correctly. * Cache loading used to collect a future per cache entry in a list! For an off heap cache that is larger than the Java heap this is a problem. Modified to bound the number of pending reads and consume read results by loading them into the cache as they come in. * Promoted errors in loading caches to info level that were debug. was (Author: aweisberg): [Have a 3.0 change for this.|https://github.com/apache/cassandra/compare/trunk...aweisberg:C-10155-3.0?expand=1] I need to make sure existing tests run, and then make sure all the new code is tested. I need to rebase it to 2.1 and then figure out the right way to get to merge forward. * There is now only one saved cache file for each kind of key (row, key, counter) * This means that loading of each kind of cache is single threaded. Previous code loaded caches synchronously at startup so it was probably single threaded anyways. * Loading is async in its own thread for each cache so key and row cache can load at the same time * Caches could load concurrently with other startup activities, but there didn't seem to be a desire to have that happen so startup blocks until they are loaded * Cache keys now contain the keyspace and CF name instead of UUID. There is no footprint change for on heap caches because they are POJOs pointing to a canonical object. Off heap will potentially see an increase in size and time since it will have to store and decode two strings instead of a UUID. * Schema now has a function that will look up a CF, even if it is an index, correctly. * Cache loading used to collect a future per cache entry in a list! For an off heap cache that is larger than the Java heap this is a problem. Modified to bound the number of pending reads and consume read results by loading them into the cache as they come in. * Promoted errors in loading caches to info level that were debug. > 2i key cache load fails > --- > > Key: CASSANDRA-10155 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10155 > Project: Cassandra > Issue Type: Bug >Reporter: Robert Stupp >Assignee: Ariel Weisberg > Fix For: 3.0.0 rc1, 2.1.10, 2.2.2 > > > CASSANDRA-9265 changed how key cache content is serialized to disk. It uses > {{UUID cfId}} to generate the file path for each {{ColumnFamilyStore}}. > Since {{cfId}} of a secondary index is the same as for the base table, the > key-cache files for 2i's and the base are the same. This will/may lead to > deserialization failures on restart for tables with at least one 2i. > /cc [~aweisberg] [~danchia] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10155) 2i key cache load fails
[ https://issues.apache.org/jira/browse/CASSANDRA-10155?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14726103#comment-14726103 ] Ariel Weisberg commented on CASSANDRA-10155: [Have a 3.0 change for this.|https://github.com/apache/cassandra/compare/trunk...aweisberg:C-10155-3.0?expand=1] I need to make sure existing tests run, and then make sure all the new code is tested. I need to rebase it to 2.1 and then figure out the right way to get to merge forward. * There is now only one saved cache file for each kind of key (row, key, counter) * This means that loading of each kind of cache is single threaded. Previous code loaded caches synchronously at startup so it was probably single threaded anyways. * Loading is async in its own thread for each cache so key and row cache can load at the same time * Caches could load concurrently with other startup activities, but there didn't seem to be a desire to have that happen so startup blocks until they are loaded * Cache keys now contain the keyspace and CF name instead of UUID. There is no footprint change for on heap caches because they are POJOs pointing to a canonical object. Off heap will potentially see an increase in size and time since it will have to store and decode two strings instead of a UUID. * Schema now has a function that will look up a CF, even if it is an index, correctly. * Cache loading used to collect a future per cache entry in a list! For an off heap cache that is larger than the Java heap this is a problem. Modified to bound the number of pending reads and consume read results by loading them into the cache as they come in. * Promoted errors in loading caches to info level that were debug. > 2i key cache load fails > --- > > Key: CASSANDRA-10155 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10155 > Project: Cassandra > Issue Type: Bug >Reporter: Robert Stupp >Assignee: Ariel Weisberg > Fix For: 3.0.0 rc1, 2.1.10, 2.2.2 > > > CASSANDRA-9265 changed how key cache content is serialized to disk. It uses > {{UUID cfId}} to generate the file path for each {{ColumnFamilyStore}}. > Since {{cfId}} of a secondary index is the same as for the base table, the > key-cache files for 2i's and the base are the same. This will/may lead to > deserialization failures on restart for tables with at least one 2i. > /cc [~aweisberg] [~danchia] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10240) sstableexpiredblockers can throw FileNotFound exceptions
[ https://issues.apache.org/jira/browse/CASSANDRA-10240?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-10240: - Description: {noformat} Exception in thread "main" java.lang.RuntimeException: java.io.FileNotFoundException: /path/to/data/system/sstable_activity/system-sstable_activity-jb-85002-Data.db (No such file or directory) at org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:47) at org.apache.cassandra.io.util.CompressedPoolingSegmentedFile.createReader(CompressedPoolingSegmentedFile.java:48) at org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:39) at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1242) at org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) at org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) at org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:173) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1599) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1418) at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:330) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) at org.apache.cassandra.cql3.statements.SelectStatement.readLocally(SelectStatement.java:296) at org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:315) at org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:64) at org.apache.cassandra.cql3.QueryProcessor.processInternal(QueryProcessor.java:208) at org.apache.cassandra.db.SystemKeyspace.getSSTableReadMeter(SystemKeyspace.java:957) at org.apache.cassandra.io.sstable.SSTableReader.(SSTableReader.java:340) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:196) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:184) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:150) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:142) at org.apache.cassandra.tools.SSTableExpiredBlockers.main(SSTableExpiredBlockers.java:77) {noformat} It appears this may be caused by sstable_activity getting compacted away while this is running. I'm not sure if sstableexpiredblockers should just try again or what, but it should probably at least handle the exception. was: {noformat} Exception in thread "main" java.lang.RuntimeException: java.io.FileNotFoundException: /path/to/data/system/sstable_activity/system-sstable_activity-jb-85002-Data.db (No such file or directory) at org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:47) at org.apache.cassandra.io.util.CompressedPoolingSegmentedFile.createReader(CompressedPoolingSegmentedFile.java:48) at org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:39) at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1242) at org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) at org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) at org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:173) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1599) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1418) at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:330) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) at org.apache.cassandra.cql3.statements.SelectStatement.readLocally(SelectStatement.java:296) at org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:315) at org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:64) at org.apache.cassandra.c
[jira] [Created] (CASSANDRA-10240) sstableexpiredblockers can throw FileNotFound exceptions
Brandon Williams created CASSANDRA-10240: Summary: sstableexpiredblockers can throw FileNotFound exceptions Key: CASSANDRA-10240 URL: https://issues.apache.org/jira/browse/CASSANDRA-10240 Project: Cassandra Issue Type: Bug Components: Core Reporter: Brandon Williams Assignee: Marcus Eriksson Fix For: 2.1.x, 2.0.x, 2.2.x, 3.0.x {noformat} Exception in thread "main" java.lang.RuntimeException: java.io.FileNotFoundException: /path/to/data/system/sstable_activity/system-sstable_activity-jb-85002-Data.db (No such file or directory) at org.apache.cassandra.io.compress.CompressedRandomAccessReader.open(CompressedRandomAccessReader.java:47) at org.apache.cassandra.io.util.CompressedPoolingSegmentedFile.createReader(CompressedPoolingSegmentedFile.java:48) at org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:39) at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1242) at org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) at org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) at org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) at org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:173) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) at org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1599) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1418) at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:330) at org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) at org.apache.cassandra.cql3.statements.SelectStatement.readLocally(SelectStatement.java:296) at org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:315) at org.apache.cassandra.cql3.statements.SelectStatement.executeInternal(SelectStatement.java:64) at org.apache.cassandra.cql3.QueryProcessor.processInternal(QueryProcessor.java:208) at org.apache.cassandra.db.SystemKeyspace.getSSTableReadMeter(SystemKeyspace.java:957) at org.apache.cassandra.io.sstable.SSTableReader.(SSTableReader.java:340) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:196) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:184) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:150) at org.apache.cassandra.io.sstable.SSTableReader.open(SSTableReader.java:142) at org.apache.cassandra.tools.SSTableExpiredBlockers.main(SSTableExpiredBlockers.java:77) {noformat} It appears this may because by sstable_activity getting compacted away while this is running. I'm not sure if sstableexpiredblockers should just try again or what, but it should probably at least handle the exception. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10238) Consolidating racks violates the RF contract
[ https://issues.apache.org/jira/browse/CASSANDRA-10238?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-10238: --- Assignee: Russ Hatch (was: Rocco Varela) > Consolidating racks violates the RF contract > > > Key: CASSANDRA-10238 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10238 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Brandon Williams >Assignee: Russ Hatch > > I have only tested this on 2.0 so far, but I suspect it will affect multiple > versions. > Repro: > * create a datacenter with rf>1 > * create more than one rack in this datacenter > * consolidate these racks into 1 > * getendpoints will reveal the RF in practice is 1, even though other tools > will report the original RF that was set > Restarting Cassandra will resolve this. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8741) Running a drain before a decommission apparently the wrong thing to do
[ https://issues.apache.org/jira/browse/CASSANDRA-8741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14725939#comment-14725939 ] Jonathan Ellis commented on CASSANDRA-8741: --- [~Jan Karlsson] ? > Running a drain before a decommission apparently the wrong thing to do > -- > > Key: CASSANDRA-8741 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8741 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Ubuntu 14.04; Cassandra 2.0.11.82 (Datastax Enterprise > 4.5.3) >Reporter: Casey Marshall >Assignee: Jan Karlsson >Priority: Trivial > Labels: lhf > Fix For: 2.1.x, 2.0.x > > Attachments: 8741.txt > > > This might simply be a documentation issue. It appears that running "nodetool > drain" is a very wrong thing to do before running a "nodetool decommission". > The idea was that I was going to safely shut off writes and flush everything > to disk before beginning the decommission. What happens is the "decommission" > call appears to fail very early on after starting, and afterwards, the node > in question is stuck in state LEAVING, but all other nodes in the ring see > that node as NORMAL, but down. No streams are ever sent from the node being > decommissioned to other nodes. > The drain command does indeed shut down the "BatchlogTasks" executor > (org/apache/cassandra/service/StorageService.java, line 3445 in git tag > "cassandra-2.0.11") but the decommission process tries using that executor > when calling the "startBatchlogReplay" function > (org/apache/cassandra/db/BatchlogManager.java, line 123) called through > org.apache.cassandra.service.StorageService.unbootstrap (see the stack trace > pasted below). > This also failed in a similar way on Cassandra 1.2.13-ish (DSE 3.2.4). > So, either something is wrong with the drain/decommission commands, or it's > very wrong to run a drain before a decommission. What's worse, there seems to > be no way to recover this node once it is in this state; you need to shut it > down and run "removenode". > My terminal output: > {code} > ubuntu@x:~$ nodetool drain > ubuntu@x:~$ tail /var/log/^C > ubuntu@x:~$ nodetool decommission > Exception in thread "main" java.util.concurrent.RejectedExecutionException: > Task > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@3008fa33 > rejected from > org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor@1d6242e8[Terminated, > pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 52] > at > java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2048) > at > java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:821) > at > java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:325) > at > java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:530) > at > java.util.concurrent.ScheduledThreadPoolExecutor.submit(ScheduledThreadPoolExecutor.java:629) > at > org.apache.cassandra.db.BatchlogManager.startBatchlogReplay(BatchlogManager.java:123) > at > org.apache.cassandra.service.StorageService.unbootstrap(StorageService.java:2966) > at > org.apache.cassandra.service.StorageService.decommission(StorageService.java:2934) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:75) > at sun.reflect.GeneratedMethodAccessor11.invoke(Unknown Source) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:606) > at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:279) > at > com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) > at > com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) > at > com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) > at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) > at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) > at > com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) > at > com.sun.jmx.mbeanserver.JmxMBeanServer.invok
[1/2] cassandra git commit: Re-apply MV updates on commitlog replay
Repository: cassandra Updated Branches: refs/heads/trunk 78dcf79c7 -> 17f8788ef Re-apply MV updates on commitlog replay patch by tjake; reviewed by carlyeks for CASSANDRA-10164 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b5868685 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b5868685 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b5868685 Branch: refs/heads/trunk Commit: b58686858c632ed642ccf355f1f3a588e28b0e8a Parents: 9c02625 Author: T Jake Luciani Authored: Thu Aug 27 13:28:04 2015 -0400 Committer: T Jake Luciani Committed: Tue Sep 1 15:30:20 2015 -0400 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/Keyspace.java | 29 +++-- .../db/commitlog/CommitLogReplayer.java | 13 +++- .../db/view/MaterializedViewBuilder.java| 13 +--- .../db/view/MaterializedViewManager.java| 8 ++--- .../apache/cassandra/service/StorageProxy.java | 33 +++- .../cassandra/streaming/StreamReceiveTask.java | 8 +++-- .../cassandra/cql3/MaterializedViewTest.java| 26 +++ 8 files changed, 79 insertions(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b5868685/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 6539792..88b99a2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.0-beta2 + * Re-apply MaterializedView updates on commitlog replay (CASSANDRA-10164) * Require AbstractType.isByteOrderComparable declaration in constructor (CASSANDRA-9901) * Avoid digest mismatch on upgrade to 3.0 (CASSANDRA-9554) * Fix Materialized View builder when adding multiple MVs (CASSANDRA-10156) http://git-wip-us.apache.org/repos/asf/cassandra/blob/b5868685/src/java/org/apache/cassandra/db/Keyspace.java -- diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index f5a047f..981209c 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -386,7 +386,17 @@ public class Keyspace public void apply(Mutation mutation, boolean writeCommitLog) { -apply(mutation, writeCommitLog, true); +apply(mutation, writeCommitLog, true, false); +} + +public void apply(Mutation mutation, boolean writeCommitLog, boolean updateIndexes) +{ +apply(mutation, writeCommitLog, updateIndexes, false); +} + +public void applyFromCommitLog(Mutation mutation) +{ +apply(mutation, false, true, true); } /** @@ -396,8 +406,9 @@ public class Keyspace * may happen concurrently, depending on the CL Executor type. * @param writeCommitLog false to disable commitlog append entirely * @param updateIndexes false to disable index updates (used by CollationController "defragmenting") + * @param isClReplay true if caller is the commitlog replayer */ -public void apply(final Mutation mutation, final boolean writeCommitLog, boolean updateIndexes) +public void apply(final Mutation mutation, final boolean writeCommitLog, boolean updateIndexes, boolean isClReplay) { if (TEST_FAIL_WRITES && metadata.name.equals(TEST_FAIL_WRITES_KS)) throw new RuntimeException("Testing write failures"); @@ -456,15 +467,15 @@ public class Keyspace { try { -Tracing.trace("Create materialized view mutations from replica"); - cfs.materializedViewManager.pushViewReplicaUpdates(upd); +Tracing.trace("Creating materialized view mutations from base table replica"); + cfs.materializedViewManager.pushViewReplicaUpdates(upd, !isClReplay); } -catch (Exception e) +catch (Throwable t) { -if (!(e instanceof WriteTimeoutException)) -logger.warn("Encountered exception when creating materialized view mutations", e); - -JVMStabilityInspector.inspectThrowable(e); +JVMStabilityInspector.inspectThrowable(t); +logger.error(String.format("Unknown exception caught while attempting to update MaterializedView! %s.%s", + upd.metadata().ksName, upd.metadata().cfName), t); +throw t; } } http://git-wip-us.apache.org/repos/asf/cassandra/blob/b586
[2/2] cassandra git commit: Merge branch 'cassandra-3.0' into trunk
Merge branch 'cassandra-3.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/17f8788e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/17f8788e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/17f8788e Branch: refs/heads/trunk Commit: 17f8788ef80ff541f43141e0181efe9baea46db9 Parents: 78dcf79 b586868 Author: T Jake Luciani Authored: Tue Sep 1 15:31:27 2015 -0400 Committer: T Jake Luciani Committed: Tue Sep 1 15:31:27 2015 -0400 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/Keyspace.java | 29 +++-- .../db/commitlog/CommitLogReplayer.java | 13 +++- .../db/view/MaterializedViewBuilder.java| 13 +--- .../db/view/MaterializedViewManager.java| 8 ++--- .../apache/cassandra/service/StorageProxy.java | 33 +++- .../cassandra/streaming/StreamReceiveTask.java | 8 +++-- .../cassandra/cql3/MaterializedViewTest.java| 26 +++ 8 files changed, 79 insertions(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/17f8788e/CHANGES.txt -- diff --cc CHANGES.txt index 0c94103,88b99a2..9a4b48f --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,8 -1,5 +1,9 @@@ +3.2 + * Add transparent data encryption core classes (CASSANDRA-9945) + + 3.0.0-beta2 + * Re-apply MaterializedView updates on commitlog replay (CASSANDRA-10164) * Require AbstractType.isByteOrderComparable declaration in constructor (CASSANDRA-9901) * Avoid digest mismatch on upgrade to 3.0 (CASSANDRA-9554) * Fix Materialized View builder when adding multiple MVs (CASSANDRA-10156)
cassandra git commit: Re-apply MV updates on commitlog replay
Repository: cassandra Updated Branches: refs/heads/cassandra-3.0 9c0262529 -> b58686858 Re-apply MV updates on commitlog replay patch by tjake; reviewed by carlyeks for CASSANDRA-10164 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b5868685 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b5868685 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b5868685 Branch: refs/heads/cassandra-3.0 Commit: b58686858c632ed642ccf355f1f3a588e28b0e8a Parents: 9c02625 Author: T Jake Luciani Authored: Thu Aug 27 13:28:04 2015 -0400 Committer: T Jake Luciani Committed: Tue Sep 1 15:30:20 2015 -0400 -- CHANGES.txt | 1 + src/java/org/apache/cassandra/db/Keyspace.java | 29 +++-- .../db/commitlog/CommitLogReplayer.java | 13 +++- .../db/view/MaterializedViewBuilder.java| 13 +--- .../db/view/MaterializedViewManager.java| 8 ++--- .../apache/cassandra/service/StorageProxy.java | 33 +++- .../cassandra/streaming/StreamReceiveTask.java | 8 +++-- .../cassandra/cql3/MaterializedViewTest.java| 26 +++ 8 files changed, 79 insertions(+), 52 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b5868685/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 6539792..88b99a2 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0.0-beta2 + * Re-apply MaterializedView updates on commitlog replay (CASSANDRA-10164) * Require AbstractType.isByteOrderComparable declaration in constructor (CASSANDRA-9901) * Avoid digest mismatch on upgrade to 3.0 (CASSANDRA-9554) * Fix Materialized View builder when adding multiple MVs (CASSANDRA-10156) http://git-wip-us.apache.org/repos/asf/cassandra/blob/b5868685/src/java/org/apache/cassandra/db/Keyspace.java -- diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index f5a047f..981209c 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -386,7 +386,17 @@ public class Keyspace public void apply(Mutation mutation, boolean writeCommitLog) { -apply(mutation, writeCommitLog, true); +apply(mutation, writeCommitLog, true, false); +} + +public void apply(Mutation mutation, boolean writeCommitLog, boolean updateIndexes) +{ +apply(mutation, writeCommitLog, updateIndexes, false); +} + +public void applyFromCommitLog(Mutation mutation) +{ +apply(mutation, false, true, true); } /** @@ -396,8 +406,9 @@ public class Keyspace * may happen concurrently, depending on the CL Executor type. * @param writeCommitLog false to disable commitlog append entirely * @param updateIndexes false to disable index updates (used by CollationController "defragmenting") + * @param isClReplay true if caller is the commitlog replayer */ -public void apply(final Mutation mutation, final boolean writeCommitLog, boolean updateIndexes) +public void apply(final Mutation mutation, final boolean writeCommitLog, boolean updateIndexes, boolean isClReplay) { if (TEST_FAIL_WRITES && metadata.name.equals(TEST_FAIL_WRITES_KS)) throw new RuntimeException("Testing write failures"); @@ -456,15 +467,15 @@ public class Keyspace { try { -Tracing.trace("Create materialized view mutations from replica"); - cfs.materializedViewManager.pushViewReplicaUpdates(upd); +Tracing.trace("Creating materialized view mutations from base table replica"); + cfs.materializedViewManager.pushViewReplicaUpdates(upd, !isClReplay); } -catch (Exception e) +catch (Throwable t) { -if (!(e instanceof WriteTimeoutException)) -logger.warn("Encountered exception when creating materialized view mutations", e); - -JVMStabilityInspector.inspectThrowable(e); +JVMStabilityInspector.inspectThrowable(t); +logger.error(String.format("Unknown exception caught while attempting to update MaterializedView! %s.%s", + upd.metadata().ksName, upd.metadata().cfName), t); +throw t; } } http://git-wip-us.apache.org/repos/asf/cas
[jira] [Commented] (CASSANDRA-6237) Allow range deletions in CQL
[ https://issues.apache.org/jira/browse/CASSANDRA-6237?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14725919#comment-14725919 ] Joshua McKenzie commented on CASSANDRA-6237: Finished first pass of review and getting familiar with the code. On the whole, the refactors look really solid and the new functionality looks thoroughly implemented. For future reference: as a reviewer it would have been very helpful to have the StatementType / StatementRestrictions refactors mentioned in the comments above kept as discrete commits leading up to the work for Range deletions, rather than as one monolithic commit to review. Some feedback: * ModificationStatement.Parsed.prepareConditions is a little odd to me, as its implicit relationship between condition precedence is {{ifExists > ifNotExists > conditions.isEmpty() > columnConditions}}. We might want to consider Assertions in the Parsed ctor to communicate the intent of these various conditions on Parsed. At the least, we should preserve the comment that was in {{ModificationStatement.addConditions}} that used to explain this relationship. * In Operations.java, rename "applyToX" to "appliesToX" to better capture the intent of the method * Same with ModificationStatement.applyOnlyTo* * nit: on UpdateStatement.java:L219, revert change of "statements" to "statement". Should read "INSERT statements are not allowed" * AbstractConditions.java: remove redundant @Override on interface implementations as per [coding style|https://wiki.apache.org/cassandra/CodeStyle] * ColumnConditions.java: inconsistent paradigms between {{getColumns()}} and {{getFunctions()}} w/regards to building your Iterable/Collections * IfExistsCondition.java: Missing apache header * IfNotExistsCondition.java: Missing apache header * Json.java: redundant @Override on abstract implementations * CBuilder.java: In the two {{buildWith}}, there's no need to change to System.arraycopy as Arrays.copyOf uses that behind the scenes. * I'm hesitant about us crystallizing our unit tests around the exact verbiage of the error messages we're throwing (see: {{assertInvalidMessage}} being added into DeleteTest and InsertTest). Checking exception type alone doesn't buy us the 100% confirmation that the exception we think was raised was actually raised, however it allows for future modifications of the details of those exceptions without having to track down failed unit tests that are explicitly dependent upon them. I'm on the fence with this one; if you're strongly attached to the {{assertInvalidMessage}} approach, stick with it. * nit: Might want to wrap the new entries in NEWS.txt to try and keep them under 100 or so characters, wherever the other lines tend to fall. The new entries stick out as incongruous. I'll take some more time tomorrow to further consider the patch and also plan on taking a look at the rebased 3.0 branch when that becomes available. > Allow range deletions in CQL > > > Key: CASSANDRA-6237 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6237 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Assignee: Benjamin Lerer >Priority: Minor > Labels: cql, docs > Fix For: 3.0 beta 2 > > Attachments: CASSANDRA-6237.txt > > > We uses RangeTombstones internally in a number of places, but we could expose > more directly too. Typically, given a table like: > {noformat} > CREATE TABLE events ( > id text, > created_at timestamp, > content text, > PRIMARY KEY (id, created_at) > ) > {noformat} > we could allow queries like: > {noformat} > DELETE FROM events WHERE id='someEvent' AND created_at < 'Jan 3, 2013'; > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10144) CASSANDRA-8989 backported CASSANDRA-6863 but didn't include the fix from CASSANDRA-8013 in that backport
[ https://issues.apache.org/jira/browse/CASSANDRA-10144?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian updated CASSANDRA-10144: --- Reviewer: Tyler Hobbs (was: Sam Tunnicliffe) > CASSANDRA-8989 backported CASSANDRA-6863 but didn't include the fix from > CASSANDRA-8013 in that backport > > > Key: CASSANDRA-10144 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10144 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Jeremiah Jordan >Assignee: Carl Yeksigian > Fix For: 2.0.17 > > > CASSANDRA-8989 backported CASSANDRA-6863 but didn't include the fix from > CASSANDRA-8013 in that backport -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10239) Failure to launch on Windows with spaces in directory.
vyacheslav zaslavskiy created CASSANDRA-10239: - Summary: Failure to launch on Windows with spaces in directory. Key: CASSANDRA-10239 URL: https://issues.apache.org/jira/browse/CASSANDRA-10239 Project: Cassandra Issue Type: Bug Components: Packaging Environment: Windows Server 2012 Oracle JDK 1.7.79 Reporter: vyacheslav zaslavskiy Repro: Download cassandra 2.2.0 Extra to C:\Program Files Execute with cassandra, cassandra.bat, powershell /file cassandra.ps1 The only way I was able to get this to work was by setting contrast_home to Progra~1. I have seen multiple resolved tickets for similar issues, but that doesn't appear to be the case. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10239) Failure to launch on Windows with spaces in directory.
[ https://issues.apache.org/jira/browse/CASSANDRA-10239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] vyacheslav zaslavskiy updated CASSANDRA-10239: -- Reproduced In: 2.2.0 > Failure to launch on Windows with spaces in directory. > -- > > Key: CASSANDRA-10239 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10239 > Project: Cassandra > Issue Type: Bug > Components: Packaging > Environment: Windows Server 2012 > Oracle JDK 1.7.79 >Reporter: vyacheslav zaslavskiy > > Repro: > Download cassandra 2.2.0 > Extra to C:\Program Files > Execute with cassandra, cassandra.bat, powershell /file cassandra.ps1 > The only way I was able to get this to work was by setting contrast_home to > Progra~1. I have seen multiple resolved tickets for similar issues, but that > doesn't appear to be the case. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10144) CASSANDRA-8989 backported CASSANDRA-6863 but didn't include the fix from CASSANDRA-8013 in that backport
[ https://issues.apache.org/jira/browse/CASSANDRA-10144?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14725883#comment-14725883 ] Carl Yeksigian commented on CASSANDRA-10144: Just pushed a new version which uses the patch as applied (instead of the patch posted on the ticket). > CASSANDRA-8989 backported CASSANDRA-6863 but didn't include the fix from > CASSANDRA-8013 in that backport > > > Key: CASSANDRA-10144 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10144 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Jeremiah Jordan >Assignee: Carl Yeksigian > Fix For: 2.0.17 > > > CASSANDRA-8989 backported CASSANDRA-6863 but didn't include the fix from > CASSANDRA-8013 in that backport -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10232) Small optimizations in index entry serialization
[ https://issues.apache.org/jira/browse/CASSANDRA-10232?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14725880#comment-14725880 ] Ariel Weisberg commented on CASSANDRA-10232: Where do we test that we can still load the old version of the serialization? I am having some conversations with Robert that make me nervous about stuff like https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/db/Serializers.java#L51 > Small optimizations in index entry serialization > > > Key: CASSANDRA-10232 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10232 > Project: Cassandra > Issue Type: Improvement >Reporter: Sylvain Lebresne >Assignee: Sylvain Lebresne > Fix For: 3.0.0 rc1 > > > While we should improve the data structure we use for our on-disk index in > future versions, it occurred to me that we had a few _very_ low hanging fruit > optimization (as in, for 3.0) we could do for the serialization of our > current entries, like using vint encodings. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-9629) Error when reading date after altering column
[ https://issues.apache.org/jira/browse/CASSANDRA-9629?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Carl Yeksigian resolved CASSANDRA-9629. --- Resolution: Not A Problem Assignee: (was: Carl Yeksigian) Fix Version/s: (was: 2.1.x) This is caused by a driver issue, not a Cassandra issue. See [JAVA-420|https://datastax-oss.atlassian.net/browse/JAVA-420]. > Error when reading date after altering column > - > > Key: CASSANDRA-9629 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9629 > Project: Cassandra > Issue Type: Bug >Reporter: Tomasz Bartczak > > Having a table with timestamp column and a prepared statement for it - after > altering the table with new column - reading date values are broken. > {code} > com.datastax.driver.core.exceptions.InvalidTypeException: Invalid 64-bits > long value, expecting 8 bytes but got 1 > at > com.datastax.driver.core.TypeCodec$LongCodec.deserializeNoBoxing(TypeCodec.java:452) > at > com.datastax.driver.core.TypeCodec$DateCodec.deserialize(TypeCodec.java:826) > at > com.datastax.driver.core.AbstractGettableByIndexData.getDate(AbstractGettableByIndexData.java:155) > at > com.datastax.driver.core.AbstractGettableData.getDate(AbstractGettableData.java:26) > at > com.datastax.driver.core.AbstractGettableData.getDate(AbstractGettableData.java:99) > at > CassandraAlterTable$delayedInit$body.apply(CassandraAlterTable.scala:19) > at scala.Function0$class.apply$mcV$sp(Function0.scala:40) > at > scala.runtime.AbstractFunction0.apply$mcV$sp(AbstractFunction0.scala:12) > at scala.App$$anonfun$main$1.apply(App.scala:71) > at scala.App$$anonfun$main$1.apply(App.scala:71) > at scala.collection.immutable.List.foreach(List.scala:318) > at > scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:32) > at scala.App$class.main(App.scala:71) > at CassandraAlterTable$.main(CassandraAlterTable.scala:8) > at CassandraAlterTable.main(CassandraAlterTable.scala) > at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) > at > sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) > at > sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) > at java.lang.reflect.Method.invoke(Method.java:497) > {code} > This bug is reproduced in : https://github.com/kretes/cassandra-alter -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-10238) Consolidating racks violates the RF contract
[ https://issues.apache.org/jira/browse/CASSANDRA-10238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14725864#comment-14725864 ] Rocco Varela commented on CASSANDRA-10238: -- I reproduced the RF inconsistency in Cassandra 2.1.8.689. *Cluster Setup:* # 3 nodes # GossipingPropertyFileSnitch # 1 DC, 3 racks (rack0, rack1, rack2), rack numbers initially match node numbers # Cassandra 2.1.8.689 # Used cassandra-stress for data load, with modification for specifying DTCS (probably not needed) *Steps:* # create 1 datacenter, and KS with RF=3 (using DTCS) # created 3 racks during setup # consolidate these racks into 1, did this in phases, first collapsing rack on node0 (rack0->rack1), then rack on node2 (rack2->rack1) # getendpoints reveals different RF on each node, see output below *Output:* Before collapsing racks: node0 {code} automaton@i-a0cf954d9:~$ nodetool getendpoints "keyspace1" "standard1" 36F41194 10.240.22.34 10.240.132.129 10.240.232.159 {code} *After collapsing racks:* node0 {code} automaton@i-a0cf954d9:~$ nodetool getendpoints "keyspace1" "standard1" 36F41194 10.240.22.34 10.240.132.129 {code} node1 {code} automaton@i-8c39d23e6:~$ nodetool getendpoints "keyspace1" "standard1" 36F41194 10.240.22.34 10.240.132.129 10.240.232.159 {code} node2 {code} automaton@i-9d29327c8:~$ nodetool getendpoints "keyspace1" "standard1" 36F41194 10.240.22.34 {code} Description of keyspace before and after collapsing racks: {code} cqlsh> desc keyspace keyspace1; CREATE KEYSPACE keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'DC1': '3'} AND durable_writes = true; CREATE TABLE keyspace1.standard1 ( key blob PRIMARY KEY, "C0" blob, "C1" blob, "C2" blob, "C3" blob, "C4" blob )... {code} > Consolidating racks violates the RF contract > > > Key: CASSANDRA-10238 > URL: https://issues.apache.org/jira/browse/CASSANDRA-10238 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Brandon Williams >Assignee: Rocco Varela > > I have only tested this on 2.0 so far, but I suspect it will affect multiple > versions. > Repro: > * create a datacenter with rf>1 > * create more than one rack in this datacenter > * consolidate these racks into 1 > * getendpoints will reveal the RF in practice is 1, even though other tools > will report the original RF that was set > Restarting Cassandra will resolve this. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-7392) Abort in-progress queries that time out
[ https://issues.apache.org/jira/browse/CASSANDRA-7392?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-7392: -- Priority: Critical (was: Major) > Abort in-progress queries that time out > --- > > Key: CASSANDRA-7392 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7392 > Project: Cassandra > Issue Type: New Feature > Components: Core >Reporter: Jonathan Ellis >Assignee: Stefania >Priority: Critical > Fix For: 3.x > > > Currently we drop queries that time out before we get to them (because node > is overloaded) but not queries that time out while being processed. > (Particularly common for index queries on data that shouldn't be indexed.) > Adding the latter and logging when we have to interrupt one gets us a poor > man's "slow query log" for free. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9893) Fix upgrade tests from #9704 that are still failing
[ https://issues.apache.org/jira/browse/CASSANDRA-9893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14725658#comment-14725658 ] Blake Eggleston edited comment on CASSANDRA-9893 at 9/1/15 5:46 PM: CASSANDRA-9554 fixed {{edge_2i_on_complex_pk_test}} and {{indexed_with_eq_test}}. {{composite_index_collections_test}} is still failing for me on cassandra-3.0 though. Here's the output: {noformat} == FAIL: composite_index_collections_test (cql_tests.TestCQL) -- Traceback (most recent call last): File "/Users/beggleston/code/cassandra-dtest/dtest.py", line 632, in wrapped f(obj) File "/Users/beggleston/code/cassandra-dtest/upgrade_tests/cql_tests.py", line 2339, in composite_index_collections_test assert rows_to_list(res) == [[1, set(['bar1', 'bar2'])], [1, set(['bar2', 'bar3'])], [2, set(['baz'])]], res AssertionError: [Row(blog_id=1, content=None), Row(blog_id=1, content=sortedset([u'bar1', u'bar2'])), Row(blog_id=1, content=sortedset([u'bar2', u'bar3'])), Row(blog_id=2, content=sortedset([u'baz']))] >> begin captured stdout << - [node1 ERROR] objc[81871]: Class JavaLaunchHelper is implemented in both /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/bin/java and /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/jre/lib/libinstrument.dylib. One of the two will be used. Which one is undefined. [node2 ERROR] objc[81870]: Class JavaLaunchHelper is implemented in both /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/bin/java and /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/jre/lib/libinstrument.dylib. One of the two will be used. Which one is undefined. - >> end captured stdout << -- >> begin captured logging << dtest: DEBUG: cluster ccm directory: /var/folders/kb/n2400lpj1rd5q1gx6pfj81tcgn/T/dtest-7OaTmZ dtest: DEBUG: removing ccm cluster test at: /var/folders/kb/n2400lpj1rd5q1gx6pfj81tcgn/T/dtest-7OaTmZ dtest: DEBUG: clearing ssl stores from [/var/folders/kb/n2400lpj1rd5q1gx6pfj81tcgn/T/dtest-7OaTmZ] directory dtest: DEBUG: cluster ccm directory: /var/folders/kb/n2400lpj1rd5q1gx6pfj81tcgn/T/dtest-SLY5pv dtest: DEBUG: Querying upgraded node - >> end captured logging << - -- Ran 1 test in 32.667s FAILED (failures=1) {noformat} was (Author: bdeggleston): Something committed in the last few days has fixed {{edge_2i_on_complex_pk_test}} and {{indexed_with_eq_test}}. {{composite_index_collections_test}} is still failing for me on cassandra-3.0 though. Here's the output: {noformat} == FAIL: composite_index_collections_test (cql_tests.TestCQL) -- Traceback (most recent call last): File "/Users/beggleston/code/cassandra-dtest/dtest.py", line 632, in wrapped f(obj) File "/Users/beggleston/code/cassandra-dtest/upgrade_tests/cql_tests.py", line 2339, in composite_index_collections_test assert rows_to_list(res) == [[1, set(['bar1', 'bar2'])], [1, set(['bar2', 'bar3'])], [2, set(['baz'])]], res AssertionError: [Row(blog_id=1, content=None), Row(blog_id=1, content=sortedset([u'bar1', u'bar2'])), Row(blog_id=1, content=sortedset([u'bar2', u'bar3'])), Row(blog_id=2, content=sortedset([u'baz']))] >> begin captured stdout << - [node1 ERROR] objc[81871]: Class JavaLaunchHelper is implemented in both /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/bin/java and /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/jre/lib/libinstrument.dylib. One of the two will be used. Which one is undefined. [node2 ERROR] objc[81870]: Class JavaLaunchHelper is implemented in both /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/bin/java and /Library/Java/JavaVirtualMachines/jdk1.8.0_40.jdk/Contents/Home/jre/lib/libinstrument.dylib. One of the two will be used. Which one is undefined. - >> end captured stdout << -- >> begin captured logging << dtest: DEBUG: cluster ccm directory: /var/folders/kb/n2400lpj1rd5q1gx6pfj81tcgn/T/dtest-7OaTmZ dtest: DEBUG: removing ccm cluster test at: /var/folders/kb/n2400lpj1rd5q1gx6pfj81tcgn/T/dtest-7OaTmZ dtest: DEBUG: clearing ssl stores from [/var/folders/kb/n2400lpj1rd5q1gx6pfj81tcgn/T/dtest-7OaTmZ] directory dtest: DEBUG: cluster ccm directory: /var/folders/kb/n2400lpj1rd5q1gx6pfj81tc00