[jira] [Comment Edited] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder edited comment on CASSANDRA-13948 at 10/13/17 1:04 AM:
--

Just a heads up, I have been seeing these deadlocks happen easily, so I am 
running your patch [~pauloricardomg] in addition to 
Marcus's[patch|https://github.com/krummas/cassandra/commits/marcuse/13215] from 
CASSANDRA-13215.

I do see a large number of "Could not acquire references for compacting 
SSTable" happening, in bursts. Will upload a log file.

I also see some of this:
{noformat}
java.lang.AssertionError: Memory was freed
at org.apache.cassandra.io.util.Memory.checkBounds(Memory.java:344) 
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at org.apache.cassandra.io.util.Memory.getInt(Memory.java:291) 
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.getPositionInSummary(IndexSummary.java:148)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.fillTemporaryKey(IndexSummary.java:162)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.binarySearch(IndexSummary.java:121)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.format.SSTableReader.getSampleIndexesForRanges(SSTableReader.java:1370)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.format.SSTableReader.estimatedKeysForRanges(SSTableReader.java:1326)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:441)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.findDroppableSSTable(LeveledCompactionStrategy.java:503)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:121)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:124)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:262)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_144]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[na:1.8.0_144]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[na:1.8.0_144]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
[na:1.8.0_144]
at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
 [apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_144]
{noformat}

UPDATE: the node does successfully complete compactions for a while but 
gradually does fewer and fewer and then stops compacting altogether, even 
though compactionstats says there are pending compactions:

{noformat}
dkinder@seu-walker-fs01:~$ nodetool compactionstats
pending tasks: 102
- walker.links: 102

dkinder@seu-walker-fs01:~$
{noformat}
It stays this way and periodically prints out some "Could not acquire 
references for compacting SSTable" messages. It is able to compact some more if 
I restart the node.

tablestats for walker.links:
{noformat}
Keyspace : walker
Read Count: 16952
Read Latency: 10.388668062765454 ms.
Write Count: 277291
Write Latency: 0.0186555207345352 ms.
Pending Flushes: 0
Table: links
SSTable count: 8507
SSTables in each level: [73/4, 81/10, 297/100, 2078/1000, 2402, 
3635, 0, 0, 0]
Space used (live): 4902698702556
Space used (total): 4902698702556
Space used by snapshots (total): 13788057680993
Off heap memory used (total): 9835235
SSTable Compression Ratio: -1.0
Number of partitions (estimate): 19996043
Memtable cell count: 360248
Memtable data size: 36729792
Memtable off heap memory used: 0
Memtable switch count: 0
Local read count: 0
Local read latency: NaN ms
Local write count: 360248
Local write latency: 0.017 ms
Pending flushes: 0
Percent repaired: 0.0
Bloom filter false positives: 0

[jira] [Commented] (CASSANDRA-7839) Support standard EC2 naming conventions in Ec2Snitch

2017-10-12 Thread Daniel Bankhead (JIRA)

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

Daniel Bankhead commented on CASSANDRA-7839:


[~jasobrown] Sure thing, I can try both.

> Support standard EC2 naming conventions in Ec2Snitch
> 
>
> Key: CASSANDRA-7839
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7839
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Gregory Ramsperger
>Assignee: Gregory Ramsperger
>  Labels: docs-impacting
> Attachments: CASSANDRA-7839-aws-naming-conventions.patch
>
>
> The EC2 snitches use datacenter and rack naming conventions inconsistent with 
> those presented in Amazon EC2 APIs as region and availability zone. A 
> discussion of this is found in CASSANDRA-4026. This has not been changed for 
> valid backwards compatibility reasons. Using SnitchProperties, it is possible 
> to switch between the legacy naming and the full, AWS-style naming. 
> Proposal:
> * introduce a property (ec2_naming_scheme) to switch naming schemes.
> * default to current/legacy naming scheme
> * add support for a new scheme ("standard") which is consistent AWS 
> conventions
> ** data centers will be the region name, including the number
> ** racks will be the availability zone name, including the region name
> Examples:
> * * legacy* : datacenter is the part of the availability zone name preceding 
> the last "\-" when the zone ends in \-1 and includes the number if not \-1. 
> Rack is the portion of the availability zone name following  the last "\-".
> ** us-west-1a => dc: us-west, rack: 1a
> ** us-west-2b => dc: us-west-2, rack: 2b; 
> * *standard* : datacenter is the part of the availability zone name preceding 
> zone letter. rack is the entire availability zone name.
> ** us-west-1a => dc: us-west-1, rack: us-west-1a
> ** us-west-2b => dc: us-west-2, rack: us-west-2b; 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Commented] (CASSANDRA-3200) Repair: compare all trees together (for a given range/cf) instead of by pair in isolation

2017-10-12 Thread Blake Eggleston (JIRA)

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

Blake Eggleston commented on CASSANDRA-3200:


bq. yeah I agree it duplicates a lot of code, but they also do different things 
- the asymmetric ones don't need the merkle trees for example since we compare 
everything outside of this class now. Let me know if you see a straight-forward 
way to do it. I'll try to break out the common code in a separate class. 
Hopefully the non-symmetric classes can be removed once we have confidence this 
works as well.

Good point, I wasn’t paying attention to the stuff going on in their respective 
base classes

bq. indentation looks good to me and they look good on github or am I 
misunderstanding you?

The formatting of the matrices looks good, they just look weird starting at 
column 0 when the rest of the method / comment is indented 8 spaces. iow, 
something like this:

{code}
 /*
  ... something ...

A   B   C   D   E
  A =   x   x   x
  B x   x   x
  C x   x
  D =
 */
{code}

Second round of review:

Everything looks good for the most part, and your optimization / stream 
reduction stuff makes sense. There are just a few minor things:

HostDifferences:
* {{hasDifferencesFor}} isEmpty check is uneccesary

ReducedDifferenceHolder
* Probably don’t need this class, ImmutableMap 
should be fine

RepairOption
* default for optimizeStreams seems to be false, but javadoc says it’s true

AsymmetricLocalSyncTask
* uncomment or remove logger info statement at line 95

AsymmetricSyncTask
* startTime is compared to Long.MIN_VALUE in {{finished}}, but it never 
initialized to that value. Unless I’m mistaken, long values that aren’t 
explicitly initialized to some value become 0 by default, so that branch in 
finished will always run, even if {{run}} wasn’t called.

