[jira] [Commented] (CASSANDRA-10540) RangeAwareCompaction

2018-08-22 Thread Marcus Eriksson (JIRA)


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

Marcus Eriksson commented on CASSANDRA-10540:
-

bq. if that's all it needed we could of had them written by now. 
well writing tests will most likely require refactoring and fixing issues 
found, so it is most likely not all that is needed 

that is the latest branch, correct

> RangeAwareCompaction
> 
>
> Key: CASSANDRA-10540
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10540
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Major
>  Labels: 4.0-feature-freeze-review-requested, compaction, lcs, 
> vnodes
> Fix For: 4.x
>
>
> Broken out from CASSANDRA-6696, we should split sstables based on ranges 
> during compaction.
> Requirements;
> * dont create tiny sstables - keep them bunched together until a single vnode 
> is big enough (configurable how big that is)
> * make it possible to run existing compaction strategies on the per-range 
> sstables
> We should probably add a global compaction strategy parameter that states 
> whether this should be enabled or not.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14358) OutboundTcpConnection can hang for many minutes when nodes restart

2018-08-22 Thread Joseph Lynch (JIRA)


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

Joseph Lynch updated CASSANDRA-14358:
-
Reproduced In: 3.0.15, 2.1.19  (was: 2.1.19, 3.0.15)
   Status: Patch Available  (was: In Progress)

