Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-07 Thread Shravan C
In fact I truncated hints table to stabilize the cluster. Through the heap 
dumps I was able to identify the table on which there were numerous queries. 
Then I focused on system_traces.session table around the time OOM occurred. It 
turned out to be a full table scan on a large table which caused OOM.


Thanks everyone of you.

From: Jeff Jirsa 
Sent: Tuesday, March 7, 2017 1:19 PM
To: user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time



On 2017-03-03 09:18 (-0800), Shravan Ch  wrote:
>
> nodetool compactionstats -H
> pending tasks: 3
> compaction typekeyspace  table   
> completed  totalunit   progress
> Compaction  system  hints 
> 28.5 GB   92.38 GB   bytes 30.85%
>
>

The hint buildup is also something that could have caused OOMs, too. Hints are 
stored for a given host in a single partition, which means it's common for a 
single row/partition to get huge if you have a single host flapping.

If you see "Compacting large row" messages for the hint rows, I suspect you'll 
find that one of the hosts/rows is responsible for most of that 92GB of hints, 
which means when you try to deliver the hints, you'll read from a huge 
partition, which creates memory pressure (see: CASSANDRA-9754) leading to GC 
pauses (or ooms), which then causes you to flap, which causes you to create 
more hints, which causes an ugly spiral.

In 3.0, hints were rewritten to avoid this problem, but short term, you may 
need to truncate your hints to get healthy (assuming it's safe for you to do 
so, where 'safe' is based on your read+write consistency levels).




Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-07 Thread Jeff Jirsa


On 2017-03-03 09:18 (-0800), Shravan Ch  wrote: 
> 
> nodetool compactionstats -H
> pending tasks: 3
> compaction typekeyspace  table   
> completed  totalunit   progress
> Compaction  system  hints 
> 28.5 GB   92.38 GB   bytes 30.85%
> 
> 

The hint buildup is also something that could have caused OOMs, too. Hints are 
stored for a given host in a single partition, which means it's common for a 
single row/partition to get huge if you have a single host flapping.

If you see "Compacting large row" messages for the hint rows, I suspect you'll 
find that one of the hosts/rows is responsible for most of that 92GB of hints, 
which means when you try to deliver the hints, you'll read from a huge 
partition, which creates memory pressure (see: CASSANDRA-9754) leading to GC 
pauses (or ooms), which then causes you to flap, which causes you to create 
more hints, which causes an ugly spiral.

In 3.0, hints were rewritten to avoid this problem, but short term, you may 
need to truncate your hints to get healthy (assuming it's safe for you to do 
so, where 'safe' is based on your read+write consistency levels).




Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-07 Thread Jeff Jirsa


On 2017-03-04 07:23 (-0800), "Thakrar, Jayesh"  
wrote: 
> LCS does not rule out frequent updates - it just says that there will be more 
> frequent compaction, which can potentially increase compaction activity 
> (which again can be throttled as needed).
> But STCS will guarantee OOM when you have large datasets.
> Did you have a look at the offheap + onheap size of our jvm using "nodetool 
> -info" ?
> 
> 

STCS does not guarantee you OOM when you have large datasets, unless by large 
datasets you mean in the tens-of-terabytes range, which is already something we 
typically recommend against.




Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-06 Thread Eric Evans
On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch  wrote:
> More than 30 plus Cassandra servers in the primary DC went down OOM
> exception below. What puzzles me is the scale at which it happened (at the
> same minute). I will share some more details below.

You'd be surprised; When it's the result of aberrant data/workload,
then having many nodes OOM at once is more common than you might
think.

> System Log: http://pastebin.com/iPeYrWVR

The traceback shows the OOM occurring during a read (a slice), not a
write.  What does your data model and queries look like?  Do you do
deletes (TTLs maybe)? Did the OOM result in a heap dump?

> GC Log: http://pastebin.com/CzNNGs0r
>
> During the OOM I saw lot of WARNings like the below (these were there for
> quite sometime may be weeks)
> WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252
> - Batch of prepared statements for [keyspace.table] is of size 225455,
> exceeding specified threshold of 65536 by 159919.
>
> Environment:
> We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C*
> nodes on SSD and apps run here)  and secondary DC (geographically remote and
> more like a DR to primary) on SAS drives.
> Cassandra config:
>
> Java 1.8.0_65
> Garbage Collector: G1GC
> memtable_allocation_type: offheap_objects
>
> Post this OOM I am seeing huge hints pile up on majority of the nodes and
> the pending hints keep going up. I have increased HintedHandoff CoreThreads
> to 6 but that did not help (I admit that I tried this on one node to try).
>
> nodetool compactionstats -H
> pending tasks: 3
> compaction typekeyspace  table
> completed  totalunit   progress
> Compaction  system  hints
> 28.5 GB   92.38 GB   bytes 30.85%