> Repair: compare all trees together (for a given range/cf) instead of by pair 
> in isolation
> -
>
> Key: CASSANDRA-3200
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3200
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Sylvain Lebresne
>Assignee: Marcus Eriksson
>Priority: Minor
>  Labels: repair
> Fix For: 4.x
>
>
> Currently, repair compare merkle trees by pair, in isolation of any other 
> tree. What that means concretely is that if I have three node A, B and C 
> (RF=3) with A and B in sync, but C having some range r inconsitent with both 
> A and B (since those are consistent), we will do the following transfer of r: 
> A -> C, C -> A, B -> C, C -> B.
> The fact that we do both A -> C and C -> A is fine, because we cannot know 
> which one is more to date from A or C. However, the transfer B -> C is 
> useless provided we do A -> C if A and B are in sync. Not doing that transfer 
> will be a 25% improvement in that case. With RF=5 and only one node 
> inconsistent with all the others, that almost a 40% improvement, etc...
> Given that this situation of one node not in sync while the others are is 
> probably fairly common (one node died so it is behind), this could be a fair 
> improvement over what is transferred. In the case where we use repair to 
> rebuild completely a node, this will be a dramatic improvement, because it 
> will avoid the rebuilded node to get RF times the data it should get.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Commented] (CASSANDRA-12673) Nodes cannot see each other in multi-DC, non-EC2 environment with two-interface nodes due to outbound node-to-node connection binding to private interface

2017-10-12 Thread Jia Tan (JIRA)

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

Jia Tan commented on CASSANDRA-12673:
-

[~mmajercik] Got it. So it's not likely that there will be further fixes for 
this issue in 3.0 releases. Unfortunately our migration to 3.11 isn't happening 
any time soon so we'll have to manually fix it. Thanks!

> Nodes cannot see each other in multi-DC, non-EC2 environment with 
> two-interface nodes due to outbound node-to-node connection binding to 
> private interface
> --
>
> Key: CASSANDRA-12673
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12673
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Multi-DC, non-EC2 environment with two-interface nodes
>Reporter: Milan Majercik
>Assignee: Milan Majercik
>Priority: Minor
> Fix For: 2.2.9, 3.0.11, 3.10
>
>
> We have a two-DC cluster in non-EC2 environment with each node containing two 
> interfaces, one using private addresses for intra-DC communication and the 
> other using public addresses for inter-DC communication. After proper 
> configuration setup needed for this kind of environment we observed nodes 
> cannot see each other.
> The configuration changes made for this purpose are as follows:
> *listen_address*: bound to private interface
> *broadcast_address*: bound to public address
> *listen_on_broadcast_address*: true
> *endpoint_snitch*: GossipingPropertyFileSnitch
> *prefer_local*=true (in cassandra-rackdc.properties)
> Upon restart, cassandra node contacts other nodes with their public addresses 
> which is essential for making contacts to foreign data centers. After 
> exhaustive investigation we found cassandra binds outbound node-to-node 
> connections to private interface (the one specified in listen_address) that 
> poses a problem for our environment as these data centers _do not allow 
> connections from private interface to public network_.
> A portion of cassandra code responsible for local binding of outbound 
> connections can be found in method 
> {{org.apache.cassandra.net.OutboundTcpConnectionPool.newSocket}}:
> {code}
> if (!Config.getOutboundBindAny())
> channel.bind(new 
> InetSocketAddress(FBUtilities.getLocalAddress(), 0));
> {code}
> After we commented out these two lines and deployed cassandra.jar across the 
> cluster, the nodes were able to see each other and everything appears to be 
> working fine, including two-DC setup.
> Do you think it's possible to remove these two lines without negative 
> consequences? Alternatively, if the local binding serves some specific 
> purpose of which I'm ignorant would it be possible to make it configurable?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Updated] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder updated CASSANDRA-13948:
---
Attachment: debug.log

> Avoid deadlock when not able to acquire references for compaction
> -
>
> Key: CASSANDRA-13948
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13948
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Paulo Motta
>Assignee: Paulo Motta
> Fix For: 3.11.x, 4.x
>
> Attachments: debug.log
>
>
> The thread dump below shows a race between an sstable replacement by the 
> {{IndexSummaryRedistribution}} and 
> {{AbstractCompactionTask.getNextBackgroundTask}}:
> {noformat}
> Thread 94580: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() 
> @bci=1, line=836 (Compiled frame)
>  - 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.util.concurrent.locks.AbstractQueuedSynchronizer$Node,
>  int) @bci=67, line=870 (Compiled frame)
>  - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(int) 
> @bci=17, line=1199 (Compiled frame)
>  - java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock() @bci=5, 
> line=943 (Compiled frame)
>  - 
> org.apache.cassandra.db.compaction.CompactionStrategyManager.handleListChangedNotification(java.lang.Iterable,
>  java.lang.Iterable) @bci=359, line=483 (Interpreted frame)
>  - 
> org.apache.cassandra.db.compaction.CompactionStrategyManager.handleNotification(org.apache.cassandra.notifications.INotification,
>  java.lang.Object) @bci=53, line=555 (Interpreted frame)
>  - 
> org.apache.cassandra.db.lifecycle.Tracker.notifySSTablesChanged(java.util.Collection,
>  java.util.Collection, org.apache.cassandra.db.compaction.OperationType, 
> java.lang.Throwable) @bci=50, line=409 (Interpreted frame)
>  - 
> org.apache.cassandra.db.lifecycle.LifecycleTransaction.doCommit(java.lang.Throwable)
>  @bci=157, line=227 (Interpreted frame)
>  - 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.commit(java.lang.Throwable)
>  @bci=61, line=116 (Compiled frame)
>  - 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.commit()
>  @bci=2, line=200 (Interpreted frame)
>  - 
> org.apache.cassandra.utils.concurrent.Transactional$AbstractTransactional.finish()
>  @bci=5, line=185 (Interpreted frame)
>  - 
> org.apache.cassandra.io.sstable.IndexSummaryRedistribution.redistributeSummaries()
>  @bci=559, line=130 (Interpreted frame)
>  - 
> org.apache.cassandra.db.compaction.CompactionManager.runIndexSummaryRedistribution(org.apache.cassandra.io.sstable.IndexSummaryRedistribution)
>  @bci=9, line=1420 (Interpreted frame)
>  - 
> org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries(org.apache.cassandra.io.sstable.IndexSummaryRedistribution)
>  @bci=4, line=250 (Interpreted frame)
>  - 
> org.apache.cassandra.io.sstable.IndexSummaryManager.redistributeSummaries() 
> @bci=30, line=228 (Interpreted frame)
>  - org.apache.cassandra.io.sstable.IndexSummaryManager$1.runMayThrow() 
> @bci=4, line=125 (Interpreted frame)
>  - org.apache.cassandra.utils.WrappedRunnable.run() @bci=1, line=28 
> (Interpreted frame)
>  - 
> org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor$UncomplainingRunnable.run()
>  @bci=4, line=118 (Compiled frame)
>  - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 
> (Compiled frame)
>  - java.util.concurrent.FutureTask.runAndReset() @bci=47, line=308 (Compiled 
> frame)
>  - 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask)
>  @bci=1, line=180 (Compiled frame)
>  - java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run() 
> @bci=37, line=294 (Compiled frame)
>  - 
> java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
>  @bci=95, line=1149 (Compiled frame)
>  - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 
> (Interpreted frame)
>  - 
> org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(java.lang.Runnable)
>  @bci=1, line=81 (Interpreted frame)
>  - org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$8.run() @bci=4 
> (Interpreted frame)
>  - java.lang.Thread.run() @bci=11, line=748 (Compiled frame)
> {noformat}
> {noformat}
> Thread 94573: (state = IN_JAVA)
>  - java.util.HashMap$HashIterator.nextNode() @bci=95, line=1441 (Compiled 
> frame; 

[jira] [Commented] (CASSANDRA-13480) nodetool repair can hang forever if we lose the notification for the repair completing/failing

2017-10-12 Thread ASF GitHub Bot (JIRA)

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

ASF GitHub Bot commented on CASSANDRA-13480:


Github user Jollyplum closed the pull request at:

https://github.com/apache/cassandra/pull/122


> nodetool repair can hang forever if we lose the notification for the repair 
> completing/failing
> --
>
> Key: CASSANDRA-13480
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13480
> Project: Cassandra
>  Issue Type: Bug
>  Components: Tools
>Reporter: Matt Byrd
>Assignee: Matt Byrd
>Priority: Minor
>  Labels: repair
> Fix For: 4.0
>
>
> When a Jmx lost notification occurs, sometimes the lost notification in 
> question is the notification which let's RepairRunner know that the repair is 
> finished (ProgressEventType.COMPLETE or even ERROR for that matter).
> This results in nodetool process running the repair hanging forever. 
> I have a test which reproduces the issue here:
> https://github.com/Jollyplum/cassandra-dtest/tree/repair_hang_test
> To fix this, If on receiving a notification that notifications have been lost 
> (JMXConnectionNotification.NOTIFS_LOST), we instead query a new endpoint via 
> Jmx to receive all the relevant notifications we're interested in, we can 
> replay those we missed and avoid this scenario.
> It's possible also that the JMXConnectionNotification.NOTIFS_LOST itself 
> might be lost and so for good measure I have made RepairRunner poll 
> periodically to see if there were any notifications that had been sent but we 
> didn't receive (scoped just to the particular tag for the given repair).
> Users who don't use nodetool but go via jmx directly, can still use this new 
> endpoint and implement similar behaviour in their clients as desired.
> I'm also expiring the notifications which have been kept on the server side.
> Please let me know if you've any questions or can think of a different 
> approach, I also tried setting:
>  JVM_OPTS="$JVM_OPTS -Djmx.remote.x.notification.buffer.size=5000"
> but this didn't fix the test. I suppose it might help under certain scenarios 
> but in this test we don't even send that many notifications so I'm not 
> surprised it doesn't fix it.
> It seems like getting lost notifications is always a potential problem with 
> jmx as far as I can tell.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Commented] (CASSANDRA-13948) Avoid deadlock when not able to acquire references for compaction

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder commented on CASSANDRA-13948:


Just a heads up, I have been seeing these deadlocks happen easily, so I am 
running your patch [~pauloricardomg] in addition to 
Marcus's[patch|https://github.com/krummas/cassandra/commits/marcuse/13215] from 
CASSANDRA-13215.

I do see a large number of "Could not acquire references for compacting 
SSTables 
[BigTableReader(path='/srv/disk2/cassandra-data/walker/links/mc-6566879-big-Data.db')]
 which is not a problem per se,unless it happens frequently, in which case it 
must be reported. Will retry later." happening, in bursts. Will upload a log 
file.

I also see some of this:
{noformat}
java.lang.AssertionError: Memory was freed
at org.apache.cassandra.io.util.Memory.checkBounds(Memory.java:344) 
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at org.apache.cassandra.io.util.Memory.getInt(Memory.java:291) 
~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.getPositionInSummary(IndexSummary.java:148)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.fillTemporaryKey(IndexSummary.java:162)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.IndexSummary.binarySearch(IndexSummary.java:121)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.format.SSTableReader.getSampleIndexesForRanges(SSTableReader.java:1370)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.io.sstable.format.SSTableReader.estimatedKeysForRanges(SSTableReader.java:1326)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.AbstractCompactionStrategy.worthDroppingTombstones(AbstractCompactionStrategy.java:441)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.findDroppableSSTable(LeveledCompactionStrategy.java:503)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(LeveledCompactionStrategy.java:121)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.CompactionStrategyManager.getNextBackgroundTask(CompactionStrategyManager.java:124)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:262)
 ~[apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
~[na:1.8.0_144]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) 
~[na:1.8.0_144]
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) 
~[na:1.8.0_144]
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) 
[na:1.8.0_144]
at 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:81)
 [apache-cassandra-3.11.0.jar:3.11.2-SNAPSHOT]
at java.lang.Thread.run(Thread.java:748) ~[na:1.8.0_144]
{noformat}

> Avoid deadlock when not able to acquire references for compaction
> -
>
> Key: CASSANDRA-13948
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13948
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
>Reporter: Paulo Motta
>Assignee: Paulo Motta
> Fix For: 3.11.x, 4.x
>
>
> The thread dump below shows a race between an sstable replacement by the 
> {{IndexSummaryRedistribution}} and 
> {{AbstractCompactionTask.getNextBackgroundTask}}:
> {noformat}
> Thread 94580: (state = BLOCKED)
>  - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information 
> may be imprecise)
>  - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
> line=175 (Compiled frame)
>  - 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() 
> @bci=1, line=836 (Compiled frame)
>  - 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(java.util.concurrent.locks.AbstractQueuedSynchronizer$Node,
>  int) @bci=67, line=870 (Compiled frame)
>  - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(int) 
> @bci=17, line=1199 (Compiled frame)
>  - java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.lock() @bci=5, 
> line=943 (Compiled frame)
>  - 
> org.apache.cassandra.db.compaction.CompactionStrategyManager.handleListChangedNotification(java.lang.Iterable,
>  java.lang.Iterable) @bci=359, line=483 

[jira] [Commented] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13943:
-

Yeah getting the lock is clearly a problem

My thinking was that we should try if we see a big improvement when lowering 
the throughput, and thus finishing fewer compactions where we grab the write 
lock to replace sstables

You might also want to try the patch in CASSANDRA-13948

> Infinite compaction of L0 SSTables in JBOD
> --
>
> Key: CASSANDRA-13943
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13943
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: Cassandra 3.11.0 / Centos 6
>Reporter: Dan Kinder
>Assignee: Marcus Eriksson
> Attachments: cassandra-jstack-2017-10-12-infinite-sstable-adding.txt, 
> cassandra-jstack-2017-10-12.txt, cassandra.yaml, debug.log, 
> debug.log-with-commit-d8f3f2780, debug.log.1.zip, debug.log.zip, jvm.options
>
>
> I recently upgraded from 2.2.6 to 3.11.0.
> I am seeing Cassandra loop infinitely compacting the same data over and over. 
> Attaching logs.
> It is compacting two tables, one on /srv/disk10, the other on /srv/disk1. It 
> does create new SSTables but immediately recompacts again. Note that I am not 
> inserting anything at the moment, there is no flushing happening on this 
> table (Memtable switch count has not changed).
> My theory is that it somehow thinks those should be compaction candidates. 
> But they shouldn't be, they are on different disks and I ran nodetool 
> relocatesstables as well as nodetool compact. So, it tries to compact them 
> together, but the compaction results in the exact same 2 SSTables on the 2 
> disks, because the keys are split by data disk.
> This is pretty serious, because all our nodes right now are consuming CPU 
> doing this for multiple tables, it seems.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Commented] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder commented on CASSANDRA-13943:


Sure, I can try, but I'm not sure how that would help, these nodes have plenty 
of cpu and iops available... But there is nonetheless a problem, because 
flushers are getting stuck.

Looking at the stack trace, it seems to come down to the readLock in 
CompactionStrategyManager. The flushers are blocked on that:
{noformat}
Thread 7385: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may 
be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, 
line=175 (Compiled frame)
 - 
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt() 
@bci=1, line=836 (Compiled frame)
 - java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(int) 
@bci=83, line=967 (Interpreted frame)
 - java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(int) 
@bci=10, line=1283 (Compiled frame)
 - java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock() @bci=5, 
line=727 (Compiled frame)
 - 
org.apache.cassandra.db.compaction.CompactionStrategyManager.createSSTableMultiWriter(org.apache.cassandra.io.sstable.Descriptor,
 long, long, org.apache. cassandra.io.sstable.metadata.MetadataCollector, 
org.apache.cassandra.db.SerializationHeader, java.util.Collection, 
org.apache.cassandra.db.lifecycle.   LifecycleTransaction) @bci=4, line=901 
(Compiled frame)
 - 
org.apache.cassandra.db.ColumnFamilyStore.createSSTableMultiWriter(org.apache.cassandra.io.sstable.Descriptor,
 long, long, org.apache.cassandra.io.   sstable.metadata.MetadataCollector, 
org.apache.cassandra.db.SerializationHeader, 
org.apache.cassandra.db.lifecycle.LifecycleTransaction) @bci=21, line=515   
(Compiled frame)
 - 
org.apache.cassandra.db.Memtable$FlushRunnable.createFlushWriter(org.apache.cassandra.db.lifecycle.LifecycleTransaction,
 java.lang.String, org.apache.cassandra.db.PartitionColumns, 
org.apache.cassandra.db.rows.EncodingStats) @bci=104, line=506 (Compiled frame)
 - 
org.apache.cassandra.db.Memtable$FlushRunnable.(org.apache.cassandra.db.Memtable,
 java.util.concurrent.ConcurrentNavigableMap, 
org.apache.cassandra.db.Directories$DataDirectory, 
org.apache.cassandra.db.PartitionPosition, 
org.apache.cassandra.db.PartitionPosition, org.apache.cassandra.db.lifecycle.   
LifecycleTransaction) @bci=253, line=447 (Compiled frame)
 - 
org.apache.cassandra.db.Memtable$FlushRunnable.(org.apache.cassandra.db.Memtable,
 org.apache.cassandra.db.PartitionPosition, org.apache.cassandra.  
db.PartitionPosition, org.apache.cassandra.db.Directories$DataDirectory, 
org.apache.cassandra.db.lifecycle.LifecycleTransaction) @bci=19, line=417 
(Compiled frame)
 - 
org.apache.cassandra.db.Memtable.createFlushRunnables(org.apache.cassandra.db.lifecycle.LifecycleTransaction)
 @bci=125, line=318 (Interpreted frame)
 - 
org.apache.cassandra.db.Memtable.flushRunnables(org.apache.cassandra.db.lifecycle.LifecycleTransaction)
 @bci=2, line=300 (Compiled frame)
 - 
org.apache.cassandra.db.ColumnFamilyStore$Flush.flushMemtable(org.apache.cassandra.db.Memtable,
 boolean) @bci=82, line=1137 (Compiled frame)
 - org.apache.cassandra.db.ColumnFamilyStore$Flush.run() @bci=85, line=1102 
(Interpreted frame)
 - 
java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
 @bci=95, line=1149 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=624 
(Compiled frame)
 - 
org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(java.lang.Runnable)
 @bci=1, line=81 (Compiled frame)
 - org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$5.run() @bci=4 
(Compiled frame)
 - java.lang.Thread.run() @bci=11, line=748 (Compiled frame)
{noformat}

All the callers in CompactionStrategyManager that *aren't* blocked on a lock 
are trying to get background tasks:
{noformat}
Thread 1468: (state = IN_JAVA)
 - java.util.HashMap$HashIterator.nextNode() @bci=95, line=1441 (Compiled 
frame; information may be imprecise)
 - java.util.HashMap$KeyIterator.next() @bci=1, line=1461 (Compiled frame)
 - java.util.AbstractCollection.toArray() @bci=39, line=141 (Compiled frame)
 - java.util.ArrayList.(java.util.Collection) @bci=6, line=177 (Compiled 
frame)
 - 
org.apache.cassandra.db.compaction.LeveledManifest.ageSortedSSTables(java.util.Collection)
 @bci=5, line=731 (Compiled frame)
 - org.apache.cassandra.db.compaction.LeveledManifest.getCandidatesFor(int) 
@bci=187, line=644 (Compiled frame)
 - org.apache.cassandra.db.compaction.LeveledManifest.getCompactionCandidates() 
@bci=290, line=385 (Compiled frame)
 - 
org.apache.cassandra.db.compaction.LeveledCompactionStrategy.getNextBackgroundTask(int)
 @bci=4, line=119 (Compiled frame)
 - 

[jira] [Commented] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13943:
-

I think you could lower the number of {{concurrent_compactors}} (maybe 10 or 
so) and you probably need to set {{compaction_throughput_mb_per_sec}} to 
something conservative, I would start with 10 and then increase if things look 
good.

> Infinite compaction of L0 SSTables in JBOD
> --
>
> Key: CASSANDRA-13943
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13943
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: Cassandra 3.11.0 / Centos 6
>Reporter: Dan Kinder
>Assignee: Marcus Eriksson
> Attachments: cassandra-jstack-2017-10-12-infinite-sstable-adding.txt, 
> cassandra-jstack-2017-10-12.txt, cassandra.yaml, debug.log, 
> debug.log-with-commit-d8f3f2780, debug.log.1.zip, debug.log.zip, jvm.options
>
>
> I recently upgraded from 2.2.6 to 3.11.0.
> I am seeing Cassandra loop infinitely compacting the same data over and over. 
> Attaching logs.
> It is compacting two tables, one on /srv/disk10, the other on /srv/disk1. It 
> does create new SSTables but immediately recompacts again. Note that I am not 
> inserting anything at the moment, there is no flushing happening on this 
> table (Memtable switch count has not changed).
> My theory is that it somehow thinks those should be compaction candidates. 
> But they shouldn't be, they are on different disks and I ran nodetool 
> relocatesstables as well as nodetool compact. So, it tries to compact them 
> together, but the compaction results in the exact same 2 SSTables on the 2 
> disks, because the keys are split by data disk.
> This is pretty serious, because all our nodes right now are consuming CPU 
> doing this for multiple tables, it seems.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Updated] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder updated CASSANDRA-13943:
---
Attachment: cassandra.yaml
jvm.options
debug.log.zip
debug.log.1.zip
cassandra-jstack-2017-10-12-infinite-sstable-adding.txt

