[jira] [Assigned] (CASSANDRA-12216) TTL Reading And Writing is Asymmetric
[ https://issues.apache.org/jira/browse/CASSANDRA-12216?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer reassigned CASSANDRA-12216: - Assignee: Russell Alexander Spitzer > TTL Reading And Writing is Asymmetric > -- > > Key: CASSANDRA-12216 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12216 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: Russell Alexander Spitzer >Assignee: Russell Alexander Spitzer >Priority: Minor > Attachments: 12216-3.7.txt > > > There is an inherent asymmetry in the way TTL's are read and Written. > An `TTL` of 0 when written becomes a `null` in C* > When read, this `TTL` becomes a `null` > The `null` cannot be written back to C* as `TTL` > This means that end users attempting to copy tables with TTL have to do > manual mapping of the null TTL values to 0 to avoid NPE. This is a bit > onerous when C* seems to have an internal logic that 0 == NULL. I don't think > C* should return values which are not directly insertable back to C*. > Even with the advent CASSANDRA-7304 this still remains a problem that the > User needs to be aware of and take care of. > The following prepared statement > {code} > INSERT INTO test.table2 (k,v) (?,?) USING TTL: ? > {code} > Will throw NPEs unless we specifically check that the value to be bound to > TTL is not null. > I think we should discuss whether `null` should be treated as 0 in TTL for > prepared statements. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-12216) TTL Reading And Writing is Asymmetric
[ https://issues.apache.org/jira/browse/CASSANDRA-12216?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-12216: -- Attachment: 12216-3.7.txt Patch attached, not sure where to write tests for this > TTL Reading And Writing is Asymmetric > -- > > Key: CASSANDRA-12216 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12216 > Project: Cassandra > Issue Type: Bug > Components: CQL >Reporter: Russell Alexander Spitzer >Priority: Minor > Attachments: 12216-3.7.txt > > > There is an inherent asymmetry in the way TTL's are read and Written. > An `TTL` of 0 when written becomes a `null` in C* > When read, this `TTL` becomes a `null` > The `null` cannot be written back to C* as `TTL` > This means that end users attempting to copy tables with TTL have to do > manual mapping of the null TTL values to 0 to avoid NPE. This is a bit > onerous when C* seems to have an internal logic that 0 == NULL. I don't think > C* should return values which are not directly insertable back to C*. > Even with the advent CASSANDRA-7304 this still remains a problem that the > User needs to be aware of and take care of. > The following prepared statement > {code} > INSERT INTO test.table2 (k,v) (?,?) USING TTL: ? > {code} > Will throw NPEs unless we specifically check that the value to be bound to > TTL is not null. > I think we should discuss whether `null` should be treated as 0 in TTL for > prepared statements. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-12216) TTL Reading And Writing is Asymmetric
Russell Alexander Spitzer created CASSANDRA-12216: - Summary: TTL Reading And Writing is Asymmetric Key: CASSANDRA-12216 URL: https://issues.apache.org/jira/browse/CASSANDRA-12216 Project: Cassandra Issue Type: Bug Components: CQL Reporter: Russell Alexander Spitzer Priority: Minor There is an inherent asymmetry in the way TTL's are read and Written. An `TTL` of 0 when written becomes a `null` in C* When read, this `TTL` becomes a `null` The `null` cannot be written back to C* as `TTL` This means that end users attempting to copy tables with TTL have to do manual mapping of the null TTL values to 0 to avoid NPE. This is a bit onerous when C* seems to have an internal logic that 0 == NULL. I don't think C* should return values which are not directly insertable back to C*. Even with the advent CASSANDRA-7304 this still remains a problem that the User needs to be aware of and take care of. The following prepared statement {code} INSERT INTO test.table2 (k,v) (?,?) USING TTL: ? {code} Will throw NPEs unless we specifically check that the value to be bound to TTL is not null. I think we should discuss whether `null` should be treated as 0 in TTL for prepared statements. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7304) Ability to distinguish between NULL and UNSET values in Prepared Statements
[ https://issues.apache.org/jira/browse/CASSANDRA-7304?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15379880#comment-15379880 ] Russell Alexander Spitzer commented on CASSANDRA-7304: -- Why is TTL unlimited and not the Table Default? > Ability to distinguish between NULL and UNSET values in Prepared Statements > --- > > Key: CASSANDRA-7304 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7304 > Project: Cassandra > Issue Type: Sub-task >Reporter: Drew Kutcharian >Assignee: Oded Peer > Labels: client-impacting, cql, protocolv4 > Fix For: 2.2.0 beta 1 > > Attachments: 7304-03.patch, 7304-04.patch, 7304-05.patch, > 7304-06.patch, 7304-07.patch, 7304-2.patch, 7304-V8.txt, 7304.patch > > > Currently Cassandra inserts tombstones when a value of a column is bound to > NULL in a prepared statement. At higher insert rates managing all these > tombstones becomes an unnecessary overhead. This limits the usefulness of the > prepared statements since developers have to either create multiple prepared > statements (each with a different combination of column names, which at times > is just unfeasible because of the sheer number of possible combinations) or > fall back to using regular (non-prepared) statements. > This JIRA is here to explore the possibility of either: > A. Have a flag on prepared statements that once set, tells Cassandra to > ignore null columns > or > B. Have an "UNSET" value which makes Cassandra skip the null columns and not > tombstone them > Basically, in the context of a prepared statement, a null value means delete, > but we don’t have anything that means "ignore" (besides creating a new > prepared statement without the ignored column). > Please refer to the original conversation on DataStax Java Driver mailing > list for more background: > https://groups.google.com/a/lists.datastax.com/d/topic/java-driver-user/cHE3OOSIXBU/discussion > *EDIT 18/12/14 - [~odpeer] Implementation Notes:* > The motivation hasn't changed. > Protocol version 4 specifies that bind variables do not require having a > value when executing a statement. Bind variables without a value are called > 'unset'. The 'unset' bind variable is serialized as the int value '-2' > without following bytes. > \\ > \\ > * An unset bind variable in an EXECUTE or BATCH request > ** On a {{value}} does not modify the value and does not create a tombstone > ** On the {{ttl}} clause is treated as 'unlimited' > ** On the {{timestamp}} clause is treated as 'now' > ** On a map key or a list index throws {{InvalidRequestException}} > ** On a {{counter}} increment or decrement operation does not change the > counter value, e.g. {{UPDATE my_tab SET c = c - ? WHERE k = 1}} does change > the value of counter {{c}} > ** On a tuple field or UDT field throws {{InvalidRequestException}} > * An unset bind variable in a QUERY request > ** On a partition column, clustering column or index column in the {{WHERE}} > clause throws {{InvalidRequestException}} > ** On the {{limit}} clause is treated as 'unlimited' -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-8700) replace the wiki with docs in the git repo
[ https://issues.apache.org/jira/browse/CASSANDRA-8700?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-8700: - Attachment: TombstonesAndGcGrace.md Wrote a small section on Tombstones and GCGrace. > replace the wiki with docs in the git repo > -- > > Key: CASSANDRA-8700 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8700 > Project: Cassandra > Issue Type: Improvement > Components: Documentation and Website >Reporter: Jon Haddad >Assignee: Sylvain Lebresne >Priority: Minor > Attachments: TombstonesAndGcGrace.md, bloom_filters.md, > compression.md, drivers_list.md, hardware.md, installation.md > > > The wiki as it stands is pretty terrible. It takes several minutes to apply > a single update, and as a result, it's almost never updated. The information > there has very little context as to what version it applies to. Most people > I've talked to that try to use the information they find there find it is > more confusing than helpful. > I'd like to propose that instead of using the wiki, the doc directory in the > cassandra repo be used for docs (already used for CQL3 spec) in a format that > can be built to a variety of output formats like HTML / epub / etc. I won't > start the bikeshedding on which markup format is preferable - but there are > several options that can work perfectly fine. I've personally use sphinx w/ > restructured text, and markdown. Both can build easily and as an added bonus > be pushed to readthedocs (or something similar) automatically. For an > example, see cqlengine's documentation, which I think is already > significantly better than the wiki: > http://cqlengine.readthedocs.org/en/latest/ > In addition to being overall easier to maintain, putting the documentation in > the git repo adds context, since it evolves with the versions of Cassandra. > If the wiki were kept even remotely up to date, I wouldn't bother with this, > but not having at least some basic documentation in the repo, or anywhere > associated with the project, is frustrating. > For reference, the last 3 updates were: > 1/15/15 - updating committers list > 1/08/15 - updating contributers and how to contribute > 12/16/14 - added a link to CQL docs from wiki frontpage (by me) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-11542) Create a benchmark to compare HDFS and Cassandra bulk read times
[ https://issues.apache.org/jira/browse/CASSANDRA-11542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15299326#comment-15299326 ] Russell Alexander Spitzer edited comment on CASSANDRA-11542 at 5/25/16 2:11 AM: The benchmark looks good to me. I would only suggest you increase the volume of data in the run so that the ratio of pulling data from C* to setting up Spark work is higher. was (Author: rspitzer): The benchmark looks good to me. I would only suggest you increase the volume of data in the run so that the ratio of pulling data from C* to setting up Spark work is lower. > Create a benchmark to compare HDFS and Cassandra bulk read times > > > Key: CASSANDRA-11542 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11542 > Project: Cassandra > Issue Type: Sub-task > Components: Testing >Reporter: Stefania >Assignee: Stefania > Fix For: 3.x > > Attachments: jfr_recordings.zip, spark-load-perf-results-001.zip, > spark-load-perf-results-002.zip, spark-load-perf-results-003.zip > > > I propose creating a benchmark for comparing Cassandra and HDFS bulk reading > performance. Simple Spark queries will be performed on data stored in HDFS or > Cassandra, and the entire duration will be measured. An example query would > be the max or min of a column or a count\(*\). > This benchmark should allow determining the impact of: > * partition size > * number of clustering columns > * number of value columns (cells) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11542) Create a benchmark to compare HDFS and Cassandra bulk read times
[ https://issues.apache.org/jira/browse/CASSANDRA-11542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15299326#comment-15299326 ] Russell Alexander Spitzer commented on CASSANDRA-11542: --- The benchmark looks good to me. I would only suggest you increase the volume of data in the run so that the ratio of pulling data from C* to setting up Spark work is lower. > Create a benchmark to compare HDFS and Cassandra bulk read times > > > Key: CASSANDRA-11542 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11542 > Project: Cassandra > Issue Type: Sub-task > Components: Testing >Reporter: Stefania >Assignee: Stefania > Fix For: 3.x > > Attachments: jfr_recordings.zip, spark-load-perf-results-001.zip, > spark-load-perf-results-002.zip, spark-load-perf-results-003.zip > > > I propose creating a benchmark for comparing Cassandra and HDFS bulk reading > performance. Simple Spark queries will be performed on data stored in HDFS or > Cassandra, and the entire duration will be measured. An example query would > be the max or min of a column or a count\(*\). > This benchmark should allow determining the impact of: > * partition size > * number of clustering columns > * number of value columns (cells) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-11542) Create a benchmark to compare HDFS and Cassandra bulk read times
[ https://issues.apache.org/jira/browse/CASSANDRA-11542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264698#comment-15264698 ] Russell Alexander Spitzer edited comment on CASSANDRA-11542 at 4/29/16 9:04 PM: -Hmm i'm a little confused that case classes don't help but Dataframes do...- https://datastax-oss.atlassian.net/browse/SPARKC-373 Saw that we are always doing the conversion to CassandraRow with RDDs, dataframes go directly to the internal SQL Type. The code you presented looks good to me, there is the potential issue of blocking on resultsets that take a long time to complete while other result-sets are already on the driver but i'm not sure if this is a big deal. Do you have any idea of the parallelization in these test? How many partitions are the different runs generating? was (Author: rspitzer): Hmm i'm a little confused that case classes don't help but Dataframes do... The code you presented looks good to me, there is the potential issue of blocking on resultsets that take a long time to complete while other result-sets are already on the driver but i'm not sure if this is a big deal. Do you have any idea of the parallelization in these test? How many partitions are the different runs generating? > Create a benchmark to compare HDFS and Cassandra bulk read times > > > Key: CASSANDRA-11542 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11542 > Project: Cassandra > Issue Type: Sub-task > Components: Testing >Reporter: Stefania >Assignee: Stefania > Fix For: 3.x > > Attachments: spark-load-perf-results-001.zip, > spark-load-perf-results-002.zip > > > I propose creating a benchmark for comparing Cassandra and HDFS bulk reading > performance. Simple Spark queries will be performed on data stored in HDFS or > Cassandra, and the entire duration will be measured. An example query would > be the max or min of a column or a count\(*\). > This benchmark should allow determining the impact of: > * partition size > * number of clustering columns > * number of value columns (cells) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11542) Create a benchmark to compare HDFS and Cassandra bulk read times
[ https://issues.apache.org/jira/browse/CASSANDRA-11542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15264698#comment-15264698 ] Russell Alexander Spitzer commented on CASSANDRA-11542: --- Hmm i'm a little confused that case classes don't help but Dataframes do... The code you presented looks good to me, there is the potential issue of blocking on resultsets that take a long time to complete while other result-sets are already on the driver but i'm not sure if this is a big deal. Do you have any idea of the parallelization in these test? How many partitions are the different runs generating? > Create a benchmark to compare HDFS and Cassandra bulk read times > > > Key: CASSANDRA-11542 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11542 > Project: Cassandra > Issue Type: Sub-task > Components: Testing >Reporter: Stefania >Assignee: Stefania > Fix For: 3.x > > Attachments: spark-load-perf-results-001.zip, > spark-load-perf-results-002.zip > > > I propose creating a benchmark for comparing Cassandra and HDFS bulk reading > performance. Simple Spark queries will be performed on data stored in HDFS or > Cassandra, and the entire duration will be measured. An example query would > be the max or min of a column or a count\(*\). > This benchmark should allow determining the impact of: > * partition size > * number of clustering columns > * number of value columns (cells) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11542) Create a benchmark to compare HDFS and Cassandra bulk read times
[ https://issues.apache.org/jira/browse/CASSANDRA-11542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15263561#comment-15263561 ] Russell Alexander Spitzer commented on CASSANDRA-11542: --- One other thing, when we were testing before we noticed that our Reads were not IO bound, with Streaming are we now IO bound on the C* side? IE is drive usage at full? > Create a benchmark to compare HDFS and Cassandra bulk read times > > > Key: CASSANDRA-11542 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11542 > Project: Cassandra > Issue Type: Sub-task > Components: Testing >Reporter: Stefania >Assignee: Stefania > Fix For: 3.x > > Attachments: spark-load-perf-results-001.zip, > spark-load-perf-results-002.zip > > > I propose creating a benchmark for comparing Cassandra and HDFS bulk reading > performance. Simple Spark queries will be performed on data stored in HDFS or > Cassandra, and the entire duration will be measured. An example query would > be the max or min of a column or a count\(*\). > This benchmark should allow determining the impact of: > * partition size > * number of clustering columns > * number of value columns (cells) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-11542) Create a benchmark to compare HDFS and Cassandra bulk read times
[ https://issues.apache.org/jira/browse/CASSANDRA-11542?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15263553#comment-15263553 ] Russell Alexander Spitzer commented on CASSANDRA-11542: --- You may also want to do tests reading into CaseClasses rather than CassandraRows, {code} case class RowName( col:Type, col2: type, ) sc.cassandraTable[RowName]{code} This may explain some of the difference between RDD and DataFrame read times as Dataframes (SqlRows vs CassandraRows) read into a different format than RDDs by default and case classes should be much more efficient than the map based CassandraRows. In addition I think the parquet versions are able to skip full counts (because of the metadata) but i'm not really sure about that which may give them the advantage over CSV ... Again not sure it could just be the compression of repeated values > Create a benchmark to compare HDFS and Cassandra bulk read times > > > Key: CASSANDRA-11542 > URL: https://issues.apache.org/jira/browse/CASSANDRA-11542 > Project: Cassandra > Issue Type: Sub-task > Components: Testing >Reporter: Stefania >Assignee: Stefania > Fix For: 3.x > > Attachments: spark-load-perf-results-001.zip, > spark-load-perf-results-002.zip > > > I propose creating a benchmark for comparing Cassandra and HDFS bulk reading > performance. Simple Spark queries will be performed on data stored in HDFS or > Cassandra, and the entire duration will be measured. An example query would > be the max or min of a column or a count\(*\). > This benchmark should allow determining the impact of: > * partition size > * number of clustering columns > * number of value columns (cells) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15118187#comment-15118187 ] Russell Alexander Spitzer commented on CASSANDRA-9318: -- [~aweisberg] I think i can get you a repo pretty easily with the SparkCassandraConnector. We tend to get a few comments a week where the driver ends up excepting out on "NoHostAvailable" exceptions. Usually folks who have done a RF=1 and then a full table scan or are doing heavy writes to a cluster with a very high number of concurrent writers. > Bound the number of in-flight requests at the coordinator > - > > Key: CASSANDRA-9318 > URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 > Project: Cassandra > Issue Type: Improvement > Components: Local Write-Read Paths, Streaming and Messaging >Reporter: Ariel Weisberg >Assignee: Ariel Weisberg > Fix For: 2.1.x, 2.2.x > > > It's possible to somewhat bound the amount of load accepted into the cluster > by bounding the number of in-flight requests and request bytes. > An implementation might do something like track the number of outstanding > bytes and requests and if it reaches a high watermark disable read on client > connections until it goes back below some low watermark. > Need to make sure that disabling read on the client connection won't > introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-10050) Secondary Index Performance Dependent on TokenRange Searched in Analytics
Russell Alexander Spitzer created CASSANDRA-10050: - Summary: Secondary Index Performance Dependent on TokenRange Searched in Analytics Key: CASSANDRA-10050 URL: https://issues.apache.org/jira/browse/CASSANDRA-10050 Project: Cassandra Issue Type: Bug Components: Core Environment: Single node, macbook, 2.1.8 Reporter: Russell Alexander Spitzer In doing some test work on the Spark Cassandra Connector I saw some odd performance when pushing down range queries with Secondary Index filters. When running the queries we see huge amount of time when the C* server is not doing any work and the query seem to be hanging. This investigation led to the work in this document https://docs.google.com/spreadsheets/d/1aJg3KX7nPnY77RJ9ZT-IfaYADgJh0A--nAxItvC6hb4/edit#gid=0 The Spark Cassandra Connector builds up token range specific queries and allows the user to pushdown relevant fields to C*. Here we should two indexed fields (size) and (color) being pushed down to C*. {code} SELECT count(*) FROM ks.tab WHERE token(store) $min AND token(store) = $max AND color = 'red' AND size = 'P' ALLOW FILTERING;{code} These queries will have different token ranges inserted and executed as separate spark tasks. Spark tasks with token ranges near the Min(token) end up executing much faster than those near Max(token) which also happen to through errors. {code} Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 0 responses. info={'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'} {code} I took the queries and ran them through CQLSH to see the difference in time. A linear relationship is seen based on where the tokenRange being queried is starting with only 2 second for queries near the beginning of the full token spectrum and over 12 seconds at the end of the spectrum. The question is, can this behavior be improved? or should we not recommend using secondary indexes with Analytics workloads? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-10050) Secondary Index Performance Dependent on TokenRange Searched in Analytics
[ https://issues.apache.org/jira/browse/CASSANDRA-10050?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-10050: -- Description: In doing some test work on the Spark Cassandra Connector I saw some odd performance when pushing down range queries with Secondary Index filters. When running the queries we see huge amount of time when the C* server is not doing any work and the query seem to be hanging. This investigation led to the work in this document https://docs.google.com/spreadsheets/d/1aJg3KX7nPnY77RJ9ZT-IfaYADgJh0A--nAxItvC6hb4/edit#gid=0 The Spark Cassandra Connector builds up token range specific queries and allows the user to pushdown relevant fields to C*. Here we have two indexed fields (size) and (color) being pushed down to C*. {code} SELECT count(*) FROM ks.tab WHERE token(store) $min AND token(store) = $max AND color = 'red' AND size = 'P' ALLOW FILTERING;{code} These queries will have different token ranges inserted and executed as separate spark tasks. Spark tasks with token ranges near the Min(token) end up executing much faster than those near Max(token) which also happen to through errors. {code} Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 0 responses. info={'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'} {code} I took the queries and ran them through CQLSH to see the difference in time. A linear relationship is seen based on where the tokenRange being queried is starting with only 2 second for queries near the beginning of the full token spectrum and over 12 seconds at the end of the spectrum. The question is, can this behavior be improved? or should we not recommend using secondary indexes with Analytics workloads? was: In doing some test work on the Spark Cassandra Connector I saw some odd performance when pushing down range queries with Secondary Index filters. When running the queries we see huge amount of time when the C* server is not doing any work and the query seem to be hanging. This investigation led to the work in this document https://docs.google.com/spreadsheets/d/1aJg3KX7nPnY77RJ9ZT-IfaYADgJh0A--nAxItvC6hb4/edit#gid=0 The Spark Cassandra Connector builds up token range specific queries and allows the user to pushdown relevant fields to C*. Here we should two indexed fields (size) and (color) being pushed down to C*. {code} SELECT count(*) FROM ks.tab WHERE token(store) $min AND token(store) = $max AND color = 'red' AND size = 'P' ALLOW FILTERING;{code} These queries will have different token ranges inserted and executed as separate spark tasks. Spark tasks with token ranges near the Min(token) end up executing much faster than those near Max(token) which also happen to through errors. {code} Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 0 responses. info={'received_responses': 0, 'required_responses': 1, 'consistency': 'ONE'} {code} I took the queries and ran them through CQLSH to see the difference in time. A linear relationship is seen based on where the tokenRange being queried is starting with only 2 second for queries near the beginning of the full token spectrum and over 12 seconds at the end of the spectrum. The question is, can this behavior be improved? or should we not recommend using secondary indexes with Analytics workloads? Secondary Index Performance Dependent on TokenRange Searched in Analytics - Key: CASSANDRA-10050 URL: https://issues.apache.org/jira/browse/CASSANDRA-10050 Project: Cassandra Issue Type: Bug Components: Core Environment: Single node, macbook, 2.1.8 Reporter: Russell Alexander Spitzer In doing some test work on the Spark Cassandra Connector I saw some odd performance when pushing down range queries with Secondary Index filters. When running the queries we see huge amount of time when the C* server is not doing any work and the query seem to be hanging. This investigation led to the work in this document https://docs.google.com/spreadsheets/d/1aJg3KX7nPnY77RJ9ZT-IfaYADgJh0A--nAxItvC6hb4/edit#gid=0 The Spark Cassandra Connector builds up token range specific queries and allows the user to pushdown relevant fields to C*. Here we have two indexed fields (size) and (color) being pushed down to C*. {code} SELECT count(*) FROM ks.tab WHERE token(store) $min AND token(store) = $max AND color = 'red' AND size = 'P' ALLOW FILTERING;{code} These queries will have different token ranges inserted and executed as separate spark tasks. Spark tasks with token ranges near the Min(token) end up executing much faster than those near
[jira] [Created] (CASSANDRA-9637) CFS selectAndReference Blocks for Compaction
Russell Alexander Spitzer created CASSANDRA-9637: Summary: CFS selectAndReference Blocks for Compaction Key: CASSANDRA-9637 URL: https://issues.apache.org/jira/browse/CASSANDRA-9637 Project: Cassandra Issue Type: Bug Components: Core Reporter: Russell Alexander Spitzer Assignee: Benedict Fix For: 2.1.x The Spark Cassandra Connector uses dsecribe_splits_ex to deterimine token information about the C* cluster. On 2.1.4 this behaves normally and responds in a few milliseconds. On 2.1.5 the function blocks while compaction is running which basically makes the function unusable. Under the hood this function access sstable information using selectAndReference. A debugging session led us to determine that the error occurred because the code would simply spin (unable to gain references to compacting tables) while compaction was happening. A fix was tested and proposed: https://github.com/belliottsmith/cassandra/tree/fix-spinning -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9259) Bulk Reading from Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-9259?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14540321#comment-14540321 ] Russell Alexander Spitzer commented on CASSANDRA-9259: -- For full data queries it may be advantageous to have C* be able to compact all of the relevant sstables into a format friendlier to analytics workloads. For example a {{compactToParquet}} command could be quite useful for doing batch analytics. This would be useful for full table work because: We are going to need to write the data to a new storage format (or spill serialized data to disk anyway) A durable copy of the data in the new format will stop us from worrying about losing spilled blocks during execution (with spark executors dying and their block stores being lost) Bulk Reading from Cassandra --- Key: CASSANDRA-9259 URL: https://issues.apache.org/jira/browse/CASSANDRA-9259 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Brian Hess This ticket is following on from the 2015 NGCC. This ticket is designed to be a place for discussing and designing an approach to bulk reading. The goal is to have a bulk reading path for Cassandra. That is, a path optimized to grab a large portion of the data for a table (potentially all of it). This is a core element in the Spark integration with Cassandra, and the speed at which Cassandra can deliver bulk data to Spark is limiting the performance of Spark-plus-Cassandra operations. This is especially of importance as Cassandra will (likely) leverage Spark for internal operations (for example CASSANDRA-8234). The core CQL to consider is the following: SELECT a, b, c FROM myKs.myTable WHERE Token(partitionKey) X AND Token(partitionKey) = Y Here, we choose X and Y to be contained within one token range (perhaps considering the primary range of a node without vnodes, for example). This query pushes 50K-100K rows/sec, which is not very fast if we are doing bulk operations via Spark (or other processing frameworks - ETL, etc). There are a few causes (e.g., inefficient paging). There are a few approaches that could be considered. First, we consider a new Streaming Compaction approach. The key observation here is that a bulk read from Cassandra is a lot like a major compaction, though instead of outputting a new SSTable we would output CQL rows to a stream/socket/etc. This would be similar to a CompactionTask, but would strip out some unnecessary things in there (e.g., some of the indexing, etc). Predicates and projections could also be encapsulated in this new StreamingCompactionTask, for example. Another approach would be an alternate storage format. For example, we might employ Parquet (just as an example) to store the same data as in the primary Cassandra storage (aka SSTables). This is akin to Global Indexes (an alternate storage of the same data optimized for a particular query). Then, Cassandra can choose to leverage this alternate storage for particular CQL queries (e.g., range scans). These are just 2 suggestions to get the conversation going. One thing to note is that it will be useful to have this storage segregated by token range so that when you extract via these mechanisms you do not get replications-factor numbers of copies of the data. That will certainly be an issue for some Spark operations (e.g., counting). Thus, we will want per-token-range storage (even for single disks), so this will likely leverage CASSANDRA-6696 (though, we'll want to also consider the single disk case). It is also worth discussing what the success criteria is here. It is unlikely to be as fast as EDW or HDFS performance (though, that is still a good goal), but being within some percentage of that performance should be set as success. For example, 2x as long as doing bulk operations on HDFS with similar node count/size/etc. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9136) Mixed 2.0.14 - 2.1.4 Cluster Error Deserializing RangeSliceCommand
[ https://issues.apache.org/jira/browse/CASSANDRA-9136?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-9136: - Description: This error occurs during a rolling upgrade between 2.0.14 and 2.1.4. h3. Repo With all the nodes on 2.0.14 make the following tables {code} CREATE KEYSPACE test WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; USE test; CREATE TABLE compact ( k int, c int, d int, PRIMARY KEY ((k), c) ) WITH COMPACT STORAGE; CREATE TABLE norm ( k int, c int, d int, PRIMARY KEY ((k), c) ) ; {code} Then load some data into these tables. I used the python driver {code} from cassandra.cluster import Cluster s = Cluster().connect() for x in range (1000): for y in range (1000): s.execute_async(INSERT INTO test.compact (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) s.execute_async(INSERT INTO test.norm (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) {code} Upgrade one node from 2.0.14 - 2.1.4 Run cqlsh on the upgraded node and certain queries will fail intermittently, easiest to repo with CL = ALL {code} cqlsh SELECT count(*) FROM test.norm where k = 22 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} cqlsh SELECT count(*) FROM test.norm where k = 21 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} {code} h3. Possible related exceptions On the 2.0.14 nodes we sometimes see these exceptions {code} ERROR [Thread-19] 2015-04-08 18:48:45,337 CassandraDaemon.java (line 258) Exception in thread Thread[Thread-19,5,main] java.lang.NullPointerException at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247) at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156) at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74) {code} was: This error occurs during a rolling upgrade between 2.0.14 and 2.1.4. h3. Repo With all the nodes on 2.0.14 make the following tables {code} CREATE KEYSPACE test WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; USE test; CREATE TABLE compact ( k int, c int, d int, PRIMARY KEY ((k), c) ) WITH COMPACT STORAGE CREATE TABLE norm ( k int, c int, d int, PRIMARY KEY ((k), c) ) {code} Then load some data into these tables. I used the python driver {code} from cassandra.cluster import Cluster s = Cluster().connect() for x in range (1000): for y in range (1000): s.execute_async(INSERT INTO test.compact (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) s.execute_async(INSERT INTO test.norm (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) {code} Upgrade one node from 2.0.14 - 2.1.4 Run cqlsh on the upgraded node and certain queries will fail intermittently, easiest to repo with CL = ALL {code} cqlsh SELECT count(*) FROM test.norm where k = 22 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} cqlsh SELECT count(*) FROM test.norm where k = 21 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} {code} h3. Possible related exceptions On the 2.0.14 nodes we sometimes see these exceptions {code} ERROR [Thread-19] 2015-04-08 18:48:45,337 CassandraDaemon.java (line 258) Exception in thread Thread[Thread-19,5,main] java.lang.NullPointerException at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247) at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156) at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74) {code} Mixed 2.0.14 - 2.1.4 Cluster Error Deserializing
[jira] [Resolved] (CASSANDRA-9136) Mixed 2.0.14 - 2.1.4 Cluster Error Deserializing RangeSliceCommand
[ https://issues.apache.org/jira/browse/CASSANDRA-9136?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer resolved CASSANDRA-9136. -- Resolution: Cannot Reproduce Mixed 2.0.14 - 2.1.4 Cluster Error Deserializing RangeSliceCommand -- Key: CASSANDRA-9136 URL: https://issues.apache.org/jira/browse/CASSANDRA-9136 Project: Cassandra Issue Type: Bug Components: Core Environment: 3 Nodes GCE, N1-Standard-2, Ubuntu 12, 1 Node on 2.1.4, 2 on 2.0.14 Reporter: Russell Alexander Spitzer Fix For: 2.1.5 This error occurs during a rolling upgrade between 2.0.14 and 2.1.4. h3. Repo With all the nodes on 2.0.14 make the following tables {code} CREATE KEYSPACE test WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; USE test; CREATE TABLE compact ( k int, c int, d int, PRIMARY KEY ((k), c) ) WITH COMPACT STORAGE; CREATE TABLE norm ( k int, c int, d int, PRIMARY KEY ((k), c) ) ; {code} Then load some data into these tables. I used the python driver {code} from cassandra.cluster import Cluster s = Cluster().connect() for x in range (1000): for y in range (1000): s.execute_async(INSERT INTO test.compact (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) s.execute_async(INSERT INTO test.norm (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) {code} Upgrade one node from 2.0.14 - 2.1.4 Run cqlsh on the upgraded node and certain queries will fail intermittently, easiest to repo with CL = ALL {code} cqlsh SELECT count(*) FROM test.norm where k = 22 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} cqlsh SELECT count(*) FROM test.norm where k = 21 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} {code} h3. Possible related exceptions On the 2.0.14 nodes we sometimes see these exceptions {code} ERROR [Thread-19] 2015-04-08 18:48:45,337 CassandraDaemon.java (line 258) Exception in thread Thread[Thread-19,5,main] java.lang.NullPointerException at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247) at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156) at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74) {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9136) Mixed 2.0.14 - 2.1.4 Cluster Read Timeout Errors
Russell Alexander Spitzer created CASSANDRA-9136: Summary: Mixed 2.0.14 - 2.1.4 Cluster Read Timeout Errors Key: CASSANDRA-9136 URL: https://issues.apache.org/jira/browse/CASSANDRA-9136 Project: Cassandra Issue Type: Bug Components: Core Environment: 3 Nodes GCE, N1-Standard-2, Ubuntu 12, 1 Node on 2.1.4, 2 on 2.0.14 Reporter: Russell Alexander Spitzer This error occurs during a rolling upgrade between 2.0.14 and 2.1.4. h3. Repo With all the nodes on 2.0.14 make the following tables {code} CREATE KEYSPACE test WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; USE test; CREATE TABLE compact ( k int, c int, d int, PRIMARY KEY ((k), c) ) WITH COMPACT STORAGE CREATE TABLE norm ( k int, c int, d int, PRIMARY KEY ((k), c) ) {code} Then load some data into these tables. I used the python driver {code} from cassandra.cluster import Cluster s = Cluster().connect() for x in range (1000): for y in range (1000): s.execute_async(INSERT INTO test.compact (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) s.execute_async(INSERT INTO test.norm (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) {code} Upgrade one node from 2.0.14 - 2.1.4 Run cqlsh on the upgraded node and certain queries will fail intermittently, easiest to repo with CL = ALL {code} cqlsh SELECT count(*) FROM test.norm where k = 22 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} cqlsh SELECT count(*) FROM test.norm where k = 21 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} {code} h3. Possible related exceptions On the 2.0.14 nodes we sometimes see these exceptions {code} ERROR [Thread-19] 2015-04-08 18:48:45,337 CassandraDaemon.java (line 258) Exception in thread Thread[Thread-19,5,main] java.lang.NullPointerException at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247) at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156) at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74) {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8576) Primary Key Pushdown For Hadoop
[ https://issues.apache.org/jira/browse/CASSANDRA-8576?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14276183#comment-14276183 ] Russell Alexander Spitzer commented on CASSANDRA-8576: -- For this particular use-case they only need EQ, but IN would be nice as well. Primary Key Pushdown For Hadoop --- Key: CASSANDRA-8576 URL: https://issues.apache.org/jira/browse/CASSANDRA-8576 Project: Cassandra Issue Type: Improvement Components: Hadoop Reporter: Russell Alexander Spitzer I've heard reports from several users that they would like to have predicate pushdown functionality for hadoop (Hive in particular) based services. Example usecase Table with wide partitions, one per customer Application team has HQL they would like to run on a single customer Currently time to complete scales with number of customers since Input Format can't pushdown primary key predicate Current implementation requires a full table scan (since it can't recognize that a single partition was specified) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-8576) Primary Key Pushdown For Hadoop
Russell Alexander Spitzer created CASSANDRA-8576: Summary: Primary Key Pushdown For Hadoop Key: CASSANDRA-8576 URL: https://issues.apache.org/jira/browse/CASSANDRA-8576 Project: Cassandra Issue Type: Improvement Components: Hadoop Reporter: Russell Alexander Spitzer I've heard reports from several users that they would like to have predicate pushdown functionality for hadoop (Hive in particular) based services. Example usecase Table with wide partitions, one per customer Application team has HQL they would like to run on a single customer Currently time to complete scales with number of customers since Input Format can't pushdown primary key predicate Current implementation requires a full table scan (since it can't recognize that a single partition was specified) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14169444#comment-14169444 ] Russell Alexander Spitzer commented on CASSANDRA-7821: -- Np, I'll work on that when i get some free time. Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7821-2.1.patch Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14160697#comment-14160697 ] Russell Alexander Spitzer commented on CASSANDRA-7821: -- We would have to write a new retry policy I think, the current ones retry immediately which causes problems when we are testing under load. Sometimes a node will become unresponsive temporarily and break the test but if we gave it a bit of time it would come back online. If we retry as fast as possible then a node which has stopped responding will stop the entire test. Code i'm looking at for retry policies on the JD: https://github.com/datastax/java-driver/blob/55b60db5767af26d4cf4618c495e0f760096c25c/driver-core/src/main/java/com/datastax/driver/core/policies/DefaultRetryPolicy.java Of course writing a new retry policy would be acceptable too, a BackingOffRetryPolicy.java or something alike? Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7821-2.1.patch Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=1451#comment-1451 ] Russell Alexander Spitzer commented on CASSANDRA-7821: -- One more commit, forgot to implement serializable Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7821-2.1.patch Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7821: - Attachment: CASSANDRA-7821-2.1.patch Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7821-2.1.patch Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7821: - Attachment: (was: CASSANDRA-7821-2.1.patch) Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7821-2.1.patch Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14109626#comment-14109626 ] Russell Alexander Spitzer commented on CASSANDRA-7821: -- Attached a patch which adds to simlpe options to C* Stress {code} backoff_strategy = {CONSTANT,LINEAR,EXPONENTIAL} CONSTANT : A constant amount of seconds based on backoff_seconds LINEAR : An amount of time based on the retry_num * backoff_seconds EXPONENTIAL: An amount of time based on backoff_seconds * 2 ^ retry_num backoff_seconds = # The number of seconds to be used as a coefficent in the above strategies {code} https://github.com/RussellSpitzer/cassandra/compare/RussellSpitzer:cassandra-2.1...CASSANDRA-7821 I also bumped up the timeout for threads up to 10 minutes but ideally we would pass through the max expected amount of retry time. [~benedict] As usual your feedback would be extremely welcome Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7821: - Attachment: CASSANDRA-7821-2.1.patch Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7821-2.1.patch Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
[ https://issues.apache.org/jira/browse/CASSANDRA-7821?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer reassigned CASSANDRA-7821: Assignee: Russell Alexander Spitzer Add Optional Backoff on Retry to Cassandra Stress - Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7821) Add Optional Backoff on Retry to Cassandra Stress
Russell Alexander Spitzer created CASSANDRA-7821: Summary: Add Optional Backoff on Retry to Cassandra Stress Key: CASSANDRA-7821 URL: https://issues.apache.org/jira/browse/CASSANDRA-7821 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Currently when stress is running against a cluster which occasionally has nodes marked as down, it will almost immediately stop. This occurs because the retry loop can execute extremely quickly if each execution terminates with a {{com.datastax.driver.core.exceptions.NoHostAvailableException}} or {{com.datastax.driver.core.exceptions.UnavailableException}}. In case of these exceptions is will most likely be unable to succeed if the retries are performed as fast as possible. To get around this, we could add an optional delay on retries giving the cluster time to recover rather than terminating the stress run. We could make this configurable, with options such as: * Constant # Delays the same amount after each retry * Linear # Backoff a set amount * the trial number * Exponential # Backoff set amount * 2 ^ trial number This may also require adjusting the thread is stuck check to make sure that the max retry timeout will not cause the thread to be terminated early. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6851) Improve anticompaction after incremental repair
[ https://issues.apache.org/jira/browse/CASSANDRA-6851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14102328#comment-14102328 ] Russell Alexander Spitzer commented on CASSANDRA-6851: -- Looks fine to me, Logging the details within the function makes more sense than after the return. +1 Improve anticompaction after incremental repair --- Key: CASSANDRA-6851 URL: https://issues.apache.org/jira/browse/CASSANDRA-6851 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Marcus Eriksson Assignee: Russell Alexander Spitzer Priority: Minor Labels: compaction, lhf Fix For: 3.0 After an incremental repair we iterate over all sstables and split them in two parts, one containing the repaired data and one the unrepaired. We could in theory double the number of sstables on a node. To avoid this we could make anticompaction also do a compaction, for example, if we are to anticompact 10 sstables, we could anticompact those to 2. Note that we need to avoid creating too big sstables though, if we anticompact all sstables on a node it would essentially be a major compaction. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7726) Give CRR a default input_cql Statement
[ https://issues.apache.org/jira/browse/CASSANDRA-7726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14094927#comment-14094927 ] Russell Alexander Spitzer commented on CASSANDRA-7726: -- I'm getting some errors with the following tables {code} CREATE TABLE mixedCase ( KeY text, Field1 int, field2 int, FielD3 int, field4 int, field5 int, PRIMARY KEY ((KeY, FielD3), field5)); CREATE TABLE mixedCaseCompact ( KeY text, Field1 int, field2 int, FielD3 int, field4 int, field5 int, PRIMARY KEY ((KeY, FielD3), field5)) WITH COMPACT STORAGE; {code} {code} mixedCase = LOAD 'cql://cnspig/mixedCase' USING CqlNativeStorage(); mixedCaseCompact = LOAD 'cql://cnspig/mixedCaseCompact' USING CqlNativeStorage(); dump mixedCase; dump mixedCaseCompact; {code} * mixedCase errors out when dumping with the following * {code} Caused by: java.io.IOException: Task process exit with nonzero status of 65. at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:258) at org.apache.pig.PigServer.openIterator(PigServer.java:856) at org.apache.pig.tools.grunt.GruntParser.processDump(GruntParser.java:683) at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:303) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:190) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:166) at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:69) at org.apache.pig.Main.run(Main.java:490) at org.apache.pig.Main.main(Main.java:111) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.hadoop.util.RunJar.main(RunJar.java:156) Caused by: org.apache.pig.backend.executionengine.ExecException: ERROR 2997: Unable to recreate exception from backed error: java.lang.Throwable: Child Error at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:271) Caused by: java.io.IOException: Task process exit with nonzero status of 65. at org.apache.hadoop.mapred.TaskRunner.run(TaskRunner.java:258) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.getErrorMessages(Launcher.java:217) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.Launcher.getStats(Launcher.java:149) at org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MapReduceLauncher.launchPig(MapReduceLauncher.java:383) at org.apache.pig.PigServer.launchPlan(PigServer.java:1279) at org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:1264) at org.apache.pig.PigServer.storeEx(PigServer.java:961) at org.apache.pig.PigServer.store(PigServer.java:928) at org.apache.pig.PigServer.openIterator(PigServer.java:841) ... 12 more {code} * mixedCaseCompact fails when attempting to load the table * {code} ERROR 2245: line 11, column 19 Cannot get schema from loadFunc org.apache.cassandra.hadoop.pig.CqlNativeStorage org.apache.pig.impl.logicalLayer.FrontendException: ERROR 2245: line 11, column 19 Cannot get schema from loadFunc org.apache.cassandra.hadoop.pig.CqlNativeStorage at org.apache.pig.newplan.logical.relational.LOLoad.getSchemaFromMetaData(LOLoad.java:155) at org.apache.pig.newplan.logical.relational.LOLoad.getSchema(LOLoad.java:110) at org.apache.pig.newplan.logical.visitor.LineageFindRelVisitor.visit(LineageFindRelVisitor.java:100) at org.apache.pig.newplan.logical.relational.LOLoad.accept(LOLoad.java:219) at org.apache.pig.newplan.DependencyOrderWalker.walk(DependencyOrderWalker.java:75) at org.apache.pig.newplan.PlanVisitor.visit(PlanVisitor.java:50) at org.apache.pig.newplan.logical.visitor.CastLineageSetter.init(CastLineageSetter.java:57) at org.apache.pig.PigServer$Graph.compile(PigServer.java:1644) at org.apache.pig.PigServer$Graph.validateQuery(PigServer.java:1575) at org.apache.pig.PigServer$Graph.registerQuery(PigServer.java:1547) at org.apache.pig.PigServer.registerQuery(PigServer.java:549) at org.apache.pig.tools.grunt.GruntParser.processPig(GruntParser.java:971) at org.apache.pig.tools.pigscript.parser.PigScriptParser.parse(PigScriptParser.java:386) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:190) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:166) at org.apache.pig.tools.grunt.Grunt.run(Grunt.java:69) at org.apache.pig.Main.run(Main.java:490)
[jira] [Commented] (CASSANDRA-7726) Give CRR a default input_cql Statement
[ https://issues.apache.org/jira/browse/CASSANDRA-7726?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14094930#comment-14094930 ] Russell Alexander Spitzer commented on CASSANDRA-7726: -- I'll write some test cases for C* tomorrow Give CRR a default input_cql Statement -- Key: CASSANDRA-7726 URL: https://issues.apache.org/jira/browse/CASSANDRA-7726 Project: Cassandra Issue Type: Improvement Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Mike Adamson Fix For: 2.0.10, 2.1.0 Attachments: 7726-2.txt, 7726-3.txt, 7726.txt Inorder to ease migration from CqlPagingRecordReader to CqlRecordReader, it would be helpful if CRR input_cql defaulted to a select statement that would mirror the behavior of CPRR. For example for a give table with partition key `((x,y,z),c1,c2)` It would automatically generate {code} input_cql = SELECT * FROM ks.tab WHERE token(x,y,z) ? AND token (x,y,z) = ? {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7725) CqlRecordReader does not validate input_cql Statments
[ https://issues.apache.org/jira/browse/CASSANDRA-7725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7725: - Description: CRR reader doesn't validate input_cql statements which can lead to some very dangerous results. In general we should make sure that the statement conforms to the required template and throw an exception if they don't. For example if a use puts in an input statement like {code} SELECT * from ks.tab {code} Will run but will run the same query for each split. https://github.com/apache/cassandra/blob/541a20dbb2ef258705c0632cddc3361ea533995c/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java#L231 was: CRR reader doesn't validate input_cql statements which can lead to some very dangerous results. In general we should make sure that the statement conforms to the required template and throw an exception. For example if a use puts in an input statement like {code} SELECT * from ks.tab {code} Will run but will run the same query for each split. https://github.com/apache/cassandra/blob/541a20dbb2ef258705c0632cddc3361ea533995c/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java#L231 CqlRecordReader does not validate input_cql Statments - Key: CASSANDRA-7725 URL: https://issues.apache.org/jira/browse/CASSANDRA-7725 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer CRR reader doesn't validate input_cql statements which can lead to some very dangerous results. In general we should make sure that the statement conforms to the required template and throw an exception if they don't. For example if a use puts in an input statement like {code} SELECT * from ks.tab {code} Will run but will run the same query for each split. https://github.com/apache/cassandra/blob/541a20dbb2ef258705c0632cddc3361ea533995c/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java#L231 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7725) CqlRecordReader does not validate input_cql Statments
Russell Alexander Spitzer created CASSANDRA-7725: Summary: CqlRecordReader does not validate input_cql Statments Key: CASSANDRA-7725 URL: https://issues.apache.org/jira/browse/CASSANDRA-7725 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer CRR reader doesn't validate input_cql statements which can lead to some very dangerous results. In general we should make sure that the statement conforms to the required template and throw an exception. For example if a use puts in an input statement like {code} SELECT * from ks.tab {code} Will run but will run the same query for each split. https://github.com/apache/cassandra/blob/541a20dbb2ef258705c0632cddc3361ea533995c/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java#L231 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7726) Give CRR a default input_cql Statement
[ https://issues.apache.org/jira/browse/CASSANDRA-7726?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7726: - Component/s: Hadoop Give CRR a default input_cql Statement -- Key: CASSANDRA-7726 URL: https://issues.apache.org/jira/browse/CASSANDRA-7726 Project: Cassandra Issue Type: Improvement Components: Hadoop Reporter: Russell Alexander Spitzer Inorder to ease migration from CqlPagingRecordReader to CqlRecordReader, it would be helpful if CRR input_cql defaulted to a select statement that would mirror the behavior of CPRR. For example for a give table with partition key `((x,y,z),c1,c2)` It would automatically generate {code} input_cql = SELECT * FROM ks.tab WHERE token(x,y,z) ? AND token (x,y,z) = ? {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7726) Give CRR a default input_cql Statement
Russell Alexander Spitzer created CASSANDRA-7726: Summary: Give CRR a default input_cql Statement Key: CASSANDRA-7726 URL: https://issues.apache.org/jira/browse/CASSANDRA-7726 Project: Cassandra Issue Type: Improvement Reporter: Russell Alexander Spitzer Inorder to ease migration from CqlPagingRecordReader to CqlRecordReader, it would be helpful if CRR input_cql defaulted to a select statement that would mirror the behavior of CPRR. For example for a give table with partition key `((x,y,z),c1,c2)` It would automatically generate {code} input_cql = SELECT * FROM ks.tab WHERE token(x,y,z) ? AND token (x,y,z) = ? {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7725) CqlRecordReader does not validate input_cql Statments
[ https://issues.apache.org/jira/browse/CASSANDRA-7725?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7725: - Assignee: Alex Liu (was: Sylvain Lebresne) CqlRecordReader does not validate input_cql Statments - Key: CASSANDRA-7725 URL: https://issues.apache.org/jira/browse/CASSANDRA-7725 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu CRR reader doesn't validate input_cql statements which can lead to some very dangerous results. In general we should make sure that the statement conforms to the required template and throw an exception if they don't. For example if a use puts in an input statement like {code} SELECT * from ks.tab {code} Will run but will run the same query for each split. https://github.com/apache/cassandra/blob/541a20dbb2ef258705c0632cddc3361ea533995c/src/java/org/apache/cassandra/hadoop/cql3/CqlRecordReader.java#L231 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7726) Give CRR a default input_cql Statement
[ https://issues.apache.org/jira/browse/CASSANDRA-7726?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7726: - Assignee: Mike Adamson Give CRR a default input_cql Statement -- Key: CASSANDRA-7726 URL: https://issues.apache.org/jira/browse/CASSANDRA-7726 Project: Cassandra Issue Type: Improvement Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Mike Adamson Inorder to ease migration from CqlPagingRecordReader to CqlRecordReader, it would be helpful if CRR input_cql defaulted to a select statement that would mirror the behavior of CPRR. For example for a give table with partition key `((x,y,z),c1,c2)` It would automatically generate {code} input_cql = SELECT * FROM ks.tab WHERE token(x,y,z) ? AND token (x,y,z) = ? {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14085347#comment-14085347 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- https://github.com/RussellSpitzer/cassandra/compare/CASSANDRA-7631 I'm going to have to leave this branch, I've fixed up as much as I could but I realized that we won't be able to get this working cross C* versions easily and that was going to be our main use case. I can try to put more time into this offline but i'm going to stop main work on this for now. [~benedict] Feel free to take this if you like in the mean time, i've basically written all the code except for skipping tokens not valid on the node that stress is run from. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14082838#comment-14082838 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- Added another commit with the code to write legacy sstables. https://github.com/RussellSpitzer/cassandra/commit/10f7db2ad1f765bc4a7c928ade722e81c0e9b899 I'll add all the warnings and such for improper stress options next. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14082875#comment-14082875 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- True, I was just going for completeness sake atm. Maybe we should just pull all of the legacy commands out now and replace them with example user profiles that do the same thing? Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7658) stress connects to all nodes when it shouldn't
[ https://issues.apache.org/jira/browse/CASSANDRA-7658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14081065#comment-14081065 ] Russell Alexander Spitzer commented on CASSANDRA-7658: -- I'm mixed on this. I really like the discovery that you get with a default connection but we should have the ability to use the whitelist policy with specific nodes as well. stress connects to all nodes when it shouldn't -- Key: CASSANDRA-7658 URL: https://issues.apache.org/jira/browse/CASSANDRA-7658 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Brandon Williams Assignee: Benedict Priority: Minor Fix For: 2.1.1 If you tell stress -node 1,2 in cluster with more nodes, stress appears to do ring discovery and connect to them all anyway (checked via netstat.) This led to the confusion on CASSANDRA-7567 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7658) stress connects to all nodes when it shouldn't
[ https://issues.apache.org/jira/browse/CASSANDRA-7658?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14081068#comment-14081068 ] Russell Alexander Spitzer commented on CASSANDRA-7658: -- Java driver does support this, http://www.datastax.com/drivers/java/2.0/com/datastax/driver/core/policies/WhiteListPolicy.html So it can be implemented stress connects to all nodes when it shouldn't -- Key: CASSANDRA-7658 URL: https://issues.apache.org/jira/browse/CASSANDRA-7658 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Brandon Williams Assignee: Benedict Priority: Minor Fix For: 2.1.1 If you tell stress -node 1,2 in cluster with more nodes, stress appears to do ring discovery and connect to them all anyway (checked via netstat.) This led to the confusion on CASSANDRA-7567 -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14081098#comment-14081098 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- I have a few idea's I'd like to try out first :) This has given me a good chance to see how new stress works. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14081134#comment-14081134 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- I eliminated my first bottleneck. I was putting every logical row onto the ABQ which ends up being bad news when there are 300k a second. Instead I know put a list of logical rows (one for each work unit) onto the ABQ and I can keep up with the generation now up to 8 generating threads with still just 1 consumer. 73k pk/s . I'll check with a greater number of generators to see if I can keep up still. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14081134#comment-14081134 ] Russell Alexander Spitzer edited comment on CASSANDRA-7631 at 7/31/14 5:16 PM: --- I eliminated my first bottleneck. I was putting every logical row onto the ABQ which ends up being bad news when there are 300k a second. Instead I now put a list of logical rows (one for each work unit) onto the ABQ and I can keep up with the generation now up to 8 generating threads with still just 1 consumer. 73k pk/s . I'll check with a greater number of generators to see if I can keep up still. was (Author: rspitzer): I eliminated my first bottleneck. I was putting every logical row onto the ABQ which ends up being bad news when there are 300k a second. Instead I know put a list of logical rows (one for each work unit) onto the ABQ and I can keep up with the generation now up to 8 generating threads with still just 1 consumer. 73k pk/s . I'll check with a greater number of generators to see if I can keep up still. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14081184#comment-14081184 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- Looks like that is as fast as I can go, CPU is pegged at max on my MBP. Let me clean up the code and I'll get a preview up. I'm relying on CQLSSTableWriter to buffer and do the writes which provides (3) for us but limits the program to 1 CQLSSTableWriter per process since it is not thread-safe. I think (4) and (5) could be very helpful though to giving that code an easier job. (2) There is much bad argument parsing I still need to add. I'm trying to track down one bug at the moment then i'll post a preliminary branch. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14081410#comment-14081410 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- https://github.com/RussellSpitzer/cassandra/compare/CASSANDRA-7631 This should work for User Profiles, but i still have to add in the code for it to work with WRITE commands. Also I need to Write to sstable dir by default? Filter out writes based on token's existing on the node being run on Check for bad combinations of options Investigate further performance boosts Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14080430#comment-14080430 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- And of course it's been a while since i've been writing java every day so I could be doing something very wrong. I'll post a branch tomorrow once i've cleaned it up a bit. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14080429#comment-14080429 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- Back on topic, I've been running through a series of experiments to see how much faster (if any) running through cqlsstablewriter would be than just using the native client. Here are some quick numbers run on my macbook against C* also running on my macbook (for native protocol) {code} NOOP = Just generate a row don't do anything with it (I know this may be optimized out) Native = Run using -mode native cql3 SSTable = Run passing rows to a queue which is consumed by a single thread running CQLSSTableWriter n=1M using the example user profile user n=100 no_warmup profile=cqlstress-example.yaml ops(insert=1) -rate threads=N -mode (sstable|native cql3) Partitions Per Second Threads NOOP Native SSTable 1 22765 10165 20917 2 38333 17247 38659 4 58089 26920 33956 8 72434 33507 29354 16 87837 34195 29354 {code} So while a single SSTable writer can keep up with the generator threads it looks like contention over the ArrayBlockingQueue puts a threshold on performance. I'm going to look into getting a threading safe version of the SSTableWriter tomorrow (there is at the very least contention on file naming), hopefully we'll be able to just tie a different SSTableWriter to each generator. If all else fails we can just have them writing to different directories then rename the sstables when we have finished. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7620) Allow user stress commands to specify the seed on the command line
[ https://issues.apache.org/jira/browse/CASSANDRA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14077801#comment-14077801 ] Russell Alexander Spitzer commented on CASSANDRA-7620: -- I definitely see the benefits to explict setting of key ranges but at the same time I think it's more conceptually simple to keep all the other parameters the same and only change the seed to get different data. If we aren't going to allow user specification though we should probably pull it out of the config file as well. As an aside related to seed specification, as it is stress currently non-deterministic. I think we may want to force repeated runs with the same parameters to generate the same data. I could start another ticket for this if that's an issue for anyone else. For example running {code} ./bin/cassandra-stress user ops(insert=1) profile=cqlstress-example.yaml n=1000 no_warmup -rate threads=200 Select * from stresscql.testtypes # (2097 rows) ./bin/cassandra-stress user ops(insert=1) profile=cqlstress-example.yaml n=1000 no_warmup -rate threads=200 Select * from stresscql.testtypes # (2954 rows) {code} I think most users would expect that you would generate the exact same data with the same command so it may be worthwhile to modify this. Allow user stress commands to specify the seed on the command line -- Key: CASSANDRA-7620 URL: https://issues.apache.org/jira/browse/CASSANDRA-7620 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Trivial Attachments: CASSANDRA-7620-2.1.patch Although I don't want to add to parameter bloat, it would be really useful if the seed for a particular profile could be set on the command line. This makes it easier to run the same command with many different seed settings without modifying the profile file. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7620) Allow user stress commands to specify the seed on the command line
[ https://issues.apache.org/jira/browse/CASSANDRA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14077821#comment-14077821 ] Russell Alexander Spitzer commented on CASSANDRA-7620: -- I think the issue is that different clustering columns are being generated, | {code} ./bin/cassandra-stress user ops(insert=1) profile=cqlstress-example.yaml n=1000 no_warmup -key populate=1..1000 -rate threads=200 cqlsh:stresscql SELECT count(*) FROM typestest ; count --- 3052 (1 rows) ./bin/cassandra-stress user ops(insert=1) profile=cqlstress-example.yaml n=1000 no_warmup -key populate=1..1000 -rate threads=200 cqlsh:stresscql SELECT count(*) FROM typestest ; count --- 4334 (1 rows) {code} Should we swap this ticket to be Remove seed from user stress? Or should I make a new ticket for that Allow user stress commands to specify the seed on the command line -- Key: CASSANDRA-7620 URL: https://issues.apache.org/jira/browse/CASSANDRA-7620 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Trivial Attachments: CASSANDRA-7620-2.1.patch Although I don't want to add to parameter bloat, it would be really useful if the seed for a particular profile could be set on the command line. This makes it easier to run the same command with many different seed settings without modifying the profile file. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7620) Allow user stress commands to specify the seed on the command line
[ https://issues.apache.org/jira/browse/CASSANDRA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14077860#comment-14077860 ] Russell Alexander Spitzer commented on CASSANDRA-7620: -- Seed is the same each time. I'm using the default exaple user profile. It has {code} Uniform for all of the columns except one clustering column population:gaussian(1.1000) Inside of insert the distributions are uniform except for perbatch: ~exp(1..3)/4 {code} So if i set n= the population size of the primary key would i get deterministic results? Or can you only get deterministic results with a fixed distribution? Allow user stress commands to specify the seed on the command line -- Key: CASSANDRA-7620 URL: https://issues.apache.org/jira/browse/CASSANDRA-7620 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Trivial Attachments: CASSANDRA-7620-2.1.patch Although I don't want to add to parameter bloat, it would be really useful if the seed for a particular profile could be set on the command line. This makes it easier to run the same command with many different seed settings without modifying the profile file. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7620) Allow user stress commands to specify the seed on the command line
[ https://issues.apache.org/jira/browse/CASSANDRA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14077882#comment-14077882 ] Russell Alexander Spitzer commented on CASSANDRA-7620: -- Ah that makes sense, Thanks for your help. Allow user stress commands to specify the seed on the command line -- Key: CASSANDRA-7620 URL: https://issues.apache.org/jira/browse/CASSANDRA-7620 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Trivial Attachments: CASSANDRA-7620-2.1.patch Although I don't want to add to parameter bloat, it would be really useful if the seed for a particular profile could be set on the command line. This makes it easier to run the same command with many different seed settings without modifying the profile file. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7631) Allow Stress to write directly to SSTables
Russell Alexander Spitzer created CASSANDRA-7631: Summary: Allow Stress to write directly to SSTables Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14076415#comment-14076415 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- I think we can implement this by writing a new client, SSTableClient which would create the directory structure and instead of executing cql statements will add lines to a CQLSSTable writer. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer reassigned CASSANDRA-7631: Assignee: Russell Alexander Spitzer Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14076906#comment-14076906 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- +1 Basically Put a TB on the cluster as fast as possible, Then run a mixed user-defined workload Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14077020#comment-14077020 ] Russell Alexander Spitzer commented on CASSANDRA-7631: -- https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java wraps SSTableSimpleUnsorted Writer so I think we are ok there. The main reason I would like this as part of stress is that we already have all the data generation code backed in for arbitrary schemas, Thanks [~tjake]! This way we could prepare for a test that uses a large amount of data and a mixed workload much faster. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-7631) Allow Stress to write directly to SSTables
[ https://issues.apache.org/jira/browse/CASSANDRA-7631?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14077020#comment-14077020 ] Russell Alexander Spitzer edited comment on CASSANDRA-7631 at 7/28/14 10:32 PM: https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java wraps SSTableSimpleUnsorted Writer so I think we are ok there. The main reason I would like this as part of stress is that we already have all the data generation code written in for arbitrary schemas, Thanks [~tjake]! This way we could prepare for a test that writes a large amount of data and then runs a mixed workload much faster. was (Author: rspitzer): https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java wraps SSTableSimpleUnsorted Writer so I think we are ok there. The main reason I would like this as part of stress is that we already have all the data generation code backed in for arbitrary schemas, Thanks [~tjake]! This way we could prepare for a test that uses a large amount of data and a mixed workload much faster. Allow Stress to write directly to SSTables -- Key: CASSANDRA-7631 URL: https://issues.apache.org/jira/browse/CASSANDRA-7631 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer One common difficulty with benchmarking machines is the amount of time it takes to initially load data. For machines with a large amount of ram this becomes especially onerous because a very large amount of data needs to be placed on the machine before page-cache can be circumvented. To remedy this I suggest we add a top level flag to Cassandra-Stress which would cause the tool to write directly to sstables rather than actually performing CQL inserts. Internally this would use CQLSStable writer to write directly to sstables while skipping any keys which are not owned by the node stress is running on. The same stress command run on each node in the cluster would then write unique sstables only containing data which that node is responsible for. Following this no further network IO would be required to distribute data as it would all already be correctly in place. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7608) StressD can't create keyspaces with Write Command
[ https://issues.apache.org/jira/browse/CASSANDRA-7608?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14074599#comment-14074599 ] Russell Alexander Spitzer commented on CASSANDRA-7608: -- [~benedict], Another Stress ticket that would really help us out :) StressD can't create keyspaces with Write Command - Key: CASSANDRA-7608 URL: https://issues.apache.org/jira/browse/CASSANDRA-7608 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Minor Fix For: 2.1.1 Attachments: CASSANDRA-7608-2.1.patch It is impossible to run the default stress command via the dameon ./stress write Because the column names are HeapByteBuffers so they get ignored during serilization (no error is thrown) and then when the object is deserialized on the server the settings.columns.names is null. This leads to a null pointer on the dameon for what would have worked had it run locally. Settings object on the Local machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@1465} maxColumnsPerKey = 5 names = {java.util.Arrays$ArrayList@1471} size = 5 [0] = {java.nio.HeapByteBuffer@1478}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [1] = {java.nio.HeapByteBuffer@1483}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [2] = {java.nio.HeapByteBuffer@1484}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [3] = {java.nio.HeapByteBuffer@1485}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [4] = {java.nio.HeapByteBuffer@1486}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] {code} Setings object on the StressD Machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@810} maxColumnsPerKey = 5 names = null {code} This leads to the null pointer in {code} Exception in thread Thread-1 java.lang.NullPointerException at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpacesThrift(SettingsSchema.java:94) at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpaces(SettingsSchema.java:67) at org.apache.cassandra.stress.settings.StressSettings.maybeCreateKeyspaces(StressSettings.java:193) at org.apache.cassandra.stress.StressAction.run(StressAction.java:59) at java.lang.Thread.run(Thread.java:745) {code} Which refers to {code} for (int i = 0; i settings.columns.names.size(); i++) standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), BytesType)); {code} Possible solution: Just use the settings.columns.namestr and convert them to byte buffers at this point in the code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
[ https://issues.apache.org/jira/browse/CASSANDRA-7559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14074605#comment-14074605 ] Russell Alexander Spitzer commented on CASSANDRA-7559: -- [~benedict], One last stress patch (for now). I would be really grateful if you could review. Switch Stress from using math3.pair because it is unserializable - Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7559-2.1.patch Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7608) StressD doesn't can't create keyspaces with Write Command
Russell Alexander Spitzer created CASSANDRA-7608: Summary: StressD doesn't can't create keyspaces with Write Command Key: CASSANDRA-7608 URL: https://issues.apache.org/jira/browse/CASSANDRA-7608 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Priority: Minor It is impossible to run the default stress command via the dameon ./stress write Because the column names are HeapByteBuffers so they get ignored during serilization (no error is thrown) and then when the object is deserialized on the server the settings.columns.names is null. This leads to a null pointer on the dameon for what would have worked had it run locally. Settings object on the Local machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@1465} maxColumnsPerKey = 5 names = {java.util.Arrays$ArrayList@1471} size = 5 [0] = {java.nio.HeapByteBuffer@1478}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [1] = {java.nio.HeapByteBuffer@1483}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [2] = {java.nio.HeapByteBuffer@1484}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [3] = {java.nio.HeapByteBuffer@1485}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [4] = {java.nio.HeapByteBuffer@1486}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] {code} Setings object on the StressD Machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@810} maxColumnsPerKey = 5 names = null {code} This leads to the null pointer in {code} Exception in thread Thread-1 java.lang.NullPointerException at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpacesThrift(SettingsSchema.java:94) at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpaces(SettingsSchema.java:67) at org.apache.cassandra.stress.settings.StressSettings.maybeCreateKeyspaces(StressSettings.java:193) at org.apache.cassandra.stress.StressAction.run(StressAction.java:59) at java.lang.Thread.run(Thread.java:745) {code} Which refers to {code} for (int i = 0; i settings.columns.names.size(); i++) standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), BytesType)); {code} Possible solution: Just use the settings.columns.namestr and convert them to byte buffers at this point in the code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7608) StressD can't create keyspaces with Write Command
[ https://issues.apache.org/jira/browse/CASSANDRA-7608?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7608: - Summary: StressD can't create keyspaces with Write Command (was: StressD doesn't can't create keyspaces with Write Command) StressD can't create keyspaces with Write Command - Key: CASSANDRA-7608 URL: https://issues.apache.org/jira/browse/CASSANDRA-7608 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Priority: Minor It is impossible to run the default stress command via the dameon ./stress write Because the column names are HeapByteBuffers so they get ignored during serilization (no error is thrown) and then when the object is deserialized on the server the settings.columns.names is null. This leads to a null pointer on the dameon for what would have worked had it run locally. Settings object on the Local machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@1465} maxColumnsPerKey = 5 names = {java.util.Arrays$ArrayList@1471} size = 5 [0] = {java.nio.HeapByteBuffer@1478}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [1] = {java.nio.HeapByteBuffer@1483}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [2] = {java.nio.HeapByteBuffer@1484}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [3] = {java.nio.HeapByteBuffer@1485}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [4] = {java.nio.HeapByteBuffer@1486}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] {code} Setings object on the StressD Machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@810} maxColumnsPerKey = 5 names = null {code} This leads to the null pointer in {code} Exception in thread Thread-1 java.lang.NullPointerException at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpacesThrift(SettingsSchema.java:94) at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpaces(SettingsSchema.java:67) at org.apache.cassandra.stress.settings.StressSettings.maybeCreateKeyspaces(StressSettings.java:193) at org.apache.cassandra.stress.StressAction.run(StressAction.java:59) at java.lang.Thread.run(Thread.java:745) {code} Which refers to {code} for (int i = 0; i settings.columns.names.size(); i++) standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), BytesType)); {code} Possible solution: Just use the settings.columns.namestr and convert them to byte buffers at this point in the code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7608) StressD can't create keyspaces with Write Command
[ https://issues.apache.org/jira/browse/CASSANDRA-7608?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer reassigned CASSANDRA-7608: Assignee: Russell Alexander Spitzer StressD can't create keyspaces with Write Command - Key: CASSANDRA-7608 URL: https://issues.apache.org/jira/browse/CASSANDRA-7608 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Minor Fix For: 2.1.1 It is impossible to run the default stress command via the dameon ./stress write Because the column names are HeapByteBuffers so they get ignored during serilization (no error is thrown) and then when the object is deserialized on the server the settings.columns.names is null. This leads to a null pointer on the dameon for what would have worked had it run locally. Settings object on the Local machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@1465} maxColumnsPerKey = 5 names = {java.util.Arrays$ArrayList@1471} size = 5 [0] = {java.nio.HeapByteBuffer@1478}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [1] = {java.nio.HeapByteBuffer@1483}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [2] = {java.nio.HeapByteBuffer@1484}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [3] = {java.nio.HeapByteBuffer@1485}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [4] = {java.nio.HeapByteBuffer@1486}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] {code} Setings object on the StressD Machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@810} maxColumnsPerKey = 5 names = null {code} This leads to the null pointer in {code} Exception in thread Thread-1 java.lang.NullPointerException at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpacesThrift(SettingsSchema.java:94) at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpaces(SettingsSchema.java:67) at org.apache.cassandra.stress.settings.StressSettings.maybeCreateKeyspaces(StressSettings.java:193) at org.apache.cassandra.stress.StressAction.run(StressAction.java:59) at java.lang.Thread.run(Thread.java:745) {code} Which refers to {code} for (int i = 0; i settings.columns.names.size(); i++) standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), BytesType)); {code} Possible solution: Just use the settings.columns.namestr and convert them to byte buffers at this point in the code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7608) StressD can't create keyspaces with Write Command
[ https://issues.apache.org/jira/browse/CASSANDRA-7608?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14073442#comment-14073442 ] Russell Alexander Spitzer commented on CASSANDRA-7608: -- Looks like there are actually a bunch of uses of columns.names, so i'll switch it to a byte array. StressD can't create keyspaces with Write Command - Key: CASSANDRA-7608 URL: https://issues.apache.org/jira/browse/CASSANDRA-7608 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Minor Fix For: 2.1.1 It is impossible to run the default stress command via the dameon ./stress write Because the column names are HeapByteBuffers so they get ignored during serilization (no error is thrown) and then when the object is deserialized on the server the settings.columns.names is null. This leads to a null pointer on the dameon for what would have worked had it run locally. Settings object on the Local machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@1465} maxColumnsPerKey = 5 names = {java.util.Arrays$ArrayList@1471} size = 5 [0] = {java.nio.HeapByteBuffer@1478}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [1] = {java.nio.HeapByteBuffer@1483}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [2] = {java.nio.HeapByteBuffer@1484}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [3] = {java.nio.HeapByteBuffer@1485}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [4] = {java.nio.HeapByteBuffer@1486}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] {code} Setings object on the StressD Machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@810} maxColumnsPerKey = 5 names = null {code} This leads to the null pointer in {code} Exception in thread Thread-1 java.lang.NullPointerException at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpacesThrift(SettingsSchema.java:94) at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpaces(SettingsSchema.java:67) at org.apache.cassandra.stress.settings.StressSettings.maybeCreateKeyspaces(StressSettings.java:193) at org.apache.cassandra.stress.StressAction.run(StressAction.java:59) at java.lang.Thread.run(Thread.java:745) {code} Which refers to {code} for (int i = 0; i settings.columns.names.size(); i++) standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), BytesType)); {code} Possible solution: Just use the settings.columns.namestr and convert them to byte buffers at this point in the code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7608) StressD can't create keyspaces with Write Command
[ https://issues.apache.org/jira/browse/CASSANDRA-7608?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7608: - Attachment: CASSANDRA-7608-2.1.patch StressD can't create keyspaces with Write Command - Key: CASSANDRA-7608 URL: https://issues.apache.org/jira/browse/CASSANDRA-7608 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Minor Fix For: 2.1.1 Attachments: CASSANDRA-7608-2.1.patch It is impossible to run the default stress command via the dameon ./stress write Because the column names are HeapByteBuffers so they get ignored during serilization (no error is thrown) and then when the object is deserialized on the server the settings.columns.names is null. This leads to a null pointer on the dameon for what would have worked had it run locally. Settings object on the Local machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@1465} maxColumnsPerKey = 5 names = {java.util.Arrays$ArrayList@1471} size = 5 [0] = {java.nio.HeapByteBuffer@1478}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [1] = {java.nio.HeapByteBuffer@1483}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [2] = {java.nio.HeapByteBuffer@1484}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [3] = {java.nio.HeapByteBuffer@1485}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [4] = {java.nio.HeapByteBuffer@1486}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] {code} Setings object on the StressD Machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@810} maxColumnsPerKey = 5 names = null {code} This leads to the null pointer in {code} Exception in thread Thread-1 java.lang.NullPointerException at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpacesThrift(SettingsSchema.java:94) at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpaces(SettingsSchema.java:67) at org.apache.cassandra.stress.settings.StressSettings.maybeCreateKeyspaces(StressSettings.java:193) at org.apache.cassandra.stress.StressAction.run(StressAction.java:59) at java.lang.Thread.run(Thread.java:745) {code} Which refers to {code} for (int i = 0; i settings.columns.names.size(); i++) standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), BytesType)); {code} Possible solution: Just use the settings.columns.namestr and convert them to byte buffers at this point in the code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7608) StressD can't create keyspaces with Write Command
[ https://issues.apache.org/jira/browse/CASSANDRA-7608?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14073571#comment-14073571 ] Russell Alexander Spitzer commented on CASSANDRA-7608: -- I ended up adjusting the serialization of the settings.columns.names so that we convert it from a bytebuffer to an array on the way out and back to a bytebuffer on the way in. StressD can't create keyspaces with Write Command - Key: CASSANDRA-7608 URL: https://issues.apache.org/jira/browse/CASSANDRA-7608 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Minor Fix For: 2.1.1 Attachments: CASSANDRA-7608-2.1.patch It is impossible to run the default stress command via the dameon ./stress write Because the column names are HeapByteBuffers so they get ignored during serilization (no error is thrown) and then when the object is deserialized on the server the settings.columns.names is null. This leads to a null pointer on the dameon for what would have worked had it run locally. Settings object on the Local machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@1465} maxColumnsPerKey = 5 names = {java.util.Arrays$ArrayList@1471} size = 5 [0] = {java.nio.HeapByteBuffer@1478}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [1] = {java.nio.HeapByteBuffer@1483}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [2] = {java.nio.HeapByteBuffer@1484}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [3] = {java.nio.HeapByteBuffer@1485}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] [4] = {java.nio.HeapByteBuffer@1486}java.nio.HeapByteBuffer[pos=0 lim=2 cap=2] {code} Setings object on the StressD Machine {code} columns = {org.apache.cassandra.stress.settings.SettingsColumn@810} maxColumnsPerKey = 5 names = null {code} This leads to the null pointer in {code} Exception in thread Thread-1 java.lang.NullPointerException at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpacesThrift(SettingsSchema.java:94) at org.apache.cassandra.stress.settings.SettingsSchema.createKeySpaces(SettingsSchema.java:67) at org.apache.cassandra.stress.settings.StressSettings.maybeCreateKeyspaces(StressSettings.java:193) at org.apache.cassandra.stress.StressAction.run(StressAction.java:59) at java.lang.Thread.run(Thread.java:745) {code} Which refers to {code} for (int i = 0; i settings.columns.names.size(); i++) standardCfDef.addToColumn_metadata(new ColumnDef(settings.columns.names.get(i), BytesType)); {code} Possible solution: Just use the settings.columns.namestr and convert them to byte buffers at this point in the code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7620) Allow user stress commands to specify the seed on the command line
[ https://issues.apache.org/jira/browse/CASSANDRA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer reassigned CASSANDRA-7620: Assignee: Russell Alexander Spitzer Allow user stress commands to specify the seed on the command line -- Key: CASSANDRA-7620 URL: https://issues.apache.org/jira/browse/CASSANDRA-7620 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Trivial Although I don't want to add to parameter bloat, it would be really useful if the seed for a particular profile could be set on the command line. This makes it easier to run the same command with many different seed settings without modifying the profile file. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7620) Allow user stress commands to specify the seed on the command line
Russell Alexander Spitzer created CASSANDRA-7620: Summary: Allow user stress commands to specify the seed on the command line Key: CASSANDRA-7620 URL: https://issues.apache.org/jira/browse/CASSANDRA-7620 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Priority: Trivial Although I don't want to add to parameter bloat, it would be really useful if the seed for a particular profile could be set on the command line. This makes it easier to run the same command with many different seed settings without modifying the profile file. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7620) Allow user stress commands to specify the seed on the command line
[ https://issues.apache.org/jira/browse/CASSANDRA-7620?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7620: - Attachment: CASSANDRA-7620-2.1.patch Allow user stress commands to specify the seed on the command line -- Key: CASSANDRA-7620 URL: https://issues.apache.org/jira/browse/CASSANDRA-7620 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Trivial Attachments: CASSANDRA-7620-2.1.patch Although I don't want to add to parameter bloat, it would be really useful if the seed for a particular profile could be set on the command line. This makes it easier to run the same command with many different seed settings without modifying the profile file. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6851) Improve anticompaction after incremental repair
[ https://issues.apache.org/jira/browse/CASSANDRA-6851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14069324#comment-14069324 ] Russell Alexander Spitzer commented on CASSANDRA-6851: -- The only thing I worry about is changing the signature on, bikeshed {code} public CollectionCollectionSSTableReader groupSSTablesForAntiCompaction(CollectionSSTableReader sstablesToGroup, int groupSize) {code} I was wondering if we might want multiple implementations, some of which may not have a set group size. I know this is bikesheding, but maybe we want to change the signature to maxGroupSize? /bikeshed I'm +1 with everything else Improve anticompaction after incremental repair --- Key: CASSANDRA-6851 URL: https://issues.apache.org/jira/browse/CASSANDRA-6851 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Marcus Eriksson Assignee: Russell Alexander Spitzer Priority: Minor Labels: compaction, lhf Fix For: 3.0 After an incremental repair we iterate over all sstables and split them in two parts, one containing the repaired data and one the unrepaired. We could in theory double the number of sstables on a node. To avoid this we could make anticompaction also do a compaction, for example, if we are to anticompact 10 sstables, we could anticompact those to 2. Note that we need to avoid creating too big sstables though, if we anticompact all sstables on a node it would essentially be a major compaction. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
[ https://issues.apache.org/jira/browse/CASSANDRA-7559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14069424#comment-14069424 ] Russell Alexander Spitzer commented on CASSANDRA-7559: -- [~tjake], This is the patch we were talking about a few days ago for fixing StressD. I would be grateful if you could review and make sure I haven't done anything destructive. Switch Stress from using math3.pair because it is unserializable - Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7559-2.1.patch Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
Russell Alexander Spitzer created CASSANDRA-7559: Summary: Switch Stress from using math3.pair because it is unserializable Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
[ https://issues.apache.org/jira/browse/CASSANDRA-7559?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer reassigned CASSANDRA-7559: Assignee: Russell Alexander Spitzer Switch Stress from using math3.pair because it is unserializable - Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
[ https://issues.apache.org/jira/browse/CASSANDRA-7559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14064340#comment-14064340 ] Russell Alexander Spitzer commented on CASSANDRA-7559: -- cassandra.util.pair is also unserializable so i'll look at just removing the stored Pair altogether. Switch Stress from using math3.pair because it is unserializable - Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
[ https://issues.apache.org/jira/browse/CASSANDRA-7559?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14064395#comment-14064395 ] Russell Alexander Spitzer commented on CASSANDRA-7559: -- Patched to keep everything as Maps right until needed by the Distribution code. https://github.com/RussellSpitzer/cassandra/compare/CASSANDRA-7559 Switch Stress from using math3.pair because it is unserializable - Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
[ https://issues.apache.org/jira/browse/CASSANDRA-7559?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7559: - Reproduced In: 2.1 rc3 Switch Stress from using math3.pair because it is unserializable - Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7559-2.1.patch Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7559) Switch Stress from using math3.pair because it is unserializable
[ https://issues.apache.org/jira/browse/CASSANDRA-7559?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7559: - Attachment: CASSANDRA-7559-2.1.patch Switch Stress from using math3.pair because it is unserializable - Key: CASSANDRA-7559 URL: https://issues.apache.org/jira/browse/CASSANDRA-7559 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Attachments: CASSANDRA-7559-2.1.patch Stress uses org.apache.commons.math3.util.Pair to hold information in settings because eventually it is used in commons.math3.distributions. This makes the settings unserializable so we can't run with StressDemon. {code} /bin/cassandra-stress user no_warmup ops(insert=1) n=1 profile=cqlstress-example.yaml -sendto 127.0.0.1 Exception in thread main java.io.NotSerializableException: org.apache.commons.math3.util.Pair at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1183) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at java.util.ArrayList.writeObject(ArrayList.java:742) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at java.io.ObjectStreamClass.invokeWriteObject(ObjectStreamClass.java:988) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1495) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeObject(ObjectOutputStream.java:347) at org.apache.cassandra.stress.Stress.main(Stress.java:78) Control-C caught. Canceling running action and shutting down... {code} To fix this we can pass around serializable pairs and convert to commons.math3 pairs before we actually pass the objects to the distribution code. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6851) Improve anticompaction after incremental repair
[ https://issues.apache.org/jira/browse/CASSANDRA-6851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14053028#comment-14053028 ] Russell Alexander Spitzer commented on CASSANDRA-6851: -- Updated branch at https://github.com/RussellSpitzer/cassandra/compare/CASSANDRA-6851?expand=1 Sorry for the delay again but DSE waits for no one. I've moved the grouping function to AbstractCompactionStrategy with the default implementation being the numeric grouper using a group size of 2. For Leveled compaction strategy I overrode the function to first bin by level, and then group. I've also added a few tests for anti-compacting a table with 10 sstables and a test for Leveled Compaction Strat to make sure it's grouping method does not return groups with sstables from mixed levels. I tried to fix up most of the style issues and fix up the imports so they are more consistent (no wildcards). As usual please let me see anything out of place or anything that can use improvement and I'll continue to try to get IDEA to respect my chosen style conventions. Improve anticompaction after incremental repair --- Key: CASSANDRA-6851 URL: https://issues.apache.org/jira/browse/CASSANDRA-6851 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Marcus Eriksson Assignee: Russell Alexander Spitzer Priority: Minor Labels: compaction, lhf Fix For: 2.1.1 After an incremental repair we iterate over all sstables and split them in two parts, one containing the repaired data and one the unrepaired. We could in theory double the number of sstables on a node. To avoid this we could make anticompaction also do a compaction, for example, if we are to anticompact 10 sstables, we could anticompact those to 2. Note that we need to avoid creating too big sstables though, if we anticompact all sstables on a node it would essentially be a major compaction. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7238) Nodetool Status performance is much slower with VNodes On
[ https://issues.apache.org/jira/browse/CASSANDRA-7238?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7238: - Reproduced In: 2.1 beta2, 2.0.8 (was: 2.1 beta2) Nodetool Status performance is much slower with VNodes On - Key: CASSANDRA-7238 URL: https://issues.apache.org/jira/browse/CASSANDRA-7238 Project: Cassandra Issue Type: Bug Components: Tools Environment: 1000 M1.Large Ubuntu 12.04 Reporter: Russell Alexander Spitzer Priority: Minor Nodetool status on a 1000 Node cluster without vnodes returns in several seconds. With vnodes on (256) there are OOM errors with the default XMX of 32. Adjusting the XMX to 128 allows nodetool status to complete but the execution takes roughly 10 minutes. Tested {code} XMX| Status 32 |OOM 64 |OOM: GC Overhead 128|Finishes in ~10 minutes 500|Finishes in ~10 minutes 1000 |Finishes in ~10 minutes {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6851) Improve anticompaction after incremental repair
[ https://issues.apache.org/jira/browse/CASSANDRA-6851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14032039#comment-14032039 ] Russell Alexander Spitzer commented on CASSANDRA-6851: -- Gave this another go, this is another rough draft and I'll need to add a few more unit tests to make sure it's behaving as schedule. I implemented a naive grouper which lets you just define a number of sstables to group together during an anti compaction. For the final version of this patch we can determine a more clever way of grouping together sstables. https://github.com/RussellSpitzer/cassandra/commit/afd9e5200e5b7b455b8ab8237a8234bed2de9d61 Sorry it took me a while to get back to this, my weekends have been pretty devoted to DSE what with the new release coming so soon. Improve anticompaction after incremental repair --- Key: CASSANDRA-6851 URL: https://issues.apache.org/jira/browse/CASSANDRA-6851 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Marcus Eriksson Assignee: Russell Alexander Spitzer Priority: Minor Labels: compaction, lhf Fix For: 2.1.1 After an incremental repair we iterate over all sstables and split them in two parts, one containing the repaired data and one the unrepaired. We could in theory double the number of sstables on a node. To avoid this we could make anticompaction also do a compaction, for example, if we are to anticompact 10 sstables, we could anticompact those to 2. Note that we need to avoid creating too big sstables though, if we anticompact all sstables on a node it would essentially be a major compaction. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6851) Improve anticompaction after incremental repair
[ https://issues.apache.org/jira/browse/CASSANDRA-6851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14003896#comment-14003896 ] Russell Alexander Spitzer commented on CASSANDRA-6851: -- Ah I see, that makes a lot more sense. We could always try to do a comparison of bloom filters for sstables, and try to anti-compact pairs with the most similar filters? IE: if dotproduct(bloomfilters) some threshold : place in the same anticompaction group. Improve anticompaction after incremental repair --- Key: CASSANDRA-6851 URL: https://issues.apache.org/jira/browse/CASSANDRA-6851 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Marcus Eriksson Assignee: Russell Alexander Spitzer Priority: Minor Labels: compaction, lhf Fix For: 2.1.1 After an incremental repair we iterate over all sstables and split them in two parts, one containing the repaired data and one the unrepaired. We could in theory double the number of sstables on a node. To avoid this we could make anticompaction also do a compaction, for example, if we are to anticompact 10 sstables, we could anticompact those to 2. Note that we need to avoid creating too big sstables though, if we anticompact all sstables on a node it would essentially be a major compaction. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6851) Improve anticompaction after incremental repair
[ https://issues.apache.org/jira/browse/CASSANDRA-6851?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14001313#comment-14001313 ] Russell Alexander Spitzer commented on CASSANDRA-6851: -- I began working on this issue (and learning about the compaction paths) by grouping the repaired and unrepaired tables and then calling a compaction on each group. This required me to add a hook into AbstractCompactionTask for the results of a compaction operation so that I could fulfill the return contract of doAntiCompaction. Currently it seems like the 2.1 tests are not passing cleanly and I'm not sure of which issues might be related to my patch. I'm running a bit short on time for this weekend so I'll keep working when I can find some time. My major question for finishing this is what should we be using a threshold for an sstable being to large? Should we let this be user configurable or put a static ceiling on the compaction? As usual I would appreciate any advice or critique for my efforts thus far Current work: https://github.com/RussellSpitzer/cassandra/compare/CASSANDRA-6851 Improve anticompaction after incremental repair --- Key: CASSANDRA-6851 URL: https://issues.apache.org/jira/browse/CASSANDRA-6851 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Priority: Minor Labels: compaction, lhf Fix For: 2.1.1 After an incremental repair we iterate over all sstables and split them in two parts, one containing the repaired data and one the unrepaired. We could in theory double the number of sstables on a node. To avoid this we could make anticompaction also do a compaction, for example, if we are to anticompact 10 sstables, we could anticompact those to 2. Note that we need to avoid creating too big sstables though, if we anticompact all sstables on a node it would essentially be a major compaction. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7238) Nodetool Status performance is much slower with VNodes On
Russell Alexander Spitzer created CASSANDRA-7238: Summary: Nodetool Status performance is much slower with VNodes On Key: CASSANDRA-7238 URL: https://issues.apache.org/jira/browse/CASSANDRA-7238 Project: Cassandra Issue Type: Bug Components: Tools Environment: 1000 M1.Large Ubuntu 12.04 Reporter: Russell Alexander Spitzer Priority: Minor Fix For: 2.1 beta2 Nodetool status on a 1000 Node cluster without vnodes returns in several seconds. With vnodes on (256) there are OOM errors with the default XMX of 32. Adjusting the XMX to 128 allows nodetool status to complete but the execution takes roughly 10 minutes. Tested {code} XMX| Status 32 |OOM 64 |OOM: GC Overhead 128|Finishes in ~10 minutes 500|Finishes in ~10 minutes 1000 |Finishes in ~10 minutes {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7239) Nodetool Status Reports Negative Load With VNodes Disabled
[ https://issues.apache.org/jira/browse/CASSANDRA-7239?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7239: - Reproduced In: 2.1 beta2 (was: 2.0 beta 2) Nodetool Status Reports Negative Load With VNodes Disabled -- Key: CASSANDRA-7239 URL: https://issues.apache.org/jira/browse/CASSANDRA-7239 Project: Cassandra Issue Type: Bug Components: Tools Environment: 1000 Nodes EC2 m1.large ubuntu 12.04 Reporter: Russell Alexander Spitzer Priority: Minor When I run stress on a large cluster without vnodes (num_token =1 initial token set) The loads reported by nodetool status are negative, or become negative after stress is run. {code} UN 10.97.155.31-447426217 bytes 1 0.2% 8d40568c-044c-4753-be26-4ab62710beba rack1 UN 10.9.132.53 -447342449 bytes 1 0.2% 58e7f255-803d-493b-a19e-58137466fb78 rack1 UN 10.37.151.202 -447298672 bytes 1 0.2% ba29b1f1-186f-45d0-9e59-6a528db8df5d rack1 {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7239) Nodetool Status Reports Negative Load With VNodes Disabled
[ https://issues.apache.org/jira/browse/CASSANDRA-7239?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14000414#comment-14000414 ] Russell Alexander Spitzer commented on CASSANDRA-7239: -- Basically the inverse of that, at that time it was about 500mb a node. Nodetool Status Reports Negative Load With VNodes Disabled -- Key: CASSANDRA-7239 URL: https://issues.apache.org/jira/browse/CASSANDRA-7239 Project: Cassandra Issue Type: Bug Components: Tools Environment: 1000 Nodes EC2 m1.large ubuntu 12.04 Reporter: Russell Alexander Spitzer Priority: Minor When I run stress on a large cluster without vnodes (num_token =1 initial token set) The loads reported by nodetool status are negative, or become negative after stress is run. {code} UN 10.97.155.31-447426217 bytes 1 0.2% 8d40568c-044c-4753-be26-4ab62710beba rack1 UN 10.9.132.53 -447342449 bytes 1 0.2% 58e7f255-803d-493b-a19e-58137466fb78 rack1 UN 10.37.151.202 -447298672 bytes 1 0.2% ba29b1f1-186f-45d0-9e59-6a528db8df5d rack1 {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7240) Altering Keyspace Replication On Large Cluster With vnodes Leads to Warns on All nodes
Russell Alexander Spitzer created CASSANDRA-7240: Summary: Altering Keyspace Replication On Large Cluster With vnodes Leads to Warns on All nodes Key: CASSANDRA-7240 URL: https://issues.apache.org/jira/browse/CASSANDRA-7240 Project: Cassandra Issue Type: Bug Components: Core Environment: 1000 Nodes M1.large ubuntu 12.04 Reporter: Russell Alexander Spitzer 1000 Node cluster started with vnodes(256) on. 25 separate Nodes began an all write workload against the first 1000 nodes. During the test I attempted to alter the key-space from simple strategy to a network topology strategy. {code} cqlsh ALTER KEYSPACE Keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'DC1': '3', 'DC2':'3'} AND durable_writes = true; errors={}, last_host=127.0.0.1 cqlsh ALTER KEYSPACE Keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'DC1': '3', 'DC2':'3'} AND durable_writes = true; ('Unable to complete the operation against any hosts', {Host: 127.0.0.1 DC1: ConnectionShutdown('Connection to 127.0.0.1 is defunct',)}) {code} All one thousand nodes then began to repeat the following in their respective logs {code} WARN [Thread-50131] 2014-05-14 23:34:07,631 IncomingTcpConnection.java:91 - UnknownColumnFamilyException reading from socket; closing org.apache.cassandra.db.UnknownColumnFamilyException: Couldn't find cfId=46b7b090-dbaf-11e3-8413-fffd4403e7d2 at org.apache.cassandra.db.ColumnFamilySerializer.deserializeCfId(ColumnFamilySerializer.java:164) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.db.ColumnFamilySerializer.deserialize(ColumnFamilySerializer.java:97) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.db.Mutation$MutationSerializer.deserializeOneCf(Mutation.java:318) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:298) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:326) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:268) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:165) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.net.IncomingTcpConnection.handleModernVersion(IncomingTcpConnection.java:147) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:82) ~[apache-cassandra-2.1.0-beta2.jar:2.1.0-beta2] {code} Stress continued but at a decreased speed {code} Excerpt from one of the 25 Stress Nodes 83222847 , 14602, 14602, 6.7, 2.1,23.1, 132.1, 292.3, 531.3, 5216.5, 0.00188 83239512 , 13888, 13888, 7.3, 2.1,31.3, 129.9, 267.9, 555.8, 5217.7, 0.00188 83258520 , 14301, 14301, 7.0, 2.1,28.8, 125.4, 297.2, 758.1, 5219.0, 0.00188 83277750 , 14023, 14023, 7.1, 2.1,28.4, 132.8, 292.3, 703.6, 5220.4, 0.00188 83301413 , 14410, 14410, 6.9, 2.1,24.5, 124.8, 391.4, 1010.1, 5222.0, 0.00188 83316846 , 12313, 12313, 8.1, 2.1,35.1, 168.2, 275.3, 467.9, 5223.3, 0.00188 83332883 , 13753, 13753, 6.9, 2.1,28.1, 132.2, 276.1, 498.9, 5224.4, 0.00188 #ALTER REQUEST HERE 83351413 ,9981,9981, 9.9, 2.1,46.7, 172.0, 447.8, 1327.9, 5226.3, 0.00188 83358381 ,4464,4464,22.7, 2.2, 125.9, 257.8, 594.6, 1650.6, 5227.8, 0.00188 83363153 ,3186,3186,31.7, 2.5, 153.0, 300.3, 477.0, 566.1, 5229.3, 0.00189 83367341 ,2967,2967,33.7, 2.4, 173.9, 311.5, 465.8, 761.9, 5230.7, 0.00190 83370738 ,2392,2392,41.4, 2.9, 208.0, 308.1, 434.8, 839.6, 5232.2, 0.00191 83373651 ,2283,2283,43.0, 2.5, 213.9, 310.5, 409.3, 503.3, 5233.4, 0.00192 {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7238) Nodetool Status performance is much slower with VNodes On
[ https://issues.apache.org/jira/browse/CASSANDRA-7238?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7238: - Reproduced In: 2.1 beta2 Nodetool Status performance is much slower with VNodes On - Key: CASSANDRA-7238 URL: https://issues.apache.org/jira/browse/CASSANDRA-7238 Project: Cassandra Issue Type: Bug Components: Tools Environment: 1000 M1.Large Ubuntu 12.04 Reporter: Russell Alexander Spitzer Priority: Minor Nodetool status on a 1000 Node cluster without vnodes returns in several seconds. With vnodes on (256) there are OOM errors with the default XMX of 32. Adjusting the XMX to 128 allows nodetool status to complete but the execution takes roughly 10 minutes. Tested {code} XMX| Status 32 |OOM 64 |OOM: GC Overhead 128|Finishes in ~10 minutes 500|Finishes in ~10 minutes 1000 |Finishes in ~10 minutes {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7239) Nodetool Status Reports Negative Load With VNodes Disabled
Russell Alexander Spitzer created CASSANDRA-7239: Summary: Nodetool Status Reports Negative Load With VNodes Disabled Key: CASSANDRA-7239 URL: https://issues.apache.org/jira/browse/CASSANDRA-7239 Project: Cassandra Issue Type: Bug Components: Tools Environment: 1000 Nodes EC2 m1.large ubuntu 12.04 Reporter: Russell Alexander Spitzer Priority: Minor When I run stress on a large cluster without vnodes (num_token =1 initial token set) The loads reported by nodetool status are negative, or become negative after stress is run. {code} UN 10.97.155.31-447426217 bytes 1 0.2% 8d40568c-044c-4753-be26-4ab62710beba rack1 UN 10.9.132.53 -447342449 bytes 1 0.2% 58e7f255-803d-493b-a19e-58137466fb78 rack1 UN 10.37.151.202 -447298672 bytes 1 0.2% ba29b1f1-186f-45d0-9e59-6a528db8df5d rack1 {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7238) Nodetool Status performance is much slower with VNodes On
[ https://issues.apache.org/jira/browse/CASSANDRA-7238?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7238: - Fix Version/s: (was: 2.1 beta2) Nodetool Status performance is much slower with VNodes On - Key: CASSANDRA-7238 URL: https://issues.apache.org/jira/browse/CASSANDRA-7238 Project: Cassandra Issue Type: Bug Components: Tools Environment: 1000 M1.Large Ubuntu 12.04 Reporter: Russell Alexander Spitzer Priority: Minor Nodetool status on a 1000 Node cluster without vnodes returns in several seconds. With vnodes on (256) there are OOM errors with the default XMX of 32. Adjusting the XMX to 128 allows nodetool status to complete but the execution takes roughly 10 minutes. Tested {code} XMX| Status 32 |OOM 64 |OOM: GC Overhead 128|Finishes in ~10 minutes 500|Finishes in ~10 minutes 1000 |Finishes in ~10 minutes {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-5547) Multi-threaded scrub
[ https://issues.apache.org/jira/browse/CASSANDRA-5547?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13979954#comment-13979954 ] Russell Alexander Spitzer commented on CASSANDRA-5547: -- Looks good to me +1, I hope this will end up being useful for folks. Multi-threaded scrub Key: CASSANDRA-5547 URL: https://issues.apache.org/jira/browse/CASSANDRA-5547 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Benjamin Coverston Assignee: Russell Alexander Spitzer Labels: lhf Fix For: 2.0.8 Attachments: 0001-5547.patch, cassandra-2.0-5547.txt Scrub (especially offline) could benefit from being multi-threaded, especially in the case where the SSTables are compressed. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-5547) Multi-threaded scrub
[ https://issues.apache.org/jira/browse/CASSANDRA-5547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-5547: - Attachment: cassandra-2.0-5547.txt Thought I would make an attempt at this. Made all three operations request a thread per file. I would be glad to make any changes/improvements requested. Example of multithreaded scrubbing {code} INFO [CompactionExecutor:29] 2014-03-22 14:29:30,783 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-38-Data.db') (18061938 bytes) INFO [CompactionExecutor:26] 2014-03-22 14:29:30,783 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-40-Data.db') (110006268 bytes) INFO [CompactionExecutor:28] 2014-03-22 14:29:30,783 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-32-Data.db') (99237382 bytes) INFO [CompactionExecutor:25] 2014-03-22 14:29:30,783 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-23-Data.db') (76191460 bytes) INFO [CompactionExecutor:32] 2014-03-22 14:29:30,783 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-27-Data.db') (189539010 bytes) INFO [CompactionExecutor:31] 2014-03-22 14:29:30,783 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-37-Data.db') (27822796 bytes) INFO [CompactionExecutor:30] 2014-03-22 14:29:30,783 OutputHandler.java (line 42) Scrubbing SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-39-Data.db') (29473004 bytes) INFO [CompactionExecutor:29] 2014-03-22 14:29:37,391 OutputHandler.java (line 42) Scrub of SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-38-Data.db') complete: 64971 rows in new sstable and 0 empty (tombstoned) rows dropped INFO [CompactionExecutor:31] 2014-03-22 14:29:40,882 OutputHandler.java (line 42) Scrub of SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-37-Data.db') complete: 100082 rows in new sstable and 0 empty (tombstoned) rows dropped INFO [CompactionExecutor:30] 2014-03-22 14:29:41,383 OutputHandler.java (line 42) Scrub of SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-39-Data.db') complete: 106018 rows in new sstable and 0 empty (tombstoned) rows dropped INFO [CompactionExecutor:25] 2014-03-22 14:29:52,562 OutputHandler.java (line 42) Scrub of SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-23-Data.db') complete: 274070 rows in new sstable and 0 empty (tombstoned) rows dropped INFO [CompactionExecutor:28] 2014-03-22 14:29:56,753 OutputHandler.java (line 42) Scrub of SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-32-Data.db') complete: 356969 rows in new sstable and 0 empty (tombstoned) rows dropped INFO [CompactionExecutor:26] 2014-03-22 14:29:58,021 OutputHandler.java (line 42) Scrub of SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-40-Data.db') complete: 395706 rows in new sstable and 0 empty (tombstoned) rows dropped INFO [CompactionExecutor:32] 2014-03-22 14:30:02,841 OutputHandler.java (line 42) Scrub of SSTableReader(path='/var/lib/cassandra/data/Keyspace1/Standard1/Keyspace1-Standard1-jb-27-Data.db') complete: 681795 rows in new sstable and 0 empty (tombstoned) rows dropped {code} Multi-threaded scrub Key: CASSANDRA-5547 URL: https://issues.apache.org/jira/browse/CASSANDRA-5547 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Benjamin Coverston Labels: lhf Attachments: cassandra-2.0-5547.txt Scrub (especially offline) could benefit from being multi-threaded, especially in the case where the SSTables are compressed. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-5547) Multi-threaded scrub
[ https://issues.apache.org/jira/browse/CASSANDRA-5547?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer reassigned CASSANDRA-5547: Assignee: Russell Alexander Spitzer Multi-threaded scrub Key: CASSANDRA-5547 URL: https://issues.apache.org/jira/browse/CASSANDRA-5547 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Benjamin Coverston Assignee: Russell Alexander Spitzer Labels: lhf Attachments: cassandra-2.0-5547.txt Scrub (especially offline) could benefit from being multi-threaded, especially in the case where the SSTables are compressed. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6451) Allow Cassandra-Stress to Set Compaction Strategy Options
[ https://issues.apache.org/jira/browse/CASSANDRA-6451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13901734#comment-13901734 ] Russell Alexander Spitzer commented on CASSANDRA-6451: -- Discussed over chat, LGTM +1 Allow Cassandra-Stress to Set Compaction Strategy Options - Key: CASSANDRA-6451 URL: https://issues.apache.org/jira/browse/CASSANDRA-6451 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Russell Alexander Spitzer Assignee: Russell Alexander Spitzer Priority: Minor Attachments: trunk-6451.txt I was just running some tests with Cassandra-Stress and discovered that I was unable to set the compaction_strategy_options I needed. I've made a small patch to add yet another parameter to stress allowing the user to set strategy_options. Usage like so: ./cassandra-stress -Z MyStrat -z option1=10,option2=5 -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6210) Repair hangs when a new datacenter is added to a cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-6210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13883665#comment-13883665 ] Russell Alexander Spitzer commented on CASSANDRA-6210: -- Newest patch looks good to me. Passes extended repair tests. Repair hangs when a new datacenter is added to a cluster Key: CASSANDRA-6210 URL: https://issues.apache.org/jira/browse/CASSANDRA-6210 Project: Cassandra Issue Type: Bug Components: Core Environment: Amazon Ec2 2 M1.large nodes Reporter: Russell Alexander Spitzer Assignee: Yuki Morishita Fix For: 2.0.5 Attachments: 6210-2.0.txt, RepairLogs.tar.gz, patch_1_logs.tar.gz Attempting to add a new datacenter to a cluster seems to cause repair operations to break. I've been reproducing this with 20~ node clusters but can get it to reliably occur on 2 node setups. {code} ##Basic Steps to reproduce #Node 1 is started using GossipingPropertyFileSnitch as dc1 #Cassandra-stress is used to insert a minimal amount of data $CASSANDRA_STRESS -t 100 -R org.apache.cassandra.locator.NetworkTopologyStrategy --num-keys=1000 --columns=10 --consistency-level=LOCAL_QUORUM --average-size-values - -compaction-strategy='LeveledCompactionStrategy' -O dc1:1 --operation=COUNTER_ADD #Alter Keyspace1 ALTER KEYSPACE Keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'dc1': 1 , 'dc2': 1 }; #Add node 2 using GossipingPropertyFileSnitch as dc2 run repair on node 1 run repair on node 2 {code} The repair task on node 1 never completes and while there are no exceptions in the logs of node1, netstat reports the following repair tasks {code} Mode: NORMAL Repair 4e71a250-36b4-11e3-bedc-1d1bb5c9abab Repair 6c64ded0-36b4-11e3-bedc-1d1bb5c9abab Read Repair Statistics: Attempted: 0 Mismatch (Blocking): 0 Mismatch (Background): 0 Pool NameActive Pending Completed Commandsn/a 0 10239 Responses n/a 0 3839 {code} Checking on node 2 we see the following exceptions {code} ERROR [STREAM-IN-/10.171.122.130] 2013-10-16 22:42:58,961 StreamSession.java (line 410) [Stream #4e71a250-36b4-11e3-bedc-1d1bb5c9abab] Streaming error occurred java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:174) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:724) ... ERROR [STREAM-IN-/10.171.122.130] 2013-10-16 22:43:49,214 StreamSession.java (line 410) [Stream #6c64ded0-36b4-11e3-bedc-1d1bb5c9abab] Streaming error occurred java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:174) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:724) {code} Netstats on node 2 reports {code} automaton@ip-10-171-15-234:~$ nodetool netstats Mode: NORMAL Repair 4e71a250-36b4-11e3-bedc-1d1bb5c9abab Read Repair Statistics: Attempted: 0 Mismatch (Blocking): 0 Mismatch (Background): 0 Pool NameActive Pending Completed Commandsn/a 0 2562 Responses n/a 0 4284 {code} -- This message was sent by Atlassian JIRA (v6.1.5#6160)
[jira] [Commented] (CASSANDRA-6210) Repair hangs when a new datacenter is added to a cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-6210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13875006#comment-13875006 ] Russell Alexander Spitzer commented on CASSANDRA-6210: -- Ran the test again last night and repair reported the following exceptions, I'll have the logs up in a moment. Setup: 4 Nodes, 2 per DC {code} ERROR [AntiEntropySessions:2] 2014-01-17 06:59:13,320 RepairSession.java (line 278) [repair #def293b0-7f44-11e3-b180-d1c68624042f] session completed with the following error org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1, (-4559856749309798061,-4559456353371206248]] Sync failed between /10.171.121.18 and /10.196.16.123 at org.apache.cassandra.repair.RepairSession.syncComplete(RepairSession.java:200) at org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:204) at org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:59) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:724) ERROR [AntiEntropySessions:2] 2014-01-17 06:59:13,325 CassandraDaemon.java (line 192) Exception in thread Thread[AntiEntropySessions:2,5,RMI Runtime] java.lang.RuntimeException: org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1, (-4559856749309798061,-4559456353371206248]] Sync failed between /10.171.121.18 and /10.196.16.123 at com.google.common.base.Throwables.propagate(Throwables.java:160) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334) at java.util.concurrent.FutureTask.run(FutureTask.java:166) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:724) Caused by: org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1, (-4559856749309798061,-4559456353371206248]] Sync failed between /10.171.121.18 and /10.196.16.123 at org.apache.cassandra.repair.RepairSession.syncComplete(RepairSession.java:200) at org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:204) at org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:59) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) ... 3 more INFO [AntiEntropySessions:4] 2014-01-17 06:59:13,328 RepairSession.java (line 236) [repair #df5d6370-7f44-11e3-b180-d1c68624042f] new session: will sync /10.171.121.18, /10.198.2.16 on range (-5516517151222322415,-5504449186624942606] for Keyspace1.[SuperCounter1, Super1, Counter3, Standard1, Counter1] {code] Repair hangs when a new datacenter is added to a cluster Key: CASSANDRA-6210 URL: https://issues.apache.org/jira/browse/CASSANDRA-6210 Project: Cassandra Issue Type: Bug Components: Core Environment: Amazon Ec2 2 M1.large nodes Reporter: Russell Alexander Spitzer Assignee: Yuki Morishita Fix For: 2.0.5 Attachments: 6210-2.0.txt, RepairLogs.tar.gz Attempting to add a new datacenter to a cluster seems to cause repair operations to break. I've been reproducing this with 20~ node clusters but can get it to reliably occur on 2 node setups. {code} ##Basic Steps to reproduce #Node 1 is started using GossipingPropertyFileSnitch as dc1 #Cassandra-stress is used to insert a minimal amount of data $CASSANDRA_STRESS -t 100 -R org.apache.cassandra.locator.NetworkTopologyStrategy --num-keys=1000 --columns=10 --consistency-level=LOCAL_QUORUM --average-size-values - -compaction-strategy='LeveledCompactionStrategy' -O dc1:1 --operation=COUNTER_ADD #Alter Keyspace1 ALTER KEYSPACE Keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'dc1': 1 , 'dc2': 1 }; #Add node 2 using GossipingPropertyFileSnitch as dc2 run repair on node 1 run repair on node 2 {code} The repair task on node 1 never completes and while there are no exceptions in the logs of node1, netstat reports the following repair tasks {code} Mode: NORMAL Repair
[jira] [Comment Edited] (CASSANDRA-6210) Repair hangs when a new datacenter is added to a cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-6210?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13875006#comment-13875006 ] Russell Alexander Spitzer edited comment on CASSANDRA-6210 at 1/17/14 5:51 PM: --- Ran the test again last night and repair reported the following exceptions, I'll have the logs up in a moment. Setup: 4 Nodes, 2 per DC {code} ERROR [AntiEntropySessions:2] 2014-01-17 06:59:13,320 RepairSession.java (line 278) [repair #def293b0-7f44-11e3-b180-d1c68624042f] session completed with the following error org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1, (-4559856749309798061,-4559456353371206248]] Sync failed between /10.171.121.18 and /10.196.16.123 at org.apache.cassandra.repair.RepairSession.syncComplete(RepairSession.java:200) at org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:204) at org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:59) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:724) ERROR [AntiEntropySessions:2] 2014-01-17 06:59:13,325 CassandraDaemon.java (line 192) Exception in thread Thread[AntiEntropySessions:2,5,RMI Runtime] java.lang.RuntimeException: org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1, (-4559856749309798061,-4559456353371206248]] Sync failed between /10.171.121.18 and /10.196.16.123 at com.google.common.base.Throwables.propagate(Throwables.java:160) at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:32) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334) at java.util.concurrent.FutureTask.run(FutureTask.java:166) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:724) Caused by: org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1, (-4559856749309798061,-4559456353371206248]] Sync failed between /10.171.121.18 and /10.196.16.123 at org.apache.cassandra.repair.RepairSession.syncComplete(RepairSession.java:200) at org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:204) at org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:59) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) ... 3 more INFO [AntiEntropySessions:4] 2014-01-17 06:59:13,328 RepairSession.java (line 236) [repair #df5d6370-7f44-11e3-b180-d1c68624042f] new session: will sync /10.171.121.18, /10.198.2.16 on range (-5516517151222322415,-5504449186624942606] for Keyspace1.[SuperCounter1, Super1, Counter3, Standard1, Counter1] {code} was (Author: rspitzer): Ran the test again last night and repair reported the following exceptions, I'll have the logs up in a moment. Setup: 4 Nodes, 2 per DC {code} ERROR [AntiEntropySessions:2] 2014-01-17 06:59:13,320 RepairSession.java (line 278) [repair #def293b0-7f44-11e3-b180-d1c68624042f] session completed with the following error org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1, (-4559856749309798061,-4559456353371206248]] Sync failed between /10.171.121.18 and /10.196.16.123 at org.apache.cassandra.repair.RepairSession.syncComplete(RepairSession.java:200) at org.apache.cassandra.service.ActiveRepairService.handleMessage(ActiveRepairService.java:204) at org.apache.cassandra.repair.RepairMessageVerbHandler.doVerb(RepairMessageVerbHandler.java:59) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:60) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:724) ERROR [AntiEntropySessions:2] 2014-01-17 06:59:13,325 CassandraDaemon.java (line 192) Exception in thread Thread[AntiEntropySessions:2,5,RMI Runtime] java.lang.RuntimeException: org.apache.cassandra.exceptions.RepairException: [repair #def293b0-7f44-11e3-b180-d1c68624042f on Keyspace1/Standard1,
[jira] [Updated] (CASSANDRA-6210) Repair hangs when a new datacenter is added to a cluster
[ https://issues.apache.org/jira/browse/CASSANDRA-6210?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-6210: - Attachment: patch_1_logs.tar.gz Logs from patched cluster running repair test Repair hangs when a new datacenter is added to a cluster Key: CASSANDRA-6210 URL: https://issues.apache.org/jira/browse/CASSANDRA-6210 Project: Cassandra Issue Type: Bug Components: Core Environment: Amazon Ec2 2 M1.large nodes Reporter: Russell Alexander Spitzer Assignee: Yuki Morishita Fix For: 2.0.5 Attachments: 6210-2.0.txt, RepairLogs.tar.gz, patch_1_logs.tar.gz Attempting to add a new datacenter to a cluster seems to cause repair operations to break. I've been reproducing this with 20~ node clusters but can get it to reliably occur on 2 node setups. {code} ##Basic Steps to reproduce #Node 1 is started using GossipingPropertyFileSnitch as dc1 #Cassandra-stress is used to insert a minimal amount of data $CASSANDRA_STRESS -t 100 -R org.apache.cassandra.locator.NetworkTopologyStrategy --num-keys=1000 --columns=10 --consistency-level=LOCAL_QUORUM --average-size-values - -compaction-strategy='LeveledCompactionStrategy' -O dc1:1 --operation=COUNTER_ADD #Alter Keyspace1 ALTER KEYSPACE Keyspace1 WITH replication = {'class': 'NetworkTopologyStrategy', 'dc1': 1 , 'dc2': 1 }; #Add node 2 using GossipingPropertyFileSnitch as dc2 run repair on node 1 run repair on node 2 {code} The repair task on node 1 never completes and while there are no exceptions in the logs of node1, netstat reports the following repair tasks {code} Mode: NORMAL Repair 4e71a250-36b4-11e3-bedc-1d1bb5c9abab Repair 6c64ded0-36b4-11e3-bedc-1d1bb5c9abab Read Repair Statistics: Attempted: 0 Mismatch (Blocking): 0 Mismatch (Background): 0 Pool NameActive Pending Completed Commandsn/a 0 10239 Responses n/a 0 3839 {code} Checking on node 2 we see the following exceptions {code} ERROR [STREAM-IN-/10.171.122.130] 2013-10-16 22:42:58,961 StreamSession.java (line 410) [Stream #4e71a250-36b4-11e3-bedc-1d1bb5c9abab] Streaming error occurred java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:174) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:724) ... ERROR [STREAM-IN-/10.171.122.130] 2013-10-16 22:43:49,214 StreamSession.java (line 410) [Stream #6c64ded0-36b4-11e3-bedc-1d1bb5c9abab] Streaming error occurred java.lang.NullPointerException at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:174) at org.apache.cassandra.streaming.StreamSession.prepare(StreamSession.java:436) at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:358) at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:293) at java.lang.Thread.run(Thread.java:724) {code} Netstats on node 2 reports {code} automaton@ip-10-171-15-234:~$ nodetool netstats Mode: NORMAL Repair 4e71a250-36b4-11e3-bedc-1d1bb5c9abab Read Repair Statistics: Attempted: 0 Mismatch (Blocking): 0 Mismatch (Background): 0 Pool NameActive Pending Completed Commandsn/a 0 2562 Responses n/a 0 4284 {code} -- This message was sent by Atlassian JIRA (v6.1.5#6160)