-- 
Eric Evans
john.eric.ev...@gmail.com


Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-04 Thread Thakrar, Jayesh
If possible, I would suggest running that command on a periodic basis (cron or 
whatever).
Also, you can run it on a single server and iterate through all the nodes in 
the cluster/DC.
Would also recommend running "nodetool compactionstats

And looked at your concern about high value for hinted handoff.
That's good (in a way), it ensures that updates are not lost.
Its possible because your DB was constantly being updated and the servers that 
come up started accumulating for the servers that were still down.
Furthermore, that may have been the situation also as the servers were going 
down.
Hence high hinted handoff is just a sign of pending updates that need to be 
applied, which is not uncommon if you had servers falling down/restarting like 
dominos and updates still coming in.

From: Shravan C 
Date: Saturday, March 4, 2017 at 11:15 AM
To: Conversant , Joaquin Casares 
, "user@cassandra.apache.org" 

Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


I was looking at nodetool info across all nodes. Consistently JVM heap used is 
~ 12GB and off heap is ~ 4-5GB.


From: Thakrar, Jayesh 
Sent: Saturday, March 4, 2017 9:23:01 AM
To: Shravan C; Joaquin Casares; user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time

LCS does not rule out frequent updates - it just says that there will be more 
frequent compaction, which can potentially increase compaction activity (which 
again can be throttled as needed).
But STCS will guarantee OOM when you have large datasets.
Did you have a look at the offheap + onheap size of our jvm using "nodetool 
-info" ?


From: Shravan C 
Date: Friday, March 3, 2017 at 11:11 PM
To: Joaquin Casares , "user@cassandra.apache.org" 

Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


We run C* at 32 GB and all servers have 96GB RAM. We use STCS . LCS is not an 
option for us as we have frequent updates.


Thanks,
Shravan

From: Thakrar, Jayesh 
Sent: Friday, March 3, 2017 3:47:27 PM
To: Joaquin Casares; user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


Had been fighting a similar battle, but am now over the hump for most part.



Get info on the server config (e.g. memory, cpu, free memory (free -g), etc)

Run "nodetool info" on the nodes to get heap and off-heap sizes

Run "nodetool tablestats" or "nodetool tablestats ." on the 
key large tables

Essentially the purpose is to see if you really had a true OOM or was your 
machine running out of memory.



Cassandra can use offheap memory very well - so "nodetool info" will give you 
both heap and offheap.



Also, what is the compaction strategy of your tables?



Personally, I have found STCS to be awful at large scale - when you have 
sstables that are 100+ GB in size.

See 
https://issues.apache.org/jira/browse/CASSANDRA-10821?focusedCommentId=15389451=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15389451



LCS seems better and should be the default (my opinion) unless you want DTCS



A good description of all three compactions is here - 
http://docs.scylladb.com/kb/compaction/
Documentation
docs.scylladb.com
Scylla is a Cassandra-compatible NoSQL data store that can handle 1 million 
transactions per second on a single server.








From: Joaquin Casares 
Date: Friday, March 3, 2017 at 11:34 AM
To: 
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time



Hello Shravan,



Typically asynchronous requests are recommended over batch statements since 
batch statements will cause more work on the coordinator node while individual 
requests, when using a TokenAwarePolicy, will hit a specific coordinator, 
perform a local disk seek, and return the requested information.



The only times that using batch statements are ideal is if writing to the same 
partition key, even if it's across multiple tables when using the same hashing 
algorithm (like murmur3).



Could you provide a bit of insight into what the batch statement was trying to 
accomplish and how many child statements were bundled up within that batch?



Cheers,



Joaquin


Joaquin Casares

Consultant

Austin, TX



Apache Cassandra Consulting

http://www.thelastpickle.com
The Last Pickle • Apache Cassandra Consulting & 
Services
www.thelastpickle.com
Apache Cassandra Consulting & Services. Our wealth of experience with Apache 
Cassandra will ensure success at all stages of a your project lifecycle.




On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch 

Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-04 Thread Priyanka


Sent from my iPhone