No, as far as I can tell no infinite compactions.

I'm doing several hundred mutation per second for one of the tables.

Attached debug logs and configuration. debug.log.1.zip contains node startup.

I also just started seeing some crazy behavior on a node I killed and 
restarted... it was adding the same SSTables over and over endlessly:
{noformat}
INFO  [CompactionExecutor:49] 2017-10-12 11:26:03,109 LeveledManifest.java:474 
- Adding high-level (L5) 
BigTableReader(path='/srv/disk11/cassandra-data/walker/links-a3d582c0a6683450a030de61dc4b9823/mc-1237377-big-Data.db')
 to candidates
INFO  [CompactionExecutor:49] 2017-10-12 11:26:03,110 LeveledManifest.java:474 
- Adding high-level (L5) 
BigTableReader(path='/srv/disk11/cassandra-data/walker/links-a3d582c0a6683450a030de61dc4b9823/mc-1237377-big-Data.db')
 to candidates
INFO  [CompactionExecutor:49] 2017-10-12 11:26:03,110 LeveledManifest.java:474 
- Adding high-level (L5) 
BigTableReader(path='/srv/disk11/cassandra-data/walker/links-a3d582c0a6683450a030de61dc4b9823/mc-1237377-big-Data.db')
 to candidates
INFO  [CompactionExecutor:49] 2017-10-12 11:26:03,110 LeveledManifest.java:474 
- Adding high-level (L5) 
BigTableReader(path='/srv/disk11/cassandra-data/walker/links-a3d582c0a6683450a030de61dc4b9823/mc-1237377-big-Data.db')
 to candidates
INFO  [CompactionExecutor:49] 2017-10-12 11:26:03,111 LeveledManifest.java:474 
- Adding high-level (L5) 
BigTableReader(path='/srv/disk11/cassandra-data/walker/links-a3d582c0a6683450a030de61dc4b9823/mc-1237377-big-Data.db')
 to candidates
INFO  [CompactionExecutor:49] 2017-10-12 11:26:03,111 LeveledManifest.java:474 
- Adding high-level (L5) 
BigTableReader(path='/srv/disk11/cassandra-data/walker/links-a3d582c0a6683450a030de61dc4b9823/mc-1237377-big-Data.db')
 to candidates
{noformat}
I took a quick jstack before killing it, attached as 
cassandra-jstack-2017-10-12-infinite-sstable-adding.txt

> Infinite compaction of L0 SSTables in JBOD
> --
>
> Key: CASSANDRA-13943
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13943
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: Cassandra 3.11.0 / Centos 6
>Reporter: Dan Kinder
>Assignee: Marcus Eriksson
> Attachments: cassandra-jstack-2017-10-12-infinite-sstable-adding.txt, 
> cassandra-jstack-2017-10-12.txt, cassandra.yaml, debug.log, 
> debug.log-with-commit-d8f3f2780, debug.log.1.zip, debug.log.zip, jvm.options
>
>
> I recently upgraded from 2.2.6 to 3.11.0.
> I am seeing Cassandra loop infinitely compacting the same data over and over. 
> Attaching logs.
> It is compacting two tables, one on /srv/disk10, the other on /srv/disk1. It 
> does create new SSTables but immediately recompacts again. Note that I am not 
> inserting anything at the moment, there is no flushing happening on this 
> table (Memtable switch count has not changed).
> My theory is that it somehow thinks those should be compaction candidates. 
> But they shouldn't be, they are on different disks and I ran nodetool 
> relocatesstables as well as nodetool compact. So, it tries to compact them 
> together, but the compaction results in the exact same 2 SSTables on the 2 
> disks, because the keys are split by data disk.
> This is pretty serious, because all our nodes right now are consuming CPU 
> doing this for multiple tables, it seems.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Comment Edited] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson edited comment on CASSANDRA-13943 at 10/12/17 5:31 PM:
---

but not seeing any infinite compactions in the logs anymore?

and could you post your cassandra.yaml and describe how much data you are 
writing?

the logs for a node would help as well


was (Author: krummas):
but not seeing any infinite compactions in the logs anymore?

and could you post your cassandra.yaml and describe how much data you are 
writing?

> Infinite compaction of L0 SSTables in JBOD
> --
>
> Key: CASSANDRA-13943
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13943
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: Cassandra 3.11.0 / Centos 6
>Reporter: Dan Kinder
>Assignee: Marcus Eriksson
> Attachments: cassandra-jstack-2017-10-12.txt, debug.log, 
> debug.log-with-commit-d8f3f2780
>
>
> I recently upgraded from 2.2.6 to 3.11.0.
> I am seeing Cassandra loop infinitely compacting the same data over and over. 
> Attaching logs.
> It is compacting two tables, one on /srv/disk10, the other on /srv/disk1. It 
> does create new SSTables but immediately recompacts again. Note that I am not 
> inserting anything at the moment, there is no flushing happening on this 
> table (Memtable switch count has not changed).
> My theory is that it somehow thinks those should be compaction candidates. 
> But they shouldn't be, they are on different disks and I ran nodetool 
> relocatesstables as well as nodetool compact. So, it tries to compact them 
> together, but the compaction results in the exact same 2 SSTables on the 2 
> disks, because the keys are split by data disk.
> This is pretty serious, because all our nodes right now are consuming CPU 
> doing this for multiple tables, it seems.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Commented] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Marcus Eriksson (JIRA)

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

Marcus Eriksson commented on CASSANDRA-13943:
-

but not seeing any infinite compactions in the logs anymore?

and could you post your cassandra.yaml and describe how much data you are 
writing?

> Infinite compaction of L0 SSTables in JBOD
> --
>
> Key: CASSANDRA-13943
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13943
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: Cassandra 3.11.0 / Centos 6
>Reporter: Dan Kinder
>Assignee: Marcus Eriksson
> Attachments: cassandra-jstack-2017-10-12.txt, debug.log, 
> debug.log-with-commit-d8f3f2780
>
>
> I recently upgraded from 2.2.6 to 3.11.0.
> I am seeing Cassandra loop infinitely compacting the same data over and over. 
> Attaching logs.
> It is compacting two tables, one on /srv/disk10, the other on /srv/disk1. It 
> does create new SSTables but immediately recompacts again. Note that I am not 
> inserting anything at the moment, there is no flushing happening on this 
> table (Memtable switch count has not changed).
> My theory is that it somehow thinks those should be compaction candidates. 
> But they shouldn't be, they are on different disks and I ran nodetool 
> relocatesstables as well as nodetool compact. So, it tries to compact them 
> together, but the compaction results in the exact same 2 SSTables on the 2 
> disks, because the keys are split by data disk.
> This is pretty serious, because all our nodes right now are consuming CPU 
> doing this for multiple tables, it seems.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Comment Edited] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder edited comment on CASSANDRA-13943 at 10/12/17 5:24 PM:
--