> OutboundTcpConnection can hang for many minutes when nodes restart
> --
>
> Key: CASSANDRA-14358
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14358
> Project: Cassandra
>  Issue Type: Bug
>  Components: Streaming and Messaging
> Environment: Cassandra 2.1.19 (also reproduced on 3.0.15), running 
> with {{internode_encryption: all}} and the EC2 multi region snitch on Linux 
> 4.13 within the same AWS region. Smallest cluster I've seen the problem on is 
> 12 nodes, reproduces more reliably on 40+ and 300 node clusters consistently 
> reproduce on at least one node in the cluster.
> So all the connections are SSL and we're connecting on the internal ip 
> addresses (not the public endpoint ones).
> Potentially relevant sysctls:
> {noformat}
> /proc/sys/net/ipv4/tcp_syn_retries = 2
> /proc/sys/net/ipv4/tcp_synack_retries = 5
> /proc/sys/net/ipv4/tcp_keepalive_time = 7200
> /proc/sys/net/ipv4/tcp_keepalive_probes = 9
> /proc/sys/net/ipv4/tcp_keepalive_intvl = 75
> /proc/sys/net/ipv4/tcp_retries2 = 15
> {noformat}
>Reporter: Joseph Lynch
>Assignee: Joseph Lynch
>Priority: Major
>  Labels: 4.0-feature-freeze-review-requested
> Fix For: 2.1.x, 2.2.x, 3.0.x, 3.11.x
>
> Attachments: 10 Minute Partition.pdf
>
>
> edit summary: This primarily impacts networks with stateful firewalls such as 
> AWS. I'm working on a proper patch for trunk but unfortunately it relies on 
> the Netty refactor in 4.0 so it will be hard to backport to previous 
> versions. A workaround for earlier versions is to set the 
> {{net.ipv4.tcp_retries2}} sysctl to ~5. This can be done with the following:
> {code:java}
> $ cat /etc/sysctl.d/20-cassandra-tuning.conf
> net.ipv4.tcp_retries2=5
> $ # Reload all sysctls
> $ sysctl --system{code}
> Original Bug Report:
> I've been trying to debug nodes not being able to see each other during 
> longer (~5 minute+) Cassandra restarts in 3.0.x and 2.1.x which can 
> contribute to {{UnavailableExceptions}} during rolling restarts of 3.0.x and 
> 2.1.x clusters for us. I think I finally have a lead. It appears that prior 
> to trunk (with the awesome Netty refactor) we do not set socket connect 
> timeouts on SSL connections (in 2.1.x, 3.0.x, or 3.11.x) nor do we set 
> {{SO_TIMEOUT}} as far as I can tell on outbound connections either. I believe 
> that this means that we could potentially block forever on {{connect}} or 
> {{recv}} syscalls, and we could block forever on the SSL Handshake as well. I 
> think that the OS will protect us somewhat (and that may be what's causing 
> the eventual timeout) but I think that given the right network conditions our 
> {{OutboundTCPConnection}} threads can just be stuck never making any progress 
> until the OS intervenes.
> I have attached some logs of such a network partition during a rolling 
> restart where an old node in the cluster has a completely foobarred 
> {{OutboundTcpConnection}} for ~10 minutes before finally getting a 
> {{java.net.SocketException: Connection timed out (Write failed)}} and 
> immediately successfully reconnecting. I conclude that the old node is the 
> problem because the new node (the one that restarted) is sending ECHOs to the 
> old node, and the old node is sending ECHOs and REQUEST_RESPONSES to the new 
> node's ECHOs, but the new node is never getting the ECHO's. This appears, to 
> me, to indicate that the old node's {{OutboundTcpConnection}} thread is just 
> stuck and can't make any forward progress. By the time we could notice this 
> and slap TRACE logging on, the only thing we see is ~10 minutes later a 
> {{SocketException}} inside {{writeConnected}}'s flush and an immediate 
> recovery. It is interesting to me that the exception happens in 
> {{writeConnected}} and it's a _connection timeout_ (and since we see {{Write 
> failure}} I believe that this can't be a connection reset), because my 
> understanding is that we should have a fully handshaked SSL connection at 
> that point in the code.
> Current theory:
>  # "New" node restarts,  "Old" node calls 
> [newSocket|https://github.com/apache/cassandra/blob/6f30677b28dcbf82bcd0a291f3294ddf87dafaac/src/java/org/apache/cassandra/net/OutboundTcpConnection.java#L433]
>  # Old node starts [creating a 
> new|https://github.com/apache/cassandra/blob/6f30677b28dcbf82bcd0a291f3294ddf87dafaac/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java#L141]
>  SSL socket 
>  # SSLSocket calls 
> 

[jira] [Updated] (CASSANDRA-14358) OutboundTcpConnection can hang for many minutes when nodes restart

2018-08-22 Thread Joseph Lynch (JIRA)


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

Joseph Lynch updated CASSANDRA-14358:
-
Fix Version/s: 4.0

> OutboundTcpConnection can hang for many minutes when nodes restart
> --
>
> Key: CASSANDRA-14358
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14358
> Project: Cassandra
>  Issue Type: Bug
>  Components: Streaming and Messaging
> Environment: Cassandra 2.1.19 (also reproduced on 3.0.15), running 
> with {{internode_encryption: all}} and the EC2 multi region snitch on Linux 
> 4.13 within the same AWS region. Smallest cluster I've seen the problem on is 
> 12 nodes, reproduces more reliably on 40+ and 300 node clusters consistently 
> reproduce on at least one node in the cluster.
> So all the connections are SSL and we're connecting on the internal ip 
> addresses (not the public endpoint ones).
> Potentially relevant sysctls:
> {noformat}
> /proc/sys/net/ipv4/tcp_syn_retries = 2
> /proc/sys/net/ipv4/tcp_synack_retries = 5
> /proc/sys/net/ipv4/tcp_keepalive_time = 7200
> /proc/sys/net/ipv4/tcp_keepalive_probes = 9
> /proc/sys/net/ipv4/tcp_keepalive_intvl = 75
> /proc/sys/net/ipv4/tcp_retries2 = 15
> {noformat}
>Reporter: Joseph Lynch
>Assignee: Joseph Lynch
>Priority: Major
>  Labels: 4.0-feature-freeze-review-requested
> Fix For: 4.0, 2.1.x, 2.2.x, 3.0.x, 3.11.x
>
> Attachments: 10 Minute Partition.pdf
>
>
> edit summary: This primarily impacts networks with stateful firewalls such as 
> AWS. I'm working on a proper patch for trunk but unfortunately it relies on 
> the Netty refactor in 4.0 so it will be hard to backport to previous 
> versions. A workaround for earlier versions is to set the 
> {{net.ipv4.tcp_retries2}} sysctl to ~5. This can be done with the following:
> {code:java}
> $ cat /etc/sysctl.d/20-cassandra-tuning.conf
> net.ipv4.tcp_retries2=5
> $ # Reload all sysctls
> $ sysctl --system{code}
> Original Bug Report:
> I've been trying to debug nodes not being able to see each other during 
> longer (~5 minute+) Cassandra restarts in 3.0.x and 2.1.x which can 
> contribute to {{UnavailableExceptions}} during rolling restarts of 3.0.x and 
> 2.1.x clusters for us. I think I finally have a lead. It appears that prior 
> to trunk (with the awesome Netty refactor) we do not set socket connect 
> timeouts on SSL connections (in 2.1.x, 3.0.x, or 3.11.x) nor do we set 
> {{SO_TIMEOUT}} as far as I can tell on outbound connections either. I believe 
> that this means that we could potentially block forever on {{connect}} or 
> {{recv}} syscalls, and we could block forever on the SSL Handshake as well. I 
> think that the OS will protect us somewhat (and that may be what's causing 
> the eventual timeout) but I think that given the right network conditions our 
> {{OutboundTCPConnection}} threads can just be stuck never making any progress 
> until the OS intervenes.
> I have attached some logs of such a network partition during a rolling 
> restart where an old node in the cluster has a completely foobarred 
> {{OutboundTcpConnection}} for ~10 minutes before finally getting a 
> {{java.net.SocketException: Connection timed out (Write failed)}} and 
> immediately successfully reconnecting. I conclude that the old node is the 
> problem because the new node (the one that restarted) is sending ECHOs to the 
> old node, and the old node is sending ECHOs and REQUEST_RESPONSES to the new 
> node's ECHOs, but the new node is never getting the ECHO's. This appears, to 
> me, to indicate that the old node's {{OutboundTcpConnection}} thread is just 
> stuck and can't make any forward progress. By the time we could notice this 
> and slap TRACE logging on, the only thing we see is ~10 minutes later a 
> {{SocketException}} inside {{writeConnected}}'s flush and an immediate 
> recovery. It is interesting to me that the exception happens in 
> {{writeConnected}} and it's a _connection timeout_ (and since we see {{Write 
> failure}} I believe that this can't be a connection reset), because my 
> understanding is that we should have a fully handshaked SSL connection at 
> that point in the code.
> Current theory:
>  # "New" node restarts,  "Old" node calls 
> [newSocket|https://github.com/apache/cassandra/blob/6f30677b28dcbf82bcd0a291f3294ddf87dafaac/src/java/org/apache/cassandra/net/OutboundTcpConnection.java#L433]
>  # Old node starts [creating a 
> new|https://github.com/apache/cassandra/blob/6f30677b28dcbf82bcd0a291f3294ddf87dafaac/src/java/org/apache/cassandra/net/OutboundTcpConnectionPool.java#L141]
>  SSL socket 
>  # SSLSocket calls 
> [createSocket|https://github.com/apache/cassandra/blob/cassandra-3.11/src/java/org/apache/cassandra/security/SSLFactory.java#L98],
>  

[jira] [Commented] (CASSANDRA-14358) OutboundTcpConnection can hang for many minutes when nodes restart

2018-08-22 Thread Joseph Lynch (JIRA)


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

Joseph Lynch commented on CASSANDRA-14358:
--

[~khuizhang] yea that looks similar with the half open keepalive connections. 
Did you try the kernel workaround and did it help?

[~aweisberg] [~jasobrown] I've got a mitigation patch so that Cassandra trunk 
at least heals the half open partitions faster. Please take a look if you have 
bandwidth for review. While testing the re-connection behavior I ran into 
CASSANDRA-14503 because the retry future was just getting clobbered by another 
message, so I couldn't test that we don't keep retrying after just connections 
are killed (as right now they just retry every message).
||trunk||
|[patch|https://github.com/apache/cassandra/compare/trunk...jolynch:CASSANDRA-14358]|
|[unit 
tests|https://circleci.com/gh/jolynch/workflows/cassandra/tree/CASSANDRA-14358]|

This patch makes the timeout configurable for internode connection (2s) and 
internode tcp user timeout (30s). The timeouts are settable via JMX (and 
nodetool).

I'm marking this as patch available as I think the operating system workaround 
is probably ok for previous releases? If it's not just let me know and I can 
try to figure out how to fix it for those ones as well.

I didn't have any new tests because the only way I'm aware to reproduce this 
behavior is by using iptables to blackhole traffic. I've been testing with a 
ccm cluster when i disable gossip (with 3 nodes the gossip is so frequent that 
we mark the node as DOWN very fast):
{noformat}
$ netstat -on | grep 7000 | grep 127.0.0.3:7000 | grep -v "0 127.0.0.3"
tcp0  0 127.0.0.1:55604 127.0.0.3:7000  ESTABLISHED 
keepalive (4093.37/0/0)
tcp0  0 127.0.0.1:55610 127.0.0.3:7000  ESTABLISHED 
keepalive (4093.53/0/0)
tcp0  0 127.0.0.1:58080 127.0.0.3:7000  ESTABLISHED 
keepalive (6601.96/0/0)

$ sudo iptables -A OUTPUT -p tcp -d 127.0.0.3 --dport 7000 --sport 55604 -j DROP
$ sudo iptables -A OUTPUT -p tcp -d 127.0.0.3 --dport 7000 --sport 55610 -j DROP
$ sudo iptables -A OUTPUT -p tcp -d 127.0.0.3 --dport 7000 --sport 58080 -j DROP
{noformat}
Then I just check that we properly reconnect. If you have ideas for how to unit 
test it I'm open to suggestions of course.

> OutboundTcpConnection can hang for many minutes when nodes restart
> --
>
> Key: CASSANDRA-14358
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14358
> Project: Cassandra
>  Issue Type: Bug
>  Components: Streaming and Messaging
> Environment: Cassandra 2.1.19 (also reproduced on 3.0.15), running 
> with {{internode_encryption: all}} and the EC2 multi region snitch on Linux 
> 4.13 within the same AWS region. Smallest cluster I've seen the problem on is 
> 12 nodes, reproduces more reliably on 40+ and 300 node clusters consistently 
> reproduce on at least one node in the cluster.
> So all the connections are SSL and we're connecting on the internal ip 
> addresses (not the public endpoint ones).
> Potentially relevant sysctls:
> {noformat}
> /proc/sys/net/ipv4/tcp_syn_retries = 2
> /proc/sys/net/ipv4/tcp_synack_retries = 5
> /proc/sys/net/ipv4/tcp_keepalive_time = 7200
> /proc/sys/net/ipv4/tcp_keepalive_probes = 9
> /proc/sys/net/ipv4/tcp_keepalive_intvl = 75
> /proc/sys/net/ipv4/tcp_retries2 = 15
> {noformat}
>Reporter: Joseph Lynch
>Assignee: Joseph Lynch
>Priority: Major
>  Labels: 4.0-feature-freeze-review-requested
> Fix For: 2.1.x, 2.2.x, 3.0.x, 3.11.x
>
> Attachments: 10 Minute Partition.pdf
>
>
> edit summary: This primarily impacts networks with stateful firewalls such as 
> AWS. I'm working on a proper patch for trunk but unfortunately it relies on 
> the Netty refactor in 4.0 so it will be hard to backport to previous 
> versions. A workaround for earlier versions is to set the 
> {{net.ipv4.tcp_retries2}} sysctl to ~5. This can be done with the following:
> {code:java}
> $ cat /etc/sysctl.d/20-cassandra-tuning.conf
> net.ipv4.tcp_retries2=5
> $ # Reload all sysctls
> $ sysctl --system{code}
> Original Bug Report:
> I've been trying to debug nodes not being able to see each other during 
> longer (~5 minute+) Cassandra restarts in 3.0.x and 2.1.x which can 
> contribute to {{UnavailableExceptions}} during rolling restarts of 3.0.x and 
> 2.1.x clusters for us. I think I finally have a lead. It appears that prior 
> to trunk (with the awesome Netty refactor) we do not set socket connect 
> timeouts on SSL connections (in 2.1.x, 3.0.x, or 3.11.x) nor do we set 
> {{SO_TIMEOUT}} as far as I can tell on outbound connections either. I believe 
> that this means that we could potentially block 

[jira] [Commented] (CASSANDRA-10540) RangeAwareCompaction

2018-08-22 Thread Kurt Greaves (JIRA)


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

Kurt Greaves commented on CASSANDRA-10540:
--

We've been asking to help for months now, if that's all it needed we could of 
had them written by now.  
[This|https://github.com/apache/cassandra/compare/trunk...krummas:marcuse/10540]
 is your latest branch correct? I'll start looking at writing tests, although 
there's minimal hope of having anything ready by the first.

> RangeAwareCompaction
> 
>
> Key: CASSANDRA-10540
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10540
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Major
>  Labels: 4.0-feature-freeze-review-requested, compaction, lcs, 
> vnodes
> Fix For: 4.x
>
>
> Broken out from CASSANDRA-6696, we should split sstables based on ranges 
> during compaction.
> Requirements;
> * dont create tiny sstables - keep them bunched together until a single vnode 
> is big enough (configurable how big that is)
> * make it possible to run existing compaction strategies on the per-range 
> sstables
> We should probably add a global compaction strategy parameter that states 
> whether this should be enabled or not.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-10726) Read repair inserts should not be blocking

2018-08-22 Thread Kurt Greaves (JIRA)


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

Kurt Greaves commented on CASSANDRA-10726:
--

Great, thanks Blake. I just needed a quick fix to get my tests running :).

> Read repair inserts should not be blocking
> --
>
> Key: CASSANDRA-10726
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10726
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
>Reporter: Richard Low
>Assignee: Blake Eggleston
>Priority: Major
> Fix For: 4.0
>
>
> Today, if there’s a digest mismatch in a foreground read repair, the insert 
> to update out of date replicas is blocking. This means, if it fails, the read 
> fails with a timeout. If a node is dropping writes (maybe it is overloaded or 
> the mutation stage is backed up for some other reason), all reads to a 
> replica set could fail. Further, replicas dropping writes get more out of 
> sync so will require more read repair.
> The comment on the code for why the writes are blocking is:
> {code}
> // wait for the repair writes to be acknowledged, to minimize impact on any 
> replica that's
> // behind on writes in case the out-of-sync row is read multiple times in 
> quick succession
> {code}
> but the bad side effect is that reads timeout. Either the writes should not 
> be blocking or we should return success for the read even if the write times 
> out.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14635) Support table level configuration of monotonic reads

2018-08-22 Thread Aleksey Yeschenko (JIRA)


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

Aleksey Yeschenko commented on CASSANDRA-14635:
---

Looks like a relatively straightforward small, correct change, so I don't have 
a lot of feedback here.

1. I'd like the get-repair-strategy-from-row logic to be moved inline in 
{{SchemaKeyspace}} - with a comment. Such logic historically belongs to 
{{SchemaKeyspace}} or its old equivalent.
2. {{cqlsh}} should be altered to add the new param for completion - it's 
relatively straightforward
3. There is a single-iteration (at most) loop in 
{{AbstractReadRepair.maybeSendAdditionalReads()}} that should probably be 
rewritten using {{Iterables.find()}} or {{Iterables.tryFind()}}
4. {{ReconcileRead}} as the metric name for what it is; we should come up with 
something better, though I myself am out of ideas right now

Outside of those three points, some general thoughts that don't need to be 
addressed:
- I'm not convinced that {{ASYNC}} is an option that will be used by folks, and 
also am concerned with complete lack of backpressure there. With {{BLOCKING}} 
we at least have effective pressure via reads being blocked, here it might be 
dangerous, like the old replicate on write stage for counters
- Not a fan of {{ReadOnlyReadRepair}} name. I know you aren't either, and I 
don't have better ideas

> Support table level configuration of monotonic reads
> 
>
> Key: CASSANDRA-14635
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14635
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
>Reporter: Ariel Weisberg
>Assignee: Blake Eggleston
>Priority: Major
> Fix For: 4.0
>
>
> In CASSANDRA-10726 it was discussed that allowing expert users to forgo 
> monotonic reads might be desirable. It is practical to control monotonicity 
> of reads at a fine grained level because it involves changing the behavior of 
> read repair on a per read basis.
> Per CASSANDRA-14593 we already don't preserve update atomicity down to the 
> column level. You could read the key out of a row and read repair the key, 
> pass the key to another process which attempts to read the value, but finds 
> the value is null because read repair only repairs the data (including 
> columns) that is part of the read. IMO it's a stretch to say that reads are 
> monotonic. It is technically correct, the best kind of correct, but far from 
> as useful as it should be.
> An initial implementation could make read repair asynchronous or forgo read 
> repair entirely. This would improve the throughput and latency of reads.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14631) Add RSS support for Cassandra blog

2018-08-22 Thread Nate McCall (JIRA)


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

Nate McCall commented on CASSANDRA-14631:
-

Will try this out later today and commit if it looks good. Thanks [~beckje01] 
and [~djoshi3] for getting it ready. 

> Add RSS support for Cassandra blog
> --
>
> Key: CASSANDRA-14631
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14631
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Documentation and Website
>Reporter: Jacques-Henri Berthemet
>Assignee: Jeff Beck
>Priority: Major
>  Labels: blog
> Attachments: 14631-site.txt, Screen Shot 2018-08-17 at 5.32.08 
> PM.png, Screen Shot 2018-08-17 at 5.32.25 PM.png
>
>
> It would be convenient to add RSS support to Cassandra blog:
> [http://cassandra.apache.org/blog/2018/08/07/faster_streaming_in_cassandra.html]
> And maybe also for other resources like new versions, but this ticket is 
> about blog.
>  
> {quote}From: Scott Andreas
> Sent: Wednesday, August 08, 2018 6:53 PM
> To: [d...@cassandra.apache.org|mailto:d...@cassandra.apache.org]
> Subject: Re: Apache Cassandra Blog is now live
>  
> Please feel free to file a ticket (label: Documentation and Website).
>  
> It looks like Jekyll, the static site generator used to build the website, 
> has a plugin that generates Atom feeds if someone would like to work on 
> adding one: [https://github.com/jekyll/jekyll-feed]
> {quote}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14660) Improve TokenMetaData cache populating performance for large cluster

2018-08-22 Thread Benedict (JIRA)


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

Benedict commented on CASSANDRA-14660:
--

bq. cache invalidation is protected by the write lock

I assume the bottleneck you are encountering is inside the 
{{cachedOnlyTokenMap}} method, and if this were updated by the writer 
performing the cache invalidation, this method would never touch either the 
read or write lock, so the readers would be completely non-blocking.  

This could be updated without the writer holding the write lock, in exactly the 
same manner as it is now; we would only have to ensure the calls to 
{{invalidateCachedRings}} were made outside of the write lock.

Thinking on it just a little, though, I think the proper solution is to replace 
these collections with immutable data structures (i.e. that re-use their 
unmodified contents on update).  We actually have such a data structure 
in-tree, just not currently exposed in a user-friendly way for this scenario 
right now (our memtable BTree fits the bill, but we would need to write a 
MultiMap wrapper).  Perhaps for 4.x we can rustle up that patch, as it will 
have zero cost clone, and approximately the same complexity for updates.  Or 
perhaps there's an existing CoW Multimap we could pull in for this patch.


> Improve TokenMetaData cache populating performance for large cluster
> 
>
> Key: CASSANDRA-14660
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14660
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
> Environment: Benchmark is on MacOSX 10.13.5, 2017 MBP
>Reporter: Pengchao Wang
>Priority: Critical
>  Labels: Performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
> Attachments: 14660-trunk.txt, TokenMetaDataBenchmark.java
>
>
> TokenMetaData#cachedOnlyTokenMap is a method C* used to get a consistent 
> token and topology view on coordinations without paying read lock cost. Upon 
> first read the method acquire a synchronize lock and generate a copy of major 
> token meta data structures and cached it, and upon every token meta data 
> changes(due to gossip changes), the cache get cleared and next read will 
> taking care of cache population.
> For small to medium size clusters this strategy works pretty well. But large 
> clusters can actually be suffered from the locking since cache populating is 
> much slower. On one of our largest cluster (~1000 nodes,  125k tokens, C* 
> 3.0.15)  each cache population take about 500~700ms, and during that there 
> are no requests can go through since synchronize lock was acquired. This 
> caused waves of timeouts errors when there are large amount gossip messages 
> propagating cross the cluster, such as in the case of cluster restarting.
> Base on profiling we found that the cost mostly comes from copying 
> tokenToEndpointMap. It is a SortedBiMultiValueMap made from a forward map use 
> TreeMap and a reverse map use guava TreeMultiMap. There is an optimization in 
> TreeMap helps reduce copying complexity from O(N*log(N)) to O(N) when copying 
> from already ordered data. But guava's TreeMultiMap copying missed that 
> optimization and make it ~10 times slower than it actually need to be on our 
> size of cluster.
> The patch attached to the issue replace the reverse TreeMultiMap to a 
> vanilla TreeMap> in SortedBiMultiValueMap to make sure we can 
> copy it O(N) time.
> I also attached a benchmark script (TokenMetaDataBenchmark.java), which 
> simulates a large cluster then measures average latency for TokenMetaData 
> cache populating.
> Benchmark result before and after that patch:
> {code:java}
> trunk: 
> before 100ms, after 13ms
> 3.0.x: 
> before 199ms, after 15ms
>  {code}
> (On 3.0.x even the forward TreeMap copying is slow, the O(N*log(N)) to O(N) 
> optimization is not applied because the key comparator is dynamically created 
> and TreeMap cannot determine the source and dest are in same order)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-12151) Audit logging for database activity

2018-08-22 Thread Mehdi (JIRA)


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

Mehdi commented on CASSANDRA-12151:
---

We are using Cassandra 3.7 and we could use this audit feature. Is there a way 
we can backport this patch to 3.7?

> Audit logging for database activity
> ---
>
> Key: CASSANDRA-12151
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12151
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: stefan setyadi
>Assignee: Vinay Chella
>Priority: Major
> Fix For: 4.0
>
> Attachments: 12151.txt, CASSANDRA_12151-benchmark.html, 
> DesignProposal_AuditingFeature_ApacheCassandra_v1.docx
>
>
> we would like a way to enable cassandra to log database activity being done 
> on our server.
> It should show username, remote address, timestamp, action type, keyspace, 
> column family, and the query statement.
> it should also be able to log connection attempt and changes to the 
> user/roles.
> I was thinking of making a new keyspace and insert an entry for every 
> activity that occurs.
> Then It would be possible to query for specific activity or a query targeting 
> a specific keyspace and column family.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14660) Improve TokenMetaData cache populating performance for large cluster