> On Mar 3, 2017, at 12:18 PM, Shravan Ch  wrote:
> 
> Hello,
> 
> More than 30 plus Cassandra servers in the primary DC went down OOM exception 
> below. What puzzles me is the scale at which it happened (at the same 
> minute). I will share some more details below. 
> 
> System Log: http://pastebin.com/iPeYrWVR
> GC Log: http://pastebin.com/CzNNGs0r
> 
> During the OOM I saw lot of WARNings like the below (these were there for 
> quite sometime may be weeks)
> WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252 
> - Batch of prepared statements for [keyspace.table] is of size 225455, 
> exceeding specified threshold of 65536 by 159919.
> 
> Environment:
> We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C* 
> nodes on SSD and apps run here)  and secondary DC (geographically remote and 
> more like a DR to primary) on SAS drives. 
> Cassandra config:
> 
> Java 1.8.0_65
> Garbage Collector: G1GC
> memtable_allocation_type: offheap_objects
> 
> Post this OOM I am seeing huge hints pile up on majority of the nodes and the 
> pending hints keep going up. I have increased HintedHandoff CoreThreads to 6 
> but that did not help (I admit that I tried this on one node to try).
> 
> nodetool compactionstats -H
> pending tasks: 3
> compaction typekeyspace  table   
> completed  totalunit   progress
> Compaction  system  hints 
> 28.5 GB   92.38 GB   bytes 30.85%
> 
> 
> Appreciate your inputs here. 
> 
> Thanks,
> Shravan


Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-04 Thread Shravan C
I was looking at nodetool info across all nodes. Consistently JVM heap used is 
~ 12GB and off heap is ~ 4-5GB.


From: Thakrar, Jayesh 
Sent: Saturday, March 4, 2017 9:23:01 AM
To: Shravan C; Joaquin Casares; user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time

LCS does not rule out frequent updates - it just says that there will be more 
frequent compaction, which can potentially increase compaction activity (which 
again can be throttled as needed).
But STCS will guarantee OOM when you have large datasets.
Did you have a look at the offheap + onheap size of our jvm using "nodetool 
-info" ?


From: Shravan C 
Date: Friday, March 3, 2017 at 11:11 PM
To: Joaquin Casares , "user@cassandra.apache.org" 

Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


We run C* at 32 GB and all servers have 96GB RAM. We use STCS . LCS is not an 
option for us as we have frequent updates.


Thanks,
Shravan

From: Thakrar, Jayesh 
Sent: Friday, March 3, 2017 3:47:27 PM
To: Joaquin Casares; user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


Had been fighting a similar battle, but am now over the hump for most part.



Get info on the server config (e.g. memory, cpu, free memory (free -g), etc)

Run "nodetool info" on the nodes to get heap and off-heap sizes

Run "nodetool tablestats" or "nodetool tablestats ." on the 
key large tables

Essentially the purpose is to see if you really had a true OOM or was your 
machine running out of memory.



Cassandra can use offheap memory very well - so "nodetool info" will give you 
both heap and offheap.



Also, what is the compaction strategy of your tables?



Personally, I have found STCS to be awful at large scale - when you have 
sstables that are 100+ GB in size.

See 
https://issues.apache.org/jira/browse/CASSANDRA-10821?focusedCommentId=15389451=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15389451



LCS seems better and should be the default (my opinion) unless you want DTCS



A good description of all three compactions is here - 
http://docs.scylladb.com/kb/compaction/
Documentation
docs.scylladb.com
Scylla is a Cassandra-compatible NoSQL data store that can handle 1 million 
transactions per second on a single server.








From: Joaquin Casares 
Date: Friday, March 3, 2017 at 11:34 AM
To: 
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time



Hello Shravan,



Typically asynchronous requests are recommended over batch statements since 
batch statements will cause more work on the coordinator node while individual 
requests, when using a TokenAwarePolicy, will hit a specific coordinator, 
perform a local disk seek, and return the requested information.



The only times that using batch statements are ideal is if writing to the same 
partition key, even if it's across multiple tables when using the same hashing 
algorithm (like murmur3).



Could you provide a bit of insight into what the batch statement was trying to 
accomplish and how many child statements were bundled up within that batch?



Cheers,



Joaquin


Joaquin Casares

Consultant

Austin, TX



Apache Cassandra Consulting

http://www.thelastpickle.com
The Last Pickle • Apache Cassandra Consulting & 
Services
www.thelastpickle.com
Apache Cassandra Consulting & Services. Our wealth of experience with Apache 
Cassandra will ensure success at all stages of a your project lifecycle.