[~krummas] after a night I am seeing most nodes start to build up 
MemtablePostFlush Pending tasks (up to several hundred) and MemtableFlushWriter 
Pending tasks (less so than MemtablePostFlush). Several nodes have blocked on 
MutationStage and grown lots of pending tasks in that, I assume because 
flushing is blocked and there's nowhere else to write.

Attached a stack dump from one of these heavily blocked nodes.

Also notable, since it seems like the blocking is in 
{{LeveledManifest.getCompactionCandidates()}}, here is what the SSTable levels 
look like for the biggest table:

 52 SSTable Level: 0
 75 SSTable Level: 1
122 SSTable Level: 2
360 SSTable Level: 3
   2724 SSTable Level: 4
  10647 SSTable Level: 5

Several other tables have comparable number of SSTables in L0 (20-40 or so), 
but those only have a few hundred or a few thousand SSTables.


was (Author: dkinder):
[~krummas] after a night I am seeing most nodes start to build up 
MemtablePostFlush Pending tasks (up to several hundred) and MemtableFlushWriter 
Pending tasks (less so than MemtablePostFlush). Several nodes have blocked on 
MutationStage and grown lots of pending tasks in that, I assume because 
flushing is blocked and there's nowhere else to write.

Attached a stack dump from one of these heavily blocked nodes.

> Infinite compaction of L0 SSTables in JBOD
> --
>
> Key: CASSANDRA-13943
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13943
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: Cassandra 3.11.0 / Centos 6
>Reporter: Dan Kinder
>Assignee: Marcus Eriksson
> Attachments: cassandra-jstack-2017-10-12.txt, debug.log, 
> debug.log-with-commit-d8f3f2780
>
>
> I recently upgraded from 2.2.6 to 3.11.0.
> I am seeing Cassandra loop infinitely compacting the same data over and over. 
> Attaching logs.
> It is compacting two tables, one on /srv/disk10, the other on /srv/disk1. It 
> does create new SSTables but immediately recompacts again. Note that I am not 
> inserting anything at the moment, there is no flushing happening on this 
> table (Memtable switch count has not changed).
> My theory is that it somehow thinks those should be compaction candidates. 
> But they shouldn't be, they are on different disks and I ran nodetool 
> relocatesstables as well as nodetool compact. So, it tries to compact them 
> together, but the compaction results in the exact same 2 SSTables on the 2 
> disks, because the keys are split by data disk.
> This is pretty serious, because all our nodes right now are consuming CPU 
> doing this for multiple tables, it seems.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Updated] (CASSANDRA-13943) Infinite compaction of L0 SSTables in JBOD

2017-10-12 Thread Dan Kinder (JIRA)

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

Dan Kinder updated CASSANDRA-13943:
---
Attachment: cassandra-jstack-2017-10-12.txt

[~krummas] after a night I am seeing most nodes start to build up 
MemtablePostFlush Pending tasks (up to several hundred) and MemtableFlushWriter 
Pending tasks (less so than MemtablePostFlush). Several nodes have blocked on 
MutationStage and grown lots of pending tasks in that, I assume because 
flushing is blocked and there's nowhere else to write.

Attached a stack dump from one of these heavily blocked nodes.

> Infinite compaction of L0 SSTables in JBOD
> --
>
> Key: CASSANDRA-13943
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13943
> Project: Cassandra
>  Issue Type: Bug
>  Components: Compaction
> Environment: Cassandra 3.11.0 / Centos 6
>Reporter: Dan Kinder
>Assignee: Marcus Eriksson
> Attachments: cassandra-jstack-2017-10-12.txt, debug.log, 
> debug.log-with-commit-d8f3f2780
>
>
> I recently upgraded from 2.2.6 to 3.11.0.
> I am seeing Cassandra loop infinitely compacting the same data over and over. 
> Attaching logs.
> It is compacting two tables, one on /srv/disk10, the other on /srv/disk1. It 
> does create new SSTables but immediately recompacts again. Note that I am not 
> inserting anything at the moment, there is no flushing happening on this 
> table (Memtable switch count has not changed).
> My theory is that it somehow thinks those should be compaction candidates. 
> But they shouldn't be, they are on different disks and I ran nodetool 
> relocatesstables as well as nodetool compact. So, it tries to compact them 
> together, but the compaction results in the exact same 2 SSTables on the 2 
> disks, because the keys are split by data disk.
> This is pretty serious, because all our nodes right now are consuming CPU 
> doing this for multiple tables, it seems.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Comment Edited] (CASSANDRA-12673) Nodes cannot see each other in multi-DC, non-EC2 environment with two-interface nodes due to outbound node-to-node connection binding to private interface

2017-10-12 Thread Milan Majercik (JIRA)

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

Milan Majercik edited comment on CASSANDRA-12673 at 10/12/17 11:47 AM:
---

[~crazytan], in my opinion the {{outboundBindAny}} static field should be 
converted into regular configuration property as I originally proposed. However 
[~pauloricardomg] insisted to keep {{outboundBindAny}} static for releases 2.2 
and 3.0 in order to avoid unintended consequences. The good news is the  
{{outboundBindAny}} static field got eliminated since branch 3.11 so there is 
no binding of outbound connections to any specific interface.


was (Author: mmajercik):
[~crazytan], in my opinion the {{outboundBindAny}} static field should be 
converted into regular configuration property as I originally proposed. I see 
no reason for it to be kept that way release 3.0. [~pauloricardomg] insisted to 
keep {{outboundBindAny}} static in order to avoid unintended consequences, but 
I thought that applied only to release 2.2. 

> Nodes cannot see each other in multi-DC, non-EC2 environment with 
> two-interface nodes due to outbound node-to-node connection binding to 
> private interface
> --
>
> Key: CASSANDRA-12673
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12673
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Multi-DC, non-EC2 environment with two-interface nodes
>Reporter: Milan Majercik
>Assignee: Milan Majercik
>Priority: Minor
> Fix For: 2.2.9, 3.0.11, 3.10
>
>
> We have a two-DC cluster in non-EC2 environment with each node containing two 
> interfaces, one using private addresses for intra-DC communication and the 
> other using public addresses for inter-DC communication. After proper 
> configuration setup needed for this kind of environment we observed nodes 
> cannot see each other.
> The configuration changes made for this purpose are as follows:
> *listen_address*: bound to private interface
> *broadcast_address*: bound to public address
> *listen_on_broadcast_address*: true
> *endpoint_snitch*: GossipingPropertyFileSnitch
> *prefer_local*=true (in cassandra-rackdc.properties)
> Upon restart, cassandra node contacts other nodes with their public addresses 
> which is essential for making contacts to foreign data centers. After 
> exhaustive investigation we found cassandra binds outbound node-to-node 
> connections to private interface (the one specified in listen_address) that 
> poses a problem for our environment as these data centers _do not allow 
> connections from private interface to public network_.
> A portion of cassandra code responsible for local binding of outbound 
> connections can be found in method 
> {{org.apache.cassandra.net.OutboundTcpConnectionPool.newSocket}}:
> {code}
> if (!Config.getOutboundBindAny())
> channel.bind(new 
> InetSocketAddress(FBUtilities.getLocalAddress(), 0));
> {code}
> After we commented out these two lines and deployed cassandra.jar across the 
> cluster, the nodes were able to see each other and everything appears to be 
> working fine, including two-DC setup.
> Do you think it's possible to remove these two lines without negative 
> consequences? Alternatively, if the local binding serves some specific 
> purpose of which I'm ignorant would it be possible to make it configurable?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Commented] (CASSANDRA-12673) Nodes cannot see each other in multi-DC, non-EC2 environment with two-interface nodes due to outbound node-to-node connection binding to private interface

