[jira] [Commented] (CASSANDRA-6234) Add metrics for native protocols
[ https://issues.apache.org/jira/browse/CASSANDRA-6234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805092#comment-13805092 ] Sylvain Lebresne commented on CASSANDRA-6234: - Not completely. The goal here is more to expose metrics on the inner working of the native server to help tuning it. Long story short, we want to at least expose metrics on the thread pool used to have some feedback on whether native_transport_max_thread needs to be bumped (and the number of thread in use at any given time is not related to the number of connected client). But I'll also check if there other things that would make sense to expose. Add metrics for native protocols Key: CASSANDRA-6234 URL: https://issues.apache.org/jira/browse/CASSANDRA-6234 Project: Cassandra Issue Type: New Feature Reporter: Adam Hattrell Assignee: Sylvain Lebresne It would be very useful to expose metrics related to the native protocol. Initially I have a user that would like to be able to monitor the usage of native transport threads. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-5201) Cassandra/Hadoop does not support current Hadoop releases
[ https://issues.apache.org/jira/browse/CASSANDRA-5201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805107#comment-13805107 ] Mck SembWever commented on CASSANDRA-5201: -- Hadoop-2 only just came out of alpha/beta with hadoop-2.2.0 Cassandra/Hadoop does not support current Hadoop releases - Key: CASSANDRA-5201 URL: https://issues.apache.org/jira/browse/CASSANDRA-5201 Project: Cassandra Issue Type: Bug Components: Hadoop Affects Versions: 1.2.0 Reporter: Brian Jeltema Assignee: Dave Brosius Attachments: 5201_a.txt Using Hadoop 0.22.0 with Cassandra results in the stack trace below. It appears that version 0.21+ changed org.apache.hadoop.mapreduce.JobContext from a class to an interface. Exception in thread main java.lang.IncompatibleClassChangeError: Found interface org.apache.hadoop.mapreduce.JobContext, but class was expected at org.apache.cassandra.hadoop.ColumnFamilyInputFormat.getSplits(ColumnFamilyInputFormat.java:103) at org.apache.hadoop.mapreduce.JobSubmitter.writeNewSplits(JobSubmitter.java:445) at org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:462) at org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:357) at org.apache.hadoop.mapreduce.Job$2.run(Job.java:1045) at org.apache.hadoop.mapreduce.Job$2.run(Job.java:1042) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1153) at org.apache.hadoop.mapreduce.Job.submit(Job.java:1042) at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:1062) at MyHadoopApp.run(MyHadoopApp.java:163) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:69) at MyHadoopApp.main(MyHadoopApp.java:82) 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:601) at org.apache.hadoop.util.RunJar.main(RunJar.java:192) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6238) LOCAL_ONE doesn't work for SimpleStrategy
[ https://issues.apache.org/jira/browse/CASSANDRA-6238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805129#comment-13805129 ] Alex Liu commented on CASSANDRA-6238: - I understand it comes as the same way as original LOCAL_QUORUM. SimpleStrategy is for a cluster without multiple DCs. NTS is for multiple DC cluster. SimpleStrategy could be treated as a special NTS that is only for a one DC cluster. The default CL of Cassandra Hadoop integration is LOCAL_ONE now, it should work for all types of strategy. LOCAL_ONE looks like a better candidate as a default CL than ONE. When people first start using C*, it is common to be a one DC cluster, then later it may grow up to a multiple DC cluster. If LOCAL_ONE work for all types of strategy, manual change the default CL is not needed. If we keep LOCAL_* requiring NTS, we has to update all the hadoop/pig examples and tests to use ONE. The main reason to make LOCAL_* work for all types of strategy is that LOCAL_* are better candidates as default CLs. The default CL should work for all types of strategy. LOCAL_ONE doesn't work for SimpleStrategy - Key: CASSANDRA-6238 URL: https://issues.apache.org/jira/browse/CASSANDRA-6238 Project: Cassandra Issue Type: Bug Components: Core Reporter: Alex Liu Assignee: Alex Liu Fix For: 1.2.12, 2.0.3 Attachments: 6238.txt, 6238-v2.txt LOCAL_ONE only works for NetworkTopologyStrategy which has DC specification. Any other strategy fails. If there is no DC specified in the strategy, we should treat LOCAL_ONE as ONE -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-5201) Cassandra/Hadoop does not support current Hadoop releases
[ https://issues.apache.org/jira/browse/CASSANDRA-5201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805165#comment-13805165 ] Nyo commented on CASSANDRA-5201: Why dropping support for old hadoop versions while you can just use a different package name (e.g org.apache.cassandra.hadoop2.SomeClass)? Imo, the code in here: https://github.com/michaelsembwever/cassandra-hadoop/ can be almost immediately used with hadoop 2.x. Then everybody would be happy. Or you can indeed make it a lot more complex by merging the two implementations and deciding which one to use based on metadata obtained by polling the cluster, but that is a too sophisticated option. For most people the first option would be sufficient. Cassandra/Hadoop does not support current Hadoop releases - Key: CASSANDRA-5201 URL: https://issues.apache.org/jira/browse/CASSANDRA-5201 Project: Cassandra Issue Type: Bug Components: Hadoop Affects Versions: 1.2.0 Reporter: Brian Jeltema Assignee: Dave Brosius Attachments: 5201_a.txt Using Hadoop 0.22.0 with Cassandra results in the stack trace below. It appears that version 0.21+ changed org.apache.hadoop.mapreduce.JobContext from a class to an interface. Exception in thread main java.lang.IncompatibleClassChangeError: Found interface org.apache.hadoop.mapreduce.JobContext, but class was expected at org.apache.cassandra.hadoop.ColumnFamilyInputFormat.getSplits(ColumnFamilyInputFormat.java:103) at org.apache.hadoop.mapreduce.JobSubmitter.writeNewSplits(JobSubmitter.java:445) at org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:462) at org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:357) at org.apache.hadoop.mapreduce.Job$2.run(Job.java:1045) at org.apache.hadoop.mapreduce.Job$2.run(Job.java:1042) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1153) at org.apache.hadoop.mapreduce.Job.submit(Job.java:1042) at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:1062) at MyHadoopApp.run(MyHadoopApp.java:163) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:69) at MyHadoopApp.main(MyHadoopApp.java:82) 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:601) at org.apache.hadoop.util.RunJar.main(RunJar.java:192) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Comment Edited] (CASSANDRA-5201) Cassandra/Hadoop does not support current Hadoop releases
[ https://issues.apache.org/jira/browse/CASSANDRA-5201?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805165#comment-13805165 ] Nyo edited comment on CASSANDRA-5201 at 10/25/13 8:37 AM: -- Why dropping support for old hadoop versions while you can just use a different package name (e.g org.apache.cassandra.hadoop2.SomeClass)? Imo, the code in here: https://github.com/michaelsembwever/cassandra-hadoop/ should just be merged with mainstream implementation, so both 1.x and 2.x users can be happy. Or you can indeed make it a lot more complex by merging the 1.x and 2.x compatible implementations and deciding which one to use based on metadata obtained by polling the cluster, but that is a bit too sophisticated. For most people the first option, where they can choose which package to use, would be sufficient. was (Author: nyo): Why dropping support for old hadoop versions while you can just use a different package name (e.g org.apache.cassandra.hadoop2.SomeClass)? Imo, the code in here: https://github.com/michaelsembwever/cassandra-hadoop/ can be almost immediately used with hadoop 2.x. Then everybody would be happy. Or you can indeed make it a lot more complex by merging the two implementations and deciding which one to use based on metadata obtained by polling the cluster, but that is a too sophisticated option. For most people the first option would be sufficient. Cassandra/Hadoop does not support current Hadoop releases - Key: CASSANDRA-5201 URL: https://issues.apache.org/jira/browse/CASSANDRA-5201 Project: Cassandra Issue Type: Bug Components: Hadoop Affects Versions: 1.2.0 Reporter: Brian Jeltema Assignee: Dave Brosius Attachments: 5201_a.txt Using Hadoop 0.22.0 with Cassandra results in the stack trace below. It appears that version 0.21+ changed org.apache.hadoop.mapreduce.JobContext from a class to an interface. Exception in thread main java.lang.IncompatibleClassChangeError: Found interface org.apache.hadoop.mapreduce.JobContext, but class was expected at org.apache.cassandra.hadoop.ColumnFamilyInputFormat.getSplits(ColumnFamilyInputFormat.java:103) at org.apache.hadoop.mapreduce.JobSubmitter.writeNewSplits(JobSubmitter.java:445) at org.apache.hadoop.mapreduce.JobSubmitter.writeSplits(JobSubmitter.java:462) at org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:357) at org.apache.hadoop.mapreduce.Job$2.run(Job.java:1045) at org.apache.hadoop.mapreduce.Job$2.run(Job.java:1042) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1153) at org.apache.hadoop.mapreduce.Job.submit(Job.java:1042) at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:1062) at MyHadoopApp.run(MyHadoopApp.java:163) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:69) at MyHadoopApp.main(MyHadoopApp.java:82) 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:601) at org.apache.hadoop.util.RunJar.main(RunJar.java:192) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Created] (CASSANDRA-6241) Assertion on MmappedSegmentedFile.floor doesn't tell us the path (filename)
Jackson Chung created CASSANDRA-6241: Summary: Assertion on MmappedSegmentedFile.floor doesn't tell us the path (filename) Key: CASSANDRA-6241 URL: https://issues.apache.org/jira/browse/CASSANDRA-6241 Project: Cassandra Issue Type: Bug Reporter: Jackson Chung for whatever reason (hardware failure, excess load, etc), we get this: {panel} ERROR [MutationStage:10] 2013-10-25 08:54:03,150 AbstractCassandraDaemon.java (line 132) Exception in thread Thread[MutationStage:10,5,main] java.lang.AssertionError: 1711300 vs 974637 at org.apache.cassandra.io.util.MmappedSegmentedFile.floor(MmappedSegmentedFile.java:62) at org.apache.cassandra.io.util.MmappedSegmentedFile.getSegment(MmappedSegmentedFile.java:77) at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:900) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.init(SSTableNamesIterator.java:63) at org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:61) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:124) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1362) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1224) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1154) at org.apache.cassandra.db.Table.readCurrentIndexedColumns(Table.java:514) at org.apache.cassandra.db.Table.apply(Table.java:452) at org.apache.cassandra.db.Table.apply(Table.java:384) at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:294) at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:51) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {panel} But the assertion error doesn't tell us the SSTable that is having the problem. So it doesn't really help us. I think we can simply append the assert error log msg with the this.path to show the filename of the problematic file. I would also suggest make 1711300 vs 974637 more clear... -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6135) Add beforeChange Notification to Gossiper State.
[ https://issues.apache.org/jira/browse/CASSANDRA-6135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805204#comment-13805204 ] Sergio Bossa commented on CASSANDRA-6135: - Agreed, V2 (03/Oct/13 12:22) is better imho. Add beforeChange Notification to Gossiper State. Key: CASSANDRA-6135 URL: https://issues.apache.org/jira/browse/CASSANDRA-6135 Project: Cassandra Issue Type: New Feature Reporter: Benjamin Coverston Assignee: Sergio Bossa Attachments: 0001-New-Gossiper-notification-to-IEndpointStateChangeSub.patch, 0002-CASSANDRA-6135.diff, CASSANDRA-6135-V3.patch We would like an internal notification to be fired before state changes happen so we can intercept them, and in some cases defer them. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805267#comment-13805267 ] Chris Burroughs commented on CASSANDRA-6127: It would be helpful to dump the interval times for a node that is flapping (dumpInterArrivalTimes on the FD) so we can see how long the heartbeats are taking. A per endpoint histogram of heartbeat arrival latency seems a worthwhile o.a.c.Metric to have all the time. [~qconner] On the topic of wait until there is enough data before doing stuff you might also be interested in the heuristic report from CASSANDRA-4288 vnodes don't scale to hundreds of nodes --- Key: CASSANDRA-6127 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127 Project: Cassandra Issue Type: Bug Components: Core Environment: Any cluster that has vnodes and consists of hundreds of physical nodes. Reporter: Tupshin Harper Assignee: Jonathan Ellis Attachments: 6000vnodes.patch, AdjustableGossipPeriod.patch, delayEstimatorUntilStatisticallyValid.patch There are a lot of gossip-related issues related to very wide clusters that also have vnodes enabled. Let's use this ticket as a master in case there are sub-tickets. The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge instances. Each node configured with 32 vnodes. Without vnodes, cluster spins up fine and is ready to handle requests within 30 minutes or less. With vnodes, nodes are reporting constant up/down flapping messages with no external load on the cluster. After a couple of hours, they were still flapping, had very high cpu load, and the cluster never looked like it was going to stabilize or be useful for traffic. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-5936) Improve the way we pick L0 compaction candidates
[ https://issues.apache.org/jira/browse/CASSANDRA-5936?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805352#comment-13805352 ] Jonathan Ellis commented on CASSANDRA-5936: --- So basically, it promotes to the first level L where L+1 has overlapping data with it, or where L+2 has more than 10*sstable size bytes overlapping with it. There's also a special case that if there are other L0 sstables overlapping it, it stays in L0. I don't think that last part fits our design (remember that LDB stops writes when L0 is full). I'm also curious why this logic is only? run for L0 compactions. But it does sound like a good improvement. Improve the way we pick L0 compaction candidates Key: CASSANDRA-5936 URL: https://issues.apache.org/jira/browse/CASSANDRA-5936 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Marcus Eriksson Assignee: Marcus Eriksson Fix For: 2.1 We could improve the way we pick compaction candidates in level 0 in LCS. The most common way for us to get behind on compaction is after repairs, we should exploit the fact that the streamed sstables are most often very narrow in range since the other nodes in the ring will have a similar sstable-range-distribution. We should in theory be able to do 10 concurrent compactions involving L1 - ie, partition L0 in buckets defined by the sstables in L1 to only keep one L1 SSTable busy for every compaction (be it L1 to L2 or L0 to L1). we will need some heuristics on when to select candidates from the buckets and when to do it the old way (since L0 sstables can span several L1 sstables) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Resolved] (CASSANDRA-6241) Assertion on MmappedSegmentedFile.floor doesn't tell us the path (filename)
[ https://issues.apache.org/jira/browse/CASSANDRA-6241?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-6241. --- Resolution: Fixed Fix Version/s: 2.0.3 1.2.12 Assignee: Jonathan Ellis added in dcbf35c552a53e7779d7da917cbce78ecf109021 Assertion on MmappedSegmentedFile.floor doesn't tell us the path (filename) --- Key: CASSANDRA-6241 URL: https://issues.apache.org/jira/browse/CASSANDRA-6241 Project: Cassandra Issue Type: Improvement Reporter: Jackson Chung Assignee: Jonathan Ellis Priority: Trivial Fix For: 1.2.12, 2.0.3 for whatever reason (hardware failure, excess load, etc), we get this: {panel} ERROR [MutationStage:10] 2013-10-25 08:54:03,150 AbstractCassandraDaemon.java (line 132) Exception in thread Thread[MutationStage:10,5,main] java.lang.AssertionError: 1711300 vs 974637 at org.apache.cassandra.io.util.MmappedSegmentedFile.floor(MmappedSegmentedFile.java:62) at org.apache.cassandra.io.util.MmappedSegmentedFile.getSegment(MmappedSegmentedFile.java:77) at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:900) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.init(SSTableNamesIterator.java:63) at org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:61) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:124) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1362) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1224) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1154) at org.apache.cassandra.db.Table.readCurrentIndexedColumns(Table.java:514) at org.apache.cassandra.db.Table.apply(Table.java:452) at org.apache.cassandra.db.Table.apply(Table.java:384) at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:294) at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:51) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {panel} But the assertion error doesn't tell us the SSTable that is having the problem. So it doesn't really help us. I think we can simply append the assert error log msg with the this.path to show the filename of the problematic file. I would also suggest make 1711300 vs 974637 more clear... -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6241) Assertion on MmappedSegmentedFile.floor doesn't tell us the path (filename)
[ https://issues.apache.org/jira/browse/CASSANDRA-6241?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-6241: -- Priority: Trivial (was: Major) Issue Type: Improvement (was: Bug) Assertion on MmappedSegmentedFile.floor doesn't tell us the path (filename) --- Key: CASSANDRA-6241 URL: https://issues.apache.org/jira/browse/CASSANDRA-6241 Project: Cassandra Issue Type: Improvement Reporter: Jackson Chung Priority: Trivial Fix For: 1.2.12, 2.0.3 for whatever reason (hardware failure, excess load, etc), we get this: {panel} ERROR [MutationStage:10] 2013-10-25 08:54:03,150 AbstractCassandraDaemon.java (line 132) Exception in thread Thread[MutationStage:10,5,main] java.lang.AssertionError: 1711300 vs 974637 at org.apache.cassandra.io.util.MmappedSegmentedFile.floor(MmappedSegmentedFile.java:62) at org.apache.cassandra.io.util.MmappedSegmentedFile.getSegment(MmappedSegmentedFile.java:77) at org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:900) at org.apache.cassandra.db.columniterator.SSTableNamesIterator.init(SSTableNamesIterator.java:63) at org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:61) at org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:79) at org.apache.cassandra.db.CollationController.collectTimeOrderedData(CollationController.java:124) at org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:64) at org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1362) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1224) at org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1154) at org.apache.cassandra.db.Table.readCurrentIndexedColumns(Table.java:514) at org.apache.cassandra.db.Table.apply(Table.java:452) at org.apache.cassandra.db.Table.apply(Table.java:384) at org.apache.cassandra.db.RowMutation.apply(RowMutation.java:294) at org.apache.cassandra.db.RowMutationVerbHandler.doVerb(RowMutationVerbHandler.java:51) at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:59) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:895) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:918) at java.lang.Thread.run(Thread.java:662) {panel} But the assertion error doesn't tell us the SSTable that is having the problem. So it doesn't really help us. I think we can simply append the assert error log msg with the this.path to show the filename of the problematic file. I would also suggest make 1711300 vs 974637 more clear... -- This message was sent by Atlassian JIRA (v6.1#6144)
[2/6] git commit: add path to MmappedSegmentedFile assertions
add path to MmappedSegmentedFile assertions Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/dcbf35c5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/dcbf35c5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/dcbf35c5 Branch: refs/heads/cassandra-2.0 Commit: dcbf35c552a53e7779d7da917cbce78ecf109021 Parents: 08a2272 Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 25 09:47:00 2013 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 25 09:47:00 2013 -0500 -- src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/dcbf35c5/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java -- diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java index cb812bc..eeaf46e 100644 --- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java +++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java @@ -54,10 +54,10 @@ public class MmappedSegmentedFile extends SegmentedFile */ private Segment floor(long position) { -assert 0 = position position length: position + vs + length; +assert 0 = position position length: String.format(%d = %d in %s, position, length, path); Segment seg = new Segment(position, null); int idx = Arrays.binarySearch(segments, seg); -assert idx != -1 : Bad position + position + in segments + Arrays.toString(segments); +assert idx != -1 : String.format(Bad position %d for segments %s in %s, position, Arrays.toString(segments), path); if (idx 0) // round down to entry at insertion point idx = -(idx + 2);
[6/6] git commit: Merge branch 'cassandra-2.0' into trunk
Merge branch 'cassandra-2.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8d1bb075 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8d1bb075 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8d1bb075 Branch: refs/heads/trunk Commit: 8d1bb07530390128c1ec5d48daf438380a7cce8e Parents: 2faac19 9385cfd Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 25 09:47:21 2013 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 25 09:47:21 2013 -0500 -- src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) --
[4/6] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/9385cfde Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/9385cfde Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/9385cfde Branch: refs/heads/trunk Commit: 9385cfded7fb6cbfdac65d195a178a754e1b4514 Parents: 3dc4e65 dcbf35c Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 25 09:47:12 2013 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 25 09:47:12 2013 -0500 -- src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/9385cfde/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java --
[5/6] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/9385cfde Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/9385cfde Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/9385cfde Branch: refs/heads/cassandra-2.0 Commit: 9385cfded7fb6cbfdac65d195a178a754e1b4514 Parents: 3dc4e65 dcbf35c Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 25 09:47:12 2013 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 25 09:47:12 2013 -0500 -- src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/9385cfde/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java --
[3/6] git commit: add path to MmappedSegmentedFile assertions
add path to MmappedSegmentedFile assertions Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/dcbf35c5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/dcbf35c5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/dcbf35c5 Branch: refs/heads/trunk Commit: dcbf35c552a53e7779d7da917cbce78ecf109021 Parents: 08a2272 Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 25 09:47:00 2013 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 25 09:47:00 2013 -0500 -- src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/dcbf35c5/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java -- diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java index cb812bc..eeaf46e 100644 --- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java +++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java @@ -54,10 +54,10 @@ public class MmappedSegmentedFile extends SegmentedFile */ private Segment floor(long position) { -assert 0 = position position length: position + vs + length; +assert 0 = position position length: String.format(%d = %d in %s, position, length, path); Segment seg = new Segment(position, null); int idx = Arrays.binarySearch(segments, seg); -assert idx != -1 : Bad position + position + in segments + Arrays.toString(segments); +assert idx != -1 : String.format(Bad position %d for segments %s in %s, position, Arrays.toString(segments), path); if (idx 0) // round down to entry at insertion point idx = -(idx + 2);
[1/6] git commit: add path to MmappedSegmentedFile assertions
Updated Branches: refs/heads/cassandra-1.2 08a22729a - dcbf35c55 refs/heads/cassandra-2.0 3dc4e6563 - 9385cfded refs/heads/trunk 2faac19e1 - 8d1bb0753 add path to MmappedSegmentedFile assertions Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/dcbf35c5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/dcbf35c5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/dcbf35c5 Branch: refs/heads/cassandra-1.2 Commit: dcbf35c552a53e7779d7da917cbce78ecf109021 Parents: 08a2272 Author: Jonathan Ellis jbel...@apache.org Authored: Fri Oct 25 09:47:00 2013 -0500 Committer: Jonathan Ellis jbel...@apache.org Committed: Fri Oct 25 09:47:00 2013 -0500 -- src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/dcbf35c5/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java -- diff --git a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java index cb812bc..eeaf46e 100644 --- a/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java +++ b/src/java/org/apache/cassandra/io/util/MmappedSegmentedFile.java @@ -54,10 +54,10 @@ public class MmappedSegmentedFile extends SegmentedFile */ private Segment floor(long position) { -assert 0 = position position length: position + vs + length; +assert 0 = position position length: String.format(%d = %d in %s, position, length, path); Segment seg = new Segment(position, null); int idx = Arrays.binarySearch(segments, seg); -assert idx != -1 : Bad position + position + in segments + Arrays.toString(segments); +assert idx != -1 : String.format(Bad position %d for segments %s in %s, position, Arrays.toString(segments), path); if (idx 0) // round down to entry at insertion point idx = -(idx + 2);
[jira] [Commented] (CASSANDRA-6109) Consider coldness in STCS compaction
[ https://issues.apache.org/jira/browse/CASSANDRA-6109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805387#comment-13805387 ] Tyler Hobbs commented on CASSANDRA-6109: bq. When they make up more than X% do we stop discriminating or merge them only with other cold sstables? I was thinking we would stop discriminating. The logic would basically be this: {noformat} total_reads = sum(sstable.reads_per_sec for sstable in sstables) total_cold_reads = 0 cold_sstables = set() for sstable in sorted(sstables, key=lambda sstable: sstable.reads_per_key_per_sec): if (sstable.reads_per_sec + total_cold_reads) / total_reads configurable_threshold: cold_sstables.add(sstable) total_cold_reads += sstable.reads_per_sec else: break getBuckets(sstable for sstable in sstables if sstable not in cold_sstables) {noformat} Consider coldness in STCS compaction Key: CASSANDRA-6109 URL: https://issues.apache.org/jira/browse/CASSANDRA-6109 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Tyler Hobbs Fix For: 2.0.2 Attachments: 6109-v1.patch, 6109-v2.patch I see two options: # Don't compact cold sstables at all # Compact cold sstables only if there is nothing more important to compact The latter is better if you have cold data that may become hot again... but it's confusing if you have a workload such that you can't keep up with *all* compaction, but you can keep up with hot sstable. (Compaction backlog stat becomes useless since we fall increasingly behind.) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6109) Consider coldness in STCS compaction
[ https://issues.apache.org/jira/browse/CASSANDRA-6109?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805394#comment-13805394 ] Jonathan Ellis commented on CASSANDRA-6109: --- Makes sense. Consider coldness in STCS compaction Key: CASSANDRA-6109 URL: https://issues.apache.org/jira/browse/CASSANDRA-6109 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Tyler Hobbs Fix For: 2.0.2 Attachments: 6109-v1.patch, 6109-v2.patch I see two options: # Don't compact cold sstables at all # Compact cold sstables only if there is nothing more important to compact The latter is better if you have cold data that may become hot again... but it's confusing if you have a workload such that you can't keep up with *all* compaction, but you can keep up with hot sstable. (Compaction backlog stat becomes useless since we fall increasingly behind.) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6238) LOCAL_ONE doesn't work for SimpleStrategy
[ https://issues.apache.org/jira/browse/CASSANDRA-6238?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805397#comment-13805397 ] Brandon Williams commented on CASSANDRA-6238: - I agree that having LOCAL_ONE be the hadoop default is advantageous, and having hadoop not work out of the box for those that just want to play with it (using SS) isn't very nice. LOCAL_ONE doesn't work for SimpleStrategy - Key: CASSANDRA-6238 URL: https://issues.apache.org/jira/browse/CASSANDRA-6238 Project: Cassandra Issue Type: Bug Components: Core Reporter: Alex Liu Assignee: Alex Liu Fix For: 1.2.12, 2.0.3 Attachments: 6238.txt, 6238-v2.txt LOCAL_ONE only works for NetworkTopologyStrategy which has DC specification. Any other strategy fails. If there is no DC specified in the strategy, we should treat LOCAL_ONE as ONE -- This message was sent by Atlassian JIRA (v6.1#6144)
[5/6] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Conflicts: src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2f248b3f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2f248b3f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2f248b3f Branch: refs/heads/cassandra-2.0 Commit: 2f248b3f905380347cf45652462d1fc5989ee8d0 Parents: 9385cfd cb506b7 Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 11:11:31 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 11:11:31 2013 -0500 -- .../cassandra/hadoop/pig/AbstractCassandraStorage.java | 4 +++- .../org/apache/cassandra/hadoop/pig/CassandraStorage.java| 6 +- src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java | 8 3 files changed, 16 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f248b3f/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f248b3f/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f248b3f/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java --
[jira] [Commented] (CASSANDRA-6083) Pig requires explicit cast from int to long to save to Cassandra
[ https://issues.apache.org/jira/browse/CASSANDRA-6083?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805416#comment-13805416 ] Chad Johnston commented on CASSANDRA-6083: -- I am unable to reproduce this in 1.2.10 or 1.2.11. I don't know what I ran into before, but I can't make it happen again. Pig requires explicit cast from int to long to save to Cassandra Key: CASSANDRA-6083 URL: https://issues.apache.org/jira/browse/CASSANDRA-6083 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Chad Johnston Assignee: Alex Liu Priority: Minor Since version 1.2.10, I have to manually cast any int values in Pig to long in order to store them into bigint Cassandra columns. I did not have to perform this cast in previous versions of Cassandra. -- This message was sent by Atlassian JIRA (v6.1#6144)
[2/6] git commit: Handle null correctly in CqlStorage Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6180
Handle null correctly in CqlStorage Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6180 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cb506b79 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cb506b79 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cb506b79 Branch: refs/heads/cassandra-2.0 Commit: cb506b793ced3563e5671f0ad5b92c4d8655d45d Parents: dcbf35c Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 11:10:09 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 11:10:09 2013 -0500 -- .../cassandra/hadoop/pig/AbstractCassandraStorage.java | 4 +++- .../org/apache/cassandra/hadoop/pig/CassandraStorage.java| 5 + src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java | 8 3 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java index a02e309..98b3a6c 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java @@ -403,11 +403,13 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store // we don't care about types, they all get casted to ByteBuffers } +protected abstract ByteBuffer nullToBB(); + /** convert object to ByteBuffer */ protected ByteBuffer objToBB(Object o) { if (o == null) -return (ByteBuffer)o; +return nullToBB(); if (o instanceof java.lang.String) return ByteBuffer.wrap(new DataByteArray((String)o).get()); if (o instanceof Integer) http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java index 0512a37..853a052 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java @@ -824,5 +824,10 @@ public class CassandraStorage extends AbstractCassandraStorage [init_address=host][rpc_port=port]]': + e.getMessage()); } } + +public ByteBuffer nullToBB() +{ +return (ByteBuffer) null; +} } http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java index 0d01383..7b31565 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java @@ -710,5 +710,13 @@ public class CqlStorage extends AbstractCassandraStorage return validator.getString(validator.decompose(obj)); return obj; } + +/** + * Thrift API can't handle null, so use empty byte array + */ +public ByteBuffer nullToBB() +{ +return ByteBuffer.wrap(new byte[0]); +} }
[3/6] git commit: Handle null correctly in CqlStorage Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6180
Handle null correctly in CqlStorage Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6180 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cb506b79 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cb506b79 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cb506b79 Branch: refs/heads/trunk Commit: cb506b793ced3563e5671f0ad5b92c4d8655d45d Parents: dcbf35c Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 11:10:09 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 11:10:09 2013 -0500 -- .../cassandra/hadoop/pig/AbstractCassandraStorage.java | 4 +++- .../org/apache/cassandra/hadoop/pig/CassandraStorage.java| 5 + src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java | 8 3 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java index a02e309..98b3a6c 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java @@ -403,11 +403,13 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store // we don't care about types, they all get casted to ByteBuffers } +protected abstract ByteBuffer nullToBB(); + /** convert object to ByteBuffer */ protected ByteBuffer objToBB(Object o) { if (o == null) -return (ByteBuffer)o; +return nullToBB(); if (o instanceof java.lang.String) return ByteBuffer.wrap(new DataByteArray((String)o).get()); if (o instanceof Integer) http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java index 0512a37..853a052 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java @@ -824,5 +824,10 @@ public class CassandraStorage extends AbstractCassandraStorage [init_address=host][rpc_port=port]]': + e.getMessage()); } } + +public ByteBuffer nullToBB() +{ +return (ByteBuffer) null; +} } http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java index 0d01383..7b31565 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java @@ -710,5 +710,13 @@ public class CqlStorage extends AbstractCassandraStorage return validator.getString(validator.decompose(obj)); return obj; } + +/** + * Thrift API can't handle null, so use empty byte array + */ +public ByteBuffer nullToBB() +{ +return ByteBuffer.wrap(new byte[0]); +} }
[1/6] git commit: Handle null correctly in CqlStorage Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6180
Updated Branches: refs/heads/cassandra-1.2 dcbf35c55 - cb506b793 refs/heads/cassandra-2.0 9385cfded - 2f248b3f9 refs/heads/trunk 8d1bb0753 - 0dccea4f3 Handle null correctly in CqlStorage Patch by Alex Liu, reviewed by brandonwilliams for CASSANDRA-6180 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cb506b79 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cb506b79 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cb506b79 Branch: refs/heads/cassandra-1.2 Commit: cb506b793ced3563e5671f0ad5b92c4d8655d45d Parents: dcbf35c Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 11:10:09 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 11:10:09 2013 -0500 -- .../cassandra/hadoop/pig/AbstractCassandraStorage.java | 4 +++- .../org/apache/cassandra/hadoop/pig/CassandraStorage.java| 5 + src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java | 8 3 files changed, 16 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java index a02e309..98b3a6c 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java @@ -403,11 +403,13 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store // we don't care about types, they all get casted to ByteBuffers } +protected abstract ByteBuffer nullToBB(); + /** convert object to ByteBuffer */ protected ByteBuffer objToBB(Object o) { if (o == null) -return (ByteBuffer)o; +return nullToBB(); if (o instanceof java.lang.String) return ByteBuffer.wrap(new DataByteArray((String)o).get()); if (o instanceof Integer) http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java index 0512a37..853a052 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java @@ -824,5 +824,10 @@ public class CassandraStorage extends AbstractCassandraStorage [init_address=host][rpc_port=port]]': + e.getMessage()); } } + +public ByteBuffer nullToBB() +{ +return (ByteBuffer) null; +} } http://git-wip-us.apache.org/repos/asf/cassandra/blob/cb506b79/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java -- diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java index 0d01383..7b31565 100644 --- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java +++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java @@ -710,5 +710,13 @@ public class CqlStorage extends AbstractCassandraStorage return validator.getString(validator.decompose(obj)); return obj; } + +/** + * Thrift API can't handle null, so use empty byte array + */ +public ByteBuffer nullToBB() +{ +return ByteBuffer.wrap(new byte[0]); +} }
[4/6] git commit: Merge branch 'cassandra-1.2' into cassandra-2.0
Merge branch 'cassandra-1.2' into cassandra-2.0 Conflicts: src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2f248b3f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2f248b3f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2f248b3f Branch: refs/heads/trunk Commit: 2f248b3f905380347cf45652462d1fc5989ee8d0 Parents: 9385cfd cb506b7 Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 11:11:31 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 11:11:31 2013 -0500 -- .../cassandra/hadoop/pig/AbstractCassandraStorage.java | 4 +++- .../org/apache/cassandra/hadoop/pig/CassandraStorage.java| 6 +- src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java | 8 3 files changed, 16 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f248b3f/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f248b3f/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2f248b3f/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java --
[6/6] git commit: Merge branch 'cassandra-2.0' into trunk
Merge branch 'cassandra-2.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0dccea4f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0dccea4f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0dccea4f Branch: refs/heads/trunk Commit: 0dccea4f3600ea29bf4e146ed2c2a49b615187c3 Parents: 8d1bb07 2f248b3 Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 11:11:41 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 11:11:41 2013 -0500 -- .../cassandra/hadoop/pig/AbstractCassandraStorage.java | 4 +++- .../org/apache/cassandra/hadoop/pig/CassandraStorage.java| 6 +- src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java | 8 3 files changed, 16 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0dccea4f/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java --
[jira] [Created] (CASSANDRA-6242) CQL LIST USERS does nothing after a user is created.
Ben Sykes created CASSANDRA-6242: Summary: CQL LIST USERS does nothing after a user is created. Key: CASSANDRA-6242 URL: https://issues.apache.org/jira/browse/CASSANDRA-6242 Project: Cassandra Issue Type: Bug Components: Tools Environment: cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0 java version 1.6.0_43 Java(TM) SE Runtime Environment (build 1.6.0_43-b01) Java HotSpot(TM) 64-Bit Server VM (build 20.14-b01, mixed mode) Windows 7 - CQL running in Cygwin. Python 2.7.3 Reporter: Ben Sykes Priority: Minor After using CREATE USER to create a new user, the LIST USERS command returns nothing to the console. After removing this user again, the command works as expected. {code} $ ./cqlsh -u cassandra -p cassandra Connected to Test Cluster at localhost:9160. [cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0] Use HELP for help. cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh CREATE USER bob WITH PASSWORD 'example' NOSUPERUSER; cqlsh LIST USERS; cqlsh SELECT * FROM system_auth.users; name | super ---+--- bob | False cassandra | True (2 rows) cqlsh DROP USER bob; cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh {code} -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6135) Add beforeChange Notification to Gossiper State.
[ https://issues.apache.org/jira/browse/CASSANDRA-6135?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805410#comment-13805410 ] Brandon Williams commented on CASSANDRA-6135: - Works for me, can you rebase that against 2.0? Add beforeChange Notification to Gossiper State. Key: CASSANDRA-6135 URL: https://issues.apache.org/jira/browse/CASSANDRA-6135 Project: Cassandra Issue Type: New Feature Reporter: Benjamin Coverston Assignee: Sergio Bossa Attachments: 0001-New-Gossiper-notification-to-IEndpointStateChangeSub.patch, 0002-CASSANDRA-6135.diff, CASSANDRA-6135-V3.patch We would like an internal notification to be fired before state changes happen so we can intercept them, and in some cases defer them. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805437#comment-13805437 ] Chad Johnston commented on CASSANDRA-6151: -- I'm having issues running my tests against the 1.2 branch. If I give a {{load}} statement with no {{where_clause}}, I get the error {{consistency level LOCAL_ONE not compatible with replication strategy (org.apache.cassandra.locator.SimpleStrategy)}}. When I try the query with the {{where_clause}}, I get an error saying that the keyspace does not exist, even though I can query the keyspace with no issues in cqlsh. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Comment Edited] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805443#comment-13805443 ] Alex Liu edited comment on CASSANDRA-6151 at 10/25/13 4:44 PM: --- It's related to CASSANDRA-6238, you can set cassandra.consistencylevel.read to ONE in Pig as a workaround. was (Author: alexliu68): It's related to CASSANDRA-6238, you can cassandra.consistencylevel.read to ONE in Pig as a workaround. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805443#comment-13805443 ] Alex Liu commented on CASSANDRA-6151: - It's related to CASSANDRA-6238, you can cassandra.consistencylevel.read to ONE in Pig as a workaround. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Created] (CASSANDRA-6243) Create dtest for authenticated logins
Ryan McGuire created CASSANDRA-6243: --- Summary: Create dtest for authenticated logins Key: CASSANDRA-6243 URL: https://issues.apache.org/jira/browse/CASSANDRA-6243 Project: Cassandra Issue Type: Test Components: Tests Reporter: Ryan McGuire Assignee: Ryan McGuire Looking at CASSANDRA-6233 we should have some automated testing that authentication doesn't break like this. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805460#comment-13805460 ] Chad Johnston commented on CASSANDRA-6151: -- That works around the consistency error. I still get the error saying the keyspace was not found whenever I specify a {{where_clause}}. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
git commit: Fix updating column_metadata from thrift
Updated Branches: refs/heads/cassandra-2.0 2f248b3f9 - 2c4b043f9 Fix updating column_metadata from thrift patch by slebresne; reviewed by jbellis for CASSANDRA-6182 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2c4b043f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2c4b043f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2c4b043f Branch: refs/heads/cassandra-2.0 Commit: 2c4b043f94d4fa4b35e60949be5e298f1efe6422 Parents: 2f248b3 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 25 19:10:07 2013 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 25 19:10:07 2013 +0200 -- CHANGES.txt| 4 src/java/org/apache/cassandra/config/CFMetaData.java | 4 ++-- src/java/org/apache/cassandra/config/ColumnDefinition.java | 6 -- 3 files changed, 10 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c4b043f/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 32c74aa..3c96770 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,7 @@ +2.0.3 + * Fix modifying column_metadata from thrift (CASSANDRA-6182) + + 2.0.2 * Update FailureDetector to use nanontime (CASSANDRA-4925) * Fix FileCacheService regressions (CASSANDRA-6149) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c4b043f/src/java/org/apache/cassandra/config/CFMetaData.java -- diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java index d63ee01..20c35b3 100644 --- a/src/java/org/apache/cassandra/config/CFMetaData.java +++ b/src/java/org/apache/cassandra/config/CFMetaData.java @@ -1461,7 +1461,7 @@ public final class CFMetaData if (fromThrift cd.type != ColumnDefinition.Type.REGULAR) continue; -cd.deleteFromSchema(rm, cfName, modificationTimestamp); +cd.deleteFromSchema(rm, cfName, getColumnDefinitionComparator(cd), modificationTimestamp); } // newly added columns @@ -1506,7 +1506,7 @@ public final class CFMetaData cf.addAtom(new RangeTombstone(builder.build(), builder.buildAsEndOfRange(), timestamp, ldt)); for (ColumnDefinition cd : column_metadata.values()) -cd.deleteFromSchema(rm, cfName, timestamp); +cd.deleteFromSchema(rm, cfName, getColumnDefinitionComparator(cd), timestamp); for (TriggerDefinition td : triggers.values()) td.deleteFromSchema(rm, cfName, timestamp); http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c4b043f/src/java/org/apache/cassandra/config/ColumnDefinition.java -- diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java index d68e7e2..7ca4d45 100644 --- a/src/java/org/apache/cassandra/config/ColumnDefinition.java +++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java @@ -230,13 +230,15 @@ public class ColumnDefinition * @param cfName The name of the parent ColumnFamily * @param timestamp The timestamp to use for column modification */ -public void deleteFromSchema(RowMutation rm, String cfName, long timestamp) +public void deleteFromSchema(RowMutation rm, String cfName, AbstractType? comparator, long timestamp) { ColumnFamily cf = rm.addOrGet(CFMetaData.SchemaColumnsCf); int ldt = (int) (System.currentTimeMillis() / 1000); ColumnNameBuilder builder = CFMetaData.SchemaColumnsCf.getCfDef().getColumnNameBuilder(); -builder.add(ByteBufferUtil.bytes(cfName)).add(name); +// Note: the following is necessary for backward compatibility. For CQL3, comparator will be UTF8 and nameBytes == name +ByteBuffer nameBytes = ByteBufferUtil.bytes(comparator.getString(name)); +builder.add(ByteBufferUtil.bytes(cfName)).add(nameBytes); cf.addAtom(new RangeTombstone(builder.build(), builder.buildAsEndOfRange(), timestamp, ldt)); }
[2/2] git commit: Merge branch 'cassandra-2.0' into trunk
Merge branch 'cassandra-2.0' into trunk Conflicts: CHANGES.txt Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0c56a5ec Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0c56a5ec Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0c56a5ec Branch: refs/heads/trunk Commit: 0c56a5ecedc8f0f1d60879b043650402168a14f8 Parents: 0dccea4 2c4b043 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 25 19:11:22 2013 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 25 19:11:22 2013 +0200 -- CHANGES.txt| 4 src/java/org/apache/cassandra/config/CFMetaData.java | 4 ++-- src/java/org/apache/cassandra/config/ColumnDefinition.java | 6 -- 3 files changed, 10 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c56a5ec/CHANGES.txt -- diff --cc CHANGES.txt index a7671c7,3c96770..549f318 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,12 -1,7 +1,16 @@@ +2.1 + * Parallelize fetching rows for low-cardinality indexes (CASSANDRA-1337) + * change logging from log4j to logback (CASSANDRA-5883) + * switch to LZ4 compression for internode communication (CASSANDRA-5887) + * Stop using Thrift-generated Index* classes internally (CASSANDRA-5971) + * Remove 1.2 network compatibility code (CASSANDRA-5960) + * Remove leveled json manifest migration code (CASSANDRA-5996) + + + 2.0.3 + * Fix modifying column_metadata from thrift (CASSANDRA-6182) + + 2.0.2 * Update FailureDetector to use nanontime (CASSANDRA-4925) * Fix FileCacheService regressions (CASSANDRA-6149) http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c56a5ec/src/java/org/apache/cassandra/config/CFMetaData.java -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0c56a5ec/src/java/org/apache/cassandra/config/ColumnDefinition.java --
[1/2] git commit: Fix updating column_metadata from thrift
Updated Branches: refs/heads/trunk 0dccea4f3 - 0c56a5ece Fix updating column_metadata from thrift patch by slebresne; reviewed by jbellis for CASSANDRA-6182 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2c4b043f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2c4b043f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2c4b043f Branch: refs/heads/trunk Commit: 2c4b043f94d4fa4b35e60949be5e298f1efe6422 Parents: 2f248b3 Author: Sylvain Lebresne sylv...@datastax.com Authored: Fri Oct 25 19:10:07 2013 +0200 Committer: Sylvain Lebresne sylv...@datastax.com Committed: Fri Oct 25 19:10:07 2013 +0200 -- CHANGES.txt| 4 src/java/org/apache/cassandra/config/CFMetaData.java | 4 ++-- src/java/org/apache/cassandra/config/ColumnDefinition.java | 6 -- 3 files changed, 10 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c4b043f/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 32c74aa..3c96770 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,7 @@ +2.0.3 + * Fix modifying column_metadata from thrift (CASSANDRA-6182) + + 2.0.2 * Update FailureDetector to use nanontime (CASSANDRA-4925) * Fix FileCacheService regressions (CASSANDRA-6149) http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c4b043f/src/java/org/apache/cassandra/config/CFMetaData.java -- diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java index d63ee01..20c35b3 100644 --- a/src/java/org/apache/cassandra/config/CFMetaData.java +++ b/src/java/org/apache/cassandra/config/CFMetaData.java @@ -1461,7 +1461,7 @@ public final class CFMetaData if (fromThrift cd.type != ColumnDefinition.Type.REGULAR) continue; -cd.deleteFromSchema(rm, cfName, modificationTimestamp); +cd.deleteFromSchema(rm, cfName, getColumnDefinitionComparator(cd), modificationTimestamp); } // newly added columns @@ -1506,7 +1506,7 @@ public final class CFMetaData cf.addAtom(new RangeTombstone(builder.build(), builder.buildAsEndOfRange(), timestamp, ldt)); for (ColumnDefinition cd : column_metadata.values()) -cd.deleteFromSchema(rm, cfName, timestamp); +cd.deleteFromSchema(rm, cfName, getColumnDefinitionComparator(cd), timestamp); for (TriggerDefinition td : triggers.values()) td.deleteFromSchema(rm, cfName, timestamp); http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c4b043f/src/java/org/apache/cassandra/config/ColumnDefinition.java -- diff --git a/src/java/org/apache/cassandra/config/ColumnDefinition.java b/src/java/org/apache/cassandra/config/ColumnDefinition.java index d68e7e2..7ca4d45 100644 --- a/src/java/org/apache/cassandra/config/ColumnDefinition.java +++ b/src/java/org/apache/cassandra/config/ColumnDefinition.java @@ -230,13 +230,15 @@ public class ColumnDefinition * @param cfName The name of the parent ColumnFamily * @param timestamp The timestamp to use for column modification */ -public void deleteFromSchema(RowMutation rm, String cfName, long timestamp) +public void deleteFromSchema(RowMutation rm, String cfName, AbstractType? comparator, long timestamp) { ColumnFamily cf = rm.addOrGet(CFMetaData.SchemaColumnsCf); int ldt = (int) (System.currentTimeMillis() / 1000); ColumnNameBuilder builder = CFMetaData.SchemaColumnsCf.getCfDef().getColumnNameBuilder(); -builder.add(ByteBufferUtil.bytes(cfName)).add(name); +// Note: the following is necessary for backward compatibility. For CQL3, comparator will be UTF8 and nameBytes == name +ByteBuffer nameBytes = ByteBufferUtil.bytes(comparator.getString(name)); +builder.add(ByteBufferUtil.bytes(cfName)).add(nameBytes); cf.addAtom(new RangeTombstone(builder.build(), builder.buildAsEndOfRange(), timestamp, ldt)); }
[jira] [Commented] (CASSANDRA-6182) Unable to modify column_metadata via thrift
[ https://issues.apache.org/jira/browse/CASSANDRA-6182?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805470#comment-13805470 ] Sylvain Lebresne commented on CASSANDRA-6182: - Committed, thanks Unable to modify column_metadata via thrift --- Key: CASSANDRA-6182 URL: https://issues.apache.org/jira/browse/CASSANDRA-6182 Project: Cassandra Issue Type: Bug Reporter: Nick Bailey Assignee: Sylvain Lebresne Fix For: 2.0.3 Attachments: 6182.txt Reproduced on 2.0 HEAD {noformat} [default@unknown] use opscenter; Authenticated to keyspace: OpsCenter [default@OpsCenter] create column family test with column_metadata = [{column_name: '', validation_class: LongType}]; 637fffa1-a10f-3d89-8be6-8a316af05dd2 [default@OpsCenter] update column family test with column_metadata=[]; e49e435b-ba2a-3a08-8af0-32b897b872b8 [default@OpsCenter] show schema; other entries removed create column family test with column_type = 'Standard' and comparator = 'BytesType' and default_validation_class = 'BytesType' and key_validation_class = 'BytesType' and read_repair_chance = 0.1 and dclocal_read_repair_chance = 0.0 and populate_io_cache_on_flush = false and gc_grace = 864000 and min_compaction_threshold = 4 and max_compaction_threshold = 32 and replicate_on_write = true and compaction_strategy = 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy' and caching = 'KEYS_ONLY' and default_time_to_live = 0 and speculative_retry = 'NONE' and column_metadata = [ {column_name : '', validation_class : LongType}] and compression_options = {'sstable_compression' : 'org.apache.cassandra.io.compress.LZ4Compressor'} and index_interval = 128; {noformat} -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Liu updated CASSANDRA-6151: Attachment: 6151-v2-1.2-branch.txt CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805478#comment-13805478 ] Alex Liu commented on CASSANDRA-6151: - v2 patch set default CL to ONE. My unit test is in the patch. {code} CREATE KEYSPACE cql3ks WITH replication = {'class': 'SimpleStrategy', 'replication_factor' : 1} CREATE TABLE compositekeytable (key1 text, key2 int, key3 int, key4 int, column1 int, column2 float, primary key((key1, key2), key3, key4)) CREATE INDEX column1 on compositekeytable (column1); INSERT INTO compositekeytable (key1, key2, key3, key4, column1, column2) values ('key1', 111, 100, 100, 100, 10.1) INSERT INTO compositekeytable (key1, key2, key3, key4, column1, column2) values ('key1', 111, 100, 101, 100, 10.1) INSERT INTO compositekeytable (key1, key2, key3, key4, column1, column2) values ('key1', 111, 100, 102, 100, 10.1) INSERT INTO compositekeytable (key1, key2, key3, key4, column1, column2) values ('key1', 111, 100, 103, 100, 10.1) INSERT INTO compositekeytable (key1, key2, key3, key4, column1, column2) values ('key1', 111, 100, 104, 100, 10.1) INSERT INTO compositekeytable (key1, key2, key3, key4, column1, column2) values ('key1', 111, 100, 105, 100, 10.1) composite_rows = LOAD 'cql://cql3ks/compositekeytable?where_clause=key1%20%3D%20%27key1%27%20and%20key2%20%3D%20111%20and%20column1%3D100page_size=2' USING CqlStorage(); dump composite_rows; {code} Can you check your test settings to mine? CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6243) Create more dtests for authenticated logins
[ https://issues.apache.org/jira/browse/CASSANDRA-6243?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan McGuire updated CASSANDRA-6243: Summary: Create more dtests for authenticated logins (was: Create dtest for authenticated logins) Create more dtests for authenticated logins --- Key: CASSANDRA-6243 URL: https://issues.apache.org/jira/browse/CASSANDRA-6243 Project: Cassandra Issue Type: Test Components: Tests Reporter: Ryan McGuire Assignee: Ryan McGuire Looking at CASSANDRA-6233 we should have some automated testing that authentication doesn't break like this. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6233) Authentication is broken for the protocol v1 on C* 2.0
[ https://issues.apache.org/jira/browse/CASSANDRA-6233?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805538#comment-13805538 ] Ryan McGuire commented on CASSANDRA-6233: - [~vchekan] there are [multiple authentication tests|https://github.com/riptano/cassandra-dtest/blob/master/auth_test.py] in cassandra-dtest. [~slebresne] - What is protocol v1? Can you give me an example of how this breaks? I tried [creating this test|https://github.com/EnigmaCurry/cassandra-dtest/commit/11042a79a3190211e40981271cfcca9c06aee456] and it's passing on cassandra-2.0, but also on 2.0.1, so I must not be exercising the same thing you're describing here. I also notice that the patch looks like it's already been committed to cassandra-2.0 branch (not sure if that was intentional or not.) Authentication is broken for the protocol v1 on C* 2.0 -- Key: CASSANDRA-6233 URL: https://issues.apache.org/jira/browse/CASSANDRA-6233 Project: Cassandra Issue Type: Bug Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Fix For: 2.0.3 Attachments: 6233.txt CASSANDRA-5664 simplified the decoding method of CredentialsMessage by using CBUtil.readStringMap (instead of duplicating the code). Unfortunately, that latter method turns his keys to uppercase (to provide some form of case insensitivity for keys), and in the case of CredentialsMessage this breaks PasswordAuthenticator that expect lowercased keys (besides, it's a bad idea to mess up with the case of the credentials map in general). Making CBUtil.readStringMap uppercase keys was probably a bad idea in the first place (as nothing in the method name imply this), so attaching patch that remove this (and uppercase keys specifically in StartupMessage where that was done on purpose). -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805541#comment-13805541 ] Chad Johnston commented on CASSANDRA-6151: -- I still get the keyspace does not exist error. This makes me wonder if I have something wrong in my environment. Is there a way I can turn on some debugging or other diagnostics? CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805566#comment-13805566 ] Alex Liu commented on CASSANDRA-6151: - Follow this link http://stackoverflow.com/questions/16627748/pig-batch-mode-how-to-set-logging-level-to-hide-info-log-messages {code} Step1: copy the log4j config file to the folder where my pig scripts are located. cp /etc/pig/conf.dist/log4j.properties log4j_WARN Step2: Edit log4j_WARN file and make sure these two lines are present log4j.logger.org.apache.pig=WARN, A log4j.logger.org.apache.hadoop = WARN, A log4j.logger.org.apache.cassandra = WARN, A Step3: Run pig script and instruct it to use the custom log4j pig -x local -4 log4j_WARN MyScript.pig {code} You can set level to DEBUG CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805570#comment-13805570 ] Alex Liu commented on CASSANDRA-6151: - Or change test/conf/log4j-junit.properties to remove log4j.logger.org.apache.hadoop=ERROR then run the unit test {code} ant pig-test {code} where CqlTableTest has the unit for this ticket. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6109) Consider coldness in STCS compaction
[ https://issues.apache.org/jira/browse/CASSANDRA-6109?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-6109: --- Attachment: 6109-v3.patch 6109-v3.patch (and [branch|https://github.com/thobbs/cassandra/tree/CASSANDRA-6109]) uses the new strategy for filtering cold sstables. Feel free to bikeshed on the config option name. Consider coldness in STCS compaction Key: CASSANDRA-6109 URL: https://issues.apache.org/jira/browse/CASSANDRA-6109 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Jonathan Ellis Assignee: Tyler Hobbs Fix For: 2.0.2 Attachments: 6109-v1.patch, 6109-v2.patch, 6109-v3.patch I see two options: # Don't compact cold sstables at all # Compact cold sstables only if there is nothing more important to compact The latter is better if you have cold data that may become hot again... but it's confusing if you have a workload such that you can't keep up with *all* compaction, but you can keep up with hot sstable. (Compaction backlog stat becomes useless since we fall increasingly behind.) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805622#comment-13805622 ] Chad Johnston commented on CASSANDRA-6151: -- Okay - I re-cloned the repo and applied the patch again. All of the {{pig-test}} tests pass, but I still get keyspace does not exist from within Pig. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Resolved] (CASSANDRA-6240) CLASSPATH logic from init script is unused, JNA isn't loaded
[ https://issues.apache.org/jira/browse/CASSANDRA-6240?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Eric Evans resolved CASSANDRA-6240. --- Resolution: Fixed CLASSPATH logic from init script is unused, JNA isn't loaded Key: CASSANDRA-6240 URL: https://issues.apache.org/jira/browse/CASSANDRA-6240 Project: Cassandra Issue Type: Bug Components: Packaging Reporter: Faidon Liambotis Assignee: Eric Evans The init script has a classpath() function that collects all the jars and even includes this piece of code to work with the standard Debian/Ubuntu libjna-jar: {code:none} # use JNA if installed in standard location [ -r /usr/share/java/jna.jar ] cp=$cp:/usr/share/java/jna.jar {code} This seems very nice and correct, however the classpath() function is never called and is entirely unused :) Instead, /usr/bin/cassandra is called, which in turn includes /usr/share/cassandra/cassandra.in.sh, which has basically similar code to collect the jars for CLASSPATH but a) without the JNA standard path trick b) without using EXTRA_CLASSPATH (from /etc/default/cassandra) at all, so Cassandra boots without either JNA nor EXTRA_CLASSPATH, contrary to expectations. There are various suggestions on the web to do ln -s /usr/share/java/jna.jar /usr/share/cassandra/lib/; I suspect this bug to be the reason for that. /usr/share/cassandra/cassandra.in.sh seems smart enough to append but not overwrite CLASSPATH, so fixing the init script's classpath() to only include JNA + EXTRA_CLASSPATH (and making sure it's actually getting called :)) should be enough for a fix. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6240) CLASSPATH logic from init script is unused, JNA isn't loaded
[ https://issues.apache.org/jira/browse/CASSANDRA-6240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805654#comment-13805654 ] Eric Evans commented on CASSANDRA-6240: --- This was fixed in the 2.0 branch as part of CASSANDRA-6101, and will be released as part of 2.0.2. There is one remaining issue with the init script that has gotten hung in review. [~paravoid] if you could have a look at CASSANDRA-6131 and comment on it there, I'd be very grateful (I'll buy you a cheesesteak sandwich in Portland next summer :)). CLASSPATH logic from init script is unused, JNA isn't loaded Key: CASSANDRA-6240 URL: https://issues.apache.org/jira/browse/CASSANDRA-6240 Project: Cassandra Issue Type: Bug Components: Packaging Reporter: Faidon Liambotis Assignee: Eric Evans The init script has a classpath() function that collects all the jars and even includes this piece of code to work with the standard Debian/Ubuntu libjna-jar: {code:none} # use JNA if installed in standard location [ -r /usr/share/java/jna.jar ] cp=$cp:/usr/share/java/jna.jar {code} This seems very nice and correct, however the classpath() function is never called and is entirely unused :) Instead, /usr/bin/cassandra is called, which in turn includes /usr/share/cassandra/cassandra.in.sh, which has basically similar code to collect the jars for CLASSPATH but a) without the JNA standard path trick b) without using EXTRA_CLASSPATH (from /etc/default/cassandra) at all, so Cassandra boots without either JNA nor EXTRA_CLASSPATH, contrary to expectations. There are various suggestions on the web to do ln -s /usr/share/java/jna.jar /usr/share/cassandra/lib/; I suspect this bug to be the reason for that. /usr/share/cassandra/cassandra.in.sh seems smart enough to append but not overwrite CLASSPATH, so fixing the init script's classpath() to only include JNA + EXTRA_CLASSPATH (and making sure it's actually getting called :)) should be enough for a fix. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6190) Cassandra 2.0 won't start up with Java 7u40 with Client JVM. (works on Server JVM, and both JVMs 7u25)
[ https://issues.apache.org/jira/browse/CASSANDRA-6190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805661#comment-13805661 ] Eric Evans commented on CASSANDRA-6190: --- +1, lgtm Cassandra 2.0 won't start up with Java 7u40 with Client JVM. (works on Server JVM, and both JVMs 7u25) --- Key: CASSANDRA-6190 URL: https://issues.apache.org/jira/browse/CASSANDRA-6190 Project: Cassandra Issue Type: Bug Components: Config Environment: Ubuntu 13.04 32- and 64-bit JDK 7u40 (tried JRE 7u25) Reporter: Steven Lowenthal Assignee: Brandon Williams Attachments: 6190.txt Java 7u40 on some platforms do not recognize the the -XX:+UseCondCardMark JVM option. 7u40 on Macintosh works correctly, If I use the tarball 7u40 version of 7, we encounter the error below. I tried 7u25 (the previous release) and it functioned correctly. ubuntu@ubuntu:~$ Unrecognized VM option 'UseCondCardMark' Error: Could not create the Java Virtual Machine. Error: A fatal exception has occurred. Program will exit. -- This message was sent by Atlassian JIRA (v6.1#6144)
[3/3] git commit: Merge branch 'cassandra-2.0' into trunk
Merge branch 'cassandra-2.0' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/db5c95cd Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/db5c95cd Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/db5c95cd Branch: refs/heads/trunk Commit: db5c95cdc5ba232c420d3a110ae1a7bc7821c22d Parents: 0c56a5e 1bd7ac3 Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 15:28:28 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 15:28:28 2013 -0500 -- conf/cassandra-env.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) --
[1/3] git commit: Only pass UseCondCardMark to 64bit JVMs Patch by brandonwilliams, reviewed by eevans for CASSANDRA-6190
Updated Branches: refs/heads/cassandra-2.0 2c4b043f9 - 1bd7ac3c4 refs/heads/trunk 0c56a5ece - db5c95cdc Only pass UseCondCardMark to 64bit JVMs Patch by brandonwilliams, reviewed by eevans for CASSANDRA-6190 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1bd7ac3c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1bd7ac3c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1bd7ac3c Branch: refs/heads/cassandra-2.0 Commit: 1bd7ac3c4b1be9f4fac19c379504eb0e95346ec0 Parents: 2c4b043 Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 15:27:40 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 15:28:22 2013 -0500 -- conf/cassandra-env.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/1bd7ac3c/conf/cassandra-env.sh -- diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh index d84c7ce..e8aa3a4 100644 --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -135,7 +135,6 @@ esac # Set this to control the amount of arenas per-thread in glibc #MALLOC_ARENA_MAX=4 - if [ x$MAX_HEAP_SIZE = x ] [ x$HEAP_NEWSIZE = x ]; then calculate_heap_sizes else @@ -208,7 +207,7 @@ JVM_OPTS=$JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75 JVM_OPTS=$JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly JVM_OPTS=$JVM_OPTS -XX:+UseTLAB # note: bash evals '1.7.x' as '1.7' so this is really a = 1.7 jvm check -if [ $JVM_VERSION \ 1.7 ] ; then +if [ $JVM_VERSION \ 1.7 ] [ $JVM_ARCH = 64-Bit ] ; then JVM_OPTS=$JVM_OPTS -XX:+UseCondCardMark fi
[2/3] git commit: Only pass UseCondCardMark to 64bit JVMs Patch by brandonwilliams, reviewed by eevans for CASSANDRA-6190
Only pass UseCondCardMark to 64bit JVMs Patch by brandonwilliams, reviewed by eevans for CASSANDRA-6190 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/1bd7ac3c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/1bd7ac3c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/1bd7ac3c Branch: refs/heads/trunk Commit: 1bd7ac3c4b1be9f4fac19c379504eb0e95346ec0 Parents: 2c4b043 Author: Brandon Williams brandonwilli...@apache.org Authored: Fri Oct 25 15:27:40 2013 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Fri Oct 25 15:28:22 2013 -0500 -- conf/cassandra-env.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/1bd7ac3c/conf/cassandra-env.sh -- diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh index d84c7ce..e8aa3a4 100644 --- a/conf/cassandra-env.sh +++ b/conf/cassandra-env.sh @@ -135,7 +135,6 @@ esac # Set this to control the amount of arenas per-thread in glibc #MALLOC_ARENA_MAX=4 - if [ x$MAX_HEAP_SIZE = x ] [ x$HEAP_NEWSIZE = x ]; then calculate_heap_sizes else @@ -208,7 +207,7 @@ JVM_OPTS=$JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75 JVM_OPTS=$JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly JVM_OPTS=$JVM_OPTS -XX:+UseTLAB # note: bash evals '1.7.x' as '1.7' so this is really a = 1.7 jvm check -if [ $JVM_VERSION \ 1.7 ] ; then +if [ $JVM_VERSION \ 1.7 ] [ $JVM_ARCH = 64-Bit ] ; then JVM_OPTS=$JVM_OPTS -XX:+UseCondCardMark fi
[jira] [Commented] (CASSANDRA-6199) Improve Stress Tool
[ https://issues.apache.org/jira/browse/CASSANDRA-6199?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805651#comment-13805651 ] Benedict commented on CASSANDRA-6199: - I have put a branch up with a completely new stress here: https://github.com/belliottsmith/cassandra/tree/iss-6199-stress Some things to expect: Reduced variability of results No start/end tail truncation Short warmup period (50K operations) with pause to get JVM warm Low garbage production in stress process New data and key generators, including random sentence generation from frequency based word list, exponential and extreme value key distributions Ability to configure distribution for column count and size Mixed mode permitting arbitrary ratios of any of the basic operations Automatic mode that ramps up thread count until cluster is saturated, pushes a bit further, then reports a summary Automatic cessation of a test based on stderr of mean (i.e. instead of asking for n operations, ask for e.g. stderr 0.01, with at least n samples) Reports latencies per period, plus entire run, as opposed to a running tally of latencies Calculates op rate accurately Supports huge keys, plus arbitrary key ranges, and keys are not dependent on number of operations performed, so a reads can be run with different op count to writes Also, there's a new command line syntax to handle all of the complexity it now supports, but there's a legacy support mode - I've tested the mapping, but it may need a little more testing to make sure I've caught any potential nooks and crannies Would be great to have some people test it out to see if anything needs changing. Improve Stress Tool --- Key: CASSANDRA-6199 URL: https://issues.apache.org/jira/browse/CASSANDRA-6199 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Benedict Assignee: Benedict Priority: Minor The stress tool could do with sprucing up. The following is a list of essential improvements and things that would be nice to have. Essential: - Reduce variability of results, especially start/end tails. Do not trash first/last 10% of readings - Reduce contention/overhead in stress to increase overall throughput - Short warm-up period, which is ignored for summary (or summarised separately), though prints progress as usual. Potentially automatic detection of rate levelling. - Better configurability and defaults for data generation - current column generation populates columns with the same value for every row, which is very easily compressible. Possibly introduce partial random data generator (possibly dictionary-based random data generator) Nice to have: - Calculate and print stdev and mean - Add batched sequential access mode (where a single thread performs batch-size sequential requests before selecting another random key) to test how key proximity affects performance - Auto-mode which attempts to establish the maximum throughput rate, by varying the thread count (or otherwise gating the number of parallel requests) for some period, then configures rate limit or thread count to test performance at e.g. 30%, 50%, 70%, 90%, 120%, 150% and unconstrained. - Auto-mode could have a target variance ratio for mean throughput and/or latency, and completes a test once this target is hit for x intervals - Fix key representation so independent of number of keys (possibly switch to 10 digit hex), and don't use String.format().getBytes() to construct it (expensive) Also, remove the skip-key setting, as it is currently ignored. Unless somebody knows the reason for it. - Fix latency stats - Read/write mode, with configurable recency-of-reads distribution - Add new exponential/extreme value distribution for value size, column count and recency-of-reads - Support more than 2^31 keys - Supports multiple concurrent stress inserts via key-offset parameter or similar -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805660#comment-13805660 ] Alex Liu commented on CASSANDRA-6151: - thanks, I can reproduce it. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805709#comment-13805709 ] Benedict commented on CASSANDRA-6146: - Hi [~mishail], You might want to take a look at my patch for [6199|https://issues.apache.org/jira/browse/CASSANDRA-6199], and make your changes there. Adding support for custom CQL operations like this should be as easy as copying the CqlReader class and, mostly, deleting a few unnecessary lines. The options parser may need a few minutes to figure out how to add another command. Take a look in Command, SettingsCommand and StressSettings. The interesting bit will be inserts - automatic detection of table structure and selection of sensible generators for those columns should be fun, and a great feature. A couple of new data generators may be needed, but should be pretty easy. If you want to get really exciting, support for interleaving multiple statements shouldn't be too difficult, as it's currently supported for other operations in mixed mode - this would require a little bit of rejigging, but probably not too much. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Comment Edited] (CASSANDRA-6146) CQL-native stress
[ https://issues.apache.org/jira/browse/CASSANDRA-6146?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805709#comment-13805709 ] Benedict edited comment on CASSANDRA-6146 at 10/25/13 9:20 PM: --- Hi [~mishail], You might want to take a look at my patch for [6199|https://issues.apache.org/jira/browse/CASSANDRA-6199], and make your changes there. Adding support for custom CQL operations like this should be as easy as copying the CqlReader class and, mostly, deleting a few unnecessary lines. The options parser may need a few minutes to figure out how to add another command. Take a look in Command, SettingsCommand and StressSettings. The interesting bit will be inserts - automatic detection of table structure and selection of sensible generators for those columns should be fun, and a great feature. A couple of new data generators may be needed, but should be pretty easy. If you want to get really exciting, support for interleaving multiple statements shouldn't be too difficult, as it's currently supported for other operations in mixed mode - this would require a little bit of rejigging, but probably not too much. Any questions, feel free to ping me here, email or irc was (Author: benedict): Hi [~mishail], You might want to take a look at my patch for [6199|https://issues.apache.org/jira/browse/CASSANDRA-6199], and make your changes there. Adding support for custom CQL operations like this should be as easy as copying the CqlReader class and, mostly, deleting a few unnecessary lines. The options parser may need a few minutes to figure out how to add another command. Take a look in Command, SettingsCommand and StressSettings. The interesting bit will be inserts - automatic detection of table structure and selection of sensible generators for those columns should be fun, and a great feature. A couple of new data generators may be needed, but should be pretty easy. If you want to get really exciting, support for interleaving multiple statements shouldn't be too difficult, as it's currently supported for other operations in mixed mode - this would require a little bit of rejigging, but probably not too much. CQL-native stress - Key: CASSANDRA-6146 URL: https://issues.apache.org/jira/browse/CASSANDRA-6146 Project: Cassandra Issue Type: New Feature Components: Tools Reporter: Jonathan Ellis The existing CQL support in stress is not worth discussing. We need to start over, and we might as well kill two birds with one stone and move to the native protocol while we're at it. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-4785) Secondary Index Sporadically Doesn't Return Rows
[ https://issues.apache.org/jira/browse/CASSANDRA-4785?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805730#comment-13805730 ] Jonathan Ellis commented on CASSANDRA-4785: --- Can someone confirm or deny that this is fixed in 1.2.11 by CASSANDRA-5732? Secondary Index Sporadically Doesn't Return Rows Key: CASSANDRA-4785 URL: https://issues.apache.org/jira/browse/CASSANDRA-4785 Project: Cassandra Issue Type: Bug Components: Core Affects Versions: 1.1.5, 1.1.6 Environment: Ubuntu 10.04 Java 6 Sun Cassandra 1.1.5 upgraded from 1.1.2 - 1.1.3 - 1.1.5 Reporter: Arya Goudarzi Assignee: Sam Tunnicliffe Attachments: entity_aliases.txt, repro.py I have a ColumnFamily with caching = ALL. I have 2 secondary indexes on it. I have noticed if I query using the secondary index in the where clause, sometimes I get the results and sometimes I don't. Until 2 weeks ago, the caching option on this CF was set to NONE. So, I suspect something happened in secondary index caching scheme. Here are things I tried: 1. I rebuild indexes for that CF on all nodes; 2. I set the caching to KEYS_ONLY and rebuild the index again; 3. I set the caching to NONE and rebuild the index again; None of the above helped. I suppose the caching still exists as this behavior looks like cache mistmatch. I did a bit research, and found CASSANDRA-4197 that could be related. Please advice. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Created] (CASSANDRA-6244) calculatePendingRanges could be asynchronous on 1.2 too
Ryan Fowler created CASSANDRA-6244: -- Summary: calculatePendingRanges could be asynchronous on 1.2 too Key: CASSANDRA-6244 URL: https://issues.apache.org/jira/browse/CASSANDRA-6244 Project: Cassandra Issue Type: Improvement Components: Core Environment: Cassandra 1.2, AWS Reporter: Ryan Fowler calculatePendingRanges can hang up the Gossip thread to the point of a node marking all the other nodes down. I noticed that the same problem was resolved with CASSANDRA-5135, so I attempted to port the patch from that issue to the 1.2 codebase. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6244) calculatePendingRanges could be asynchronous on 1.2 too
[ https://issues.apache.org/jira/browse/CASSANDRA-6244?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan Fowler updated CASSANDRA-6244: --- Attachment: 6244.txt calculatePendingRanges could be asynchronous on 1.2 too --- Key: CASSANDRA-6244 URL: https://issues.apache.org/jira/browse/CASSANDRA-6244 Project: Cassandra Issue Type: Improvement Components: Core Environment: Cassandra 1.2, AWS Reporter: Ryan Fowler Fix For: 1.2.12 Attachments: 6244.txt calculatePendingRanges can hang up the Gossip thread to the point of a node marking all the other nodes down. I noticed that the same problem was resolved with CASSANDRA-5135, so I attempted to port the patch from that issue to the 1.2 codebase. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Issue Comment Deleted] (CASSANDRA-6244) calculatePendingRanges could be asynchronous on 1.2 too
[ https://issues.apache.org/jira/browse/CASSANDRA-6244?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan Fowler updated CASSANDRA-6244: --- Comment: was deleted (was: commit b94f21925a41736682554c9ff934b1ff591f4711 Author: Ryan Fowler ryan.fow...@singlewire.com Date: Fri Oct 25 14:20:11 2013 -0500 Calculate pending ranges asynchronously Port of changes from 713bba to 1.2 branch. diff --git a/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java new file mode 100644 index 000..b408c75 --- /dev/null +++ b/src/java/org/apache/cassandra/service/PendingRangeCalculatorService.java @@ -0,0 +1,209 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.service; + +import org.apache.cassandra.utils.BiMultiValMap; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multimap; +import com.google.common.collect.Sets; + +import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor; +import org.apache.cassandra.concurrent.NamedThreadFactory; +import org.apache.cassandra.config.Schema; +import org.apache.cassandra.db.Table; +import org.apache.cassandra.dht.Range; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.locator.AbstractReplicationStrategy; +import org.apache.cassandra.locator.TokenMetadata; +import org.apache.cassandra.utils.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.Collection; +import java.util.concurrent.*; + + +public class PendingRangeCalculatorService extends PendingRangeCalculatorServiceMBean +{ +public static final PendingRangeCalculatorService instance = new PendingRangeCalculatorService(); + +private static Logger logger = LoggerFactory.getLogger(PendingRangeCalculatorService.class); +private final JMXEnabledThreadPoolExecutor executor = new JMXEnabledThreadPoolExecutor(1, Integer.MAX_VALUE, TimeUnit.SECONDS, +new LinkedBlockingQueueRunnable(1), new NamedThreadFactory(PendingRangeCalculator), internal); + +public PendingRangeCalculatorService() +{ +executor.setRejectedExecutionHandler(new ThreadPoolExecutor.DiscardPolicy()); +} + +private class PendingRangeTask implements Runnable +{ +public void run() +{ +long start = System.currentTimeMillis(); +for (String table : Schema.instance.getNonSystemTables()) +{ + calculatePendingRanges(Table.open(table).getReplicationStrategy(), table); +} +logger.debug(finished calculation for {} keyspaces in {}ms, Schema.instance.getNonSystemTables().size(), System.currentTimeMillis() - start); +} +} + +public Future? update() +{ +return executor.submit(new PendingRangeTask()); +} + +public void blockUntilFinished() +{ +while (true) +{ +if (executor.getActiveCount() + executor.getPendingTasks() == 0) +break; +try +{ +Thread.sleep(100); +} +catch (InterruptedException e) +{ +throw new RuntimeException(e); +} +} +} + + + +/** + * Calculate pending ranges according to bootsrapping and leaving nodes. Reasoning is: + * + * (1) When in doubt, it is better to write too much to a node than too little. That is, if + * there are multiple nodes moving, calculate the biggest ranges a node could have. Cleaning + * up unneeded data afterwards is better than missing writes during movement. + * (2) When a node leaves, ranges for other nodes can only grow (a node might get additional + * ranges, but it will not lose any of its current ranges as a result of a leave). Therefore + * we will first remove _all_ leaving tokens for the sake of calculation and then check what + * ranges would
[jira] [Created] (CASSANDRA-6245) nodetool refresh design is unsafe
Robert Coli created CASSANDRA-6245: -- Summary: nodetool refresh design is unsafe Key: CASSANDRA-6245 URL: https://issues.apache.org/jira/browse/CASSANDRA-6245 Project: Cassandra Issue Type: Bug Components: Core Reporter: Robert Coli Priority: Minor CASSANDRA-2991 added a nodetool refresh feature by which Cassandra is able to discover non-live SSTables in the datadir and make them live. It does this by : 1) looking for SSTable files in the data dir 2) renaming SSTables it finds into the current SSTable id sequence This implementation is exposed to a race with a chance of silent data loss. 1) Node's SSTable id sequence is on sstable #2, the next table to flush will get 2 as its numeric part 2) Copy SSTable with 2 as its numeric part into data dir 3) nodetool flush 4) notice that your 2 SSTable has been silently overwritten by a just-flushed 2 SSTable 5) nodetool refresh would still succeed, but would now be a no-op A simple solution would be to create a subdirectory of the datadir called refresh/ to serve as the location to refresh from. Alternately/additionally, there is probably not really a compelling reason for Cassandra to completely ignore existing files at write time.. a check for existing files at a given index and inflating the index to avoid overwriting them them seems trivial and inexpensive. I will gladly file a JIRA for this change in isolation if there is interest. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6245) nodetool refresh design is unsafe
[ https://issues.apache.org/jira/browse/CASSANDRA-6245?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805772#comment-13805772 ] Jonathan Ellis commented on CASSANDRA-6245: --- Isn't don't copy sstables over a soon-to-be-used sequence number a pretty straightforward extension of the don't copy sstables over existing in-use sequence numbers? nodetool refresh design is unsafe --- Key: CASSANDRA-6245 URL: https://issues.apache.org/jira/browse/CASSANDRA-6245 Project: Cassandra Issue Type: Bug Components: Core Reporter: Robert Coli Priority: Minor CASSANDRA-2991 added a nodetool refresh feature by which Cassandra is able to discover non-live SSTables in the datadir and make them live. It does this by : 1) looking for SSTable files in the data dir 2) renaming SSTables it finds into the current SSTable id sequence This implementation is exposed to a race with a chance of silent data loss. 1) Node's SSTable id sequence is on sstable #2, the next table to flush will get 2 as its numeric part 2) Copy SSTable with 2 as its numeric part into data dir 3) nodetool flush 4) notice that your 2 SSTable has been silently overwritten by a just-flushed 2 SSTable 5) nodetool refresh would still succeed, but would now be a no-op A simple solution would be to create a subdirectory of the datadir called refresh/ to serve as the location to refresh from. Alternately/additionally, there is probably not really a compelling reason for Cassandra to completely ignore existing files at write time.. a check for existing files at a given index and inflating the index to avoid overwriting them them seems trivial and inexpensive. I will gladly file a JIRA for this change in isolation if there is interest. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805838#comment-13805838 ] Quentin Conner commented on CASSANDRA-6127: --- I grabbed some sample log files from 10 nodes of 256 in a run today. [flap-intervals.tar.gz|http://qconner.s3.amazonaws.com/flap-intervals.tar.gz] Convictions are happening with only 1 to 5 intervals recorded. Patch #3 is looking like the winner but we should do the math by hand to be sure (volunteers?). Also, I just tested [Patch #3|https://issues.apache.org/jira/secure/attachment/12610117/delayEstimatorUntilStatisticallyValid.patch] and found 0 flaps for the same setup as yesterday (256 nodes, phi=8, normal 1000 ms gossip period). vnodes don't scale to hundreds of nodes --- Key: CASSANDRA-6127 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127 Project: Cassandra Issue Type: Bug Components: Core Environment: Any cluster that has vnodes and consists of hundreds of physical nodes. Reporter: Tupshin Harper Assignee: Jonathan Ellis Attachments: 6000vnodes.patch, AdjustableGossipPeriod.patch, delayEstimatorUntilStatisticallyValid.patch There are a lot of gossip-related issues related to very wide clusters that also have vnodes enabled. Let's use this ticket as a master in case there are sub-tickets. The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge instances. Each node configured with 32 vnodes. Without vnodes, cluster spins up fine and is ready to handle requests within 30 minutes or less. With vnodes, nodes are reporting constant up/down flapping messages with no external load on the cluster. After a couple of hours, they were still flapping, had very high cpu load, and the cluster never looked like it was going to stabilize or be useful for traffic. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Liu updated CASSANDRA-6151: Attachment: 6151-v3-1.2-branch.txt CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt, 6151-v3-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805854#comment-13805854 ] Alex Liu commented on CASSANDRA-6151: - v3 patch is attached which fixes the keyspace can't be found issue. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt, 6151-v3-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805880#comment-13805880 ] Jonathan Ellis commented on CASSANDRA-6127: --- Patch 1 will break things since later on we write the length of the string as two bytes. I think we're fine with 1700 vnodes per machine TBH, although it would be better to limit that in the config instead of failing at an assert later on. vnodes don't scale to hundreds of nodes --- Key: CASSANDRA-6127 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127 Project: Cassandra Issue Type: Bug Components: Core Environment: Any cluster that has vnodes and consists of hundreds of physical nodes. Reporter: Tupshin Harper Assignee: Jonathan Ellis Attachments: 6000vnodes.patch, AdjustableGossipPeriod.patch, delayEstimatorUntilStatisticallyValid.patch There are a lot of gossip-related issues related to very wide clusters that also have vnodes enabled. Let's use this ticket as a master in case there are sub-tickets. The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge instances. Each node configured with 32 vnodes. Without vnodes, cluster spins up fine and is ready to handle requests within 30 minutes or less. With vnodes, nodes are reporting constant up/down flapping messages with no external load on the cluster. After a couple of hours, they were still flapping, had very high cpu load, and the cluster never looked like it was going to stabilize or be useful for traffic. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805890#comment-13805890 ] Tupshin Harper commented on CASSANDRA-6127: --- I'd just set a max of 1024. No one could ever need more than that. (Famous last words) vnodes don't scale to hundreds of nodes --- Key: CASSANDRA-6127 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127 Project: Cassandra Issue Type: Bug Components: Core Environment: Any cluster that has vnodes and consists of hundreds of physical nodes. Reporter: Tupshin Harper Assignee: Jonathan Ellis Attachments: 6000vnodes.patch, AdjustableGossipPeriod.patch, delayEstimatorUntilStatisticallyValid.patch There are a lot of gossip-related issues related to very wide clusters that also have vnodes enabled. Let's use this ticket as a master in case there are sub-tickets. The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge instances. Each node configured with 32 vnodes. Without vnodes, cluster spins up fine and is ready to handle requests within 30 minutes or less. With vnodes, nodes are reporting constant up/down flapping messages with no external load on the cluster. After a couple of hours, they were still flapping, had very high cpu load, and the cluster never looked like it was going to stabilize or be useful for traffic. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Liu updated CASSANDRA-6151: Description: From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. h3. Solution was: From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt, 6151-v3-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. h3. Solution -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6151) CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated
[ https://issues.apache.org/jira/browse/CASSANDRA-6151?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Liu updated CASSANDRA-6151: Description: From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. h3. Solution If it have EQUAL clauses for all the partitioning keys, we use Query {code} SELECT * FROM data WHERE occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} instead of {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} The base line implementation is to retrieve all data of all rows around the ring. This new feature is to retrieve all data of a wide row. It's a one level lower than the base line. It helps for the use case where user is only interested in a specific wide row, so the user doesn't spend whole job to retrieve all the rows around the ring. was: From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND token(occurday,seqnumber) = ? AND occurday='A Great Day' AND seqnumber=1 LIMIT 1000 ALLOW FILTERING {code} Exception {code} InvalidRequestException(why:occurday cannot be restricted by more than one relation if it includes an Equal) {code} I'm not sure it is worth the special case but, a modification to not use the paging record reader when the entire partition key is specified would solve this issue. h3. Solution CqlPagingRecorderReader Used when Partition Key Is Explicitly Stated Key: CASSANDRA-6151 URL: https://issues.apache.org/jira/browse/CASSANDRA-6151 Project: Cassandra Issue Type: Bug Components: Hadoop Reporter: Russell Alexander Spitzer Assignee: Alex Liu Priority: Minor Attachments: 6151-1.2-branch.txt, 6151-v2-1.2-branch.txt, 6151-v3-1.2-branch.txt From http://stackoverflow.com/questions/19189649/composite-key-in-cassandra-with-pig/19211546#19211546 The user was attempting to load a single partition using a where clause in a pig load statement. CQL Table {code} CREATE table data ( occurday text, seqnumber int, occurtimems bigint, unique bigint, fields maptext, text, primary key ((occurday, seqnumber), occurtimems, unique) ) {code} Pig Load statement Query {code} data = LOAD 'cql://ks/data?where_clause=seqnumber%3D10%20AND%20occurday%3D%272013-10-01%27' USING CqlStorage(); {code} This results in an exception when processed by the the CqlPagingRecordReader which attempts to page this query even though it contains at most one partition key. This leads to an invalid CQL statement. CqlPagingRecordReader Query {code} SELECT * FROM data WHERE token(occurday,seqnumber) ? AND
[jira] [Commented] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805928#comment-13805928 ] Brandon Williams commented on CASSANDRA-6127: - Patch #3 will make it take much longer for a rebooted node to know who's actually up or down, exacerbating CASSANDRA-4288. I'd still like to know *why* things are taking longer with vnodes, and I'm especially hesitant to make any adjustments to the gossiper or FD since we know they work fine with single tokens, and also because they *have no knowledge about tokens*, it's just another opaque state to them. I suspect something in StorageService is blocking the gossiper long enough to cause this, perhaps CASSANDRA-6244 or something similar. vnodes don't scale to hundreds of nodes --- Key: CASSANDRA-6127 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127 Project: Cassandra Issue Type: Bug Components: Core Environment: Any cluster that has vnodes and consists of hundreds of physical nodes. Reporter: Tupshin Harper Assignee: Jonathan Ellis Attachments: 6000vnodes.patch, AdjustableGossipPeriod.patch, delayEstimatorUntilStatisticallyValid.patch There are a lot of gossip-related issues related to very wide clusters that also have vnodes enabled. Let's use this ticket as a master in case there are sub-tickets. The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge instances. Each node configured with 32 vnodes. Without vnodes, cluster spins up fine and is ready to handle requests within 30 minutes or less. With vnodes, nodes are reporting constant up/down flapping messages with no external load on the cluster. After a couple of hours, they were still flapping, had very high cpu load, and the cluster never looked like it was going to stabilize or be useful for traffic. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805977#comment-13805977 ] Jonathan Ellis commented on CASSANDRA-6127: --- Couldn't we tie the thrift/native server startup to I have enough gossip data now? vnodes don't scale to hundreds of nodes --- Key: CASSANDRA-6127 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127 Project: Cassandra Issue Type: Bug Components: Core Environment: Any cluster that has vnodes and consists of hundreds of physical nodes. Reporter: Tupshin Harper Assignee: Jonathan Ellis Attachments: 6000vnodes.patch, AdjustableGossipPeriod.patch, delayEstimatorUntilStatisticallyValid.patch There are a lot of gossip-related issues related to very wide clusters that also have vnodes enabled. Let's use this ticket as a master in case there are sub-tickets. The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge instances. Each node configured with 32 vnodes. Without vnodes, cluster spins up fine and is ready to handle requests within 30 minutes or less. With vnodes, nodes are reporting constant up/down flapping messages with no external load on the cluster. After a couple of hours, they were still flapping, had very high cpu load, and the cluster never looked like it was going to stabilize or be useful for traffic. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6127) vnodes don't scale to hundreds of nodes
[ https://issues.apache.org/jira/browse/CASSANDRA-6127?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13805981#comment-13805981 ] Brandon Williams commented on CASSANDRA-6127: - That might confuse autodiscovery clients, at least without further changes. vnodes don't scale to hundreds of nodes --- Key: CASSANDRA-6127 URL: https://issues.apache.org/jira/browse/CASSANDRA-6127 Project: Cassandra Issue Type: Bug Components: Core Environment: Any cluster that has vnodes and consists of hundreds of physical nodes. Reporter: Tupshin Harper Assignee: Jonathan Ellis Attachments: 6000vnodes.patch, AdjustableGossipPeriod.patch, delayEstimatorUntilStatisticallyValid.patch There are a lot of gossip-related issues related to very wide clusters that also have vnodes enabled. Let's use this ticket as a master in case there are sub-tickets. The most obvious symptom I've seen is with 1000 nodes in EC2 with m1.xlarge instances. Each node configured with 32 vnodes. Without vnodes, cluster spins up fine and is ready to handle requests within 30 minutes or less. With vnodes, nodes are reporting constant up/down flapping messages with no external load on the cluster. After a couple of hours, they were still flapping, had very high cpu load, and the cluster never looked like it was going to stabilize or be useful for traffic. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Created] (CASSANDRA-6246) EPaxos
Jonathan Ellis created CASSANDRA-6246: - Summary: EPaxos Key: CASSANDRA-6246 URL: https://issues.apache.org/jira/browse/CASSANDRA-6246 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Jonathan Ellis Priority: Minor One reason we haven't optimized our Paxos implementation with Multi-paxos is that Multi-paxos requires leader election and hence, a period of unavailability when the leader dies. EPaxos is a Paxos variant that requires (1) less messages than multi-paxos, (2) is particularly useful across multiple datacenters, and (3) allows any node to act as coordinator: http://sigops.org/sosp/sosp13/papers/p358-moraru.pdf However, there is substantial additional complexity involved if we choose to implement it. -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-3578) Multithreaded commitlog
[ https://issues.apache.org/jira/browse/CASSANDRA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13806006#comment-13806006 ] Jonathan Ellis commented on CASSANDRA-3578: --- bq. The above takes a different approach, we update commit log as a part of the mutation thread and no more threads to deal with serialization. I like it. bq. Doesn't have a end of segment on each mutation, we just have header which will hold the end. Meaning the cfLastWrite business, right? I must be missing where this gets persisted back to disk, but in any case, I'm not a fan; we got rid of the old per-segment header in part because seeking back and forth between the header and the tail where the appends happen was bad for performance. mfiguiere had a good summary of the options for dealing with end-of-segment above. Note that with any of these, I think allocate needs to write the length to the segment before returning. That way if I have two threads writing mutations X and Y, if Y's is written before a crash and X's is not, at least the length will be there that allows replay to skip the [unfinished] write of X and replay Y. Multithreaded commitlog --- Key: CASSANDRA-3578 URL: https://issues.apache.org/jira/browse/CASSANDRA-3578 Project: Cassandra Issue Type: Improvement Reporter: Jonathan Ellis Assignee: Vijay Priority: Minor Labels: performance Attachments: 0001-CASSANDRA-3578.patch, ComitlogStress.java, Current-CL.png, Multi-Threded-CL.png, parallel_commit_log_2.patch Brian Aker pointed out a while ago that allowing multiple threads to modify the commitlog simultaneously (reserving space for each with a CAS first, the way we do in the SlabAllocator.Region.allocate) can improve performance, since you're not bottlenecking on a single thread to do all the copying and CRC computation. Now that we use mmap'd CommitLog segments (CASSANDRA-3411) this becomes doable. (moved from CASSANDRA-622, which was getting a bit muddled.) -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Assigned] (CASSANDRA-6242) CQL LIST USERS does nothing after a user is created.
[ https://issues.apache.org/jira/browse/CASSANDRA-6242?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mikhail Stepura reassigned CASSANDRA-6242: -- Assignee: Mikhail Stepura CQL LIST USERS does nothing after a user is created. Key: CASSANDRA-6242 URL: https://issues.apache.org/jira/browse/CASSANDRA-6242 Project: Cassandra Issue Type: Bug Components: Tools Environment: cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0 java version 1.6.0_43 Java(TM) SE Runtime Environment (build 1.6.0_43-b01) Java HotSpot(TM) 64-Bit Server VM (build 20.14-b01, mixed mode) Windows 7 - CQL running in Cygwin. Python 2.7.3 Reporter: Ben Sykes Assignee: Mikhail Stepura Priority: Minor After using CREATE USER to create a new user, the LIST USERS command returns nothing to the console. After removing this user again, the command works as expected. {code} $ ./cqlsh -u cassandra -p cassandra Connected to Test Cluster at localhost:9160. [cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0] Use HELP for help. cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh CREATE USER bob WITH PASSWORD 'example' NOSUPERUSER; cqlsh LIST USERS; cqlsh SELECT * FROM system_auth.users; name | super ---+--- bob | False cassandra | True (2 rows) cqlsh DROP USER bob; cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh {code} -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Commented] (CASSANDRA-6242) CQL LIST USERS does nothing after a user is created.
[ https://issues.apache.org/jira/browse/CASSANDRA-6242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=13806007#comment-13806007 ] Mikhail Stepura commented on CASSANDRA-6242: Issue in cqlsh CQL LIST USERS does nothing after a user is created. Key: CASSANDRA-6242 URL: https://issues.apache.org/jira/browse/CASSANDRA-6242 Project: Cassandra Issue Type: Bug Components: Tools Environment: cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0 java version 1.6.0_43 Java(TM) SE Runtime Environment (build 1.6.0_43-b01) Java HotSpot(TM) 64-Bit Server VM (build 20.14-b01, mixed mode) Windows 7 - CQL running in Cygwin. Python 2.7.3 Reporter: Ben Sykes Assignee: Mikhail Stepura Priority: Minor After using CREATE USER to create a new user, the LIST USERS command returns nothing to the console. After removing this user again, the command works as expected. {code} $ ./cqlsh -u cassandra -p cassandra Connected to Test Cluster at localhost:9160. [cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0] Use HELP for help. cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh CREATE USER bob WITH PASSWORD 'example' NOSUPERUSER; cqlsh LIST USERS; cqlsh SELECT * FROM system_auth.users; name | super ---+--- bob | False cassandra | True (2 rows) cqlsh DROP USER bob; cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh {code} -- This message was sent by Atlassian JIRA (v6.1#6144)
[jira] [Updated] (CASSANDRA-6242) CQL LIST USERS does nothing after a user is created.
[ https://issues.apache.org/jira/browse/CASSANDRA-6242?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mikhail Stepura updated CASSANDRA-6242: --- Attachment: cassandra-2.0-6242.patch Patch to print out results for LIST commands as well CQL LIST USERS does nothing after a user is created. Key: CASSANDRA-6242 URL: https://issues.apache.org/jira/browse/CASSANDRA-6242 Project: Cassandra Issue Type: Bug Components: Tools Environment: cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0 java version 1.6.0_43 Java(TM) SE Runtime Environment (build 1.6.0_43-b01) Java HotSpot(TM) 64-Bit Server VM (build 20.14-b01, mixed mode) Windows 7 - CQL running in Cygwin. Python 2.7.3 Reporter: Ben Sykes Assignee: Mikhail Stepura Priority: Minor Attachments: cassandra-2.0-6242.patch After using CREATE USER to create a new user, the LIST USERS command returns nothing to the console. After removing this user again, the command works as expected. {code} $ ./cqlsh -u cassandra -p cassandra Connected to Test Cluster at localhost:9160. [cqlsh 4.0.1 | Cassandra 2.0.1 | CQL spec 3.1.1 | Thrift protocol 19.37.0] Use HELP for help. cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh CREATE USER bob WITH PASSWORD 'example' NOSUPERUSER; cqlsh LIST USERS; cqlsh SELECT * FROM system_auth.users; name | super ---+--- bob | False cassandra | True (2 rows) cqlsh DROP USER bob; cqlsh LIST USERS; name | super ---+--- cassandra | True cqlsh {code} -- This message was sent by Atlassian JIRA (v6.1#6144)