2018-08-22 Thread Pengchao Wang (JIRA)


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

Pengchao Wang commented on CASSANDRA-14660:
---

[~benedict]: Thanks for replying. I will work on a 3.0 branch patch as well and 
let you guys decide what's the best schedule for the patch to go. The 
inefficient token map copying issue can be trace back to very early versions of 
TokenMetaData implementation, so it may not be that urgent to go with next 
release if it was already in feature freeze stage. But I do hope we can get 
this into the trunk so people have large clusters can benefits from the fix in 
the future.

Regarding recalculate during read or write: I think the problem is cache 
invalidation is protected by the write lock – so no matter we do it under read 
or write we will need pay full cost on blocking anyway. And long write locks 
looks more harmful? But I do agree with you that the locking design of 
TokenMetaData looks quite heavy and probably need some rethink.

> Improve TokenMetaData cache populating performance for large cluster
> 
>
> Key: CASSANDRA-14660
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14660
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
> Environment: Benchmark is on MacOSX 10.13.5, 2017 MBP
>Reporter: Pengchao Wang
>Priority: Critical
>  Labels: Performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
> Attachments: 14660-trunk.txt, TokenMetaDataBenchmark.java
>
>
> TokenMetaData#cachedOnlyTokenMap is a method C* used to get a consistent 
> token and topology view on coordinations without paying read lock cost. Upon 
> first read the method acquire a synchronize lock and generate a copy of major 
> token meta data structures and cached it, and upon every token meta data 
> changes(due to gossip changes), the cache get cleared and next read will 
> taking care of cache population.
> For small to medium size clusters this strategy works pretty well. But large 
> clusters can actually be suffered from the locking since cache populating is 
> much slower. On one of our largest cluster (~1000 nodes,  125k tokens, C* 
> 3.0.15)  each cache population take about 500~700ms, and during that there 
> are no requests can go through since synchronize lock was acquired. This 
> caused waves of timeouts errors when there are large amount gossip messages 
> propagating cross the cluster, such as in the case of cluster restarting.
> Base on profiling we found that the cost mostly comes from copying 
> tokenToEndpointMap. It is a SortedBiMultiValueMap made from a forward map use 
> TreeMap and a reverse map use guava TreeMultiMap. There is an optimization in 
> TreeMap helps reduce copying complexity from O(N*log(N)) to O(N) when copying 
> from already ordered data. But guava's TreeMultiMap copying missed that 
> optimization and make it ~10 times slower than it actually need to be on our 
> size of cluster.
> The patch attached to the issue replace the reverse TreeMultiMap to a 
> vanilla TreeMap> in SortedBiMultiValueMap to make sure we can 
> copy it O(N) time.
> I also attached a benchmark script (TokenMetaDataBenchmark.java), which 
> simulates a large cluster then measures average latency for TokenMetaData 
> cache populating.
> Benchmark result before and after that patch:
> {code:java}
> trunk: 
> before 100ms, after 13ms
> 3.0.x: 
> before 199ms, after 15ms
>  {code}
> (On 3.0.x even the forward TreeMap copying is slow, the O(N*log(N)) to O(N) 
> optimization is not applied because the key comparator is dynamically created 
> and TreeMap cannot determine the source and dest are in same order)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14659) Disable old protocol versions on demand

2018-08-22 Thread Dinesh Joshi (JIRA)


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

Dinesh Joshi updated CASSANDRA-14659:
-
Status: Ready to Commit  (was: Patch Available)

