cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Michael Penick (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-6534) Slow inserts with collections into a single partition (Pathological GC behavior)
Date Tue, 31 Dec 2013 20:04:50 GMT

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

Michael Penick edited comment on CASSANDRA-6534 at 12/31/13 8:03 PM:
---------------------------------------------------------------------

Sorry, That's a pretty big detail. The inserts are being run in parallel via executeAsync()
in java-driver. This is a wide row, so the collection values are always being inserted into
a new column. So I don't think we're paying the cost to deserialize and replace values in
an existing map (if that's what you were getting at?).

I just don't understand how the the collection serialization (or collection processing) is
making that big a difference compared to the non-colleciton value. Wouldn't they both just
end up as ByteBuffers in the storage engine?   The serialized size of the collections should
be less than the largish values we were inserting with the non-colleciton test. So where's
all the extra temporary heap memory coming from?


was (Author: mpenick):
Sorry, That's a pretty big detail. The inserts are being run in parallel via executeAsync()
in java-driver. This is a wide row, so the collection values are being inserted into a new
column. So I don't think we're paying the cost to deserialize and replace values in an existing
map (if that's what you were getting at?).

I just don't understand how the the collection serialization (or collection processing) is
making that big a difference compared to the non-colleciton value. Wouldn't they both just
end up as ByteBuffers in the storage engine?   The serialized size of the collections should
be less than the largish values we were inserting with the non-colleciton test. So where's
all the extra temporary heap memory coming from?

> Slow inserts with collections into a single partition (Pathological GC behavior)
> --------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-6534
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6534
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>         Environment: dsc12-1.2.12-1.noarch.rpm
> cassandra12-1.2.12-1.noarch.rpm
> centos 6.4
>            Reporter: Michael Penick
>             Fix For: 1.2.12
>
>         Attachments: GC_behavior.png
>
>
> We noticed extremely slow insertion rates to a single partition key, using composite
column with a collection value. We were not able to replicate the issue using the same schema,
but with a non-colleciton value and using much larger values.  During the collection insertion
tests we have tons of these messages in the system.log:
> "GCInspector.java (line 119) GC for ConcurrentMarkSweep: 1287 ms for 2 collections, 1233256368
used; max is 8375238656"
> We are inserting a tiny amounts of data 32-64 bytes and seeing the issue after only a
couple 10k inserts. The amount of memory being used by C*/JVM is no where near proportional
to the amount data being inserted. Why is C* consuming so much memory?
> Attached is a picture of the GC under one of the pathological tests. Keep in mind we
are only inserting 128KB - 256KB of data and we are almost hitting the limit of the heap.
> GC flags:
> -XX:+UseThreadPriorities
> -XX:ThreadPriorityPolicy=42
> -Xms8192M
> -Xmx8192M
> -Xmn2048M
> -XX:+HeapDumpOnOutOfMemoryError
> -Xss180k
> -XX:+UseParNewGC
> -XX:+UseConcMarkSweepGC
> -XX:+CMSParallelRemarkEnabled
> -XX:SurvivorRatio=8
> -XX:MaxTenuringThreshold=1
> -XX:CMSInitiatingOccupancyFraction=75
> -XX:+UseCMSInitiatingOccupancyOnly
> -XX:+UseTLAB
> Example schemas:
> Note: The type of collection or primitive type in the collection doesn't seem to matter.
> {code}
> CREATE TABLE test.test (
> row_key text, 
> column_key uuid,
>  column_value list<int>, 
> PRIMARY KEY(row_key, column_key));
> CREATE TABLE test.test (
> row_key text, 
> column_key uuid, 
> column_value map<text, text>, 
> PRIMARY KEY(row_key, column_key));
> {code}
> Example inserts:
> Note: This issue is able to be replicated with extremely small inserts (a well as larger
~1KB)
> {code}
> INSERT INTO test.test 
> (row_key, column_key, column_value)
> VALUES 
> ('0000000001', e0138677-7246-11e3-ac78-016ae7083d37, [0, 1, 2, 3]);
> INSERT INTO test.test 
> (row_key, column_key, column_value) 
> VALUES
> ('0000000022', 1ac5770a-7247-11e3-80e4-016ae7083d37, { 'a': '0123456701234567012345670',
 'b': '0123456701234567012345670' });
> {code}
> As a comparison, I was able to run the same tests with the following schema with no issue:
> Note: This test was able to run at a much faster insertion speed, for much longer and
much bigger column sizes (1KB) without any GC issues.
> {code}
> CREATE TABLE test.test (
> row_key text, 
> column_key uuid, 
> column_value text, 
> PRIMARY KEY(row_key, column_key) )
> {code}



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Mime
View raw message