2017-10-12 Thread Milan Majercik (JIRA)

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

Milan Majercik commented on CASSANDRA-12673:


[~crazytan], in my opinion the {{outboundBindAny}} static field should be 
converted into regular configuration property as I originally proposed. I see 
no reason for it to be kept that way release 3.0. [~pauloricardomg] insisted to 
keep {{outboundBindAny}} static in order to avoid unintended consequences, but 
I thought that applied only to release 2.2. 

> Nodes cannot see each other in multi-DC, non-EC2 environment with 
> two-interface nodes due to outbound node-to-node connection binding to 
> private interface
> --
>
> Key: CASSANDRA-12673
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12673
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
> Environment: Multi-DC, non-EC2 environment with two-interface nodes
>Reporter: Milan Majercik
>Assignee: Milan Majercik
>Priority: Minor
> Fix For: 2.2.9, 3.0.11, 3.10
>
>
> We have a two-DC cluster in non-EC2 environment with each node containing two 
> interfaces, one using private addresses for intra-DC communication and the 
> other using public addresses for inter-DC communication. After proper 
> configuration setup needed for this kind of environment we observed nodes 
> cannot see each other.
> The configuration changes made for this purpose are as follows:
> *listen_address*: bound to private interface
> *broadcast_address*: bound to public address
> *listen_on_broadcast_address*: true
> *endpoint_snitch*: GossipingPropertyFileSnitch
> *prefer_local*=true (in cassandra-rackdc.properties)
> Upon restart, cassandra node contacts other nodes with their public addresses 
> which is essential for making contacts to foreign data centers. After 
> exhaustive investigation we found cassandra binds outbound node-to-node 
> connections to private interface (the one specified in listen_address) that 
> poses a problem for our environment as these data centers _do not allow 
> connections from private interface to public network_.
> A portion of cassandra code responsible for local binding of outbound 
> connections can be found in method 
> {{org.apache.cassandra.net.OutboundTcpConnectionPool.newSocket}}:
> {code}
> if (!Config.getOutboundBindAny())
> channel.bind(new 
> InetSocketAddress(FBUtilities.getLocalAddress(), 0));
> {code}
> After we commented out these two lines and deployed cassandra.jar across the 
> cluster, the nodes were able to see each other and everything appears to be 
> working fine, including two-DC setup.
> Do you think it's possible to remove these two lines without negative 
> consequences? Alternatively, if the local binding serves some specific 
> purpose of which I'm ignorant would it be possible to make it configurable?



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Updated] (CASSANDRA-12245) initial view build can be parallel

2017-10-12 Thread JIRA

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

Andrés de la Peña updated CASSANDRA-12245:
--
Status: Patch Available  (was: Open)

> initial view build can be parallel
> --
>
> Key: CASSANDRA-12245
> URL: https://issues.apache.org/jira/browse/CASSANDRA-12245
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Materialized Views
>Reporter: Tom van der Woerdt
>Assignee: Andrés de la Peña
> Fix For: 4.x
>
>
> On a node with lots of data (~3TB) building a materialized view takes several 
> weeks, which is not ideal. It's doing this in a single thread.
> There are several potential ways this can be optimized :
>  * do vnodes in parallel, instead of going through the entire range in one 
> thread
>  * just iterate through sstables, not worrying about duplicates, and include 
> the timestamp of the original write in the MV mutation. since this doesn't 
> exclude duplicates it does increase the amount of work and could temporarily 
> surface ghost rows (yikes) but I guess that's why they call it eventual 
> consistency. doing it this way can avoid holding references to all tables on 
> disk, allows parallelization, and removes the need to check other sstables 
> for existing data. this is essentially the 'do a full repair' path



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Comment Edited] (CASSANDRA-12245) initial view build can be parallel

2017-10-12 Thread JIRA

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

Andrés de la Peña edited comment on CASSANDRA-12245 at 10/12/17 8:10 AM:
-

Sorry for the delay, and thank for the thorough review. I've updated [the 
patch|https://github.com/apache/cassandra/compare/trunk...adelapena:12245-trunk]
 addressing the comments.