> Disable old protocol versions on demand
> ---
>
> Key: CASSANDRA-14659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14659
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: usability
>
> This patch allows the operators to disable older protocol versions on demand. 
> To use it, you can set {{native_transport_allow_older_protocols}} to false or 
> use nodetool disableolderprotocolversions. Cassandra will reject requests 
> from client coming in on any version except the current version. This will 
> help operators selectively reject connections from clients that do not 
> support the latest protoocol.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14659) Disable old protocol versions on demand

2018-08-22 Thread Dinesh Joshi (JIRA)


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

Dinesh Joshi commented on CASSANDRA-14659:
--

[~molsson], I wanted to keep this feature simple for now. I did consider a more 
complex version of this. However, I felt it would be over complicating things. 
This is useful to run a "scream" test prior to a major version upgrade.

> Disable old protocol versions on demand
> ---
>
> Key: CASSANDRA-14659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14659
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: usability
>
> This patch allows the operators to disable older protocol versions on demand. 
> To use it, you can set {{native_transport_allow_older_protocols}} to false or 
> use nodetool disableolderprotocolversions. Cassandra will reject requests 
> from client coming in on any version except the current version. This will 
> help operators selectively reject connections from clients that do not 
> support the latest protoocol.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-13304) Add checksumming to the native protocol

2018-08-22 Thread Sam Tunnicliffe (JIRA)


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

Sam Tunnicliffe commented on CASSANDRA-13304:
-

 Pushed a couple of updates to : 
* memoize the transformers supporting the various compression/checksum 
combinations
* remove unnecessary use of {{Optional}}
* remove log statement at {{WARN}} when Snappy is available (not selected)
* clean up javadoc


||branch||CI||
|[13304-trunk|https://github.com/beobal/cassandra/tree/13304-trunk]|[CircleCI|https://circleci.com/gh/beobal/cassandra/tree/cci%2F13304-trunk]|

Looks like there were a couple of dtest failures on the previous round, but at 
first glance they don't look related. I'll investigate further.


> Add checksumming to the native protocol
> ---
>
> Key: CASSANDRA-13304
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13304
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Michael Kjellman
>Assignee: Sam Tunnicliffe
>Priority: Blocker
>  Labels: client-impacting
> Fix For: 4.x
>
> Attachments: 13304_v1.diff, boxplot-read-throughput.png, 
> boxplot-write-throughput.png
>
>
> The native binary transport implementation doesn't include checksums. This 
> makes it highly susceptible to silently inserting corrupted data either due 
> to hardware issues causing bit flips on the sender/client side, C*/receiver 
> side, or network in between.
> Attaching an implementation that makes checksum'ing mandatory (assuming both 
> client and server know about a protocol version that supports checksums) -- 
> and also adds checksumming to clients that request compression.
> The serialized format looks something like this:
> {noformat}
>  *  1 1 1 1 1 1 1 1 1 1 2 2 2 2 2 2 2 2 2 2 3 3
>  *  0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1 2 3 4 5 6 7 8 9 0 1
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |  Number of Compressed Chunks  | Compressed Length (e1)/
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * /  Compressed Length cont. (e1) |Uncompressed Length (e1)   /
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * | Uncompressed Length cont. (e1)| CRC32 Checksum of Lengths (e1)|
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * | Checksum of Lengths cont. (e1)|Compressed Bytes (e1)+//
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |  CRC32 Checksum (e1) ||
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |Compressed Length (e2) |
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |   Uncompressed Length (e2)|
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |CRC32 Checksum of Lengths (e2) |
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * | Compressed Bytes (e2)   +//
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |  CRC32 Checksum (e2) ||
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |Compressed Length (en) |
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |   Uncompressed Length (en)|
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |CRC32 Checksum of Lengths (en) |
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |  Compressed Bytes (en)  +//
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
>  * |  CRC32 Checksum (en) ||
>  * +-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+ 
> {noformat}
> The first pass here adds checksums only to the actual contents of the frame 
> body itself (and doesn't actually checksum lengths and headers). While it 
> would be great to fully add checksuming across the entire protocol, the 
> proposed implementation will ensure we at least catch corrupted data and 
> likely protect ourselves pretty well anyways.
> I didn't go to the trouble of implementing a Snappy Checksum'ed Compressor 
> implementation as it's been deprecated for a while -- is really slow and 
> crappy compared to LZ4 -- and we should do everything in our power to make 
> sure no one in the community is still using it. I left it in (for obvious 
> backwards 

[jira] [Commented] (CASSANDRA-14659) Disable old protocol versions on demand

2018-08-22 Thread Marcus Olsson (JIRA)


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

Marcus Olsson commented on CASSANDRA-14659:
---

It seems like we support V3, V4 and V5 protocols on trunk as of today. Would it 
make sense to be able to be a bit more selective than to say only latest 
protocol or all of them? One option could be to specify the minimum protocol 
version to allow from clients.

 

I don't have a strong opinion on this but I thought it might potentially be 
useful to have a bit more flexibility with this feature when multiple protocol 
versions are supported.

> Disable old protocol versions on demand
> ---
>
> Key: CASSANDRA-14659
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14659
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
>Reporter: Dinesh Joshi
>Assignee: Dinesh Joshi
>Priority: Major
>  Labels: usability
>
> This patch allows the operators to disable older protocol versions on demand. 
> To use it, you can set {{native_transport_allow_older_protocols}} to false or 
> use nodetool disableolderprotocolversions. Cassandra will reject requests 
> from client coming in on any version except the current version. This will 
> help operators selectively reject connections from clients that do not 
> support the latest protoocol.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14621) Refactor CompactionStrategyManager

2018-08-22 Thread Blake Eggleston (JIRA)


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

Blake Eggleston commented on CASSANDRA-14621:
-

pulled in your changes and pushed up some commits addressing review comments

> Refactor CompactionStrategyManager
> --
>
> Key: CASSANDRA-14621
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14621
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Compaction
>Reporter: Blake Eggleston
>Assignee: Blake Eggleston
>Priority: Minor
> Fix For: 4.x
>
>
> CompactionStrategyManager grew a decent amount of duplicated code as part of 
> CASSANDRA-9143, which added pendingRepairs alongside the repaired and 
> unrepaired buckets. At this point, the logic that routes sstables between the 
> different buckets, and the different partition range divisions has gotten a 
> little complex, and editing it is tedious and error prone. With transient 
> replication requiring yet another bucket for, this seems like a good time to 
> split some of the functionality of CSM into other classes, and make sstable 
> routing a bit more generalized.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14409) Transient Replication: Support ring changes when transient replication is in use (add/remove node, change RF, add/remove DC)

2018-08-22 Thread Benedict (JIRA)


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

Benedict commented on CASSANDRA-14409:
--