On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch 
> wrote:

Hello,

More than 30 plus Cassandra servers in the primary DC went down OOM exception 
below. What puzzles me is the scale at which it happened (at the same minute). 
I will share some more details below.

System Log: http://pastebin.com/iPeYrWVR

GC Log: http://pastebin.com/CzNNGs0r

During the OOM I saw lot of WARNings like the below (these were there for quite 
sometime may be weeks)
WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252 - 
Batch of prepared statements for [keyspace.table] is of size 225455, exceeding 
specified threshold of 65536 by 159919.

Environment:
We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C* 
nodes on SSD and apps run here)  and secondary DC (geographically remote and 
more like a DR to primary) on SAS drives.
Cassandra config:

Java 1.8.0_65
Garbage Collector: G1GC
memtable_allocation_type: offheap_objects

Post this OOM I am seeing huge hints pile up on majority of the nodes and the 
pending hints keep going up. I have increased HintedHandoff CoreThreads to 6 
but 

Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-04 Thread Edward Capriolo
On Saturday, March 4, 2017, Thakrar, Jayesh 
wrote:

> LCS does not rule out frequent updates - it just says that there will be
> more frequent compaction, which can potentially increase compaction
> activity (which again can be throttled as needed).
>
> But STCS will guarantee OOM when you have large datasets.
>
> Did you have a look at the offheap + onheap size of our jvm using
> "nodetool -info" ?
>
>
>
>
>
> *From: *Shravan C  >
> *Date: *Friday, March 3, 2017 at 11:11 PM
> *To: *Joaquin Casares  >, "
> user@cassandra.apache.org
> " <
> user@cassandra.apache.org
> >
> *Subject: *Re: OOM on Apache Cassandra on 30 Plus node at the same time
>
>
>
> We run C* at 32 GB and all servers have 96GB RAM. We use STCS . LCS is not
> an option for us as we have frequent updates.
>
>
>
> Thanks,
>
> Shravan
> --
>
> *From:* Thakrar, Jayesh  >
> *Sent:* Friday, March 3, 2017 3:47:27 PM
> *To:* Joaquin Casares; user@cassandra.apache.org
> 
> *Subject:* Re: OOM on Apache Cassandra on 30 Plus node at the same time
>
>
>
> Had been fighting a similar battle, but am now over the hump for most part.
>
>
>
> Get info on the server config (e.g. memory, cpu, free memory (free -g),
> etc)
>
> Run "nodetool info" on the nodes to get heap and off-heap sizes
>
> Run "nodetool tablestats" or "nodetool tablestats ."
> on the key large tables
>
> Essentially the purpose is to see if you really had a true OOM or was your
> machine running out of memory.
>
>
>
> Cassandra can use offheap memory very well - so "nodetool info" will give
> you both heap and offheap.
>
>
>
> Also, what is the compaction strategy of your tables?
>
>
>
> Personally, I have found STCS to be awful at large scale - when you have
> sstables that are 100+ GB in size.
>
> See https://issues.apache.org/jira/browse/CASSANDRA-10821?
> focusedCommentId=15389451=com.atlassian.jira.
> plugin.system.issuetabpanels:comment-tabpanel#comment-15389451
>
>
>
> LCS seems better and should be the default (my opinion) unless you want
> DTCS
>
>
>
> A good description of all three compactions is here -
> http://docs.scylladb.com/kb/compaction/
>
> Documentation 
>
> docs.scylladb.com
>
> Scylla is a Cassandra-compatible NoSQL data store that can handle 1
> million transactions per second on a single server.
>
>
>
>
>
>
>
> *From: *Joaquin Casares  >
> *Date: *Friday, March 3, 2017 at 11:34 AM
> *To: * >
> *Subject: *Re: OOM on Apache Cassandra on 30 Plus node at the same time
>
>
>
> Hello Shravan,
>
>
>
> Typically asynchronous requests are recommended over batch statements
> since batch statements will cause more work on the coordinator node while
> individual requests, when using a TokenAwarePolicy, will hit a specific
> coordinator, perform a local disk seek, and return the requested
> information.
>
>
>
> The only times that using batch statements are ideal is if writing to the
> same partition key, even if it's across multiple tables when using the same
> hashing algorithm (like murmur3).
>
>
>
> Could you provide a bit of insight into what the batch statement was
> trying to accomplish and how many child statements were bundled up within
> that batch?
>
>
>
> Cheers,
>
>
>
> Joaquin
>
>
> Joaquin Casares
>
> Consultant
>
> Austin, TX
>
>
>
> Apache Cassandra Consulting
>
> http://www.thelastpickle.com
>
> The Last Pickle • Apache Cassandra Consulting & Services
> 
>
> www.thelastpickle.com
>
> Apache Cassandra Consulting & Services. Our wealth of experience with
> Apache Cassandra will ensure success at all stages of a your project
> lifecycle.
>
>
>
> On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch  > wrote:
>
> Hello,
>
> More than 30 plus Cassandra servers in the primary DC went down OOM
> exception below. What puzzles me is the scale at which it happened (at the
> same minute). I will share some more details below.
>
> System Log: http://pastebin.com/iPeYrWVR
>
> GC Log: http://pastebin.com/CzNNGs0r
>
> During the OOM I saw lot of WARNings like the below (these were there for
> quite sometime may be weeks)
> *WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209
> BatchStatement.java:252 - Batch of prepared statements for [keyspace.table]
> is of size 225455, exceeding 

Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-04 Thread Thakrar, Jayesh
LCS does not rule out frequent updates - it just says that there will be more 
frequent compaction, which can potentially increase compaction activity (which 
again can be throttled as needed).
But STCS will guarantee OOM when you have large datasets.
Did you have a look at the offheap + onheap size of our jvm using "nodetool 
-info" ?