bq. It seems like the way the number of tokens in a range was computed by 
abs(range.right - range.left) may not work correctly for some wrap-around 
cases, as shown by [this test 
case|https://github.com/pauloricardomg/cassandra/blob/2760bbbc25a2ad9a9bbf9d29a0dc19e1e3bfb237/test/unit/org/apache/cassandra/dht/SplitterTest.java#L184].
 Even though this shouldn't break when local ranges are used , I fixed it on 
[this 
commit|https://github.com/pauloricardomg/cassandra/commit/2760bbbc25a2ad9a9bbf9d29a0dc19e1e3bfb237]
 to make sure split works correctly for wrap-around ranges. Can you confirm 
this is correct?

I think this is right, merged.

bq. Other than that, it seems like you added unit tests only for 
Murmur3Partitioner, would you mind extending testSplit() to RandomPartitioner?

Done 
[here|https://github.com/apache/cassandra/commit/6f92541abaa61ed789f50f67c8800af3f97162a5].

bq. I think having a dedicated executor will ensure view building doesn't 
compete with compactions for the compaction executor, good job! One problem I 
see though is that if the user is finding its view building slow it will try to 
increase the number of concurrent view builders via nodetool, but it will have 
no effect since the range was split in the previously number of concurrent view 
builders. Given this will be a pretty common scenario for large datasets, how 
about splitting the range in multiple smaller tasks, so that if the user 
increases {{concurrent_view_builders}} the other tasks immediately start 
executing?
bq. We could use a simple approach of splitting the local range in let's say 
1000 hard-coded parts, or be smarter and make each split have ~100MB or so. In 
this way we can keep {{concurrent_materialized_view_builders=1}} by default, 
and users with large base tables are able to increase it and see immediate 
effect via nodetool. WDYT?

This makes a lot of sense. I'm worried about creating thousands of tasks for 
large datasets if the number of tasks is relative to the amount of data. 
Instead, I think we could fix the number of partitions to the higher reasonable 
number of parallel tasks, something like [a multiple of the number of available 
processors|https://github.com/adelapena/cassandra/blob/e460b3b76935cad60a9dc1e00c8d3af8bfa9584a/src/java/org/apache/cassandra/db/view/ViewBuilder.java#L59].
 This would provide the desired immediate performance improvement if the user 
increases the number of concurrent view builders while keeping the number of 
tasks limited, independently of the amount of data. What do you think? Does it 
make any sense?

bq. Great, looks much cleaner indeed! One minor thing is that if there's a 
failure after some {{ViewBuildTasks}} were completed, it will resume that 
subtask from its last token while it already finished. Could we maybe set the 
last_token = end_token when the task is finished to flag it was already 
finished and avoid resuming the task when that is the case?

Done 
[here|https://github.com/adelapena/cassandra/blob/d505d8014524b72422ba1eee036494bda39f53f0/src/java/org/apache/cassandra/db/view/ViewBuilderTask.java#L166-L168].

One case that we hadn't considered is that if the token ranges change or are 
split in a different way when resuming a build then the progress would be lost, 
because {{ViewBuildTask}} won't found any entry for the new range at 
{{system.view_builds_in_progress}}. This would be specially true if we split 
the ranges by their data size. So, independently of how we finally split the 
ranges, I think it makes sense to load all the ranges with any progress from 
{{system.view_builds_in_progress}} at {{ViewBuilder}} before splitting the 
local ranges, create a task for those of them that are not already finish, and 
then split any remaining uncovered local range. It also has the advantage of 
skipping the creation of tasks for already completed ranges. What do you think?

I have also removed the method {{SystemKeyspace.beginViewBuild}} because I 
don't see any need of saving a range without progress. Indeed, if the view 
build is restarted it is probably better to don't restore the task without 
progress and let their tokens to be processed by the split logic. 

bq. The dtest looks mostly good, except for the following nits:
bq.  * {{concurrent_materialized_view_builders=1}} when the nodes are 
restarted. Can you set the configuration value during cluster setup phase 
(instead of setting via nodetool) to make sure the restarted view builds will 
be parallel?


[jira] [Commented] (CASSANDRA-13215) Cassandra nodes startup time 20x more after upgarding to 3.x

2017-10-12 Thread Corentin Chary (JIRA)

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

Corentin Chary commented on CASSANDRA-13215:


I'll try to test that in our test env in the next days :)

> Cassandra nodes startup time 20x more after upgarding to 3.x
> 
>
> Key: CASSANDRA-13215
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13215
> Project: Cassandra
>  Issue Type: Improvement
>  Components: Core
> Environment: Cluster setup: two datacenters (dc-main, dc-backup).
> dc-main - 9 servers, no vnodes
> dc-backup - 6 servers, vnodes
>Reporter: Viktor Kuzmin
>Assignee: Marcus Eriksson
> Attachments: simple-cache.patch
>
>
> CompactionStrategyManage.getCompactionStrategyIndex is called on each sstable 
> at startup. And this function calls StorageService.getDiskBoundaries. And 
> getDiskBoundaries calls AbstractReplicationStrategy.getAddressRanges.
> It appears that last function can be really slow. In our environment we have 
> 1545 tokens and with NetworkTopologyStrategy it can make 1545*1545 
> computations in worst case (maybe I'm wrong, but it really takes lot's of 
> cpu).
> Also this function can affect runtime later, cause it is called not only 
> during startup.
> I've tried to implement simple cache for getDiskBoundaries results and now 
> startup time is about one minute instead of 25m, but I'm not sure if it's a 
> good solution.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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



[jira] [Commented] (CASSANDRA-7839) Support standard EC2 naming conventions in Ec2Snitch

2017-10-12 Thread Jason Brown (JIRA)

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

Jason Brown commented on CASSANDRA-7839:


I've rebased [~ramsperger]'s patch on trunk and running tests here:

||7839||
|[branch|https://github.com/jasobrown/cassandra/tree/7839]|
|[dtest|https://builds.apache.org/view/A-D/view/Cassandra/job/Cassandra-devbranch-dtest/366/]|
|[utests|https://circleci.com/gh/jasobrown/cassandra/tree/7839]|

I cleaned up some constants from the original patch, but added a NEWS entry and 
the cross-schema naming check, as per [~pauloricardomg].

bq. should we add a bootstrap check (similar to checkForEndpointCollision) 
failing to boostrap a node if it's using the new scheme and nodes are detected 
in gossip using the legacy scheme

FWIW, this would prevent an operator from adding a new region in the standard 
style to an existing cluster (they would always need to set {{legacy}}). I am 
totally ok with that and it's not worth the extra effort to allow it. I'm just 
noting this point here for posterity.

Further, we should check at all node startups, not just bootstrap, because if 
the value in the {{conf/cassandra-rackdc.properties}} somehow gets flipped to 
the other value, it's gonna mess up the cluster metadata big time as peers will 
think the node is in a different DC/rack now and incorrectly adjust their view 
of the cluster.

I want to add some unit tests for 
{{Ec2Snitch@#hasConflictingDatacenterOrRack}}, and I'd like some feedback on 
the implementation, as well. [~pauloricardomg], if you don't mind taking a 
look, since you've done so in the past for this ticket, I'd appreciate it.

> Support standard EC2 naming conventions in Ec2Snitch
> 
>
> Key: CASSANDRA-7839
> URL: https://issues.apache.org/jira/browse/CASSANDRA-7839
> Project: Cassandra
>  Issue Type: Improvement
>Reporter: Gregory Ramsperger
>Assignee: Gregory Ramsperger
>  Labels: docs-impacting
> Attachments: CASSANDRA-7839-aws-naming-conventions.patch
>
>
> The EC2 snitches use datacenter and rack naming conventions inconsistent with 
> those presented in Amazon EC2 APIs as region and availability zone. A 
> discussion of this is found in CASSANDRA-4026. This has not been changed for 
> valid backwards compatibility reasons. Using SnitchProperties, it is possible 
> to switch between the legacy naming and the full, AWS-style naming. 
> Proposal:
> * introduce a property (ec2_naming_scheme) to switch naming schemes.
> * default to current/legacy naming scheme
> * add support for a new scheme ("standard") which is consistent AWS 
> conventions
> ** data centers will be the region name, including the number
> ** racks will be the availability zone name, including the region name
> Examples:
> * * legacy* : datacenter is the part of the availability zone name preceding 
> the last "\-" when the zone ends in \-1 and includes the number if not \-1. 
> Rack is the portion of the availability zone name following  the last "\-".
> ** us-west-1a => dc: us-west, rack: 1a
> ** us-west-2b => dc: us-west-2, rack: 2b; 
> * *standard* : datacenter is the part of the availability zone name preceding 
> zone letter. rack is the entire availability zone name.
> ** us-west-1a => dc: us-west-1, rack: us-west-1a
> ** us-west-2b => dc: us-west-2, rack: us-west-2b; 



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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