I have [pushed a 
fix|https://github.com/belliottsmith/cassandra/commit/a08cb8c14eb5cdccf843b7d134a73dac21fe24c9]
 for a bug I introduced in the collection refactor.

I have also [pushed a 
fix|https://github.com/belliottsmith/cassandra/commit/7d3f6cf21895f8d79b906bf5b8f68c8cb791925f]
 for a few subtle bugs in calculateRangesToFetchWithPreferredEndpoints.  I have 
not completed review of the overall logic of this (and related methods), but we 
were at least not applying the provided snitch sort order in all cases, and in 
the case that strictness was requested but impossible to meet we would fail if 
any of the endpoints met the provided filters, even though we only need one 
matching source.

I have also pushed a couple of tidying commits, 
[here|https://github.com/belliottsmith/cassandra/commit/7e7d0e30e909d06a8458e5d55e2c415dea23ecd2]
 and 
[here|https://github.com/belliottsmith/cassandra/commit/293fd16ba7be2884be454aa84df1a6cc5b3fb6df]

> Transient Replication: Support ring changes when transient replication is in 
> use (add/remove node, change RF, add/remove DC)
> 
>
> Key: CASSANDRA-14409
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14409
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: Coordination, Core, Documentation and Website
>Reporter: Ariel Weisberg
>Assignee: Ariel Weisberg
>Priority: Major
> Fix For: 4.0
>
>
> The additional state transitions that transient replication introduces 
> require streaming and nodetool cleanup to behave differently. We already have 
> code that does the streaming, but in some cases we shouldn't stream any data 
> and in others when we stream to receive data we have to make sure we stream 
> from a full replica and not a transient replica.
> Transitioning from not replicated to transiently replicated means that a node 
> must stay pending until the next incremental repair completes at which point 
> the data for that range is known to be available at full replicas.
> Transitioning from transiently replicated to fully replicated requires 
> streaming from a full replica and is identical to how we stream from not 
> replicated to replicated. The transition must be managed so the transient 
> replica is not read from as a full replica until streaming completes. It can 
> be used immediately for a write quorum.
> Transitioning from fully replicated to transiently replicated requires 
> cleanup to remove repaired data from the transiently replicated range to 
> reclaim space. It can be used immediately for a write quorum.
> Transitioning from transiently replicated to not replicated requires cleanup 
> to be run to remove the formerly transiently replicated data.
> nodetool move, removenode, cleanup, decommission, and rebuild need to handle 
> these issues as does bootstrap.
> Update web site, documentation, NEWS.txt with a description of the steps for 
> doing common operations. Add/remove DC, Add/remove node(s), replace node, 
> change RF.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14406) Transient Replication: Implement cheap quorum write optimizations

2018-08-22 Thread Benedict (JIRA)


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

Benedict commented on CASSANDRA-14406:
--

My review for this is in progress alongside all of the other transient 
replication reviews, so I will report here as I find noteworthy things to 
discuss.

First and most importantly, I'm concerned about pending ranges and transient 
replicas.  It's not clear to me that ignoring transient replicas for pending 
ranges is acceptable.  I'm not sufficiently clued up on the order of ceremony 
for range movements, but it seems likely that when a transient range is moved, 
there will be a period of time where both transient replicas would need to 
receive any write intended for the logical transient replica.  If only the 
current one receives it, then after the movement takes effect a read quorum may 
return a bad response.

StorageProxy.mutate looks to have had a bug around maybeTryAdditionalReplicas 
and counters when a node was marked down.  In this case it would have tried to 
perform a regular write to the down nodes.  [I have pushed a 
fix|https://github.com/belliottsmith/cassandra/commit/9e74e1dd1f2f69f0c0117c4f94f79da9d5b6fe8f].

StorageProxy.mutate also unnecessarily allocated a HashMap, that it only used 
for pair-wise iteration.  In fact, by requiring a list to be furnished to the 
method, we could simply collect a new list of the response handlers, and 
iterate them both by index.  Since all callers already did construct a list, 
this just involved propagating some type parameter changes.  [I have pushed a 
refactor to the 
method|https://github.com/belliottsmith/cassandra/commit/4860cc0be054bfedf31220a2c87ff90fa3f16648]

> Transient Replication: Implement cheap quorum write optimizations
> -
>
> Key: CASSANDRA-14406
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14406
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: Coordination
>Reporter: Ariel Weisberg
>Assignee: Blake Eggleston
>Priority: Major
>
> Writes should never be sent to transient replicas unless necessary to satisfy 
> the requested consistency level. Such as RF not being sufficient for strong 
> consistency or not enough full replicas marked as alive.
> If a write doesn't receive sufficient responses in time additional replicas 
> should be sent the write similar to Rapid Read Protection.
> Hints should never be written for a transient replica.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-10726) Read repair inserts should not be blocking

2018-08-22 Thread Blake Eggleston (JIRA)


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

Blake Eggleston commented on CASSANDRA-10726:
-

Thanks Kurt. Opening the response in a try block like that would close the 
response too early though. I fixed it without auto-closing it 
[here|https://github.com/apache/cassandra/commit/7082b64fc9a9a14c9f9cb9a492c0b25d109f4edc].
 I also noticed there's at least one place {{StorageProxy#query}} can could 
throw an exception, so I also added a catch block to close any previously 
opened responses.

> Read repair inserts should not be blocking
> --
>
> Key: CASSANDRA-10726
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10726
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
>Reporter: Richard Low
>Assignee: Blake Eggleston
>Priority: Major
> Fix For: 4.0
>
>
> Today, if there’s a digest mismatch in a foreground read repair, the insert 
> to update out of date replicas is blocking. This means, if it fails, the read 
> fails with a timeout. If a node is dropping writes (maybe it is overloaded or 
> the mutation stage is backed up for some other reason), all reads to a 
> replica set could fail. Further, replicas dropping writes get more out of 
> sync so will require more read repair.
> The comment on the code for why the writes are blocking is:
> {code}
> // wait for the repair writes to be acknowledged, to minimize impact on any 
> replica that's
> // behind on writes in case the out-of-sync row is read multiple times in 
> quick succession
> {code}
> but the bad side effect is that reads timeout. Either the writes should not 
> be blocking or we should return success for the read even if the write times 
> out.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



cassandra git commit: ninja: fix resource leak warning

2018-08-22 Thread bdeggleston
Repository: cassandra
Updated Branches:
  refs/heads/trunk 644676b08 -> 7082b64fc


ninja: fix resource leak warning


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7082b64f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7082b64f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7082b64f

Branch: refs/heads/trunk
Commit: 7082b64fc9a9a14c9f9cb9a492c0b25d109f4edc
Parents: 644676b
Author: Blake Eggleston 
Authored: Wed Aug 22 08:15:55 2018 -0700
Committer: Blake Eggleston 
Committed: Wed Aug 22 08:15:55 2018 -0700

--
 .../apache/cassandra/service/StorageProxy.java  | 21 +++-
 1 file changed, 16 insertions(+), 5 deletions(-)
--


http://git-wip-us.apache.org/repos/asf/cassandra/blob/7082b64f/src/java/org/apache/cassandra/service/StorageProxy.java
--
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java 
b/src/java/org/apache/cassandra/service/StorageProxy.java
index 7fdf591..58f08d4 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -2243,12 +2243,23 @@ public class StorageProxy implements StorageProxyMBean
 {
 List concurrentQueries = new 
ArrayList<>(concurrencyFactor);
 List readRepairs = new ArrayList<>(concurrencyFactor);
-for (int i = 0; i < concurrencyFactor && ranges.hasNext(); i++)
+
+try
+{
+for (int i = 0; i < concurrencyFactor && ranges.hasNext(); i++)
+{
+@SuppressWarnings("resource") // response will be closed 
by concatAndBlockOnRepair, or in the catch block below
+SingleRangeResponse response = query(ranges.next(), i == 
0);
+concurrentQueries.add(response);
+readRepairs.add(response.readRepair);
+++rangesQueried;
+}
+}
+catch (Throwable t)
 {
-SingleRangeResponse response = query(ranges.next(), i == 0);
-concurrentQueries.add(response);
-readRepairs.add(response.readRepair);
-++rangesQueried;
+for (PartitionIterator response: concurrentQueries)
+response.close();
+throw t;
 }
 
 Tracing.trace("Submitted {} concurrent range requests", 
concurrentQueries.size());


-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14409) Transient Replication: Support ring changes when transient replication is in use (add/remove node, change RF, add/remove DC)

2018-08-22 Thread Ariel Weisberg (JIRA)


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

Ariel Weisberg commented on CASSANDRA-14409:


I'm going to rebase on top of Blake's work now, get all the tests passing, 
incorporate/review the collection refactor, get all the tests passing, apply 
Alex's review comments. I've done a few already.

{quote}Looks like StorageProxy#calculateRangesToStreamWithPreferredEndpoints 
can be simplified to something like this. Most likely the same logic can (and 
probably even should) belong to calculateStreamAndFetchRanges, since it seems 
that we unnecessarily trying to loop and intersect the ranges again.
This seems to have been caused by the fact 
calculateRangesToFetchWithPreferredEndpoints and 
calculateRangesToStreamWithPreferredEndpoints were receiving Range, not 
replica as they do right now, so in both cases we additionally call 
calculateNaturalReplicas (which we potentially might avoid). I think we should 
do all range-related calculations (intersections, etc) in 
calculateStreamAndFetchRanges. The old code logic was just making a diff 
between old and current set per range like here, so it seems that we can 
achieve the same without splitting the logic in two parts.{quote}

I'll look at this after once I have the collections and the tests passing. I 
want a more stable representative base to start from before making that kind of 
change to one of the thorniest parts of the patch.

> Transient Replication: Support ring changes when transient replication is in 
> use (add/remove node, change RF, add/remove DC)
> 
>
> Key: CASSANDRA-14409
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14409
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: Coordination, Core, Documentation and Website
>Reporter: Ariel Weisberg
>Assignee: Ariel Weisberg
>Priority: Major
> Fix For: 4.0
>
>
> The additional state transitions that transient replication introduces 
> require streaming and nodetool cleanup to behave differently. We already have 
> code that does the streaming, but in some cases we shouldn't stream any data 
> and in others when we stream to receive data we have to make sure we stream 
> from a full replica and not a transient replica.
> Transitioning from not replicated to transiently replicated means that a node 
> must stay pending until the next incremental repair completes at which point 
> the data for that range is known to be available at full replicas.
> Transitioning from transiently replicated to fully replicated requires 
> streaming from a full replica and is identical to how we stream from not 
> replicated to replicated. The transition must be managed so the transient 
> replica is not read from as a full replica until streaming completes. It can 
> be used immediately for a write quorum.
> Transitioning from fully replicated to transiently replicated requires 
> cleanup to remove repaired data from the transiently replicated range to 
> reclaim space. It can be used immediately for a write quorum.
> Transitioning from transiently replicated to not replicated requires cleanup 
> to be run to remove the formerly transiently replicated data.
> nodetool move, removenode, cleanup, decommission, and rebuild need to handle 
> these issues as does bootstrap.
> Update web site, documentation, NEWS.txt with a description of the steps for 
> doing common operations. Add/remove DC, Add/remove node(s), replace node, 
> change RF.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14409) Transient Replication: Support ring changes when transient replication is in use (add/remove node, change RF, add/remove DC)

2018-08-22 Thread Alex Petrov (JIRA)


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

Alex Petrov commented on CASSANDRA-14409:
-

I'm definitely in favour of refactor proposed by [~benedict]. First of all, it 
removes unclarity related to subtle differences and sorting imposed by Set and 
List, which were subsequently blurred by the use of Collection in previous 
version, so it was difficult to say what comes from where.

It seems that we did not have any places where {{Set}} was serving a meaning of 
{{ReplicaSet}}, e.g. removing duplicates endpoint/token range pairs in code, 
but rather {{Set}}.

It'd be great if we could finish up ReplicaPlan changes that this change made 
possible and attempt to hide abstract class and interface in the package, so we 
could be always specific on the code level and have clear interfaces. I have 
several minor comments / suggestions, I'll push them to the branch later today.

> Transient Replication: Support ring changes when transient replication is in 
> use (add/remove node, change RF, add/remove DC)
> 
>
> Key: CASSANDRA-14409
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14409
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: Coordination, Core, Documentation and Website
>Reporter: Ariel Weisberg
>Assignee: Ariel Weisberg
>Priority: Major
> Fix For: 4.0
>
>
> The additional state transitions that transient replication introduces 
> require streaming and nodetool cleanup to behave differently. We already have 
> code that does the streaming, but in some cases we shouldn't stream any data 
> and in others when we stream to receive data we have to make sure we stream 
> from a full replica and not a transient replica.
> Transitioning from not replicated to transiently replicated means that a node 
> must stay pending until the next incremental repair completes at which point 
> the data for that range is known to be available at full replicas.
> Transitioning from transiently replicated to fully replicated requires 
> streaming from a full replica and is identical to how we stream from not 
> replicated to replicated. The transition must be managed so the transient 
> replica is not read from as a full replica until streaming completes. It can 
> be used immediately for a write quorum.
> Transitioning from fully replicated to transiently replicated requires 
> cleanup to remove repaired data from the transiently replicated range to 
> reclaim space. It can be used immediately for a write quorum.
> Transitioning from transiently replicated to not replicated requires cleanup 
> to be run to remove the formerly transiently replicated data.
> nodetool move, removenode, cleanup, decommission, and rebuild need to handle 
> these issues as does bootstrap.
> Update web site, documentation, NEWS.txt with a description of the steps for 
> doing common operations. Add/remove DC, Add/remove node(s), replace node, 
> change RF.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14662) Refactor AuthCache

2018-08-22 Thread Sam Tunnicliffe (JIRA)


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

Sam Tunnicliffe updated CASSANDRA-14662:

Reviewer: Sam Tunnicliffe

I can review this in the next day or two

> Refactor AuthCache
> --
>
> Key: CASSANDRA-14662
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14662
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Auth
>Reporter: Kurt Greaves
>Assignee: Kurt Greaves
>Priority: Major
>  Labels: security
> Fix For: 4.x
>
>
> When building an LDAP IAuthenticator plugin I ran into a few issues when 
> trying to reuse the AuthCache similar to how PasswordAuthenticator implements 
> it. Most of the problems stemmed from the underlying cache being inaccessible 
> and not being able to override {{initCache}} properly.
> Anyway, I've had a stab at refactoring AuthCache with the following 
> improvements:
> # Make it possible to extend and override all necessary methods (initCache, 
> init, validate)
> # Makes it possible to specify a {{CacheLoader}} rather than just a 
> {{Function}}, allowing you to have a get/load that throws exceptions.
> # Use AuthCache on its own rather than extending it for each use case 
> ({{invalidate(K)}} moved to be part of MBean)
> # Provided a builder that uses sane defaults so we don't have unnecessary 
> repeated code everywhere
> The refactor made all the extensions of AuthCache unnecessary, so I've 
> simplified those cases to use AuthCache and removed any classes extending 
> AuthCache. I also removed some noop compatibility classes that were marked to 
> be removed in 4.0.
> Also added some tests in AuthCacheTest.
> |[trunk|https://github.com/apache/cassandra/compare/trunk...kgreav:authcache]|
> |[utests|https://circleci.com/gh/kgreav/cassandra/206]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14662) Refactor AuthCache

2018-08-22 Thread Jeremy Hanna (JIRA)


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

Jeremy Hanna updated CASSANDRA-14662:
-
Labels: security  (was: )

> Refactor AuthCache
> --
>
> Key: CASSANDRA-14662
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14662
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Auth
>Reporter: Kurt Greaves
>Assignee: Kurt Greaves
>Priority: Major
>  Labels: security
> Fix For: 4.x
>
>
> When building an LDAP IAuthenticator plugin I ran into a few issues when 
> trying to reuse the AuthCache similar to how PasswordAuthenticator implements 
> it. Most of the problems stemmed from the underlying cache being inaccessible 
> and not being able to override {{initCache}} properly.
> Anyway, I've had a stab at refactoring AuthCache with the following 
> improvements:
> # Make it possible to extend and override all necessary methods (initCache, 
> init, validate)
> # Makes it possible to specify a {{CacheLoader}} rather than just a 
> {{Function}}, allowing you to have a get/load that throws exceptions.
> # Use AuthCache on its own rather than extending it for each use case 
> ({{invalidate(K)}} moved to be part of MBean)
> # Provided a builder that uses sane defaults so we don't have unnecessary 
> repeated code everywhere
> The refactor made all the extensions of AuthCache unnecessary, so I've 
> simplified those cases to use AuthCache and removed any classes extending 
> AuthCache. I also removed some noop compatibility classes that were marked to 
> be removed in 4.0.
> Also added some tests in AuthCacheTest.
> |[trunk|https://github.com/apache/cassandra/compare/trunk...kgreav:authcache]|
> |[utests|https://circleci.com/gh/kgreav/cassandra/206]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14660) Improve TokenMetaData cache populating performance for large cluster

2018-08-22 Thread Benedict (JIRA)


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

Benedict commented on CASSANDRA-14660:
--

[~wpc] thanks for the patch.  I'm going to go out on a limb and say that people 
are stretched too thin to review this in time for the 4.0 feature freeze, which 
is a shame.  That said, while I haven't taken a proper look at the patch, it 
looks fairly straightforward so I might be wrong.  

I would say there's a reasonable chance this qualifies for inclusion in 3.0+, 
as well, since this sounds like pretty bad behaviour.  In which case it may not 
be affected by the freeze.

However, while where at it, it's not clear to me why we recalculate during 
read, as opposed to on invalidation.  This might marginally slow down responses 
to ring changes, but these have no fixed schedule for notification anyway, so 
we must be robust to this, and all operations (including those entirely 
not-overlapping with the change) stopping briefly during a ring change seems 
unnecessarily burdensome.

> Improve TokenMetaData cache populating performance for large cluster
> 
>
> Key: CASSANDRA-14660
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14660
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
> Environment: Benchmark is on MacOSX 10.13.5, 2017 MBP
>Reporter: Pengchao Wang
>Priority: Critical
>  Labels: Performance
> Fix For: 3.0.x, 3.11.x, 4.x
>
> Attachments: 14660-trunk.txt, TokenMetaDataBenchmark.java
>
>
> TokenMetaData#cachedOnlyTokenMap is a method C* used to get a consistent 
> token and topology view on coordinations without paying read lock cost. Upon 
> first read the method acquire a synchronize lock and generate a copy of major 
> token meta data structures and cached it, and upon every token meta data 
> changes(due to gossip changes), the cache get cleared and next read will 
> taking care of cache population.
> For small to medium size clusters this strategy works pretty well. But large 
> clusters can actually be suffered from the locking since cache populating is 
> much slower. On one of our largest cluster (~1000 nodes,  125k tokens, C* 
> 3.0.15)  each cache population take about 500~700ms, and during that there 
> are no requests can go through since synchronize lock was acquired. This 
> caused waves of timeouts errors when there are large amount gossip messages 
> propagating cross the cluster, such as in the case of cluster restarting.
> Base on profiling we found that the cost mostly comes from copying 
> tokenToEndpointMap. It is a SortedBiMultiValueMap made from a forward map use 
> TreeMap and a reverse map use guava TreeMultiMap. There is an optimization in 
> TreeMap helps reduce copying complexity from O(N*log(N)) to O(N) when copying 
> from already ordered data. But guava's TreeMultiMap copying missed that 
> optimization and make it ~10 times slower than it actually need to be on our 
> size of cluster.
> The patch attached to the issue replace the reverse TreeMultiMap to a 
> vanilla TreeMap> in SortedBiMultiValueMap to make sure we can 
> copy it O(N) time.
> I also attached a benchmark script (TokenMetaDataBenchmark.java), which 
> simulates a large cluster then measures average latency for TokenMetaData 
> cache populating.
> Benchmark result before and after that patch:
> {code:java}
> trunk: 
> before 100ms, after 13ms
> 3.0.x: 
> before 199ms, after 15ms
>  {code}
> (On 3.0.x even the forward TreeMap copying is slow, the O(N*log(N)) to O(N) 
> optimization is not applied because the key comparator is dynamically created 
> and TreeMap cannot determine the source and dest are in same order)



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14503) Internode connection management is race-prone

2018-08-22 Thread Jason Brown (JIRA)


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

Jason Brown commented on CASSANDRA-14503:
-

Working on a solution that will also resolve the problems [~sbtourist] raised 
with CASSANDRA-14507. I've spoken with some of the netty maintainers and 
they've given me some solid advice about to rework 
{{OutboubndMessageConnection}}.

> Internode connection management is race-prone
> -
>
> Key: CASSANDRA-14503
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14503
> Project: Cassandra
>  Issue Type: Bug
>  Components: Streaming and Messaging
>Reporter: Sergio Bossa
>Assignee: Jason Brown
>Priority: Major
>
> Following CASSANDRA-8457, internode connection management has been rewritten 
> to rely on Netty, but the new implementation in 
> {{OutboundMessagingConnection}} seems quite race prone to me, in particular 
> on those two cases:
> * {{#finishHandshake()}} racing with {{#close()}}: i.e. in such case the 
> former could run into an NPE if the latter nulls the {{channelWriter}} (but 
> this is just an example, other conflicts might happen).
> * Connection timeout and retry racing with state changing methods: 
> {{connectionRetryFuture}} and {{connectionTimeoutFuture}} are cancelled when 
> handshaking or closing, but there's no guarantee those will be actually 
> cancelled (as they might be already running), so they might end up changing 
> the connection state concurrently with other methods (i.e. by unexpectedly 
> closing the channel or clearing the backlog).
> Overall, the thread safety of {{OutboundMessagingConnection}} is very 
> difficult to assess given the current implementation: I would suggest to 
> refactor it into a single-thread model, where all connection state changing 
> actions are enqueued on a single threaded scheduler, so that state 
> transitions can be clearly defined and checked.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Comment Edited] (CASSANDRA-14503) Internode connection management is race-prone

2018-08-22 Thread Jason Brown (JIRA)


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

Jason Brown edited comment on CASSANDRA-14503 at 8/22/18 11:48 AM:
---

Working on a solution that will also resolve the problems [~sbtourist] raised 
with CASSANDRA-14507. I've spoken with some of the netty maintainers and 
they've given me some solid advice how to rework {{OutboubndMessageConnection}}.


was (Author: jasobrown):
Working on a solution that will also resolve the problems [~sbtourist] raised 
with CASSANDRA-14507. I've spoken with some of the netty maintainers and 
they've given me some solid advice about to rework 
{{OutboubndMessageConnection}}.

> Internode connection management is race-prone
> -
>
> Key: CASSANDRA-14503
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14503
> Project: Cassandra
>  Issue Type: Bug
>  Components: Streaming and Messaging
>Reporter: Sergio Bossa
>Assignee: Jason Brown
>Priority: Major
>
> Following CASSANDRA-8457, internode connection management has been rewritten 
> to rely on Netty, but the new implementation in 
> {{OutboundMessagingConnection}} seems quite race prone to me, in particular 
> on those two cases:
> * {{#finishHandshake()}} racing with {{#close()}}: i.e. in such case the 
> former could run into an NPE if the latter nulls the {{channelWriter}} (but 
> this is just an example, other conflicts might happen).
> * Connection timeout and retry racing with state changing methods: 
> {{connectionRetryFuture}} and {{connectionTimeoutFuture}} are cancelled when 
> handshaking or closing, but there's no guarantee those will be actually 
> cancelled (as they might be already running), so they might end up changing 
> the connection state concurrently with other methods (i.e. by unexpectedly 
> closing the channel or clearing the backlog).
> Overall, the thread safety of {{OutboundMessagingConnection}} is very 
> difficult to assess given the current implementation: I would suggest to 
> refactor it into a single-thread model, where all connection state changing 
> actions are enqueued on a single threaded scheduler, so that state 
> transitions can be clearly defined and checked.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Assigned] (CASSANDRA-14503) Internode connection management is race-prone

2018-08-22 Thread Jason Brown (JIRA)


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

Jason Brown reassigned CASSANDRA-14503:
---

Assignee: Jason Brown

> Internode connection management is race-prone
> -
>
> Key: CASSANDRA-14503
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14503
> Project: Cassandra
>  Issue Type: Bug
>  Components: Streaming and Messaging
>Reporter: Sergio Bossa
>Assignee: Jason Brown
>Priority: Major
>
> Following CASSANDRA-8457, internode connection management has been rewritten 
> to rely on Netty, but the new implementation in 
> {{OutboundMessagingConnection}} seems quite race prone to me, in particular 
> on those two cases:
> * {{#finishHandshake()}} racing with {{#close()}}: i.e. in such case the 
> former could run into an NPE if the latter nulls the {{channelWriter}} (but 
> this is just an example, other conflicts might happen).
> * Connection timeout and retry racing with state changing methods: 
> {{connectionRetryFuture}} and {{connectionTimeoutFuture}} are cancelled when 
> handshaking or closing, but there's no guarantee those will be actually 
> cancelled (as they might be already running), so they might end up changing 
> the connection state concurrently with other methods (i.e. by unexpectedly 
> closing the channel or clearing the backlog).
> Overall, the thread safety of {{OutboundMessagingConnection}} is very 
> difficult to assess given the current implementation: I would suggest to 
> refactor it into a single-thread model, where all connection state changing 
> actions are enqueued on a single threaded scheduler, so that state 
> transitions can be clearly defined and checked.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Resolved] (CASSANDRA-14507) OutboundMessagingConnection backlog is not fully written in case of race conditions

2018-08-22 Thread Jason Brown (JIRA)


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

Jason Brown resolved CASSANDRA-14507.
-
Resolution: Feedback Received

Thanks for reporting this, [~sbtourist]. I am working on a solution that is 
hand-in-hand with CASSANDRA-14503. Thus, I'll move the conversation and work 
over there.

> OutboundMessagingConnection backlog is not fully written in case of race 
> conditions
> ---
>
> Key: CASSANDRA-14507
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14507
> Project: Cassandra
>  Issue Type: Bug
>  Components: Streaming and Messaging
>Reporter: Sergio Bossa
>Priority: Major
>
> The {{OutboundMessagingConnection}} writes into a backlog queue before the 
> connection handshake is successfully completed, and then writes such backlog 
> to the channel as soon as the successful handshake moves the channel state to 
> {{READY}}.
> This is unfortunately race prone, as the following could happen:
> 1) One or more writer threads see the channel state as {{NOT_READY}} in 
> {{#sendMessage()}} and are about to enqueue to the backlog, but they get 
> descheduled by the OS.
> 2) The handshake thread is scheduled by the OS and moves the channel state to 
> {{READY}}, emptying the backlog.
> 3) The writer threads are scheduled back and add to the backlog, but the 
> channel state is {{READY}} at this point, so those writes would sit in the 
> backlog and expire.
> Please note a similar race condition exists between 
> {{OutboundMessagingConnection#sendMessage()}} and 
> {{MessageOutHandler#channelWritabilityChanged()}}, which is way more serious 
> as the channel writability could frequently change, luckily it looks like 
> {{ChannelWriter#write()}} never gets invoked with {{checkWritability}} at 
> {{true}} (so writes never go to the backlog when the channel is not writable).



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-10540) RangeAwareCompaction

2018-08-22 Thread Marcus Eriksson (JIRA)


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

Marcus Eriksson commented on CASSANDRA-10540:
-

[~KurtG] I'm afraid not, the patch has basically no unit tests, I would need to 
spend a couple of weeks on getting it in an acceptable state

> RangeAwareCompaction
> 
>
> Key: CASSANDRA-10540
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10540
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Major
>  Labels: 4.0-feature-freeze-review-requested, compaction, lcs, 
> vnodes
> Fix For: 4.x
>
>
> Broken out from CASSANDRA-6696, we should split sstables based on ranges 
> during compaction.
> Requirements;
> * dont create tiny sstables - keep them bunched together until a single vnode 
> is big enough (configurable how big that is)
> * make it possible to run existing compaction strategies on the per-range 
> sstables
> We should probably add a global compaction strategy parameter that states 
> whether this should be enabled or not.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14662) Refactor AuthCache

2018-08-22 Thread Kurt Greaves (JIRA)


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

Kurt Greaves updated CASSANDRA-14662:
-
Status: Patch Available  (was: Open)

> Refactor AuthCache
> --
>
> Key: CASSANDRA-14662
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14662
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Auth
>Reporter: Kurt Greaves
>Assignee: Kurt Greaves
>Priority: Major
>
> When building an LDAP IAuthenticator plugin I ran into a few issues when 
> trying to reuse the AuthCache similar to how PasswordAuthenticator implements 
> it. Most of the problems stemmed from the underlying cache being inaccessible 
> and not being able to override {{initCache}} properly.
> Anyway, I've had a stab at refactoring AuthCache with the following 
> improvements:
> # Make it possible to extend and override all necessary methods (initCache, 
> init, validate)
> # Makes it possible to specify a {{CacheLoader}} rather than just a 
> {{Function}}, allowing you to have a get/load that throws exceptions.
> # Use AuthCache on its own rather than extending it for each use case 
> ({{invalidate(K)}} moved to be part of MBean)
> # Provided a builder that uses sane defaults so we don't have unnecessary 
> repeated code everywhere
> The refactor made all the extensions of AuthCache unnecessary, so I've 
> simplified those cases to use AuthCache and removed any classes extending 
> AuthCache. I also removed some noop compatibility classes that were marked to 
> be removed in 4.0.
> Also added some tests in AuthCacheTest.
> |[trunk|https://github.com/apache/cassandra/compare/trunk...kgreav:authcache]|
> |[utests|https://circleci.com/gh/kgreav/cassandra/206]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Created] (CASSANDRA-14662) Refactor AuthCache

2018-08-22 Thread Kurt Greaves (JIRA)
Kurt Greaves created CASSANDRA-14662:


 Summary: Refactor AuthCache
 Key: CASSANDRA-14662
 URL: https://issues.apache.org/jira/browse/CASSANDRA-14662
 Project: Cassandra
  Issue Type: Improvement
  Components: Auth
Reporter: Kurt Greaves
Assignee: Kurt Greaves


When building an LDAP IAuthenticator plugin I ran into a few issues when trying 
to reuse the AuthCache similar to how PasswordAuthenticator implements it. Most 
of the problems stemmed from the underlying cache being inaccessible and not 
being able to override {{initCache}} properly.

Anyway, I've had a stab at refactoring AuthCache with the following 
improvements:
# Make it possible to extend and override all necessary methods (initCache, 
init, validate)
# Makes it possible to specify a {{CacheLoader}} rather than just a 
{{Function}}, allowing you to have a get/load that throws exceptions.
# Use AuthCache on its own rather than extending it for each use case 
({{invalidate(K)}} moved to be part of MBean)
# Provided a builder that uses sane defaults so we don't have unnecessary 
repeated code everywhere

The refactor made all the extensions of AuthCache unnecessary, so I've 
simplified those cases to use AuthCache and removed any classes extending 
AuthCache. I also removed some noop compatibility classes that were marked to 
be removed in 4.0.

Also added some tests in AuthCacheTest.

|[trunk|https://github.com/apache/cassandra/compare/trunk...kgreav:authcache]|
|[utests|https://circleci.com/gh/kgreav/cassandra/206]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14662) Refactor AuthCache

2018-08-22 Thread Kurt Greaves (JIRA)


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

Kurt Greaves updated CASSANDRA-14662:
-
Fix Version/s: 4.x

> Refactor AuthCache
> --
>
> Key: CASSANDRA-14662
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14662
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Auth
>Reporter: Kurt Greaves
>Assignee: Kurt Greaves
>Priority: Major
> Fix For: 4.x
>
>
> When building an LDAP IAuthenticator plugin I ran into a few issues when 
> trying to reuse the AuthCache similar to how PasswordAuthenticator implements 
> it. Most of the problems stemmed from the underlying cache being inaccessible 
> and not being able to override {{initCache}} properly.
> Anyway, I've had a stab at refactoring AuthCache with the following 
> improvements:
> # Make it possible to extend and override all necessary methods (initCache, 
> init, validate)
> # Makes it possible to specify a {{CacheLoader}} rather than just a 
> {{Function}}, allowing you to have a get/load that throws exceptions.
> # Use AuthCache on its own rather than extending it for each use case 
> ({{invalidate(K)}} moved to be part of MBean)
> # Provided a builder that uses sane defaults so we don't have unnecessary 
> repeated code everywhere
> The refactor made all the extensions of AuthCache unnecessary, so I've 
> simplified those cases to use AuthCache and removed any classes extending 
> AuthCache. I also removed some noop compatibility classes that were marked to 
> be removed in 4.0.
> Also added some tests in AuthCacheTest.
> |[trunk|https://github.com/apache/cassandra/compare/trunk...kgreav:authcache]|
> |[utests|https://circleci.com/gh/kgreav/cassandra/206]|



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-10540) RangeAwareCompaction

2018-08-22 Thread Kurt Greaves (JIRA)


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

Kurt Greaves commented on CASSANDRA-10540:
--

[~krummas] Any hope of getting this in by 4.0? I'd be committed to 
testing/verifying/benchmarking post freeze if we can get it in.
Code-wise seems fine to me, and our testing so far seems to show that it's 
viable. Any reason we can't get this committed and fix/revert during freeze if 
issues come up?

> RangeAwareCompaction
> 
>
> Key: CASSANDRA-10540
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10540
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Major
>  Labels: 4.0-feature-freeze-review-requested, compaction, lcs, 
> vnodes
> Fix For: 4.x
>
>
> Broken out from CASSANDRA-6696, we should split sstables based on ranges 
> during compaction.
> Requirements;
> * dont create tiny sstables - keep them bunched together until a single vnode 
> is big enough (configurable how big that is)
> * make it possible to run existing compaction strategies on the per-range 
> sstables
> We should probably add a global compaction strategy parameter that states 
> whether this should be enabled or not.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-10726) Read repair inserts should not be blocking

2018-08-22 Thread Kurt Greaves (JIRA)


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

Kurt Greaves commented on CASSANDRA-10726:
--

[~bdeggleston], looks like this [broke 
eclipse-warnings|https://circleci.com/gh/kgreav/cassandra/204] because we don't 
close the SingleRangeResponse in {{sendNextRequests}}. I fixed it with [this 
change|https://github.com/apache/cassandra/commit/75f9bcb40bd98f4c0c364f9d26d46be88a1f107f].

> Read repair inserts should not be blocking
> --
>
> Key: CASSANDRA-10726
> URL: https://issues.apache.org/jira/browse/CASSANDRA-10726
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Coordination
>Reporter: Richard Low
>Assignee: Blake Eggleston
>Priority: Major
> Fix For: 4.0
>
>
> Today, if there’s a digest mismatch in a foreground read repair, the insert 
> to update out of date replicas is blocking. This means, if it fails, the read 
> fails with a timeout. If a node is dropping writes (maybe it is overloaded or 
> the mutation stage is backed up for some other reason), all reads to a 
> replica set could fail. Further, replicas dropping writes get more out of 
> sync so will require more read repair.
> The comment on the code for why the writes are blocking is:
> {code}
> // wait for the repair writes to be acknowledged, to minimize impact on any 
> replica that's
> // behind on writes in case the out-of-sync row is read multiple times in 
> quick succession
> {code}
> but the bad side effect is that reads timeout. Either the writes should not 
> be blocking or we should return success for the read even if the write times 
> out.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14409) Transient Replication: Support ring changes when transient replication is in use (add/remove node, change RF, add/remove DC)

2018-08-22 Thread Alex Petrov (JIRA)


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

Alex Petrov commented on CASSANDRA-14409:
-

Looks like {{StorageProxy#calculateRangesToStreamWithPreferredEndpoints}} can 
be simplified to something like 
[this|https://gist.github.com/ifesdjeen/298ea3f90935d66461dbaa550325f37f]. Most 
likely the same logic can (and probably even should) belong to 
{{calculateStreamAndFetchRanges}}, since it seems that we unnecessarily trying 
to loop and intersect the ranges again. 

This seems to have been caused by the fact 
{{calculateRangesToFetchWithPreferredEndpoints}} and 
{{calculateRangesToStreamWithPreferredEndpoints}} were receiving 
{{Range}}, not replica as they do right now, so in both cases we 
additionally call {{calculateNaturalReplicas}} (which we potentially might 
avoid). I think we should do _all_ range-related calculations (intersections, 
etc) in {{calculateStreamAndFetchRanges}}. The old code logic was just making a 
diff between old and current set per range like 
[here|https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/service/StorageService.java#L4337-L4342],
 so it seems that we can achieve the same without splitting the logic in two 
parts. 

> Transient Replication: Support ring changes when transient replication is in 
> use (add/remove node, change RF, add/remove DC)
> 
>
> Key: CASSANDRA-14409
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14409
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: Coordination, Core, Documentation and Website
>Reporter: Ariel Weisberg
>Assignee: Ariel Weisberg
>Priority: Major
> Fix For: 4.0
>
>
> The additional state transitions that transient replication introduces 
> require streaming and nodetool cleanup to behave differently. We already have 
> code that does the streaming, but in some cases we shouldn't stream any data 
> and in others when we stream to receive data we have to make sure we stream 
> from a full replica and not a transient replica.
> Transitioning from not replicated to transiently replicated means that a node 
> must stay pending until the next incremental repair completes at which point 
> the data for that range is known to be available at full replicas.
> Transitioning from transiently replicated to fully replicated requires 
> streaming from a full replica and is identical to how we stream from not 
> replicated to replicated. The transition must be managed so the transient 
> replica is not read from as a full replica until streaming completes. It can 
> be used immediately for a write quorum.
> Transitioning from fully replicated to transiently replicated requires 
> cleanup to remove repaired data from the transiently replicated range to 
> reclaim space. It can be used immediately for a write quorum.
> Transitioning from transiently replicated to not replicated requires cleanup 
> to be run to remove the formerly transiently replicated data.
> nodetool move, removenode, cleanup, decommission, and rebuild need to handle 
> these issues as does bootstrap.
> Update web site, documentation, NEWS.txt with a description of the steps for 
> doing common operations. Add/remove DC, Add/remove node(s), replace node, 
> change RF.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Commented] (CASSANDRA-14409) Transient Replication: Support ring changes when transient replication is in use (add/remove node, change RF, add/remove DC)

2018-08-22 Thread Benedict (JIRA)


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

Benedict commented on CASSANDRA-14409:
--

FWIW, I have separated out my review notes, my follow-up improvements, and my 
ReplicaCollection refactor, to simplify review.

[14409-replicas|https://github.com/belliottsmith/cassandra/tree/14409-replicas] 
contains the refactor only, and any necessary modifications for this to 
compile, squashed
[14409-improve|https://github.com/belliottsmith/cassandra/tree/14409-improve] 
contains my follow-up improvements, and I will continue pushing things that 
appear in review here

I have removed my own review notes into a separate branch.

> Transient Replication: Support ring changes when transient replication is in 
> use (add/remove node, change RF, add/remove DC)
> 
>
> Key: CASSANDRA-14409
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14409
> Project: Cassandra
>  Issue Type: Sub-task
>  Components: Coordination, Core, Documentation and Website
>Reporter: Ariel Weisberg
>Assignee: Ariel Weisberg
>Priority: Major
> Fix For: 4.0
>
>
> The additional state transitions that transient replication introduces 
> require streaming and nodetool cleanup to behave differently. We already have 
> code that does the streaming, but in some cases we shouldn't stream any data 
> and in others when we stream to receive data we have to make sure we stream 
> from a full replica and not a transient replica.
> Transitioning from not replicated to transiently replicated means that a node 
> must stay pending until the next incremental repair completes at which point 
> the data for that range is known to be available at full replicas.
> Transitioning from transiently replicated to fully replicated requires 
> streaming from a full replica and is identical to how we stream from not 
> replicated to replicated. The transition must be managed so the transient 
> replica is not read from as a full replica until streaming completes. It can 
> be used immediately for a write quorum.
> Transitioning from fully replicated to transiently replicated requires 
> cleanup to remove repaired data from the transiently replicated range to 
> reclaim space. It can be used immediately for a write quorum.
> Transitioning from transiently replicated to not replicated requires cleanup 
> to be run to remove the formerly transiently replicated data.
> nodetool move, removenode, cleanup, decommission, and rebuild need to handle 
> these issues as does bootstrap.
> Update web site, documentation, NEWS.txt with a description of the steps for 
> doing common operations. Add/remove DC, Add/remove node(s), replace node, 
> change RF.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14656) Full query log needs to log the keyspace

2018-08-22 Thread Marcus Eriksson (JIRA)


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

Marcus Eriksson updated CASSANDRA-14656:

Fix Version/s: 4.x
   Status: Patch Available  (was: Open)

patch: https://github.com/krummas/cassandra/commits/marcuse/14656
tests: https://circleci.com/gh/krummas/cassandra/tree/marcuse%2F14656

> Full query log needs to log the keyspace
> 
>
> Key: CASSANDRA-14656
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14656
> Project: Cassandra
>  Issue Type: New Feature
>Reporter: Marcus Eriksson
>Assignee: Marcus Eriksson
>Priority: Major
> Fix For: 4.x
>
>
> If the full query log is enabled and a set of clients have already executed 
> "USE " we can't figure out which keyspace the following queries are 
> executed against.
> We need this for CASSANDRA-14618



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org



[jira] [Updated] (CASSANDRA-14592) Reconcile should not be dependent on nowInSec

2018-08-22 Thread Aleksey Yeschenko (JIRA)


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

Aleksey Yeschenko updated CASSANDRA-14592:
--
Status: Ready to Commit  (was: Patch Available)

> Reconcile should not be dependent on nowInSec
> -
>
> Key: CASSANDRA-14592
> URL: https://issues.apache.org/jira/browse/CASSANDRA-14592
> Project: Cassandra
>  Issue Type: Bug
>Reporter: Benedict
>Assignee: Benedict
>Priority: Major
> Fix For: 4.0
>
>
> To have the arrival time of a mutation on a replica determine the 
> reconciliation priority seems to provide for unintuitive database behaviour.  
> It seems we should formalise our reconciliation logic in a manner that does 
> not depend on this, and modify our internal APIs to prevent this dependency.
>  
> Take the following example, where both writes have the same timestamp:
>  
> Write X with a value A, TTL of 1s
> Write Y with a value B, no TTL
>  
> If X and Y arrive on replicas in < 1s, X and Y are both live, so record Y 
> wins the reconciliation.  The value B appears in the database.
> However, if X and Y arrive on replicas in > 1s, X is now (effectively) a 
> tombstone.  This wins the reconciliation race, and NO value is the result.
>  
> Note that the weirdness of this is more pronounced than it might first 
> appear.  If write X gets stuck in hints for a period on the coordinator to 
> one replica, the value B appears in the database until the hint is replayed.  
> So now we’re in a very uncertain state - will hints get replayed or not?  If 
> they do, the value B will disappear; if they don’t it won’t.  This is despite 
> a QUORUM of replicas ACKing both writes, and a QUORUM of readers being 
> engaged on read; the database still changes state to the user suddenly at 
> some arbitrary future point in time.
>  
> It seems to me that a simple solution to this, is to permit TTL’d data to 
> always win a reconciliation against non-TTL’d data (of same timestamp), so 
> that we are consistent across TTLs being transformed into tombstones.
>  
> 4.0 seems like a good opportunity to fix this behaviour, and mention in 
> CHANGES.txt.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

-
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org