From: Shravan C 
Date: Friday, March 3, 2017 at 11:11 PM
To: Joaquin Casares , "user@cassandra.apache.org" 

Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


We run C* at 32 GB and all servers have 96GB RAM. We use STCS . LCS is not an 
option for us as we have frequent updates.


Thanks,
Shravan

From: Thakrar, Jayesh 
Sent: Friday, March 3, 2017 3:47:27 PM
To: Joaquin Casares; user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


Had been fighting a similar battle, but am now over the hump for most part.



Get info on the server config (e.g. memory, cpu, free memory (free -g), etc)

Run "nodetool info" on the nodes to get heap and off-heap sizes

Run "nodetool tablestats" or "nodetool tablestats ." on the 
key large tables

Essentially the purpose is to see if you really had a true OOM or was your 
machine running out of memory.



Cassandra can use offheap memory very well - so "nodetool info" will give you 
both heap and offheap.



Also, what is the compaction strategy of your tables?



Personally, I have found STCS to be awful at large scale - when you have 
sstables that are 100+ GB in size.

See 
https://issues.apache.org/jira/browse/CASSANDRA-10821?focusedCommentId=15389451=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15389451



LCS seems better and should be the default (my opinion) unless you want DTCS



A good description of all three compactions is here - 
http://docs.scylladb.com/kb/compaction/
Documentation
docs.scylladb.com
Scylla is a Cassandra-compatible NoSQL data store that can handle 1 million 
transactions per second on a single server.








From: Joaquin Casares 
Date: Friday, March 3, 2017 at 11:34 AM
To: 
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time



Hello Shravan,



Typically asynchronous requests are recommended over batch statements since 
batch statements will cause more work on the coordinator node while individual 
requests, when using a TokenAwarePolicy, will hit a specific coordinator, 
perform a local disk seek, and return the requested information.



The only times that using batch statements are ideal is if writing to the same 
partition key, even if it's across multiple tables when using the same hashing 
algorithm (like murmur3).



Could you provide a bit of insight into what the batch statement was trying to 
accomplish and how many child statements were bundled up within that batch?



Cheers,



Joaquin


Joaquin Casares

Consultant

Austin, TX



Apache Cassandra Consulting

http://www.thelastpickle.com
The Last Pickle • Apache Cassandra Consulting & 
Services
www.thelastpickle.com
Apache Cassandra Consulting & Services. Our wealth of experience with Apache 
Cassandra will ensure success at all stages of a your project lifecycle.




On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch 
> wrote:

Hello,

More than 30 plus Cassandra servers in the primary DC went down OOM exception 
below. What puzzles me is the scale at which it happened (at the same minute). 
I will share some more details below.

System Log: http://pastebin.com/iPeYrWVR

GC Log: http://pastebin.com/CzNNGs0r

During the OOM I saw lot of WARNings like the below (these were there for quite 
sometime may be weeks)
WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252 - 
Batch of prepared statements for [keyspace.table] is of size 225455, exceeding 
specified threshold of 65536 by 159919.

Environment:
We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C* 
nodes on SSD and apps run here)  and secondary DC (geographically remote and 
more like a DR to primary) on SAS drives.
Cassandra config:

Java 1.8.0_65
Garbage Collector: G1GC
memtable_allocation_type: offheap_objects

Post this OOM I am seeing huge hints pile up on majority of the nodes and the 
pending hints keep going up. I have increased HintedHandoff CoreThreads to 6 
but that did not help (I admit that I tried this on one node to try).

nodetool compactionstats -H
pending tasks: 3
compaction typekeyspace  table   completed  
totalunit   progress
Compaction  system  hints 28.5 
GB   92.38 GB   bytes 30.85%


Appreciate your inputs here.


Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-03 Thread Shravan C
We run C* at 32 GB and all servers have 96GB RAM. We use STCS . LCS is not an 
option for us as we have frequent updates.


Thanks,
Shravan

From: Thakrar, Jayesh 
Sent: Friday, March 3, 2017 3:47:27 PM
To: Joaquin Casares; user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time


Had been fighting a similar battle, but am now over the hump for most part.



Get info on the server config (e.g. memory, cpu, free memory (free -g), etc)

Run "nodetool info" on the nodes to get heap and off-heap sizes

Run "nodetool tablestats" or "nodetool tablestats ." on the 
key large tables

Essentially the purpose is to see if you really had a true OOM or was your 
machine running out of memory.



Cassandra can use offheap memory very well - so "nodetool info" will give you 
both heap and offheap.



Also, what is the compaction strategy of your tables?



Personally, I have found STCS to be awful at large scale - when you have 
sstables that are 100+ GB in size.

See 
https://issues.apache.org/jira/browse/CASSANDRA-10821?focusedCommentId=15389451=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15389451



LCS seems better and should be the default (my opinion) unless you want DTCS



A good description of all three compactions is here - 
http://docs.scylladb.com/kb/compaction/

Documentation
docs.scylladb.com
Scylla is a Cassandra-compatible NoSQL data store that can handle 1 million 
transactions per second on a single server.








From: Joaquin Casares 
Date: Friday, March 3, 2017 at 11:34 AM
To: 
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time



Hello Shravan,



Typically asynchronous requests are recommended over batch statements since 
batch statements will cause more work on the coordinator node while individual 
requests, when using a TokenAwarePolicy, will hit a specific coordinator, 
perform a local disk seek, and return the requested information.



The only times that using batch statements are ideal is if writing to the same 
partition key, even if it's across multiple tables when using the same hashing 
algorithm (like murmur3).



Could you provide a bit of insight into what the batch statement was trying to 
accomplish and how many child statements were bundled up within that batch?



Cheers,



Joaquin


Joaquin Casares

Consultant

Austin, TX



Apache Cassandra Consulting

http://www.thelastpickle.com

The Last Pickle • Apache Cassandra Consulting & 
Services
www.thelastpickle.com
Apache Cassandra Consulting & Services. Our wealth of experience with Apache 
Cassandra will ensure success at all stages of a your project lifecycle.




On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch 
> wrote:

Hello,

More than 30 plus Cassandra servers in the primary DC went down OOM exception 
below. What puzzles me is the scale at which it happened (at the same minute). 
I will share some more details below.

System Log: http://pastebin.com/iPeYrWVR

GC Log: http://pastebin.com/CzNNGs0r

During the OOM I saw lot of WARNings like the below (these were there for quite 
sometime may be weeks)
WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252 - 
Batch of prepared statements for [keyspace.table] is of size 225455, exceeding 
specified threshold of 65536 by 159919.

Environment:
We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C* 
nodes on SSD and apps run here)  and secondary DC (geographically remote and 
more like a DR to primary) on SAS drives.
Cassandra config:

Java 1.8.0_65
Garbage Collector: G1GC
memtable_allocation_type: offheap_objects

Post this OOM I am seeing huge hints pile up on majority of the nodes and the 
pending hints keep going up. I have increased HintedHandoff CoreThreads to 6 
but that did not help (I admit that I tried this on one node to try).

nodetool compactionstats -H
pending tasks: 3
compaction typekeyspace  table   completed  
totalunit   progress
Compaction  system  hints 28.5 
GB   92.38 GB   bytes 30.85%


Appreciate your inputs here.

Thanks,

Shravan




Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-03 Thread Thakrar, Jayesh
Had been fighting a similar battle, but am now over the hump for most part.

Get info on the server config (e.g. memory, cpu, free memory (free -g), etc)
Run "nodetool info" on the nodes to get heap and off-heap sizes
Run "nodetool tablestats" or "nodetool tablestats ." on the 
key large tables
Essentially the purpose is to see if you really had a true OOM or was your 
machine running out of memory.

Cassandra can use offheap memory very well - so "nodetool info" will give you 
both heap and offheap.

Also, what is the compaction strategy of your tables?

Personally, I have found STCS to be awful at large scale - when you have 
sstables that are 100+ GB in size.
See 
https://issues.apache.org/jira/browse/CASSANDRA-10821?focusedCommentId=15389451=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15389451

LCS seems better and should be the default (my opinion) unless you want DTCS

A good description of all three compactions is here - 
http://docs.scylladb.com/kb/compaction/



From: Joaquin Casares 
Date: Friday, March 3, 2017 at 11:34 AM
To: 
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time

Hello Shravan,

Typically asynchronous requests are recommended over batch statements since 
batch statements will cause more work on the coordinator node while individual 
requests, when using a TokenAwarePolicy, will hit a specific coordinator, 
perform a local disk seek, and return the requested information.

The only times that using batch statements are ideal is if writing to the same 
partition key, even if it's across multiple tables when using the same hashing 
algorithm (like murmur3).

Could you provide a bit of insight into what the batch statement was trying to 
accomplish and how many child statements were bundled up within that batch?

Cheers,

Joaquin

Joaquin Casares
Consultant
Austin, TX

Apache Cassandra Consulting
http://www.thelastpickle.com

On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch 
> wrote:
Hello,

More than 30 plus Cassandra servers in the primary DC went down OOM exception 
below. What puzzles me is the scale at which it happened (at the same minute). 
I will share some more details below.
System Log: http://pastebin.com/iPeYrWVR
GC Log: http://pastebin.com/CzNNGs0r

During the OOM I saw lot of WARNings like the below (these were there for quite 
sometime may be weeks)
WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252 - 
Batch of prepared statements for [keyspace.table] is of size 225455, exceeding 
specified threshold of 65536 by 159919.

Environment:
We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C* 
nodes on SSD and apps run here)  and secondary DC (geographically remote and 
more like a DR to primary) on SAS drives.
Cassandra config:

Java 1.8.0_65
Garbage Collector: G1GC
memtable_allocation_type: offheap_objects

Post this OOM I am seeing huge hints pile up on majority of the nodes and the 
pending hints keep going up. I have increased HintedHandoff CoreThreads to 6 
but that did not help (I admit that I tried this on one node to try).
nodetool compactionstats -H
pending tasks: 3
compaction typekeyspace  table   completed  
totalunit   progress
Compaction  system  hints 28.5 
GB   92.38 GB   bytes 30.85%


Appreciate your inputs here.
Thanks,
Shravan



Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-03 Thread Shravan C
Hi Joaquin,


We have inserts going into a tracking table. Tracking table is a simple table 
[PRIMARY KEY (comid, status_timestamp) ] with a few tracking attributes and 
sorted by status_timestamp. From a volume perspective it is not a whole lot.


Thanks,
Shravan

From: Joaquin Casares 
Sent: Friday, March 3, 2017 11:34:58 AM
To: user@cassandra.apache.org
Subject: Re: OOM on Apache Cassandra on 30 Plus node at the same time

Hello Shravan,

Typically asynchronous requests are recommended over batch statements since 
batch statements will cause more work on the coordinator node while individual 
requests, when using a TokenAwarePolicy, will hit a specific coordinator, 
perform a local disk seek, and return the requested information.

The only times that using batch statements are ideal is if writing to the same 
partition key, even if it's across multiple tables when using the same hashing 
algorithm (like murmur3).

Could you provide a bit of insight into what the batch statement was trying to 
accomplish and how many child statements were bundled up within that batch?

Cheers,

Joaquin

Joaquin Casares
Consultant
Austin, TX

Apache Cassandra Consulting
http://www.thelastpickle.com

On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch 
> wrote:
Hello,

More than 30 plus Cassandra servers in the primary DC went down OOM exception 
below. What puzzles me is the scale at which it happened (at the same minute). 
I will share some more details below.

System Log: http://pastebin.com/iPeYrWVR
GC Log: http://pastebin.com/CzNNGs0r

During the OOM I saw lot of WARNings like the 
below (these were there for quite sometime may be weeks)
WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252 - 
Batch of prepared statements for [keyspace.table] is of size 225455, exceeding 
specified threshold of 65536 by 159919.

Environment:
We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C* 
nodes on SSD and apps run here)  and secondary DC (geographically remote and 
more like a DR to primary) on SAS drives.
Cassandra config:

Java 1.8.0_65
Garbage Collector: G1GC
memtable_allocation_type: offheap_objects

Post this OOM I am seeing huge hints pile up on majority of the nodes and the 
pending hints keep going up. I have increased HintedHandoff CoreThreads to 6 
but that did not help (I admit that I tried this on one node to try).

nodetool compactionstats -H
pending tasks: 3
compaction typekeyspace  table   completed  
totalunit   progress
Compaction  system  hints 28.5 
GB   92.38 GB   bytes 30.85%


Appreciate your inputs here.

Thanks,
Shravan



Re: OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-03 Thread Joaquin Casares
Hello Shravan,

Typically asynchronous requests are recommended over batch statements since
batch statements will cause more work on the coordinator node while
individual requests, when using a TokenAwarePolicy, will hit a specific
coordinator, perform a local disk seek, and return the requested
information.

The only times that using batch statements are ideal is if writing to the
same partition key, even if it's across multiple tables when using the same
hashing algorithm (like murmur3).

Could you provide a bit of insight into what the batch statement was trying
to accomplish and how many child statements were bundled up within that
batch?

Cheers,

Joaquin

Joaquin Casares
Consultant
Austin, TX

Apache Cassandra Consulting
http://www.thelastpickle.com

On Fri, Mar 3, 2017 at 11:18 AM, Shravan Ch  wrote:

> Hello,
>
> More than 30 plus Cassandra servers in the primary DC went down OOM
> exception below. What puzzles me is the scale at which it happened (at the
> same minute). I will share some more details below.
>
> System Log: http://pastebin.com/iPeYrWVR
> GC Log: http://pastebin.com/CzNNGs0r
>
> During the OOM I saw lot of WARNings like
> the below (these were there for quite sometime may be weeks)
> *WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209
> BatchStatement.java:252 - Batch of prepared statements for [keyspace.table]
> is of size 225455, exceeding specified threshold of 65536 by 159919.*
>
> * Environment:*
> We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more
> C* nodes on SSD and apps run here)  and secondary DC (geographically remote
> and more like a DR to primary) on SAS drives.
> Cassandra config:
>
> Java 1.8.0_65
> Garbage Collector: G1GC
> memtable_allocation_type: offheap_objects
>
> Post this OOM I am seeing huge hints pile up on majority of the nodes and
> the pending hints keep going up. I have increased HintedHandoff
> CoreThreads to 6 but that did not help (I admit that I tried this on one
> node to try).
>
> nodetool compactionstats -H
> pending tasks: 3
> compaction typekeyspace  table
> completed  totalunit   progress
> Compaction  system  hints
> 28.5 GB   92.38 GB   bytes 30.85%
>
>
> Appreciate your inputs here.
>
> Thanks,
> Shravan
>


OOM on Apache Cassandra on 30 Plus node at the same time

2017-03-03 Thread Shravan Ch
Hello,

More than 30 plus Cassandra servers in the primary DC went down OOM exception 
below. What puzzles me is the scale at which it happened (at the same minute). 
I will share some more details below.

System Log: http://pastebin.com/iPeYrWVR
GC Log: http://pastebin.com/CzNNGs0r

During the OOM I saw lot of WARNings like the 
below (these were there for quite sometime may be weeks)
WARN  [SharedPool-Worker-81] 2017-03-01 19:55:41,209 BatchStatement.java:252 - 
Batch of prepared statements for [keyspace.table] is of size 225455, exceeding 
specified threshold of 65536 by 159919.

Environment:
We are using ApacheCassandra-2.1.9 on Multi DC cluster. Primary DC (more C* 
nodes on SSD and apps run here)  and secondary DC (geographically remote and 
more like a DR to primary) on SAS drives.
Cassandra config:

Java 1.8.0_65
Garbage Collector: G1GC
memtable_allocation_type: offheap_objects

Post this OOM I am seeing huge hints pile up on majority of the nodes and the 
pending hints keep going up. I have increased HintedHandoff CoreThreads to 6 
but that did not help (I admit that I tried this on one node to try).

nodetool compactionstats -H
pending tasks: 3
compaction typekeyspace  table   completed  
totalunit   progress
Compaction  system  hints 28.5 
GB   92.38 GB   bytes 30.85%


Appreciate your inputs here.

Thanks,
Shravan