[jira] [Updated] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-7278: -- Fix Version/s: 2.0.10 > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Fix For: 2.0.10 > > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[4/8] git commit: Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM operations to incorrect become full QUORUM patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7345
Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM operations to incorrect become full QUORUM patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7345 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2b973b9e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2b973b9e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2b973b9e Branch: refs/heads/trunk Commit: 2b973b9ed1c1c6bd1c5b0c756cae12db434f02c0 Parents: 900f29c Author: Jonathan Ellis Authored: Thu Jun 26 21:35:50 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 21:35:50 2014 -0500 -- CHANGES.txt | 2 ++ src/java/org/apache/cassandra/service/StorageProxy.java | 11 ++- 2 files changed, 8 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2b973b9e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index c74b27d..365722f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,7 @@ 2.0.10 * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + * Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM + operations to incorrect become full QUORUM (CASSANDRA-7345) 2.0.9 http://git-wip-us.apache.org/repos/asf/cassandra/blob/2b973b9e/src/java/org/apache/cassandra/service/StorageProxy.java -- diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 3b10cff..c606d75 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -217,7 +217,7 @@ public class StorageProxy implements StorageProxyMBean List liveEndpoints = p.left; int requiredParticipants = p.right; -UUID ballot = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos); +UUID ballot = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos, consistencyForCommit); // read the current values and check they validate the conditions Tracing.trace("Reading existing values for CAS precondition"); @@ -302,7 +302,7 @@ public class StorageProxy implements StorageProxyMBean * @return the Paxos ballot promised by the replicas if no in-progress requests were seen and a quorum of * nodes have seen the mostRecentCommit. Otherwise, return null. */ -private static UUID beginAndRepairPaxos(long start, ByteBuffer key, CFMetaData metadata, List liveEndpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos) +private static UUID beginAndRepairPaxos(long start, ByteBuffer key, CFMetaData metadata, List liveEndpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos, ConsistencyLevel consistencyForCommit) throws WriteTimeoutException { long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout()); @@ -338,7 +338,7 @@ public class StorageProxy implements StorageProxyMBean Commit refreshedInProgress = Commit.newProposal(inProgress.key, ballot, inProgress.update); if (proposePaxos(refreshedInProgress, liveEndpoints, requiredParticipants, false, consistencyForPaxos)) { -commitPaxos(refreshedInProgress, ConsistencyLevel.QUORUM); +commitPaxos(refreshedInProgress, consistencyForCommit); } else { @@ -1140,16 +1140,17 @@ public class StorageProxy implements StorageProxyMBean int requiredParticipants = p.right; // does the work of applying in-progress writes; throws UAE or timeout if it can't +final ConsistencyLevel consistencyForCommitOrFetch = consistency_level == ConsistencyLevel.LOCAL_SERIAL ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM; try { -beginAndRepairPaxos(start, command.key, metadata, liveEndpoints, requiredParticipants, consistency_level); +beginAndRepairPaxos(start, command.key, metadata, liveEndpoints, requiredParticipants, consistency_level, consistencyForCommitOrFetch); } catch (WriteTimeoutException e) { throw new ReadTimeoutException(consistency_level, 0, consistency_level.blockFor(Keyspace.open(command.ksName)), false); } -rows = fetchRows(commands, consistency_level == Con
[6/8] git commit: merge from 2.0
merge from 2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8aa2b791 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8aa2b791 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8aa2b791 Branch: refs/heads/trunk Commit: 8aa2b791d946f6308b6e672293b2c1e0f92f546a Parents: c18e740 2b973b9 Author: Jonathan Ellis Authored: Thu Jun 26 21:36:55 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 21:36:55 2014 -0500 -- CHANGES.txt | 3 +++ src/java/org/apache/cassandra/service/StorageProxy.java | 11 ++- 2 files changed, 9 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8aa2b791/CHANGES.txt -- diff --cc CHANGES.txt index 971a33e,365722f..95921ff --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,38 -1,29 +1,41 @@@ -2.0.10 +2.1.0 +Merged from 2.0: + * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394) + * Support DISTINCT for static columns and fix behaviour when DISTINC is + not use (CASSANDRA-7305). + * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + * Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM +operations to incorrect become full QUORUM (CASSANDRA-7345) -2.0.9 - * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394) - * Fix assertion error in CL.ANY timeout handling (CASSANDRA-7364) - * Handle empty CFs in Memtable#maybeUpdateLiveRatio() (CASSANDRA-7401) +2.1.0-rc2 + * Fix heap size calculation for CompoundSparseCellName and + CompoundSparseCellName.WithCollection (CASSANDRA-7421) + * Allow counter mutations in UNLOGGED batches (CASSANDRA-7351) + * Modify reconcile logic to always pick a tombstone over a counter cell + (CASSANDRA-7346) + * Avoid incremental compaction on Windows (CASSANDRA-7365) + * Fix exception when querying a composite-keyed table with a collection index + (CASSANDRA-7372) + * Use node's host id in place of counter ids (CASSANDRA-7366) * Fix native protocol CAS batches (CASSANDRA-7337) + * Reduce likelihood of contention on local paxos locking (CASSANDRA-7359) + * Upgrade to Pig 0.12.1 (CASSANDRA-6556) + * Make sure we clear out repair sessions from netstats (CASSANDRA-7329) + * Don't fail streams on failure detector downs (CASSANDRA-3569) + * Add optional keyspace to DROP INDEX statement (CASSANDRA-7314) + * Reduce run time for CQL tests (CASSANDRA-7327) + * Fix heap size calculation on Windows (CASSANDRA-7352, 7353) + * RefCount native frames from netty (CASSANDRA-7245) + * Use tarball dir instead of /var for default paths (CASSANDRA-7136) + * Remove rows_per_partition_to_cache keyword (CASSANDRA-7193) + * Fix schema change response in native protocol v3 (CASSANDRA-7413) +Merged from 2.0: + * Fix assertion error in CL.ANY timeout handling (CASSANDRA-7364) * Add per-CF range read request latency metrics (CASSANDRA-7338) * Fix NPE in StreamTransferTask.createMessageForRetry() (CASSANDRA-7323) - * Add conditional CREATE/DROP USER support (CASSANDRA-7264) - * Swap local and global default read repair chances (CASSANDRA-7320) - * Add missing iso8601 patterns for date strings (CASSANDRA-6973) - * Support selecting multiple rows in a partition using IN (CASSANDRA-6875) - * cqlsh: always emphasize the partition key in DESC output (CASSANDRA-7274) - * Copy compaction options to make sure they are reloaded (CASSANDRA-7290) - * Add option to do more aggressive tombstone compactions (CASSANDRA-6563) - * Don't try to compact already-compacting files in HHOM (CASSANDRA-7288) - * Add authentication support to shuffle (CASSANDRA-6484) - * Cqlsh counts non-empty lines for "Blank lines" warning (CASSANDRA-7325) * Make StreamSession#closeSession() idempotent (CASSANDRA-7262) * Fix infinite loop on exception while streaming (CASSANDRA-7330) - * Reference sstables before populating key cache (CASSANDRA-7234) * Account for range tombstones in min/max column names (CASSANDRA-7235) * Improve sub range repair validation (CASSANDRA-7317) * Accept subtypes for function results, type casts (CASSANDRA-6766) http://git-wip-us.apache.org/repos/asf/cassandra/blob/8aa2b791/src/java/org/apache/cassandra/service/StorageProxy.java --
[8/8] git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/beea5d68 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/beea5d68 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/beea5d68 Branch: refs/heads/trunk Commit: beea5d68cfadc2065c70bb605db3105130a9c351 Parents: 652a7ea 8aa2b79 Author: Jonathan Ellis Authored: Thu Jun 26 21:37:05 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 21:37:05 2014 -0500 -- CHANGES.txt | 3 +++ src/java/org/apache/cassandra/service/StorageProxy.java | 11 ++- 2 files changed, 9 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/beea5d68/CHANGES.txt --
[2/8] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c18e740c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c18e740c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c18e740c Branch: refs/heads/trunk Commit: c18e740c8d26798c70cc1145f9ddea6ac505d7b1 Parents: e8500c7 900f29c Author: Jonathan Ellis Authored: Thu Jun 26 19:18:52 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 19:18:52 2014 -0500 -- --
[5/8] git commit: Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM operations to incorrect become full QUORUM patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7345
Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM operations to incorrect become full QUORUM patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7345 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2b973b9e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2b973b9e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2b973b9e Branch: refs/heads/cassandra-2.1 Commit: 2b973b9ed1c1c6bd1c5b0c756cae12db434f02c0 Parents: 900f29c Author: Jonathan Ellis Authored: Thu Jun 26 21:35:50 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 21:35:50 2014 -0500 -- CHANGES.txt | 2 ++ src/java/org/apache/cassandra/service/StorageProxy.java | 11 ++- 2 files changed, 8 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2b973b9e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index c74b27d..365722f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,7 @@ 2.0.10 * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + * Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM + operations to incorrect become full QUORUM (CASSANDRA-7345) 2.0.9 http://git-wip-us.apache.org/repos/asf/cassandra/blob/2b973b9e/src/java/org/apache/cassandra/service/StorageProxy.java -- diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 3b10cff..c606d75 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -217,7 +217,7 @@ public class StorageProxy implements StorageProxyMBean List liveEndpoints = p.left; int requiredParticipants = p.right; -UUID ballot = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos); +UUID ballot = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos, consistencyForCommit); // read the current values and check they validate the conditions Tracing.trace("Reading existing values for CAS precondition"); @@ -302,7 +302,7 @@ public class StorageProxy implements StorageProxyMBean * @return the Paxos ballot promised by the replicas if no in-progress requests were seen and a quorum of * nodes have seen the mostRecentCommit. Otherwise, return null. */ -private static UUID beginAndRepairPaxos(long start, ByteBuffer key, CFMetaData metadata, List liveEndpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos) +private static UUID beginAndRepairPaxos(long start, ByteBuffer key, CFMetaData metadata, List liveEndpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos, ConsistencyLevel consistencyForCommit) throws WriteTimeoutException { long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout()); @@ -338,7 +338,7 @@ public class StorageProxy implements StorageProxyMBean Commit refreshedInProgress = Commit.newProposal(inProgress.key, ballot, inProgress.update); if (proposePaxos(refreshedInProgress, liveEndpoints, requiredParticipants, false, consistencyForPaxos)) { -commitPaxos(refreshedInProgress, ConsistencyLevel.QUORUM); +commitPaxos(refreshedInProgress, consistencyForCommit); } else { @@ -1140,16 +1140,17 @@ public class StorageProxy implements StorageProxyMBean int requiredParticipants = p.right; // does the work of applying in-progress writes; throws UAE or timeout if it can't +final ConsistencyLevel consistencyForCommitOrFetch = consistency_level == ConsistencyLevel.LOCAL_SERIAL ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM; try { -beginAndRepairPaxos(start, command.key, metadata, liveEndpoints, requiredParticipants, consistency_level); +beginAndRepairPaxos(start, command.key, metadata, liveEndpoints, requiredParticipants, consistency_level, consistencyForCommitOrFetch); } catch (WriteTimeoutException e) { throw new ReadTimeoutException(consistency_level, 0, consistency_level.blockFor(Keyspace.open(command.ksName)), false); } -rows = fetchRows(commands, consistency_level
[7/8] git commit: merge from 2.0
merge from 2.0 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8aa2b791 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8aa2b791 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8aa2b791 Branch: refs/heads/cassandra-2.1 Commit: 8aa2b791d946f6308b6e672293b2c1e0f92f546a Parents: c18e740 2b973b9 Author: Jonathan Ellis Authored: Thu Jun 26 21:36:55 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 21:36:55 2014 -0500 -- CHANGES.txt | 3 +++ src/java/org/apache/cassandra/service/StorageProxy.java | 11 ++- 2 files changed, 9 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8aa2b791/CHANGES.txt -- diff --cc CHANGES.txt index 971a33e,365722f..95921ff --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,38 -1,29 +1,41 @@@ -2.0.10 +2.1.0 +Merged from 2.0: + * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394) + * Support DISTINCT for static columns and fix behaviour when DISTINC is + not use (CASSANDRA-7305). + * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + * Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM +operations to incorrect become full QUORUM (CASSANDRA-7345) -2.0.9 - * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394) - * Fix assertion error in CL.ANY timeout handling (CASSANDRA-7364) - * Handle empty CFs in Memtable#maybeUpdateLiveRatio() (CASSANDRA-7401) +2.1.0-rc2 + * Fix heap size calculation for CompoundSparseCellName and + CompoundSparseCellName.WithCollection (CASSANDRA-7421) + * Allow counter mutations in UNLOGGED batches (CASSANDRA-7351) + * Modify reconcile logic to always pick a tombstone over a counter cell + (CASSANDRA-7346) + * Avoid incremental compaction on Windows (CASSANDRA-7365) + * Fix exception when querying a composite-keyed table with a collection index + (CASSANDRA-7372) + * Use node's host id in place of counter ids (CASSANDRA-7366) * Fix native protocol CAS batches (CASSANDRA-7337) + * Reduce likelihood of contention on local paxos locking (CASSANDRA-7359) + * Upgrade to Pig 0.12.1 (CASSANDRA-6556) + * Make sure we clear out repair sessions from netstats (CASSANDRA-7329) + * Don't fail streams on failure detector downs (CASSANDRA-3569) + * Add optional keyspace to DROP INDEX statement (CASSANDRA-7314) + * Reduce run time for CQL tests (CASSANDRA-7327) + * Fix heap size calculation on Windows (CASSANDRA-7352, 7353) + * RefCount native frames from netty (CASSANDRA-7245) + * Use tarball dir instead of /var for default paths (CASSANDRA-7136) + * Remove rows_per_partition_to_cache keyword (CASSANDRA-7193) + * Fix schema change response in native protocol v3 (CASSANDRA-7413) +Merged from 2.0: + * Fix assertion error in CL.ANY timeout handling (CASSANDRA-7364) * Add per-CF range read request latency metrics (CASSANDRA-7338) * Fix NPE in StreamTransferTask.createMessageForRetry() (CASSANDRA-7323) - * Add conditional CREATE/DROP USER support (CASSANDRA-7264) - * Swap local and global default read repair chances (CASSANDRA-7320) - * Add missing iso8601 patterns for date strings (CASSANDRA-6973) - * Support selecting multiple rows in a partition using IN (CASSANDRA-6875) - * cqlsh: always emphasize the partition key in DESC output (CASSANDRA-7274) - * Copy compaction options to make sure they are reloaded (CASSANDRA-7290) - * Add option to do more aggressive tombstone compactions (CASSANDRA-6563) - * Don't try to compact already-compacting files in HHOM (CASSANDRA-7288) - * Add authentication support to shuffle (CASSANDRA-6484) - * Cqlsh counts non-empty lines for "Blank lines" warning (CASSANDRA-7325) * Make StreamSession#closeSession() idempotent (CASSANDRA-7262) * Fix infinite loop on exception while streaming (CASSANDRA-7330) - * Reference sstables before populating key cache (CASSANDRA-7234) * Account for range tombstones in min/max column names (CASSANDRA-7235) * Improve sub range repair validation (CASSANDRA-7317) * Accept subtypes for function results, type casts (CASSANDRA-6766) http://git-wip-us.apache.org/repos/asf/cassandra/blob/8aa2b791/src/java/org/apache/cassandra/service/StorageProxy.java --
[1/8] git commit: Fix race in FileCacheService RemovalListener patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7278
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 900f29c7f -> 2b973b9ed refs/heads/cassandra-2.1 c18e740c8 -> 8aa2b791d refs/heads/trunk 652a7ea71 -> beea5d68c Fix race in FileCacheService RemovalListener patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7278 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/900f29c7 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/900f29c7 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/900f29c7 Branch: refs/heads/trunk Commit: 900f29c7f7e1d563c4b0c63eae0da8877766813f Parents: ac1d5bd Author: Jonathan Ellis Authored: Thu Jun 26 19:16:12 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 19:17:58 2014 -0500 -- CHANGES.txt | 4 src/java/org/apache/cassandra/service/FileCacheService.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/900f29c7/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 2b3ace3..c74b27d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,7 @@ +2.0.10 + * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + + 2.0.9 * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394) * Fix assertion error in CL.ANY timeout handling (CASSANDRA-7364) http://git-wip-us.apache.org/repos/asf/cassandra/blob/900f29c7/src/java/org/apache/cassandra/service/FileCacheService.java -- diff --git a/src/java/org/apache/cassandra/service/FileCacheService.java b/src/java/org/apache/cassandra/service/FileCacheService.java index faa53f0..59b5548 100644 --- a/src/java/org/apache/cassandra/service/FileCacheService.java +++ b/src/java/org/apache/cassandra/service/FileCacheService.java @@ -69,7 +69,7 @@ public class FileCacheService if (cachedInstances.size() > 0) logger.debug("Evicting cold readers for {}", cachedInstances.peek().getPath()); -for (RandomAccessReader reader : cachedInstances) +for (RandomAccessReader reader = cachedInstances.poll(); reader != null; reader = cachedInstances.poll()) { memoryUsage.addAndGet(-1 * reader.getTotalBufferSize()); reader.deallocate();
[3/8] git commit: Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM operations to incorrect become full QUORUM patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7345
Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM operations to incorrect become full QUORUM patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7345 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2b973b9e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2b973b9e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2b973b9e Branch: refs/heads/cassandra-2.0 Commit: 2b973b9ed1c1c6bd1c5b0c756cae12db434f02c0 Parents: 900f29c Author: Jonathan Ellis Authored: Thu Jun 26 21:35:50 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 21:35:50 2014 -0500 -- CHANGES.txt | 2 ++ src/java/org/apache/cassandra/service/StorageProxy.java | 11 ++- 2 files changed, 8 insertions(+), 5 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2b973b9e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index c74b27d..365722f 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,7 @@ 2.0.10 * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + * Fix inconsistent use of consistencyForCommit that allowed LOCAL_QUORUM + operations to incorrect become full QUORUM (CASSANDRA-7345) 2.0.9 http://git-wip-us.apache.org/repos/asf/cassandra/blob/2b973b9e/src/java/org/apache/cassandra/service/StorageProxy.java -- diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java index 3b10cff..c606d75 100644 --- a/src/java/org/apache/cassandra/service/StorageProxy.java +++ b/src/java/org/apache/cassandra/service/StorageProxy.java @@ -217,7 +217,7 @@ public class StorageProxy implements StorageProxyMBean List liveEndpoints = p.left; int requiredParticipants = p.right; -UUID ballot = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos); +UUID ballot = beginAndRepairPaxos(start, key, metadata, liveEndpoints, requiredParticipants, consistencyForPaxos, consistencyForCommit); // read the current values and check they validate the conditions Tracing.trace("Reading existing values for CAS precondition"); @@ -302,7 +302,7 @@ public class StorageProxy implements StorageProxyMBean * @return the Paxos ballot promised by the replicas if no in-progress requests were seen and a quorum of * nodes have seen the mostRecentCommit. Otherwise, return null. */ -private static UUID beginAndRepairPaxos(long start, ByteBuffer key, CFMetaData metadata, List liveEndpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos) +private static UUID beginAndRepairPaxos(long start, ByteBuffer key, CFMetaData metadata, List liveEndpoints, int requiredParticipants, ConsistencyLevel consistencyForPaxos, ConsistencyLevel consistencyForCommit) throws WriteTimeoutException { long timeout = TimeUnit.MILLISECONDS.toNanos(DatabaseDescriptor.getCasContentionTimeout()); @@ -338,7 +338,7 @@ public class StorageProxy implements StorageProxyMBean Commit refreshedInProgress = Commit.newProposal(inProgress.key, ballot, inProgress.update); if (proposePaxos(refreshedInProgress, liveEndpoints, requiredParticipants, false, consistencyForPaxos)) { -commitPaxos(refreshedInProgress, ConsistencyLevel.QUORUM); +commitPaxos(refreshedInProgress, consistencyForCommit); } else { @@ -1140,16 +1140,17 @@ public class StorageProxy implements StorageProxyMBean int requiredParticipants = p.right; // does the work of applying in-progress writes; throws UAE or timeout if it can't +final ConsistencyLevel consistencyForCommitOrFetch = consistency_level == ConsistencyLevel.LOCAL_SERIAL ? ConsistencyLevel.LOCAL_QUORUM : ConsistencyLevel.QUORUM; try { -beginAndRepairPaxos(start, command.key, metadata, liveEndpoints, requiredParticipants, consistency_level); +beginAndRepairPaxos(start, command.key, metadata, liveEndpoints, requiredParticipants, consistency_level, consistencyForCommitOrFetch); } catch (WriteTimeoutException e) { throw new ReadTimeoutException(consistency_level, 0, consistency_level.blockFor(Keyspace.open(command.ksName)), false); } -rows = fetchRows(commands, consistency_level
[jira] [Resolved] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-7278. --- Resolution: Fixed Reviewer: Jonathan Ellis you're right. committed as 900f29c7f7e1d563c4b0c63eae0da8877766813f > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[2/3] git commit: Fix race in FileCacheService RemovalListener patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7278
Fix race in FileCacheService RemovalListener patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7278 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/900f29c7 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/900f29c7 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/900f29c7 Branch: refs/heads/cassandra-2.1 Commit: 900f29c7f7e1d563c4b0c63eae0da8877766813f Parents: ac1d5bd Author: Jonathan Ellis Authored: Thu Jun 26 19:16:12 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 19:17:58 2014 -0500 -- CHANGES.txt | 4 src/java/org/apache/cassandra/service/FileCacheService.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/900f29c7/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 2b3ace3..c74b27d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,7 @@ +2.0.10 + * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + + 2.0.9 * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394) * Fix assertion error in CL.ANY timeout handling (CASSANDRA-7364) http://git-wip-us.apache.org/repos/asf/cassandra/blob/900f29c7/src/java/org/apache/cassandra/service/FileCacheService.java -- diff --git a/src/java/org/apache/cassandra/service/FileCacheService.java b/src/java/org/apache/cassandra/service/FileCacheService.java index faa53f0..59b5548 100644 --- a/src/java/org/apache/cassandra/service/FileCacheService.java +++ b/src/java/org/apache/cassandra/service/FileCacheService.java @@ -69,7 +69,7 @@ public class FileCacheService if (cachedInstances.size() > 0) logger.debug("Evicting cold readers for {}", cachedInstances.peek().getPath()); -for (RandomAccessReader reader : cachedInstances) +for (RandomAccessReader reader = cachedInstances.poll(); reader != null; reader = cachedInstances.poll()) { memoryUsage.addAndGet(-1 * reader.getTotalBufferSize()); reader.deallocate();
[3/3] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c18e740c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c18e740c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c18e740c Branch: refs/heads/cassandra-2.1 Commit: c18e740c8d26798c70cc1145f9ddea6ac505d7b1 Parents: e8500c7 900f29c Author: Jonathan Ellis Authored: Thu Jun 26 19:18:52 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 19:18:52 2014 -0500 -- --
[1/3] git commit: Fix race in FileCacheService RemovalListener patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7278
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 ac1d5bd56 -> 900f29c7f refs/heads/cassandra-2.1 e8500c72c -> c18e740c8 Fix race in FileCacheService RemovalListener patch by Sankalp Kohli; reviewed by jbellis for CASSANDRA-7278 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/900f29c7 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/900f29c7 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/900f29c7 Branch: refs/heads/cassandra-2.0 Commit: 900f29c7f7e1d563c4b0c63eae0da8877766813f Parents: ac1d5bd Author: Jonathan Ellis Authored: Thu Jun 26 19:16:12 2014 -0500 Committer: Jonathan Ellis Committed: Thu Jun 26 19:17:58 2014 -0500 -- CHANGES.txt | 4 src/java/org/apache/cassandra/service/FileCacheService.java | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/900f29c7/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 2b3ace3..c74b27d 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,7 @@ +2.0.10 + * Fix race in FileCacheService RemovalListener (CASSANDRA-7278) + + 2.0.9 * Fix CC#collectTimeOrderedData() tombstone optimisations (CASSANDRA-7394) * Fix assertion error in CL.ANY timeout handling (CASSANDRA-7364) http://git-wip-us.apache.org/repos/asf/cassandra/blob/900f29c7/src/java/org/apache/cassandra/service/FileCacheService.java -- diff --git a/src/java/org/apache/cassandra/service/FileCacheService.java b/src/java/org/apache/cassandra/service/FileCacheService.java index faa53f0..59b5548 100644 --- a/src/java/org/apache/cassandra/service/FileCacheService.java +++ b/src/java/org/apache/cassandra/service/FileCacheService.java @@ -69,7 +69,7 @@ public class FileCacheService if (cachedInstances.size() > 0) logger.debug("Evicting cold readers for {}", cachedInstances.peek().getPath()); -for (RandomAccessReader reader : cachedInstances) +for (RandomAccessReader reader = cachedInstances.poll(); reader != null; reader = cachedInstances.poll()) { memoryUsage.addAndGet(-1 * reader.getTotalBufferSize()); reader.deallocate();
[jira] [Updated] (CASSANDRA-7238) Nodetool Status performance is much slower with VNodes On
[ https://issues.apache.org/jira/browse/CASSANDRA-7238?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Russell Alexander Spitzer updated CASSANDRA-7238: - Reproduced In: 2.1 beta2, 2.0.8 (was: 2.1 beta2) > Nodetool Status performance is much slower with VNodes On > - > > Key: CASSANDRA-7238 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7238 > Project: Cassandra > Issue Type: Bug > Components: Tools > Environment: 1000 M1.Large Ubuntu 12.04 >Reporter: Russell Alexander Spitzer >Priority: Minor > > Nodetool status on a 1000 Node cluster without vnodes returns in several > seconds. With vnodes on (256) there are OOM errors with the default XMX of > 32. Adjusting the XMX to 128 allows nodetool status to complete but the > execution takes roughly 10 minutes. > Tested > {code} > XMX| Status > 32 |OOM > 64 |OOM: GC Overhead > 128|Finishes in ~10 minutes > 500|Finishes in ~10 minutes > 1000 |Finishes in ~10 minutes > {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7459) CqlRecordWriter doesn't close socket connections
[ https://issues.apache.org/jira/browse/CASSANDRA-7459?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045341#comment-14045341 ] Peter Williams commented on CASSANDRA-7459: --- The attached patch is pretty simple, it just calls the existing `closeInternal` method before shutting down the client thread. > CqlRecordWriter doesn't close socket connections > > > Key: CASSANDRA-7459 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7459 > Project: Cassandra > Issue Type: Bug >Reporter: Peter Williams > Fix For: 2.0.9 > > Attachments: cassandra-2.0-7459.txt > > > Using spark with `CqlOutputFormat` results in thousands of open connections > to cassandra (and then crashes unless they get garbage collected fast > enough). This is because each save to cassandra task creates a > `CqlRecordWriter` which opens a bunch of connections but calling close on > those writers does not actually close their socket connections to cassandra. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Issue Comment Deleted] (CASSANDRA-7459) CqlRecordWriter doesn't close socket connections
[ https://issues.apache.org/jira/browse/CASSANDRA-7459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Williams updated CASSANDRA-7459: -- Comment: was deleted (was: make sure `CqlRecordWriter$RangeClient` closes its internal socket connection when it is closed.) > CqlRecordWriter doesn't close socket connections > > > Key: CASSANDRA-7459 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7459 > Project: Cassandra > Issue Type: Bug >Reporter: Peter Williams > Fix For: 2.0.9 > > Attachments: cassandra-2.0-7459.txt > > > Using spark with `CqlOutputFormat` results in thousands of open connections > to cassandra (and then crashes unless they get garbage collected fast > enough). This is because each save to cassandra task creates a > `CqlRecordWriter` which opens a bunch of connections but calling close on > those writers does not actually close their socket connections to cassandra. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Issue Comment Deleted] (CASSANDRA-7459) CqlRecordWriter doesn't close socket connections
[ https://issues.apache.org/jira/browse/CASSANDRA-7459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Williams updated CASSANDRA-7459: -- Comment: was deleted (was: Close socket connections when CqlRecordWriter$RangeClient thread stops.) > CqlRecordWriter doesn't close socket connections > > > Key: CASSANDRA-7459 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7459 > Project: Cassandra > Issue Type: Bug >Reporter: Peter Williams > Fix For: 2.0.9 > > Attachments: cassandra-2.0-7459.txt > > > Using spark with `CqlOutputFormat` results in thousands of open connections > to cassandra (and then crashes unless they get garbage collected fast > enough). This is because each save to cassandra task creates a > `CqlRecordWriter` which opens a bunch of connections but calling close on > those writers does not actually close their socket connections to cassandra. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Issue Comment Deleted] (CASSANDRA-7459) CqlRecordWriter doesn't close socket connections
[ https://issues.apache.org/jira/browse/CASSANDRA-7459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Williams updated CASSANDRA-7459: -- Comment: was deleted (was: Close underlying sockets when a `CqlRecordWriter` is closed.) > CqlRecordWriter doesn't close socket connections > > > Key: CASSANDRA-7459 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7459 > Project: Cassandra > Issue Type: Bug >Reporter: Peter Williams > Fix For: 2.0.9 > > Attachments: cassandra-2.0-7459.txt > > > Using spark with `CqlOutputFormat` results in thousands of open connections > to cassandra (and then crashes unless they get garbage collected fast > enough). This is because each save to cassandra task creates a > `CqlRecordWriter` which opens a bunch of connections but calling close on > those writers does not actually close their socket connections to cassandra. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7459) CqlRecordWriter doesn't close socket connections
[ https://issues.apache.org/jira/browse/CASSANDRA-7459?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Peter Williams updated CASSANDRA-7459: -- Attachment: cassandra-2.0-7459.txt make sure `CqlRecordWriter$RangeClient` closes its internal socket connection when it is closed. > CqlRecordWriter doesn't close socket connections > > > Key: CASSANDRA-7459 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7459 > Project: Cassandra > Issue Type: Bug >Reporter: Peter Williams > Fix For: 2.0.9 > > Attachments: cassandra-2.0-7459.txt > > > Using spark with `CqlOutputFormat` results in thousands of open connections > to cassandra (and then crashes unless they get garbage collected fast > enough). This is because each save to cassandra task creates a > `CqlRecordWriter` which opens a bunch of connections but calling close on > those writers does not actually close their socket connections to cassandra. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045328#comment-14045328 ] sankalp kohli commented on CASSANDRA-7278: -- I think my fix will work. Here is why while ((reader = cachedInstances.poll()) != null) Here cachedInstances is a ConcurrentLinkedQueue. This will enforce that poll only gives a reader to one caller irrespective of GC pause. So its either the thread doing the get() will get it or the removalListener. So if FileCacheService.get retrieves the reader from the queue, the removalListener won't get it. And if removalListener gets it, the get() method won't get it. > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7459) CqlRecordWriter doesn't close socket connections
Peter Williams created CASSANDRA-7459: - Summary: CqlRecordWriter doesn't close socket connections Key: CASSANDRA-7459 URL: https://issues.apache.org/jira/browse/CASSANDRA-7459 Project: Cassandra Issue Type: Bug Reporter: Peter Williams Using spark with `CqlOutputFormat` results in thousands of open connections to cassandra (and then crashes unless they get garbage collected fast enough). This is because each save to cassandra task creates a `CqlRecordWriter` which opens a bunch of connections but calling close on those writers does not actually close their socket connections to cassandra. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7414) After cleanup we can end up with non-compacting high level sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-7414?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045300#comment-14045300 ] Yuki Morishita commented on CASSANDRA-7414: --- Is it good to loop beyond targetLevel in getOverlappingStarvedSSTables? I think we can ignore lower leveles. Oherwise +1. > After cleanup we can end up with non-compacting high level sstables > --- > > Key: CASSANDRA-7414 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7414 > Project: Cassandra > Issue Type: Bug >Reporter: Marcus Eriksson >Assignee: Marcus Eriksson >Priority: Minor > Attachments: > 0001-Include-high-level-sstables-in-lower-level-compactio-v2.patch, > 0001-Include-high-level-sstables-in-lower-level-compactio.patch > > > If we run cleanup (or increase sstable size) on a node with LCS, we could end > up with a bunch of sstables in higher levels that are "never" compacted. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7457) java.lang.IllegalArgumentException when performing query from CQL
[ https://issues.apache.org/jira/browse/CASSANDRA-7457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-7457: -- Reproduced In: 2.0.7 Fix Version/s: (was: 2.0.7) > java.lang.IllegalArgumentException when performing query from CQL > - > > Key: CASSANDRA-7457 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7457 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Linux Mint 16 Petro >Reporter: Jeff Sabin > > Using python driver 2.0.2 with Cassandra 2.0.7. > Created schema listed below. Inserted data into multiple partitions. > Performed several queries and everything works great. After more data was > inserted, queries without WHERE clauses stopped working and returns error. > Restarting Cassandra does not fix the problem. > Query: > session.execute("SELECT * FROM timeseries.data_points;") > Error: > cassandra.protocol.ServerError: message="java.lang.IllegalArgumentException"> > Error in Cassandra log file: > ERROR 15:14:53,442 Unexpected exception during request > java.lang.IllegalArgumentException > at java.nio.Buffer.limit(Buffer.java:267) > at > org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:63) > at > org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72) > at > org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138) > at > org.apache.cassandra.cql3.statements.SelectStatement.processColumnFamily(SelectStatement.java:973) > at > org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:953) > at > org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:274) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:232) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:62) > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:158) > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:175) > at > org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:119) > at > org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:304) > at > org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) > at > org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) > at > org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) > at > org.jboss.netty.handler.execution.ChannelUpstreamEventRunnable.doRun(ChannelUpstreamEventRunnable.java:43) > at > org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunnable.java:67) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Schema: > session.execute("""CREATE TABLE timeseries.data_points ( > metric_name text, > value int, > time timestamp, > tags text, > PRIMARY KEY ((metric_name, tags), time) > ) WITH COMPACT STORAGE;""") -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7457) java.lang.IllegalArgumentException when performing query from CQL
[ https://issues.apache.org/jira/browse/CASSANDRA-7457?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis reassigned CASSANDRA-7457: - Assignee: Tyler Hobbs > java.lang.IllegalArgumentException when performing query from CQL > - > > Key: CASSANDRA-7457 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7457 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Linux Mint 16 Petro >Reporter: Jeff Sabin >Assignee: Tyler Hobbs > > Using python driver 2.0.2 with Cassandra 2.0.7. > Created schema listed below. Inserted data into multiple partitions. > Performed several queries and everything works great. After more data was > inserted, queries without WHERE clauses stopped working and returns error. > Restarting Cassandra does not fix the problem. > Query: > session.execute("SELECT * FROM timeseries.data_points;") > Error: > cassandra.protocol.ServerError: message="java.lang.IllegalArgumentException"> > Error in Cassandra log file: > ERROR 15:14:53,442 Unexpected exception during request > java.lang.IllegalArgumentException > at java.nio.Buffer.limit(Buffer.java:267) > at > org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:63) > at > org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72) > at > org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138) > at > org.apache.cassandra.cql3.statements.SelectStatement.processColumnFamily(SelectStatement.java:973) > at > org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:953) > at > org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:274) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:232) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:62) > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:158) > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:175) > at > org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:119) > at > org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:304) > at > org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) > at > org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) > at > org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) > at > org.jboss.netty.handler.execution.ChannelUpstreamEventRunnable.doRun(ChannelUpstreamEventRunnable.java:43) > at > org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunnable.java:67) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > Schema: > session.execute("""CREATE TABLE timeseries.data_points ( > metric_name text, > value int, > time timestamp, > tags text, > PRIMARY KEY ((metric_name, tags), time) > ) WITH COMPACT STORAGE;""") -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-7456) Hinted Handoff: java.lang.ArithmeticException: / by zero
[ https://issues.apache.org/jira/browse/CASSANDRA-7456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-7456. --- Resolution: Duplicate > Hinted Handoff: java.lang.ArithmeticException: / by zero > > > Key: CASSANDRA-7456 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7456 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: 2.0.6 >Reporter: sankalp kohli >Assignee: sankalp kohli >Priority: Minor > > We are seeing this exception. This is similar to CASSANDRA-5990 but here the > exception is coming because hintStore.getMeanRowSize() is returning zero i > think. This is blowing up in > return Math.max(2, Math.min(PAGE_SIZE, > DatabaseDescriptor.getInMemoryCompactionLimit() / averageColumnSize)); > java.lang.ArithmeticException: / by zero > at > org.apache.cassandra.db.HintedHandOffManager.calculatePageSize(HintedHandOffManager.java:504) > at > org.apache.cassandra.db.HintedHandOffManager.doDeliverHintsToEndpoint(HintedHandOffManager.java:352) > at > org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:331) > at > org.apache.cassandra.db.HintedHandOffManager.access$300(HintedHandOffManager.java:92) > at > org.apache.cassandra.db.HintedHandOffManager$5.run(HintedHandOffManager.java:558) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045279#comment-14045279 ] Jonathan Ellis commented on CASSANDRA-7278: --- I think you've evaluated the problem correctly, but doesn't your fix still have a window for a race? I can check that buffer != null in get(), but it could still get nulled out by the removalListener thread after being returned. > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7458) functional indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-7458?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-7458: -- Description: Indexing information derived from the row can be powerful. For example, using the hypothetical {{extract_date}} function, {code} create table ticks ( symbol text, ticked_at datetime, price int, tags set, PRIMARY KEY (symbol, ticked_at) ); CREATE INDEX ticks_by_day ON ticks(extract_date(ticked_at)); {code} http://www.postgresql.org/docs/9.3/static/indexes-expressional.html was: Indexing information derived from the row can be powerful. For example, using the hypothetical {{extract_date}} function, {code} create table ticks ( symbol text, ticked_at datetime, price int, tags set, PRIMARY KEY (symbol, ticked_at) ); CREATE INDEX ticks_by_day ON ticks(extract_date(ticked_at)); {code} See also http://www.postgresql.org/docs/9.3/static/sql-createindex.html > functional indexes > -- > > Key: CASSANDRA-7458 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7458 > Project: Cassandra > Issue Type: New Feature > Components: API, Core >Reporter: Jonathan Ellis > Fix For: 3.0 > > > Indexing information derived from the row can be powerful. For example, > using the hypothetical {{extract_date}} function, > {code} > create table ticks ( > symbol text, > ticked_at datetime, > price int, > tags set, > PRIMARY KEY (symbol, ticked_at) > ); > CREATE INDEX ticks_by_day ON ticks(extract_date(ticked_at)); > {code} > http://www.postgresql.org/docs/9.3/static/indexes-expressional.html -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7458) functional indexes
Jonathan Ellis created CASSANDRA-7458: - Summary: functional indexes Key: CASSANDRA-7458 URL: https://issues.apache.org/jira/browse/CASSANDRA-7458 Project: Cassandra Issue Type: New Feature Components: API, Core Reporter: Jonathan Ellis Fix For: 3.0 Indexing information derived from the row can be powerful. For example, using the hypothetical {{extract_date}} function, {code} create table ticks ( symbol text, ticked_at datetime, price int, tags set, PRIMARY KEY (symbol, ticked_at) ); CREATE INDEX ticks_by_day ON ticks(extract_date(ticked_at)); {code} See also http://www.postgresql.org/docs/9.3/static/sql-createindex.html -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6621) STCS fallback is not optimal when bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045237#comment-14045237 ] sankalp kohli commented on CASSANDRA-6621: -- +1 on [~yukim] comment. > STCS fallback is not optimal when bootstrapping > --- > > Key: CASSANDRA-6621 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6621 > Project: Cassandra > Issue Type: Improvement >Reporter: Bartłomiej Romański >Assignee: Marcus Eriksson >Priority: Minor > Labels: compaction, streaming > Fix For: 2.0.9 > > Attachments: 0001-option-to-disallow-L0-stcs.patch, > 0001-wip-keep-sstable-level-when-bootstrapping.patch > > > The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my > last comment here... > After streaming (e.g. during boostrap) Cassandra places all sstables at L0. > At the end of the process we end up with huge number of sstables at the > lowest level. > Currently, Cassandra falls back to STCS until the number of sstables at L0 > reaches the reasonable level (32 or something). > I'm not sure if falling back to STCS is the best way to handle this > particular situation. I've read the comment in the code and I'm aware why it > is a good thing to do if we have to many sstables at L0 as a result of too > many random inserts. We have a lot of sstables, each of them covers the whole > ring, there's simply no better option. > However, after the bootstrap situation looks a bit different. The loaded > sstables already have very small ranges! We just have to tidy up a bit and > everything should be OK. STCS ignores that completely and after a while we > have a bit less sstables but each of them covers the whole ring instead of > just a small part. I believe that in that case letting LCS do the job is a > better option that allowing STCS mix everything up before. > Is there a way to disable STCS fallback? I'd like to test that scenario in > practice during our next bootstrap... > Does Cassandra really have to put streamed sstables at L0? The only thing we > have to assure is that sstables at any given level do not overlap. If we > stream different regions from different nodes how can we get any overlaps? -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7457) java.lang.IllegalArgumentException when performing query from CQL
Jeff Sabin created CASSANDRA-7457: - Summary: java.lang.IllegalArgumentException when performing query from CQL Key: CASSANDRA-7457 URL: https://issues.apache.org/jira/browse/CASSANDRA-7457 Project: Cassandra Issue Type: Bug Components: Core Environment: Linux Mint 16 Petro Reporter: Jeff Sabin Fix For: 2.0.7 Using python driver 2.0.2 with Cassandra 2.0.7. Created schema listed below. Inserted data into multiple partitions. Performed several queries and everything works great. After more data was inserted, queries without WHERE clauses stopped working and returns error. Restarting Cassandra does not fix the problem. Query: session.execute("SELECT * FROM timeseries.data_points;") Error: cassandra.protocol.ServerError: Error in Cassandra log file: ERROR 15:14:53,442 Unexpected exception during request java.lang.IllegalArgumentException at java.nio.Buffer.limit(Buffer.java:267) at org.apache.cassandra.db.marshal.AbstractCompositeType.getBytes(AbstractCompositeType.java:63) at org.apache.cassandra.db.marshal.AbstractCompositeType.getWithShortLength(AbstractCompositeType.java:72) at org.apache.cassandra.db.marshal.AbstractCompositeType.split(AbstractCompositeType.java:138) at org.apache.cassandra.cql3.statements.SelectStatement.processColumnFamily(SelectStatement.java:973) at org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:953) at org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:274) at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:232) at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:62) at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:158) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:175) at org.apache.cassandra.transport.messages.QueryMessage.execute(QueryMessage.java:119) at org.apache.cassandra.transport.Message$Dispatcher.messageReceived(Message.java:304) at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70) at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:791) at org.jboss.netty.handler.execution.ChannelUpstreamEventRunnable.doRun(ChannelUpstreamEventRunnable.java:43) at org.jboss.netty.handler.execution.ChannelEventRunnable.run(ChannelEventRunnable.java:67) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) Schema: session.execute("""CREATE TABLE timeseries.data_points ( metric_name text, value int, time timestamp, tags text, PRIMARY KEY ((metric_name, tags), time) ) WITH COMPACT STORAGE;""") -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7254) NPE on startup if another Cassandra instance is already running
[ https://issues.apache.org/jira/browse/CASSANDRA-7254?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045223#comment-14045223 ] Brandon Williams commented on CASSANDRA-7254: - Yeah, I would put it in 2.0. > NPE on startup if another Cassandra instance is already running > --- > > Key: CASSANDRA-7254 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7254 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tyler Hobbs >Assignee: Chris Lohfink >Priority: Minor > Fix For: 2.1.0, 2.0.10 > > Attachments: 7254.txt > > > After CASSANDRA-7087, if you try to start cassandra while another instance is > already running, you'll see something like this: > {noformat} > $ bin/cassandra -f > Error: Exception thrown by the agent : java.lang.NullPointerException > {noformat} > This is probably a JVM bug, but we should confirm that, open a JVM ticket, > and see if we can give a more useful error message on the C* side. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7254) NPE on startup if another Cassandra instance is already running
[ https://issues.apache.org/jira/browse/CASSANDRA-7254?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-7254: Fix Version/s: 2.0.10 > NPE on startup if another Cassandra instance is already running > --- > > Key: CASSANDRA-7254 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7254 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tyler Hobbs >Assignee: Chris Lohfink >Priority: Minor > Fix For: 2.1.0, 2.0.10 > > Attachments: 7254.txt > > > After CASSANDRA-7087, if you try to start cassandra while another instance is > already running, you'll see something like this: > {noformat} > $ bin/cassandra -f > Error: Exception thrown by the agent : java.lang.NullPointerException > {noformat} > This is probably a JVM bug, but we should confirm that, open a JVM ticket, > and see if we can give a more useful error message on the C* side. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7254) NPE on startup if another Cassandra instance is already running
[ https://issues.apache.org/jira/browse/CASSANDRA-7254?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045212#comment-14045212 ] Jeremiah Jordan commented on CASSANDRA-7254: Since CASSANDRA-7087 went in to 2.0, I assume this will as well? > NPE on startup if another Cassandra instance is already running > --- > > Key: CASSANDRA-7254 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7254 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tyler Hobbs >Assignee: Chris Lohfink >Priority: Minor > Fix For: 2.1.0 > > Attachments: 7254.txt > > > After CASSANDRA-7087, if you try to start cassandra while another instance is > already running, you'll see something like this: > {noformat} > $ bin/cassandra -f > Error: Exception thrown by the agent : java.lang.NullPointerException > {noformat} > This is probably a JVM bug, but we should confirm that, open a JVM ticket, > and see if we can give a more useful error message on the C* side. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7456) Hinted Handoff: java.lang.ArithmeticException: / by zero
[ https://issues.apache.org/jira/browse/CASSANDRA-7456?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] sankalp kohli reassigned CASSANDRA-7456: Assignee: sankalp kohli > Hinted Handoff: java.lang.ArithmeticException: / by zero > > > Key: CASSANDRA-7456 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7456 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: 2.0.6 >Reporter: sankalp kohli >Assignee: sankalp kohli >Priority: Minor > > We are seeing this exception. This is similar to CASSANDRA-5990 but here the > exception is coming because hintStore.getMeanRowSize() is returning zero i > think. This is blowing up in > return Math.max(2, Math.min(PAGE_SIZE, > DatabaseDescriptor.getInMemoryCompactionLimit() / averageColumnSize)); > java.lang.ArithmeticException: / by zero > at > org.apache.cassandra.db.HintedHandOffManager.calculatePageSize(HintedHandOffManager.java:504) > at > org.apache.cassandra.db.HintedHandOffManager.doDeliverHintsToEndpoint(HintedHandOffManager.java:352) > at > org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:331) > at > org.apache.cassandra.db.HintedHandOffManager.access$300(HintedHandOffManager.java:92) > at > org.apache.cassandra.db.HintedHandOffManager$5.run(HintedHandOffManager.java:558) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7455) AssertionError with static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7455?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045163#comment-14045163 ] graham sanderson commented on CASSANDRA-7455: - I was going to turn this down to minor, but it is probably worth understanding the exact path that causes this because users might easily get bitten... though if it requires an empty partition key then maybe that is less of an issue > AssertionError with static columns > -- > > Key: CASSANDRA-7455 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7455 > Project: Cassandra > Issue Type: Bug >Reporter: graham sanderson > > Depending on how you insert static column values, regular values (and > implicitly static column row markers and regular column rows markers), you > can cause an AssertionError on select (collision between some of the empty > composite name fragments) if you have an empty partition key > Example: > {code} > cqlsh:test> create table select_error(pkey text, ckey text, value text, > static_value text static, PRIMARY KEY(pkey, ckey)); > cqlsh:test> insert into select_error(pkey, static_value) VALUES('partition1', > 'static value'); > cqlsh:test> insert into select_error(pkey, ckey, value) VALUES('partition1', > '', 'value'); > cqlsh:test> select * from select_error; > TSocket read 0 bytes > {code} > Causes > {code} > java.lang.AssertionError > at > org.apache.cassandra.cql3.statements.ColumnGroupMap.add(ColumnGroupMap.java:64) > at > org.apache.cassandra.cql3.statements.ColumnGroupMap.access$200(ColumnGroupMap.java:32) > at > org.apache.cassandra.cql3.statements.ColumnGroupMap$Builder.add(ColumnGroupMap.java:151) > at > org.apache.cassandra.cql3.statements.SelectStatement.processColumnFamily(SelectStatement.java:1202) > at > org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:1078) > at > org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:280) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:257) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:222) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:60) > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:158) > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:175) > at > org.apache.cassandra.thrift.CassandraServer.execute_cql3_query(CassandraServer.java:1958) > at > org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4486) > at > org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4470) > at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) > at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) > at > org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:201) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7456) Hinted Handoff: java.lang.ArithmeticException: / by zero
sankalp kohli created CASSANDRA-7456: Summary: Hinted Handoff: java.lang.ArithmeticException: / by zero Key: CASSANDRA-7456 URL: https://issues.apache.org/jira/browse/CASSANDRA-7456 Project: Cassandra Issue Type: Bug Components: Core Environment: 2.0.6 Reporter: sankalp kohli Priority: Minor We are seeing this exception. This is similar to CASSANDRA-5990 but here the exception is coming because hintStore.getMeanRowSize() is returning zero i think. This is blowing up in return Math.max(2, Math.min(PAGE_SIZE, DatabaseDescriptor.getInMemoryCompactionLimit() / averageColumnSize)); java.lang.ArithmeticException: / by zero at org.apache.cassandra.db.HintedHandOffManager.calculatePageSize(HintedHandOffManager.java:504) at org.apache.cassandra.db.HintedHandOffManager.doDeliverHintsToEndpoint(HintedHandOffManager.java:352) at org.apache.cassandra.db.HintedHandOffManager.deliverHintsToEndpoint(HintedHandOffManager.java:331) at org.apache.cassandra.db.HintedHandOffManager.access$300(HintedHandOffManager.java:92) at org.apache.cassandra.db.HintedHandOffManager$5.run(HintedHandOffManager.java:558) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603) -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7455) AssertionError with static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] graham sanderson updated CASSANDRA-7455: Priority: Major (was: Minor) > AssertionError with static columns > -- > > Key: CASSANDRA-7455 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7455 > Project: Cassandra > Issue Type: Bug >Reporter: graham sanderson > > Depending on how you insert static column values, regular values (and > implicitly static column row markers and regular column rows markers), you > can cause an AssertionError on select (collision between some of the empty > composite name fragments) if you have an empty partition key > Example: > {code} > cqlsh:test> create table select_error(pkey text, ckey text, value text, > static_value text static, PRIMARY KEY(pkey, ckey)); > cqlsh:test> insert into select_error(pkey, static_value) VALUES('partition1', > 'static value'); > cqlsh:test> insert into select_error(pkey, ckey, value) VALUES('partition1', > '', 'value'); > cqlsh:test> select * from select_error; > TSocket read 0 bytes > {code} > Causes > {code} > java.lang.AssertionError > at > org.apache.cassandra.cql3.statements.ColumnGroupMap.add(ColumnGroupMap.java:64) > at > org.apache.cassandra.cql3.statements.ColumnGroupMap.access$200(ColumnGroupMap.java:32) > at > org.apache.cassandra.cql3.statements.ColumnGroupMap$Builder.add(ColumnGroupMap.java:151) > at > org.apache.cassandra.cql3.statements.SelectStatement.processColumnFamily(SelectStatement.java:1202) > at > org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:1078) > at > org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:280) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:257) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:222) > at > org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:60) > at > org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:158) > at > org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:175) > at > org.apache.cassandra.thrift.CassandraServer.execute_cql3_query(CassandraServer.java:1958) > at > org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4486) > at > org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4470) > at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) > at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) > at > org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:201) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045161#comment-14045161 ] graham sanderson commented on CASSANDRA-7449: - Actually I'm probably an idiot, for static columns to work, the static columns must always be returned as the first group In any case I've opened CASSANDRA-7455 for the other AssertionError issue > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 108 > (1 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point B > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 87 > 123 |2 | gibbon | this is large and should not be inserted > every time | 111 > (2 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > 123 > (2 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point C > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 67 > 123 |2 | gibbon | this is large and should not be inserted > every time | 91 > 123 |3 | firebucket | this is large and should not be inserted > every
[jira] [Created] (CASSANDRA-7455) AssertionError with static columns
graham sanderson created CASSANDRA-7455: --- Summary: AssertionError with static columns Key: CASSANDRA-7455 URL: https://issues.apache.org/jira/browse/CASSANDRA-7455 Project: Cassandra Issue Type: Bug Reporter: graham sanderson Priority: Minor Depending on how you insert static column values, regular values (and implicitly static column row markers and regular column rows markers), you can cause an AssertionError on select (collision between some of the empty composite name fragments) if you have an empty partition key Example: {code} cqlsh:test> create table select_error(pkey text, ckey text, value text, static_value text static, PRIMARY KEY(pkey, ckey)); cqlsh:test> insert into select_error(pkey, static_value) VALUES('partition1', 'static value'); cqlsh:test> insert into select_error(pkey, ckey, value) VALUES('partition1', '', 'value'); cqlsh:test> select * from select_error; TSocket read 0 bytes {code} Causes {code} java.lang.AssertionError at org.apache.cassandra.cql3.statements.ColumnGroupMap.add(ColumnGroupMap.java:64) at org.apache.cassandra.cql3.statements.ColumnGroupMap.access$200(ColumnGroupMap.java:32) at org.apache.cassandra.cql3.statements.ColumnGroupMap$Builder.add(ColumnGroupMap.java:151) at org.apache.cassandra.cql3.statements.SelectStatement.processColumnFamily(SelectStatement.java:1202) at org.apache.cassandra.cql3.statements.SelectStatement.process(SelectStatement.java:1078) at org.apache.cassandra.cql3.statements.SelectStatement.processResults(SelectStatement.java:280) at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:257) at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:222) at org.apache.cassandra.cql3.statements.SelectStatement.execute(SelectStatement.java:60) at org.apache.cassandra.cql3.QueryProcessor.processStatement(QueryProcessor.java:158) at org.apache.cassandra.cql3.QueryProcessor.process(QueryProcessor.java:175) at org.apache.cassandra.thrift.CassandraServer.execute_cql3_query(CassandraServer.java:1958) at org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4486) at org.apache.cassandra.thrift.Cassandra$Processor$execute_cql3_query.getResult(Cassandra.java:4470) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:201) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7254) NPE on startup if another Cassandra instance is already running
[ https://issues.apache.org/jira/browse/CASSANDRA-7254?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Brandon Williams updated CASSANDRA-7254: Attachment: 7254.txt So, one nice thing about this bug is we don't even have to try to run a class, simply invoking java with our usual JVM_OPTS is enough. After that it's simply a matter of checking grep's return. Patch that does this. > NPE on startup if another Cassandra instance is already running > --- > > Key: CASSANDRA-7254 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7254 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tyler Hobbs >Assignee: Chris Lohfink >Priority: Minor > Fix For: 2.1.0 > > Attachments: 7254.txt > > > After CASSANDRA-7087, if you try to start cassandra while another instance is > already running, you'll see something like this: > {noformat} > $ bin/cassandra -f > Error: Exception thrown by the agent : java.lang.NullPointerException > {noformat} > This is probably a JVM bug, but we should confirm that, open a JVM ticket, > and see if we can give a more useful error message on the C* side. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7454) NPE When Prepared Statement ID is not Found
[ https://issues.apache.org/jira/browse/CASSANDRA-7454?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-7454: --- Attachment: 7475.txt 7475.txt avoids the NPE. > NPE When Prepared Statement ID is not Found > --- > > Key: CASSANDRA-7454 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7454 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tyler Hobbs >Assignee: Tyler Hobbs > Fix For: 2.1.0 > > Attachments: 7475.txt > > > CASSANDRA-6855 introduced a NullPointerException when an unknown prepared > statement ID is used. > You'll see a stack trace like this: > {noformat} > ERROR [SharedPool-Worker-4] 2014-06-26 15:02:04,911 ErrorMessage.java:218 - > Unexpected exception during request > java.lang.NullPointerException: null > at > org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:105) > ~[main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:412) > [main/:na] > at > org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:309) > [main/:na] > at > io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:103) > [netty-all-4.0.20.Final.jar:4.0.20.Final] > at > io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:332) > [netty-all-4.0.20.Final.jar:4.0.20.Final] > at > io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:31) > [netty-all-4.0.20.Final.jar:4.0.20.Final] > at > io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:323) > [netty-all-4.0.20.Final.jar:4.0.20.Final] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) > [na:1.7.0_40] > at > org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) > [main/:na] > at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) > [main/:na] > at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40] > {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7444) Performance drops when creating large amount of tables
[ https://issues.apache.org/jira/browse/CASSANDRA-7444?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-7444: - Fix Version/s: (was: 2.1.0) 2.1.1 > Performance drops when creating large amount of tables > --- > > Key: CASSANDRA-7444 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7444 > Project: Cassandra > Issue Type: Improvement > Components: Core > Environment: [cqlsh 3.1.8 | Cassandra 1.2.15.1 | CQL spec 3.0.0 | > Thrift protocol 19.36.2][cqlsh 4.1.1 | Cassandra 2.0.7.31 | CQL spec 3.1.1 | > Thrift protocol 19.39.0] >Reporter: Jose Martinez Poblete >Assignee: Aleksey Yeschenko >Priority: Minor > Labels: cassandra > Fix For: 2.1.1 > > Attachments: 7444.txt > > > We are creating 4000 tables from a script and using cqlsh to create the > tables. As the tables are being created, the time taken grows exponentially > and it becomes very slow and takes a lot of time. > We read a file get the keyspace append a random number and then create > keyspace with this new name example Airplane_12345678, Airplane_123575849... > then fed into cqlsh via script > Similarly each table is created via script use Airplane_12345678; create > table1...table25 , then use Airplane_123575849; create table1...create table25 > It is all done in singleton fashion, doing one after the other in a loop. > We tested using the following bash script > {noformat} > #!/bin/bash > SEED=0 > ITERATIONS=20 > while [ ${SEED} -lt ${ITERATIONS} ]; do >COUNT=0 >KEYSPACE=t10789_${SEED} >echo "CREATE KEYSPACE ${KEYSPACE} WITH replication = { 'class': > 'NetworkTopologyStrategy', 'Cassandra': '1' };" > ${KEYSPACE}.ddl >echo "USE ${KEYSPACE};" >> ${KEYSPACE}.ddl >while [ ${COUNT} -lt 25 ]; do > echo "CREATE TABLE user_colors${COUNT} (user_id int PRIMARY KEY, colors > list );" >> ${KEYSPACE}.ddl > ((COUNT++)) >done >((SEED++)) >time cat ${KEYSPACE}.ddl | cqlsh >if [ "$?" -gt 0 ]; then > echo "[ERROR] Failure at ${KEYSPACE}" > exit 1 >else > echo "[OK]Created ${KEYSPACE}" >fi >echo "===" >sleep 3 > done > #EOF > {noformat} > The timing we got on an otherwise idle system were inconsistent > {noformat} > real0m42.649s > user0m0.332s > sys 0m0.092s > [OK]Created t10789_0 > === > real1m22.211s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_1 > === > real2m45.907s > user0m0.304s > sys 0m0.124s > [OK]Created t10789_2 > === > real3m24.098s > user0m0.340s > sys 0m0.108s > [OK]Created t10789_3 > === > real2m38.930s > user0m0.324s > sys 0m0.116s > [OK]Created t10789_4 > === > real3m4.186s > user0m0.336s > sys 0m0.104s > [OK]Created t10789_5 > === > real2m55.391s > user0m0.344s > sys 0m0.092s > [OK]Created t10789_6 > === > real2m14.290s > user0m0.328s > sys 0m0.108s > [OK]Created t10789_7 > === > real2m44.880s > user0m0.344s > sys 0m0.092s > [OK]Created t10789_8 > === > real1m52.785s > user0m0.336s > sys 0m0.128s > [OK]Created t10789_9 > === > real1m18.404s > user0m0.344s > sys 0m0.108s > [OK]Created t10789_10 > === > real2m20.681s > user0m0.348s > sys 0m0.104s > [OK]Created t10789_11 > === > real1m11.860s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_12 > === > real1m37.887s > user0m0.324s > sys 0m0.100s > [OK]Created t10789_13 > === > real1m31.616s > user0m0.316s > sys 0m0.132s > [OK]Created t10789_14 > === > real1m12.103s > user0m0.360s > sys 0m0.088s > [OK]Created t10789_15 > === > real0m36.378s > user0m0.340s > sys 0m0.092s > [OK]Created t10789_16 > === > real0m40.883s > user0m0.352s > sys 0m0.096s > [OK]Created t10789_17 > === > real0m40.661s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_18 > === > real0m44.943s > user0m0.324s > sys 0m0.104s > [OK]Created t10789_19 > === > {noformat} -- This message was sent by Atlassian JIRA (v6.2#6
[jira] [Commented] (CASSANDRA-7444) Performance drops when creating large amount of tables
[ https://issues.apache.org/jira/browse/CASSANDRA-7444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045103#comment-14045103 ] Aleksey Yeschenko commented on CASSANDRA-7444: -- Sure. Guess it's time for a separate cassandra-2.1.0 branch though. > Performance drops when creating large amount of tables > --- > > Key: CASSANDRA-7444 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7444 > Project: Cassandra > Issue Type: Improvement > Components: Core > Environment: [cqlsh 3.1.8 | Cassandra 1.2.15.1 | CQL spec 3.0.0 | > Thrift protocol 19.36.2][cqlsh 4.1.1 | Cassandra 2.0.7.31 | CQL spec 3.1.1 | > Thrift protocol 19.39.0] >Reporter: Jose Martinez Poblete >Assignee: Aleksey Yeschenko >Priority: Minor > Labels: cassandra > Fix For: 2.1.1 > > Attachments: 7444.txt > > > We are creating 4000 tables from a script and using cqlsh to create the > tables. As the tables are being created, the time taken grows exponentially > and it becomes very slow and takes a lot of time. > We read a file get the keyspace append a random number and then create > keyspace with this new name example Airplane_12345678, Airplane_123575849... > then fed into cqlsh via script > Similarly each table is created via script use Airplane_12345678; create > table1...table25 , then use Airplane_123575849; create table1...create table25 > It is all done in singleton fashion, doing one after the other in a loop. > We tested using the following bash script > {noformat} > #!/bin/bash > SEED=0 > ITERATIONS=20 > while [ ${SEED} -lt ${ITERATIONS} ]; do >COUNT=0 >KEYSPACE=t10789_${SEED} >echo "CREATE KEYSPACE ${KEYSPACE} WITH replication = { 'class': > 'NetworkTopologyStrategy', 'Cassandra': '1' };" > ${KEYSPACE}.ddl >echo "USE ${KEYSPACE};" >> ${KEYSPACE}.ddl >while [ ${COUNT} -lt 25 ]; do > echo "CREATE TABLE user_colors${COUNT} (user_id int PRIMARY KEY, colors > list );" >> ${KEYSPACE}.ddl > ((COUNT++)) >done >((SEED++)) >time cat ${KEYSPACE}.ddl | cqlsh >if [ "$?" -gt 0 ]; then > echo "[ERROR] Failure at ${KEYSPACE}" > exit 1 >else > echo "[OK]Created ${KEYSPACE}" >fi >echo "===" >sleep 3 > done > #EOF > {noformat} > The timing we got on an otherwise idle system were inconsistent > {noformat} > real0m42.649s > user0m0.332s > sys 0m0.092s > [OK]Created t10789_0 > === > real1m22.211s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_1 > === > real2m45.907s > user0m0.304s > sys 0m0.124s > [OK]Created t10789_2 > === > real3m24.098s > user0m0.340s > sys 0m0.108s > [OK]Created t10789_3 > === > real2m38.930s > user0m0.324s > sys 0m0.116s > [OK]Created t10789_4 > === > real3m4.186s > user0m0.336s > sys 0m0.104s > [OK]Created t10789_5 > === > real2m55.391s > user0m0.344s > sys 0m0.092s > [OK]Created t10789_6 > === > real2m14.290s > user0m0.328s > sys 0m0.108s > [OK]Created t10789_7 > === > real2m44.880s > user0m0.344s > sys 0m0.092s > [OK]Created t10789_8 > === > real1m52.785s > user0m0.336s > sys 0m0.128s > [OK]Created t10789_9 > === > real1m18.404s > user0m0.344s > sys 0m0.108s > [OK]Created t10789_10 > === > real2m20.681s > user0m0.348s > sys 0m0.104s > [OK]Created t10789_11 > === > real1m11.860s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_12 > === > real1m37.887s > user0m0.324s > sys 0m0.100s > [OK]Created t10789_13 > === > real1m31.616s > user0m0.316s > sys 0m0.132s > [OK]Created t10789_14 > === > real1m12.103s > user0m0.360s > sys 0m0.088s > [OK]Created t10789_15 > === > real0m36.378s > user0m0.340s > sys 0m0.092s > [OK]Created t10789_16 > === > real0m40.883s > user0m0.352s > sys 0m0.096s > [OK]Created t10789_17 > === > real0m40.661s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_18 > === > real0m44.943s > user0m0.324s > sys 0m0.104s > [OK]Created t10789_19 > === >
[jira] [Created] (CASSANDRA-7454) NPE When Prepared Statement ID is not Found
Tyler Hobbs created CASSANDRA-7454: -- Summary: NPE When Prepared Statement ID is not Found Key: CASSANDRA-7454 URL: https://issues.apache.org/jira/browse/CASSANDRA-7454 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Tyler Hobbs Fix For: 2.1.0 CASSANDRA-6855 introduced a NullPointerException when an unknown prepared statement ID is used. You'll see a stack trace like this: {noformat} ERROR [SharedPool-Worker-4] 2014-06-26 15:02:04,911 ErrorMessage.java:218 - Unexpected exception during request java.lang.NullPointerException: null at org.apache.cassandra.transport.messages.ExecuteMessage.execute(ExecuteMessage.java:105) ~[main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:412) [main/:na] at org.apache.cassandra.transport.Message$Dispatcher.channelRead0(Message.java:309) [main/:na] at io.netty.channel.SimpleChannelInboundHandler.channelRead(SimpleChannelInboundHandler.java:103) [netty-all-4.0.20.Final.jar:4.0.20.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:332) [netty-all-4.0.20.Final.jar:4.0.20.Final] at io.netty.channel.AbstractChannelHandlerContext.access$700(AbstractChannelHandlerContext.java:31) [netty-all-4.0.20.Final.jar:4.0.20.Final] at io.netty.channel.AbstractChannelHandlerContext$8.run(AbstractChannelHandlerContext.java:323) [netty-all-4.0.20.Final.jar:4.0.20.Final] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) [na:1.7.0_40] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) [main/:na] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) [main/:na] at java.lang.Thread.run(Thread.java:724) [na:1.7.0_40] {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045095#comment-14045095 ] sankalp kohli commented on CASSANDRA-7278: -- This is fixed in https://issues.apache.org/jira/browse/CASSANDRA-6916 in 2.1. Can we do my change to fix it in 2.0 as well? > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6621) STCS fallback is not optimal when bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045067#comment-14045067 ] T Jake Luciani commented on CASSANDRA-6621: --- Yeah I agree with [~yukim] it should be perhaps a property > STCS fallback is not optimal when bootstrapping > --- > > Key: CASSANDRA-6621 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6621 > Project: Cassandra > Issue Type: Improvement >Reporter: Bartłomiej Romański >Assignee: Marcus Eriksson >Priority: Minor > Labels: compaction, streaming > Fix For: 2.0.9 > > Attachments: 0001-option-to-disallow-L0-stcs.patch, > 0001-wip-keep-sstable-level-when-bootstrapping.patch > > > The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my > last comment here... > After streaming (e.g. during boostrap) Cassandra places all sstables at L0. > At the end of the process we end up with huge number of sstables at the > lowest level. > Currently, Cassandra falls back to STCS until the number of sstables at L0 > reaches the reasonable level (32 or something). > I'm not sure if falling back to STCS is the best way to handle this > particular situation. I've read the comment in the code and I'm aware why it > is a good thing to do if we have to many sstables at L0 as a result of too > many random inserts. We have a lot of sstables, each of them covers the whole > ring, there's simply no better option. > However, after the bootstrap situation looks a bit different. The loaded > sstables already have very small ranges! We just have to tidy up a bit and > everything should be OK. STCS ignores that completely and after a while we > have a bit less sstables but each of them covers the whole ring instead of > just a small part. I believe that in that case letting LCS do the job is a > better option that allowing STCS mix everything up before. > Is there a way to disable STCS fallback? I'd like to test that scenario in > practice during our next bootstrap... > Does Cassandra really have to put streamed sstables at L0? The only thing we > have to assure is that sstables at any given level do not overlap. If we > stream different regions from different nodes how can we get any overlaps? -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6621) STCS fallback is not optimal when bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045053#comment-14045053 ] Yuki Morishita commented on CASSANDRA-6621: --- [~krummas] Controling through compaction option is nice idea. But do we want to change all nodes at once? When we just want to disable in bootstrapping node, then we need to add JMX interface or system property to control. Also, let's move Marcus's first patch to new JIRA and target it to 2.1.x release. > STCS fallback is not optimal when bootstrapping > --- > > Key: CASSANDRA-6621 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6621 > Project: Cassandra > Issue Type: Improvement >Reporter: Bartłomiej Romański >Assignee: Marcus Eriksson >Priority: Minor > Labels: compaction, streaming > Fix For: 2.0.9 > > Attachments: 0001-option-to-disallow-L0-stcs.patch, > 0001-wip-keep-sstable-level-when-bootstrapping.patch > > > The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my > last comment here... > After streaming (e.g. during boostrap) Cassandra places all sstables at L0. > At the end of the process we end up with huge number of sstables at the > lowest level. > Currently, Cassandra falls back to STCS until the number of sstables at L0 > reaches the reasonable level (32 or something). > I'm not sure if falling back to STCS is the best way to handle this > particular situation. I've read the comment in the code and I'm aware why it > is a good thing to do if we have to many sstables at L0 as a result of too > many random inserts. We have a lot of sstables, each of them covers the whole > ring, there's simply no better option. > However, after the bootstrap situation looks a bit different. The loaded > sstables already have very small ranges! We just have to tidy up a bit and > everything should be OK. STCS ignores that completely and after a while we > have a bit less sstables but each of them covers the whole ring instead of > just a small part. I believe that in that case letting LCS do the job is a > better option that allowing STCS mix everything up before. > Is there a way to disable STCS fallback? I'd like to test that scenario in > practice during our next bootstrap... > Does Cassandra really have to put streamed sstables at L0? The only thing we > have to assure is that sstables at any given level do not overlap. If we > stream different regions from different nodes how can we get any overlaps? -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7440) Unrecognized opcode or flag causes ArrayIndexOutOfBoundsException
[ https://issues.apache.org/jira/browse/CASSANDRA-7440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-7440: --- Fix Version/s: 2.0.10 > Unrecognized opcode or flag causes ArrayIndexOutOfBoundsException > - > > Key: CASSANDRA-7440 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7440 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tyler Hobbs >Assignee: Tyler Hobbs >Priority: Minor > Fix For: 2.1.0, 2.0.10 > > Attachments: 7440.txt > > > If an unrecognized opcode or flag is used in a native protocol message, it > results in an unhandled ArrayIndexOutOfBoundsException server-side. This can > happen when features from a new protocol version are used with an older > version of Cassandra. > Here's one example stacktrace: > {noformat} > ERROR [Native-Transport-Requests:5] 2014-06-24 11:09:50,167 ErrorMessage.java > (line 210) Unexpected exception during request > java.lang.ArrayIndexOutOfBoundsException: 15 > at org.apache.cassandra.transport.Message$Type.fromOpcode(Message.java:102) > at org.apache.cassandra.transport.Frame$Decoder.decode(Frame.java:179) > at > org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:425) > at > org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303) > at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268) > at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255) > at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) > at > org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:109) > at > org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:312) > at > org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:90) > at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7440) Unrecognized opcode or flag causes ArrayIndexOutOfBoundsException
[ https://issues.apache.org/jira/browse/CASSANDRA-7440?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-7440: --- Attachment: 7440.txt 7440.txt (and [branch|https://github.com/thobbs/cassandra/tree/CASSANDRA-7440]) raises a ProtocolException is the opcode is unrecognized, and tolerates unrecognized Frame flags (matching the behavior for QueryOptions flags). > Unrecognized opcode or flag causes ArrayIndexOutOfBoundsException > - > > Key: CASSANDRA-7440 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7440 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tyler Hobbs >Assignee: Tyler Hobbs >Priority: Minor > Fix For: 2.1.0, 2.0.10 > > Attachments: 7440.txt > > > If an unrecognized opcode or flag is used in a native protocol message, it > results in an unhandled ArrayIndexOutOfBoundsException server-side. This can > happen when features from a new protocol version are used with an older > version of Cassandra. > Here's one example stacktrace: > {noformat} > ERROR [Native-Transport-Requests:5] 2014-06-24 11:09:50,167 ErrorMessage.java > (line 210) Unexpected exception during request > java.lang.ArrayIndexOutOfBoundsException: 15 > at org.apache.cassandra.transport.Message$Type.fromOpcode(Message.java:102) > at org.apache.cassandra.transport.Frame$Decoder.decode(Frame.java:179) > at > org.jboss.netty.handler.codec.frame.FrameDecoder.callDecode(FrameDecoder.java:425) > at > org.jboss.netty.handler.codec.frame.FrameDecoder.messageReceived(FrameDecoder.java:303) > at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268) > at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255) > at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88) > at > org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:109) > at > org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:312) > at > org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:90) > at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:745) > {noformat} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7411) Node enables vnodes when bounced
[ https://issues.apache.org/jira/browse/CASSANDRA-7411?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045017#comment-14045017 ] Vladimir Rudev commented on CASSANDRA-7411: --- Exactly this way of configuring node for first start lead to such consequences in future: [Mailing list|https://www.mail-archive.com/user@cassandra.apache.org/msg36689.html] > Node enables vnodes when bounced > > > Key: CASSANDRA-7411 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7411 > Project: Cassandra > Issue Type: Improvement > Environment: OSX 9 >Reporter: Philip Thompson >Assignee: Philip Thompson >Priority: Minor > Fix For: 2.1.0, 2.0.10 > > Attachments: 7411.txt, system.log > > > According to cassandra.yaml, in the information for the num_tokens setting, > "Specifying initial_token will override this setting." So if exactly one > initial token is set, then vnodes are disabled, regardless of if or what > num_tokens are set to. This behavior is inconsistent when a node is started, > versus if it has been bounced. > From a fresh checkout of C*, if I build, then edit cassandra.yaml so that: > num_tokens: 256 > initial_token: -9223372036854775808 > then run bin/cassandra, C* will start correctly. I can run bin/nodetool ring > and see that the node has exactly one token and it is what I set in > initial_token. If I gracefully shutdown C*, then restart the node, running > bin/nodetool ring shows that the node now has vnodes enabled and has 256 > tokens. > I have been able to reproduce this locally on OSX using 2.0.8, 2.1 rc1, and > trunk. I have not yet tested in Linux or Windows to see if it occurs there. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-5263) Increase merkle tree depth as needed
[ https://issues.apache.org/jira/browse/CASSANDRA-5263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045015#comment-14045015 ] sankalp kohli commented on CASSANDRA-5263: -- Right. What I was proposing what that if we take the cardinality of the whole sstable, we will get a sense of overlapping. You are right, it won't be accurate. So not sure whether it is a good idea to use it. > Increase merkle tree depth as needed > > > Key: CASSANDRA-5263 > URL: https://issues.apache.org/jira/browse/CASSANDRA-5263 > Project: Cassandra > Issue Type: Improvement >Affects Versions: 1.1.9 >Reporter: Ahmed Bashir >Assignee: Yuki Morishita > Fix For: 2.1.1 > > Attachments: 5263-2.1-v1.txt > > > Currently, the maximum depth allowed for Merkle trees is hardcoded as 15. > This value should be configurable, just like phi_convict_treshold and other > properties. > Given a cluster with nodes responsible for a large number of row keys, Merkle > tree comparisons can result in a large amount of unnecessary row keys being > streamed. > Empirical testing indicates that reasonable changes to this depth (18, 20, > etc) don't affect the Merkle tree generation and differencing timings all > that much, and they can significantly reduce the amount of data being > streamed during repair. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7341) Emit metrics related to CAS/Paxos
[ https://issues.apache.org/jira/browse/CASSANDRA-7341?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14045008#comment-14045008 ] sankalp kohli commented on CASSANDRA-7341: -- I will get a new patch soon. > Emit metrics related to CAS/Paxos > - > > Key: CASSANDRA-7341 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7341 > Project: Cassandra > Issue Type: Improvement >Reporter: sankalp kohli >Assignee: sankalp kohli >Priority: Minor > Attachments: CASClientRequestMetrics.java, trunk-7341.diff > > > We can emit metrics based on Paxos. One of them is when there is contention. > I will add more metric in this JIRA if it is helpful. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044997#comment-14044997 ] sankalp kohli commented on CASSANDRA-7278: -- ping [~jbellis] since annotate is pointing to you :) > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044995#comment-14044995 ] sankalp kohli commented on CASSANDRA-7278: -- We are also hitting when there is lot of Java GC activity. I think this is caused by a race caused due to a pause. Here are the steps which will cause this error 1) A thread A calls FileCacheService.get to get a RandomAccessReader from cache. It grabs Queue from cache. 2) There is a big GC pause > 512 milli which is expiring time of the cache 3) After the pause, cache detects the Queue has expired and removes it and calls RemovalListener. 4) RemovalListener iterates over the Queue and calls dellocate() over all RandomAccessReader which sets buffer to null. 5) Thread A resumes, takes one RandomAccessReader from the Queue and proceeds. Since the buffer is null, it will blow up in memoryUsage.addAndGet(-result.getTotalBufferSize());. I know this is very unlikely but I could not think of anything else which will cause this. If this is indeed happening, we can fix it by changing the for loop in RemovalListener to a while loop like this RandomAccessReader reader = null; while ((reader = cachedInstances.poll()) != null) If this does not work, I will upload the patch soon. > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7278) NPE in StorageProxy.java:1920
[ https://issues.apache.org/jira/browse/CASSANDRA-7278?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] sankalp kohli reassigned CASSANDRA-7278: Assignee: sankalp kohli > NPE in StorageProxy.java:1920 > - > > Key: CASSANDRA-7278 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7278 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: Cassandra 2.07, x86-64 ubuntu 12.04 >Reporter: Duncan Sands >Assignee: sankalp kohli >Priority: Minor > Attachments: sl > > > Got this this morning under heavy load: > ERROR [ReadStage:128] 2014-05-21 07:59:03,274 CassandraDaemon.java (line 198) > Exception in thread Thread[ReadStage:128,5,main] > java.lang.RuntimeException: java.lang.NullPointerException > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1920) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) > at java.lang.Thread.run(Thread.java:744) > Caused by: java.lang.NullPointerException > at > org.apache.cassandra.io.util.RandomAccessReader.getTotalBufferSize(RandomAccessReader.java:157) > at > org.apache.cassandra.io.compress.CompressedRandomAccessReader.getTotalBufferSize(CompressedRandomAccessReader.java:159) > at > org.apache.cassandra.service.FileCacheService.get(FileCacheService.java:96) > at > org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(PoolingSegmentedFile.java:36) > at > org.apache.cassandra.io.sstable.SSTableReader.getFileDataInput(SSTableReader.java:1195) > at > org.apache.cassandra.db.columniterator.SimpleSliceReader.(SimpleSliceReader.java:57) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.createReader(SSTableSliceIterator.java:65) > at > org.apache.cassandra.db.columniterator.SSTableSliceIterator.(SSTableSliceIterator.java:42) > at > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(SliceQueryFilter.java:167) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:62) > at > org.apache.cassandra.db.CollationController.collectAllData(CollationController.java:250) > at > org.apache.cassandra.db.CollationController.getTopLevelColumns(CollationController.java:53) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1540) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1369) > at org.apache.cassandra.db.Keyspace.getRow(Keyspace.java:327) > at > org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:65) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:1352) > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:1916) > ... 3 more > There had just been a 20 second GC pause, and the system was dropping > messages like mad, see attached log snippet. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044857#comment-14044857 ] graham sanderson commented on CASSANDRA-7449: - Yeah a more general solution would be nice... it is possible but ugly to do because a static only row in this case {code} cqlsh:test> select id, series_order, writetime(large_data) from expiring_series; id | series_order | writetime(large_data) -+--+--- 123 | null | 140380161870 {code} but of course with data you get multiple rows per id (note I use writetime to avoid fetching large_data but check it is present and series_order nullness to detect lack of regular rows) so {code} select id, series_order, writetime(large_data) from expiring_series PLIMIT 1; {code} or whatever would be fine Note however that (and I'm sure you are thinking about this already), CASSANDRA-5762 continues to be a pain in terms of the efficiency of this query... my patch allows you to select just 2 rows the static one and one other... it seems you are able to guarantee the static column always appears first (due to all null keys I guess sorting first in all data types?), so here I guess PLIMIT or whatever would allow not fetching the entire row. > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 108 > (1 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point B > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 87 > 123 |2 | gibbon | this is large and should not be inserted > every time |
[jira] [Comment Edited] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044857#comment-14044857 ] graham sanderson edited comment on CASSANDRA-7449 at 6/26/14 5:01 PM: -- Thanks for your comments. Yeah a more general solution would be nice... it is possible but ugly to do today because with a static only row in this case: {code} cqlsh:test> select id, series_order, writetime(large_data) from expiring_series; id | series_order | writetime(large_data) -+--+--- 123 | null | 140380161870 {code} but of course with data you get multiple rows per id (note I use writetime to avoid fetching large_data but check it is present and series_order nullness to detect lack of regular rows) so {code} select id, series_order, writetime(large_data) from expiring_series PLIMIT 1; {code} or whatever would be fine Note however that (and I'm sure you are thinking about this already), CASSANDRA-5762 continues to be a pain in terms of the efficiency of this query... my patch allows you to select just 2 rows the static one and one other... it seems you are able to guarantee the static column always appears first (due to all null keys I guess sorting first in all data types?), so here I guess PLIMIT or whatever would allow not fetching the entire row. was (Author: graham sanderson): Yeah a more general solution would be nice... it is possible but ugly to do today because with a static only row in this case: {code} cqlsh:test> select id, series_order, writetime(large_data) from expiring_series; id | series_order | writetime(large_data) -+--+--- 123 | null | 140380161870 {code} but of course with data you get multiple rows per id (note I use writetime to avoid fetching large_data but check it is present and series_order nullness to detect lack of regular rows) so {code} select id, series_order, writetime(large_data) from expiring_series PLIMIT 1; {code} or whatever would be fine Note however that (and I'm sure you are thinking about this already), CASSANDRA-5762 continues to be a pain in terms of the efficiency of this query... my patch allows you to select just 2 rows the static one and one other... it seems you are able to guarantee the static column always appears first (due to all null keys I guess sorting first in all data types?), so here I guess PLIMIT or whatever would allow not fetching the entire row. > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A >
[jira] [Comment Edited] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044857#comment-14044857 ] graham sanderson edited comment on CASSANDRA-7449 at 6/26/14 5:00 PM: -- Yeah a more general solution would be nice... it is possible but ugly to do today because with a static only row in this case: {code} cqlsh:test> select id, series_order, writetime(large_data) from expiring_series; id | series_order | writetime(large_data) -+--+--- 123 | null | 140380161870 {code} but of course with data you get multiple rows per id (note I use writetime to avoid fetching large_data but check it is present and series_order nullness to detect lack of regular rows) so {code} select id, series_order, writetime(large_data) from expiring_series PLIMIT 1; {code} or whatever would be fine Note however that (and I'm sure you are thinking about this already), CASSANDRA-5762 continues to be a pain in terms of the efficiency of this query... my patch allows you to select just 2 rows the static one and one other... it seems you are able to guarantee the static column always appears first (due to all null keys I guess sorting first in all data types?), so here I guess PLIMIT or whatever would allow not fetching the entire row. was (Author: graham sanderson): Yeah a more general solution would be nice... it is possible but ugly to do because a static only row in this case {code} cqlsh:test> select id, series_order, writetime(large_data) from expiring_series; id | series_order | writetime(large_data) -+--+--- 123 | null | 140380161870 {code} but of course with data you get multiple rows per id (note I use writetime to avoid fetching large_data but check it is present and series_order nullness to detect lack of regular rows) so {code} select id, series_order, writetime(large_data) from expiring_series PLIMIT 1; {code} or whatever would be fine Note however that (and I'm sure you are thinking about this already), CASSANDRA-5762 continues to be a pain in terms of the efficiency of this query... my patch allows you to select just 2 rows the static one and one other... it seems you are able to guarantee the static column always appears first (due to all null keys I guess sorting first in all data types?), so here I guess PLIMIT or whatever would allow not fetching the entire row. > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A > {code} > cqlsh:test> SELECT id, series_
[jira] [Created] (CASSANDRA-7453) Geo-replication in Cassandra
Sergio Esteves created CASSANDRA-7453: - Summary: Geo-replication in Cassandra Key: CASSANDRA-7453 URL: https://issues.apache.org/jira/browse/CASSANDRA-7453 Project: Cassandra Issue Type: Wish Reporter: Sergio Esteves Priority: Minor Currently, a Cassandra cluster spanned across different datacenters replicates all data to all datacenters when an update is performed. This is a problem for the scalability of Cassandra as the number of datacenters increases. It would be desirable to have some way to make Cassandra aware of the location of data requests so that it could place replicas close to users and avoid replicating to remote datacenters that are far away. To this end, we thought of implementing a new replication strategy and some possible solutions to achieve our goals are: 1) Using a byte from every row key to identify the location of the primary datacenter where data should be stored (i.e., where it is likely to be accessed). 2) Using an additional CF for every row to specify the origin of the data. 3) Replicating only to the 2 closest datacenters from the user (for reliability reasons) upon a write update. For reads, a user would try to fetch data from the 2 closest datacenters; if data is not available it would try the other remaining datacenters. If data fails to be retrieved too many times, it means that the client has moved to other part of the planet, and thus data should be migrated accordingly. We could have some problems here, like having the same rows, but with different CFs in different DCs (i.e., if users perform updates to the same rows from different remote places). What would be the best way to do this? Thanks. -- This message was sent by Atlassian JIRA (v6.2#6252)
buildbot success in ASF Buildbot on cassandra-2.0
The Buildbot has detected a restored build on builder cassandra-2.0 while building cassandra. Full details are available at: http://ci.apache.org/builders/cassandra-2.0/builds/99 Buildbot URL: http://ci.apache.org/ Buildslave for this Build: portunus_ubuntu Build Reason: scheduler Build Source Stamp: [branch cassandra-2.0] ac1d5bd56f5ae96808270358fc45cb19e487e3d8 Blamelist: Brandon Williams Build succeeded! sincerely, -The Buildbot
[jira] [Commented] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044829#comment-14044829 ] Sylvain Lebresne commented on CASSANDRA-7449: - Honestly, that sounds super specific and imo not worth adding a new notion just for that. I'd rather spend time on adding CASSANDRA-7017 which is more generally useful (and will basically allow you to do what you want here). bq. In CFDefinition - a bug that should be fixed separately if this feature isn't implemented Created CASSANDRA-7452 for that (it's currently harmless). bq. 2) If you insert an empty (zero byte) partition key value and you have a static column, you get an assertion error on regular select * - I'll make a better repro for this and file an issue That would be cool, thanks. > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 108 > (1 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point B > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 87 > 123 |2 | gibbon | this is large and should not be inserted > every time | 111 > (2 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > 123 > (2 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point C > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > --
[jira] [Updated] (CASSANDRA-7452) Fix typo in CFDefinition (regarding static columns)
[ https://issues.apache.org/jira/browse/CASSANDRA-7452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne updated CASSANDRA-7452: Attachment: 7452.txt Trivial fix attached > Fix typo in CFDefinition (regarding static columns) > --- > > Key: CASSANDRA-7452 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7452 > Project: Cassandra > Issue Type: Bug >Reporter: Sylvain Lebresne >Priority: Trivial > Attachments: 7452.txt > > > As noted on CASSANDRA-7449, there is a typo in CFDefinition: > {noformat} > public Collection staticColumns() > { > return regularColumns.values(); > } > {noformat} > I'll note that this is currently completely inconsequential since this method > is only used by AbstractCassandraStorage, but in a case where there can't be > any static columns (because the CFDefinition uses comes from a > CFMetadata.fromThrift() that can't have them). Not a reason to leave the typo > of course. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Assigned] (CASSANDRA-7452) Fix typo in CFDefinition (regarding static columns)
[ https://issues.apache.org/jira/browse/CASSANDRA-7452?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne reassigned CASSANDRA-7452: --- Assignee: Sylvain Lebresne > Fix typo in CFDefinition (regarding static columns) > --- > > Key: CASSANDRA-7452 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7452 > Project: Cassandra > Issue Type: Bug >Reporter: Sylvain Lebresne >Assignee: Sylvain Lebresne >Priority: Trivial > Attachments: 7452.txt > > > As noted on CASSANDRA-7449, there is a typo in CFDefinition: > {noformat} > public Collection staticColumns() > { > return regularColumns.values(); > } > {noformat} > I'll note that this is currently completely inconsequential since this method > is only used by AbstractCassandraStorage, but in a case where there can't be > any static columns (because the CFDefinition uses comes from a > CFMetadata.fromThrift() that can't have them). Not a reason to leave the typo > of course. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7452) Fix typo in CFDefinition (regarding static columns)
Sylvain Lebresne created CASSANDRA-7452: --- Summary: Fix typo in CFDefinition (regarding static columns) Key: CASSANDRA-7452 URL: https://issues.apache.org/jira/browse/CASSANDRA-7452 Project: Cassandra Issue Type: Bug Reporter: Sylvain Lebresne Priority: Trivial As noted on CASSANDRA-7449, there is a typo in CFDefinition: {noformat} public Collection staticColumns() { return regularColumns.values(); } {noformat} I'll note that this is currently completely inconsequential since this method is only used by AbstractCassandraStorage, but in a case where there can't be any static columns (because the CFDefinition uses comes from a CFMetadata.fromThrift() that can't have them). Not a reason to leave the typo of course. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7451) Launch scripts on Windows don't handle spaces gracefully
[ https://issues.apache.org/jira/browse/CASSANDRA-7451?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044796#comment-14044796 ] Jorge Bay commented on CASSANDRA-7451: -- Looks good! +1 > Launch scripts on Windows don't handle spaces gracefully > > > Key: CASSANDRA-7451 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7451 > Project: Cassandra > Issue Type: Bug > Environment: 2.1.X >Reporter: Joshua McKenzie >Assignee: Joshua McKenzie >Priority: Minor > Labels: Windows > Fix For: 2.1.0 > > Attachments: 7451_v1.txt > > > There's also some .ps1 problems after we get past just the .bat. Should be > some trivial escaping to fix. > C:\src - Copy\cassandra\bin>cassandra.bat > Detected powershell execution permissions. Running with enhanced startup > scripts. > Processing -File 'C:\src' failed because the file does not have a '.ps1' > extension. Specify a valid PowerShell script file name, and then try again. -- This message was sent by Atlassian JIRA (v6.2#6252)
[03/13] git commit: Refuse range queries with strict bounds on compact tables
Refuse range queries with strict bounds on compact tables patch by slebresne; reviewed by iamaleksey for CASSANDRA-7059 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/79a4dd58 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/79a4dd58 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/79a4dd58 Branch: refs/heads/trunk Commit: 79a4dd58cd060cc28f32e0fc17001fe1179552a2 Parents: 6e4dca0 Author: Sylvain Lebresne Authored: Thu May 8 17:51:29 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:12:59 2014 +0200 -- CHANGES.txt | 2 ++ .../cassandra/cql3/SingleColumnRelation.java| 10 ++ .../cql3/statements/SelectStatement.java| 34 3 files changed, 46 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/79a4dd58/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index c3fe8d7..2b3ace3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -23,6 +23,8 @@ * Accept subtypes for function results, type casts (CASSANDRA-6766) * Support DISTINCT for static columns and fix behaviour when DISTINC is not use (CASSANDRA-7305). + * Refuse range queries with strict bounds on compact tables since they + are broken (CASSANDRA-7059) Merged from 1.2: * Expose global ColumnFamily metrics (CASSANDRA-7273) * cqlsh: Fix CompositeType columns in DESCRIBE TABLE output (CASSANDRA-7399) http://git-wip-us.apache.org/repos/asf/cassandra/blob/79a4dd58/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java -- diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java index 5464c23..642be66 100644 --- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java +++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java @@ -84,6 +84,16 @@ public class SingleColumnRelation extends Relation return false; } +public SingleColumnRelation withNonStrictOperator() +{ +switch (relationType) +{ +case GT: return new SingleColumnRelation(entity, Type.GTE, value); +case LT: return new SingleColumnRelation(entity, Type.LTE, value); +default: return this; +} +} + @Override public String toString() { http://git-wip-us.apache.org/repos/asf/cassandra/blob/79a4dd58/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java -- diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index f106402..98bd99a 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -1977,6 +1977,40 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache "thus may have unpredictable performance. If you want to execute " + "this query despite the performance unpredictability, use ALLOW FILTERING"); } + +// We don't internally support exclusive slice bounds on non-composite tables. To deal with it we do an +// inclusive slice and remove post-query the value that shouldn't be returned. One problem however is that +// if there is a user limit, that limit may make the query return before the end of the slice is reached, +// in which case, once we'll have removed bound post-query, we might end up with less results than +// requested which would be incorrect. For single-partition query, this is not a problem, we just ask for +// one more result (see updateLimitForQuery()) since that's enough to compensate for that problem. For key +// range however, each returned row may include one result that will have to be trimmed, so we would have +// to bump the query limit by N where N is the number of rows we will return, but we don't know that in +// advance. So, since we currently don't have a good way to handle such query, we refuse it (#7059) rather +// than answering with something that is wrong. +if (stmt.sliceRestriction != null && stmt.isKeyRange && limit != null) +{ +SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(stmt.cfDef); +throw new InvalidRequestException(String.format("The query request
[13/13] git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/652a7ea7 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/652a7ea7 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/652a7ea7 Branch: refs/heads/trunk Commit: 652a7ea7152da4b77df1575a16ddd1a64c32a053 Parents: ae0f634 e8500c7 Author: Brandon Williams Authored: Thu Jun 26 11:02:17 2014 -0500 Committer: Brandon Williams Committed: Thu Jun 26 11:02:17 2014 -0500 -- CHANGES.txt | 4 +++ conf/cassandra.yaml | 3 +- .../cassandra/cql3/SingleColumnRelation.java| 10 ++ .../cql3/statements/SelectStatement.java| 34 4 files changed, 50 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/652a7ea7/CHANGES.txt --
[07/13] git commit: Document initial_token/num_tokens behavior when both are set
Document initial_token/num_tokens behavior when both are set Patch by Philip Thompson, reviewed by brandonwilliams for CASSANDRA-7411 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ac1d5bd5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ac1d5bd5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ac1d5bd5 Branch: refs/heads/trunk Commit: ac1d5bd56f5ae96808270358fc45cb19e487e3d8 Parents: 5b878ce Author: Brandon Williams Authored: Thu Jun 26 10:53:24 2014 -0500 Committer: Brandon Williams Committed: Thu Jun 26 10:54:10 2014 -0500 -- conf/cassandra.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac1d5bd5/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index ef8194e..ea4d955 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -17,7 +17,8 @@ cluster_name: 'Test Cluster' # If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, # and will use the initial_token as described below. # -# Specifying initial_token will override this setting. +# Specifying initial_token will override this setting on the node's initial start, +# on subsequent starts, this setting will apply even if initial token is set. # # If you already have a cluster with 1 token per node, and wish to migrate to # multiple tokens per node, see http://wiki.apache.org/cassandra/Operations
[05/13] git commit: Support Thrift tables clustering columns on CqlPagingInputFormat
Support Thrift tables clustering columns on CqlPagingInputFormat patch by pauloricardomg; reviewed by alexliu68 for CASSANDRA-7445 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c9cef44a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c9cef44a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c9cef44a Branch: refs/heads/trunk Commit: c9cef44a1a6c10036200f410c8a26942e64c8f12 Parents: 87c4efe Author: Sylvain Lebresne Authored: Thu Jun 26 10:40:22 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:40:22 2014 +0200 -- CHANGES.txt | 3 + .../hadoop/cql3/CqlPagingRecordReader.java | 16 ++--- .../cassandra/pig/ThriftColumnFamilyTest.java | 61 +--- 3 files changed, 65 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9cef44a/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 6adef97..9fbcd9a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +1.2.18 + * Support Thrift tables clustering columns on CqlPagingInputFormat (CASSANDRA-7445) + 1.2.17 * cqlsh: Fix CompositeType columns in DESCRIBE TABLE output (CASSANDRA-7399) * Expose global ColmunFamily metrics (CASSANDRA-7273) http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9cef44a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java -- diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java index b6e793c..0542f7e 100644 --- a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java +++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java @@ -261,7 +261,7 @@ public class CqlPagingRecordReader extends RecordReader, { value.clear(); value.putAll(getCurrentValue()); - + keys.clear(); keys.putAll(getCurrentKey()); @@ -703,7 +703,7 @@ public class CqlPagingRecordReader extends RecordReader, clusterColumns.add(new BoundColumn(key)); parseKeyValidators(ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(2).getValue(; - + Column rawComparator = cqlRow.columns.get(3); String comparator = ByteBufferUtil.string(ByteBuffer.wrap(rawComparator.getValue())); logger.debug("comparator: {}", comparator); @@ -719,8 +719,8 @@ public class CqlPagingRecordReader extends RecordReader, } } -/** - * retrieve the fake partition keys and cluster keys for classic thrift table +/** + * retrieve the fake partition keys and cluster keys for classic thrift table * use CFDefinition to get keys and columns * */ private void retrieveKeysForThriftTables() throws Exception @@ -732,8 +732,10 @@ public class CqlPagingRecordReader extends RecordReader, { CFMetaData cfMeta = CFMetaData.fromThrift(cfDef); CFDefinition cfDefinition = new CFDefinition(cfMeta); -for (ColumnIdentifier columnIdentifier : cfDefinition.keys.keySet()) -partitionBoundColumns.add(new BoundColumn(columnIdentifier.toString())); +for (ColumnIdentifier key : cfDefinition.keys.keySet()) +partitionBoundColumns.add(new BoundColumn(key.toString())); +for (ColumnIdentifier column : cfDefinition.columns.keySet()) +clusterColumns.add(new BoundColumn(column.toString())); parseKeyValidators(cfDef.key_validation_class); return; } @@ -814,7 +816,7 @@ public class CqlPagingRecordReader extends RecordReader, this.name = name; } } - + /** get string from a ByteBuffer, catch the exception and throw it as runtime exception*/ private static String stringValue(ByteBuffer value) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9cef44a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java -- diff --git a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java index 223cbf4..6f6aa0b 100644 --- a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java +++ b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java @@ -47,7 +47,7 @@ import org.junit.BeforeClass; import org.junit.Test; public class ThriftColumnFamilyTest extends PigTestBase -{ +{ private static String[] statemen
[02/13] git commit: Support DISTINCT for static columns
Support DISTINCT for static columns This also fix the behaviour when DISTINCT is not used. patch by slebresne; reviewed by thobbs for CASSANDRA-7305 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6e4dca02 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6e4dca02 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6e4dca02 Branch: refs/heads/trunk Commit: 6e4dca02a720ac9277370ba1d4cf387ef1a3cfd4 Parents: 77bbcc1 Author: Sylvain Lebresne Authored: Wed Jun 25 11:30:53 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:12:58 2014 +0200 -- CHANGES.txt | 2 + NEWS.txt| 9 ++- .../cql3/statements/SelectStatement.java| 72 +++- 3 files changed, 48 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6e4dca02/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 84be96d..c3fe8d7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -21,6 +21,8 @@ * Account for range tombstones in min/max column names (CASSANDRA-7235) * Improve sub range repair validation (CASSANDRA-7317) * Accept subtypes for function results, type casts (CASSANDRA-6766) + * Support DISTINCT for static columns and fix behaviour when DISTINC is + not use (CASSANDRA-7305). Merged from 1.2: * Expose global ColumnFamily metrics (CASSANDRA-7273) * cqlsh: Fix CompositeType columns in DESCRIBE TABLE output (CASSANDRA-7399) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6e4dca02/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 422330c..0fbc20f 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -16,11 +16,16 @@ using the provided 'sstableupgrade' tool. 2.0.9 = -Operations --- +Upgrading +- - Default values for read_repair_chance and local_read_repair_chance have been swapped. Namely, default read_repair_chance is now set to 0.0, and default local_read_repair_chance to 0.1. +- Queries selecting only CQL static columns were (mistakenly) not returning one + result per row in the partition. This has been fixed and a SELECT DISTINCT + can be used when only the static column of a partition needs to be fetch + without fetching the whole partition. But if you use static columns, please + make sure this won't affect you (see CASSANDRA-7305 for details). 2.0.8 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6e4dca02/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java -- diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 7a91517..f106402 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -427,12 +427,26 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache } } +private ColumnSlice makeStaticSlice() +{ +ColumnNameBuilder staticPrefix = cfDef.cfm.getStaticColumnNameBuilder(); +// Note: we could use staticPrefix.build() for the start bound, but EMPTY_BYTE_BUFFER gives us the +// same effect while saving a few CPU cycles. +return isReversed + ? new ColumnSlice(staticPrefix.buildAsEndOfRange(), ByteBufferUtil.EMPTY_BYTE_BUFFER) + : new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, staticPrefix.buildAsEndOfRange()); +} + private IDiskAtomFilter makeFilter(List variables, int limit) throws InvalidRequestException { +int toGroup = cfDef.isCompact ? -1 : cfDef.clusteringColumnsCount(); if (parameters.isDistinct) { -return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, -1); +// For distinct, we only care about fetching the beginning of each partition. If we don't have +// static columns, we in fact only care about the first cell, so we query only that (we don't "group"). +// If we do have static columns, we do need to fetch the first full group (to have the static columns values). +return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, selectsStaticColumns ? toGroup : -1); } else if (isColumnRange()) { @@ -440,7 +454,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache // to account for the grouping of columns. // Since that doesn't work for
[7/7] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: build.xml debian/changelog src/java/org/apache/cassandra/cql3/statements/SelectStatement.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d5e5659e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d5e5659e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d5e5659e Branch: refs/heads/cassandra-2.1 Commit: d5e5659e419833a39e19b20e267312c41f99d685 Parents: e88b888 5b878ce Author: Sylvain Lebresne Authored: Thu Jun 26 18:01:35 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 18:01:35 2014 +0200 -- CHANGES.txt | 4 +++ .../cassandra/cql3/SingleColumnRelation.java| 10 ++ .../cql3/statements/SelectStatement.java| 34 3 files changed, 48 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5e5659e/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5e5659e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java -- diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 1e1b03f,98bd99a..a4a5553 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@@ -1950,24 -1977,42 +1950,58 @@@ public class SelectStatement implement "thus may have unpredictable performance. If you want to execute " + "this query despite the performance unpredictability, use ALLOW FILTERING"); } + + // We don't internally support exclusive slice bounds on non-composite tables. To deal with it we do an + // inclusive slice and remove post-query the value that shouldn't be returned. One problem however is that + // if there is a user limit, that limit may make the query return before the end of the slice is reached, + // in which case, once we'll have removed bound post-query, we might end up with less results than + // requested which would be incorrect. For single-partition query, this is not a problem, we just ask for + // one more result (see updateLimitForQuery()) since that's enough to compensate for that problem. For key + // range however, each returned row may include one result that will have to be trimmed, so we would have + // to bump the query limit by N where N is the number of rows we will return, but we don't know that in + // advance. So, since we currently don't have a good way to handle such query, we refuse it (#7059) rather + // than answering with something that is wrong. + if (stmt.sliceRestriction != null && stmt.isKeyRange && limit != null) + { -SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(stmt.cfDef); ++SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(stmt.cfm); + throw new InvalidRequestException(String.format("The query requests a restriction of rows with a strict bound (%s) over a range of partitions. " + + "This is not supported by the underlying storage engine for COMPACT tables if a LIMIT is provided. " + + "Please either make the condition non strict (%s) or remove the user LIMIT", rel, rel.withNonStrictOperator())); + } } -private SingleColumnRelation findInclusiveClusteringRelationForCompact(CFDefinition cfDef) +private int indexOf(ColumnDefinition def, Selection selection) +{ +return indexOf(def, selection.getColumns().iterator()); +} + +private int indexOf(final ColumnDefinition def, Iterator defs) +{ +return Iterators.indexOf(defs, new Predicate() + { + public boolean apply(ColumnDefinition n) + { + return def.name.equals(n.name); + } + }); +} + ++private SingleColumnRelation findInclusiveClusteringRelationForCompact(CFMetaData cfm) + { + for (Relation r : whereClause) + {
[11/13] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e8500c72 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e8500c72 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e8500c72 Branch: refs/heads/trunk Commit: e8500c72c7ee117bcb55ac728ee6bbd612eba5b3 Parents: d5e5659 ac1d5bd Author: Brandon Williams Authored: Thu Jun 26 11:02:06 2014 -0500 Committer: Brandon Williams Committed: Thu Jun 26 11:02:06 2014 -0500 -- conf/cassandra.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8500c72/conf/cassandra.yaml --
[08/13] git commit: Document initial_token/num_tokens behavior when both are set
Document initial_token/num_tokens behavior when both are set Patch by Philip Thompson, reviewed by brandonwilliams for CASSANDRA-7411 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ac1d5bd5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ac1d5bd5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ac1d5bd5 Branch: refs/heads/cassandra-2.1 Commit: ac1d5bd56f5ae96808270358fc45cb19e487e3d8 Parents: 5b878ce Author: Brandon Williams Authored: Thu Jun 26 10:53:24 2014 -0500 Committer: Brandon Williams Committed: Thu Jun 26 10:54:10 2014 -0500 -- conf/cassandra.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac1d5bd5/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index ef8194e..ea4d955 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -17,7 +17,8 @@ cluster_name: 'Test Cluster' # If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, # and will use the initial_token as described below. # -# Specifying initial_token will override this setting. +# Specifying initial_token will override this setting on the node's initial start, +# on subsequent starts, this setting will apply even if initial token is set. # # If you already have a cluster with 1 token per node, and wish to migrate to # multiple tokens per node, see http://wiki.apache.org/cassandra/Operations
[1/7] git commit: Update versions for 1.2.17 release
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 e88b88891 -> d5e5659e4 Update versions for 1.2.17 release Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/87c4efe8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/87c4efe8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/87c4efe8 Branch: refs/heads/cassandra-2.1 Commit: 87c4efe819b36c9f99f9f616e076fc61cbf3316e Parents: d84b692 Author: Sylvain Lebresne Authored: Wed Jun 25 18:25:30 2014 +0200 Committer: Sylvain Lebresne Committed: Wed Jun 25 18:25:30 2014 +0200 -- NEWS.txt | 8 build.xml| 2 +- debian/changelog | 6 ++ 3 files changed, 15 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/87c4efe8/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index f297634..97d32f2 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -13,6 +13,14 @@ restore snapshots created with the previous major version using the 'sstableloader' tool. You can upgrade the file format of your snapshots using the provided 'sstableupgrade' tool. +1.2.17 +== + +Upgrading +- +- Nothing specific to this release, but please see 1.2.16 if you are upgrading + from a previous version. + 1.2.16 == http://git-wip-us.apache.org/repos/asf/cassandra/blob/87c4efe8/build.xml -- diff --git a/build.xml b/build.xml index 5db0a6a..1310d64 100644 --- a/build.xml +++ b/build.xml @@ -25,7 +25,7 @@ - + http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree"/> http://git-wip-us.apache.org/repos/asf/cassandra/blob/87c4efe8/debian/changelog -- diff --git a/debian/changelog b/debian/changelog index 50318c8..adb7a37 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (1.2.17) unstable; urgency=medium + + * New release + + -- Sylvain Lebresne Wed, 25 Jun 2014 17:54:01 +0200 + cassandra (1.2.16) unstable; urgency=low * New release
[2/7] git commit: Support DISTINCT for static columns
Support DISTINCT for static columns This also fix the behaviour when DISTINCT is not used. patch by slebresne; reviewed by thobbs for CASSANDRA-7305 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6e4dca02 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6e4dca02 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6e4dca02 Branch: refs/heads/cassandra-2.1 Commit: 6e4dca02a720ac9277370ba1d4cf387ef1a3cfd4 Parents: 77bbcc1 Author: Sylvain Lebresne Authored: Wed Jun 25 11:30:53 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:12:58 2014 +0200 -- CHANGES.txt | 2 + NEWS.txt| 9 ++- .../cql3/statements/SelectStatement.java| 72 +++- 3 files changed, 48 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6e4dca02/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 84be96d..c3fe8d7 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -21,6 +21,8 @@ * Account for range tombstones in min/max column names (CASSANDRA-7235) * Improve sub range repair validation (CASSANDRA-7317) * Accept subtypes for function results, type casts (CASSANDRA-6766) + * Support DISTINCT for static columns and fix behaviour when DISTINC is + not use (CASSANDRA-7305). Merged from 1.2: * Expose global ColumnFamily metrics (CASSANDRA-7273) * cqlsh: Fix CompositeType columns in DESCRIBE TABLE output (CASSANDRA-7399) http://git-wip-us.apache.org/repos/asf/cassandra/blob/6e4dca02/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index 422330c..0fbc20f 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -16,11 +16,16 @@ using the provided 'sstableupgrade' tool. 2.0.9 = -Operations --- +Upgrading +- - Default values for read_repair_chance and local_read_repair_chance have been swapped. Namely, default read_repair_chance is now set to 0.0, and default local_read_repair_chance to 0.1. +- Queries selecting only CQL static columns were (mistakenly) not returning one + result per row in the partition. This has been fixed and a SELECT DISTINCT + can be used when only the static column of a partition needs to be fetch + without fetching the whole partition. But if you use static columns, please + make sure this won't affect you (see CASSANDRA-7305 for details). 2.0.8 http://git-wip-us.apache.org/repos/asf/cassandra/blob/6e4dca02/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java -- diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 7a91517..f106402 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -427,12 +427,26 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache } } +private ColumnSlice makeStaticSlice() +{ +ColumnNameBuilder staticPrefix = cfDef.cfm.getStaticColumnNameBuilder(); +// Note: we could use staticPrefix.build() for the start bound, but EMPTY_BYTE_BUFFER gives us the +// same effect while saving a few CPU cycles. +return isReversed + ? new ColumnSlice(staticPrefix.buildAsEndOfRange(), ByteBufferUtil.EMPTY_BYTE_BUFFER) + : new ColumnSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER, staticPrefix.buildAsEndOfRange()); +} + private IDiskAtomFilter makeFilter(List variables, int limit) throws InvalidRequestException { +int toGroup = cfDef.isCompact ? -1 : cfDef.clusteringColumnsCount(); if (parameters.isDistinct) { -return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, -1); +// For distinct, we only care about fetching the beginning of each partition. If we don't have +// static columns, we in fact only care about the first cell, so we query only that (we don't "group"). +// If we do have static columns, we do need to fetch the first full group (to have the static columns values). +return new SliceQueryFilter(ColumnSlice.ALL_COLUMNS_ARRAY, false, 1, selectsStaticColumns ? toGroup : -1); } else if (isColumnRange()) { @@ -440,7 +454,6 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache // to account for the grouping of columns. // Since that doesn't w
[4/7] git commit: Versions update for 2.0.9 release
Versions update for 2.0.9 release Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/79f3874d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/79f3874d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/79f3874d Branch: refs/heads/cassandra-2.1 Commit: 79f3874d81cfe2edffd1272876f563b7b0be16cc Parents: 79a4dd5 Author: Sylvain Lebresne Authored: Thu Jun 26 10:34:53 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:34:53 2014 +0200 -- build.xml| 2 +- debian/changelog | 6 ++ 2 files changed, 7 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/79f3874d/build.xml -- diff --git a/build.xml b/build.xml index 3828d96..ddf74e4 100644 --- a/build.xml +++ b/build.xml @@ -25,7 +25,7 @@ - + http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree"/> http://git-wip-us.apache.org/repos/asf/cassandra/blob/79f3874d/debian/changelog -- diff --git a/debian/changelog b/debian/changelog index 1ab0b75..73fea4c 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (2.0.9) unstable; urgency=medium + + * New release + + -- Sylvain Lebresne Thu, 26 Jun 2014 10:30:22 +0200 + cassandra (2.0.8) unstable; urgency=medium * New release
[06/13] git commit: Merge commit 'c9cef44a1a6c10036200f410c8a26942e64c8f12' into cassandra-2.0
Merge commit 'c9cef44a1a6c10036200f410c8a26942e64c8f12' 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/5b878ceb Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5b878ceb Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5b878ceb Branch: refs/heads/trunk Commit: 5b878cebdfbd5ad1ea63eac5aa76f838b05fc038 Parents: 79f3874 c9cef44 Author: Sylvain Lebresne Authored: Thu Jun 26 10:49:22 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:49:22 2014 +0200 -- --
[12/13] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e8500c72 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e8500c72 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e8500c72 Branch: refs/heads/cassandra-2.1 Commit: e8500c72c7ee117bcb55ac728ee6bbd612eba5b3 Parents: d5e5659 ac1d5bd Author: Brandon Williams Authored: Thu Jun 26 11:02:06 2014 -0500 Committer: Brandon Williams Committed: Thu Jun 26 11:02:06 2014 -0500 -- conf/cassandra.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8500c72/conf/cassandra.yaml --
[10/13] git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: build.xml debian/changelog src/java/org/apache/cassandra/cql3/statements/SelectStatement.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d5e5659e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d5e5659e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d5e5659e Branch: refs/heads/trunk Commit: d5e5659e419833a39e19b20e267312c41f99d685 Parents: e88b888 5b878ce Author: Sylvain Lebresne Authored: Thu Jun 26 18:01:35 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 18:01:35 2014 +0200 -- CHANGES.txt | 4 +++ .../cassandra/cql3/SingleColumnRelation.java| 10 ++ .../cql3/statements/SelectStatement.java| 34 3 files changed, 48 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5e5659e/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/d5e5659e/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java -- diff --cc src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index 1e1b03f,98bd99a..a4a5553 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@@ -1950,24 -1977,42 +1950,58 @@@ public class SelectStatement implement "thus may have unpredictable performance. If you want to execute " + "this query despite the performance unpredictability, use ALLOW FILTERING"); } + + // We don't internally support exclusive slice bounds on non-composite tables. To deal with it we do an + // inclusive slice and remove post-query the value that shouldn't be returned. One problem however is that + // if there is a user limit, that limit may make the query return before the end of the slice is reached, + // in which case, once we'll have removed bound post-query, we might end up with less results than + // requested which would be incorrect. For single-partition query, this is not a problem, we just ask for + // one more result (see updateLimitForQuery()) since that's enough to compensate for that problem. For key + // range however, each returned row may include one result that will have to be trimmed, so we would have + // to bump the query limit by N where N is the number of rows we will return, but we don't know that in + // advance. So, since we currently don't have a good way to handle such query, we refuse it (#7059) rather + // than answering with something that is wrong. + if (stmt.sliceRestriction != null && stmt.isKeyRange && limit != null) + { -SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(stmt.cfDef); ++SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(stmt.cfm); + throw new InvalidRequestException(String.format("The query requests a restriction of rows with a strict bound (%s) over a range of partitions. " + + "This is not supported by the underlying storage engine for COMPACT tables if a LIMIT is provided. " + + "Please either make the condition non strict (%s) or remove the user LIMIT", rel, rel.withNonStrictOperator())); + } } -private SingleColumnRelation findInclusiveClusteringRelationForCompact(CFDefinition cfDef) +private int indexOf(ColumnDefinition def, Selection selection) +{ +return indexOf(def, selection.getColumns().iterator()); +} + +private int indexOf(final ColumnDefinition def, Iterator defs) +{ +return Iterators.indexOf(defs, new Predicate() + { + public boolean apply(ColumnDefinition n) + { + return def.name.equals(n.name); + } + }); +} + ++private SingleColumnRelation findInclusiveClusteringRelationForCompact(CFMetaData cfm) + { + for (Relation r : whereClause) + { +
[01/13] git commit: Update versions for 1.2.17 release
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 5b878cebd -> ac1d5bd56 refs/heads/cassandra-2.1 d5e5659e4 -> e8500c72c refs/heads/trunk ae0f63463 -> 652a7ea71 Update versions for 1.2.17 release Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/87c4efe8 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/87c4efe8 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/87c4efe8 Branch: refs/heads/trunk Commit: 87c4efe819b36c9f99f9f616e076fc61cbf3316e Parents: d84b692 Author: Sylvain Lebresne Authored: Wed Jun 25 18:25:30 2014 +0200 Committer: Sylvain Lebresne Committed: Wed Jun 25 18:25:30 2014 +0200 -- NEWS.txt | 8 build.xml| 2 +- debian/changelog | 6 ++ 3 files changed, 15 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/87c4efe8/NEWS.txt -- diff --git a/NEWS.txt b/NEWS.txt index f297634..97d32f2 100644 --- a/NEWS.txt +++ b/NEWS.txt @@ -13,6 +13,14 @@ restore snapshots created with the previous major version using the 'sstableloader' tool. You can upgrade the file format of your snapshots using the provided 'sstableupgrade' tool. +1.2.17 +== + +Upgrading +- +- Nothing specific to this release, but please see 1.2.16 if you are upgrading + from a previous version. + 1.2.16 == http://git-wip-us.apache.org/repos/asf/cassandra/blob/87c4efe8/build.xml -- diff --git a/build.xml b/build.xml index 5db0a6a..1310d64 100644 --- a/build.xml +++ b/build.xml @@ -25,7 +25,7 @@ - + http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree"/> http://git-wip-us.apache.org/repos/asf/cassandra/blob/87c4efe8/debian/changelog -- diff --git a/debian/changelog b/debian/changelog index 50318c8..adb7a37 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (1.2.17) unstable; urgency=medium + + * New release + + -- Sylvain Lebresne Wed, 25 Jun 2014 17:54:01 +0200 + cassandra (1.2.16) unstable; urgency=low * New release
[09/13] git commit: Document initial_token/num_tokens behavior when both are set
Document initial_token/num_tokens behavior when both are set Patch by Philip Thompson, reviewed by brandonwilliams for CASSANDRA-7411 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ac1d5bd5 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ac1d5bd5 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ac1d5bd5 Branch: refs/heads/cassandra-2.0 Commit: ac1d5bd56f5ae96808270358fc45cb19e487e3d8 Parents: 5b878ce Author: Brandon Williams Authored: Thu Jun 26 10:53:24 2014 -0500 Committer: Brandon Williams Committed: Thu Jun 26 10:54:10 2014 -0500 -- conf/cassandra.yaml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ac1d5bd5/conf/cassandra.yaml -- diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml index ef8194e..ea4d955 100644 --- a/conf/cassandra.yaml +++ b/conf/cassandra.yaml @@ -17,7 +17,8 @@ cluster_name: 'Test Cluster' # If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, # and will use the initial_token as described below. # -# Specifying initial_token will override this setting. +# Specifying initial_token will override this setting on the node's initial start, +# on subsequent starts, this setting will apply even if initial token is set. # # If you already have a cluster with 1 token per node, and wish to migrate to # multiple tokens per node, see http://wiki.apache.org/cassandra/Operations
[04/13] git commit: Versions update for 2.0.9 release
Versions update for 2.0.9 release Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/79f3874d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/79f3874d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/79f3874d Branch: refs/heads/trunk Commit: 79f3874d81cfe2edffd1272876f563b7b0be16cc Parents: 79a4dd5 Author: Sylvain Lebresne Authored: Thu Jun 26 10:34:53 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:34:53 2014 +0200 -- build.xml| 2 +- debian/changelog | 6 ++ 2 files changed, 7 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/79f3874d/build.xml -- diff --git a/build.xml b/build.xml index 3828d96..ddf74e4 100644 --- a/build.xml +++ b/build.xml @@ -25,7 +25,7 @@ - + http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=tree"/> http://git-wip-us.apache.org/repos/asf/cassandra/blob/79f3874d/debian/changelog -- diff --git a/debian/changelog b/debian/changelog index 1ab0b75..73fea4c 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,3 +1,9 @@ +cassandra (2.0.9) unstable; urgency=medium + + * New release + + -- Sylvain Lebresne Thu, 26 Jun 2014 10:30:22 +0200 + cassandra (2.0.8) unstable; urgency=medium * New release
[6/7] git commit: Merge commit 'c9cef44a1a6c10036200f410c8a26942e64c8f12' into cassandra-2.0
Merge commit 'c9cef44a1a6c10036200f410c8a26942e64c8f12' 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/5b878ceb Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/5b878ceb Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/5b878ceb Branch: refs/heads/cassandra-2.1 Commit: 5b878cebdfbd5ad1ea63eac5aa76f838b05fc038 Parents: 79f3874 c9cef44 Author: Sylvain Lebresne Authored: Thu Jun 26 10:49:22 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:49:22 2014 +0200 -- --
[5/7] git commit: Support Thrift tables clustering columns on CqlPagingInputFormat
Support Thrift tables clustering columns on CqlPagingInputFormat patch by pauloricardomg; reviewed by alexliu68 for CASSANDRA-7445 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/c9cef44a Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/c9cef44a Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/c9cef44a Branch: refs/heads/cassandra-2.1 Commit: c9cef44a1a6c10036200f410c8a26942e64c8f12 Parents: 87c4efe Author: Sylvain Lebresne Authored: Thu Jun 26 10:40:22 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:40:22 2014 +0200 -- CHANGES.txt | 3 + .../hadoop/cql3/CqlPagingRecordReader.java | 16 ++--- .../cassandra/pig/ThriftColumnFamilyTest.java | 61 +--- 3 files changed, 65 insertions(+), 15 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9cef44a/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 6adef97..9fbcd9a 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,3 +1,6 @@ +1.2.18 + * Support Thrift tables clustering columns on CqlPagingInputFormat (CASSANDRA-7445) + 1.2.17 * cqlsh: Fix CompositeType columns in DESCRIBE TABLE output (CASSANDRA-7399) * Expose global ColmunFamily metrics (CASSANDRA-7273) http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9cef44a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java -- diff --git a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java b/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java index b6e793c..0542f7e 100644 --- a/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java +++ b/src/java/org/apache/cassandra/hadoop/cql3/CqlPagingRecordReader.java @@ -261,7 +261,7 @@ public class CqlPagingRecordReader extends RecordReader, { value.clear(); value.putAll(getCurrentValue()); - + keys.clear(); keys.putAll(getCurrentKey()); @@ -703,7 +703,7 @@ public class CqlPagingRecordReader extends RecordReader, clusterColumns.add(new BoundColumn(key)); parseKeyValidators(ByteBufferUtil.string(ByteBuffer.wrap(cqlRow.columns.get(2).getValue(; - + Column rawComparator = cqlRow.columns.get(3); String comparator = ByteBufferUtil.string(ByteBuffer.wrap(rawComparator.getValue())); logger.debug("comparator: {}", comparator); @@ -719,8 +719,8 @@ public class CqlPagingRecordReader extends RecordReader, } } -/** - * retrieve the fake partition keys and cluster keys for classic thrift table +/** + * retrieve the fake partition keys and cluster keys for classic thrift table * use CFDefinition to get keys and columns * */ private void retrieveKeysForThriftTables() throws Exception @@ -732,8 +732,10 @@ public class CqlPagingRecordReader extends RecordReader, { CFMetaData cfMeta = CFMetaData.fromThrift(cfDef); CFDefinition cfDefinition = new CFDefinition(cfMeta); -for (ColumnIdentifier columnIdentifier : cfDefinition.keys.keySet()) -partitionBoundColumns.add(new BoundColumn(columnIdentifier.toString())); +for (ColumnIdentifier key : cfDefinition.keys.keySet()) +partitionBoundColumns.add(new BoundColumn(key.toString())); +for (ColumnIdentifier column : cfDefinition.columns.keySet()) +clusterColumns.add(new BoundColumn(column.toString())); parseKeyValidators(cfDef.key_validation_class); return; } @@ -814,7 +816,7 @@ public class CqlPagingRecordReader extends RecordReader, this.name = name; } } - + /** get string from a ByteBuffer, catch the exception and throw it as runtime exception*/ private static String stringValue(ByteBuffer value) { http://git-wip-us.apache.org/repos/asf/cassandra/blob/c9cef44a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java -- diff --git a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java index 223cbf4..6f6aa0b 100644 --- a/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java +++ b/test/unit/org/apache/cassandra/pig/ThriftColumnFamilyTest.java @@ -47,7 +47,7 @@ import org.junit.BeforeClass; import org.junit.Test; public class ThriftColumnFamilyTest extends PigTestBase -{ +{ private static String[]
[3/7] git commit: Refuse range queries with strict bounds on compact tables
Refuse range queries with strict bounds on compact tables patch by slebresne; reviewed by iamaleksey for CASSANDRA-7059 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/79a4dd58 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/79a4dd58 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/79a4dd58 Branch: refs/heads/cassandra-2.1 Commit: 79a4dd58cd060cc28f32e0fc17001fe1179552a2 Parents: 6e4dca0 Author: Sylvain Lebresne Authored: Thu May 8 17:51:29 2014 +0200 Committer: Sylvain Lebresne Committed: Thu Jun 26 10:12:59 2014 +0200 -- CHANGES.txt | 2 ++ .../cassandra/cql3/SingleColumnRelation.java| 10 ++ .../cql3/statements/SelectStatement.java| 34 3 files changed, 46 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/79a4dd58/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index c3fe8d7..2b3ace3 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -23,6 +23,8 @@ * Accept subtypes for function results, type casts (CASSANDRA-6766) * Support DISTINCT for static columns and fix behaviour when DISTINC is not use (CASSANDRA-7305). + * Refuse range queries with strict bounds on compact tables since they + are broken (CASSANDRA-7059) Merged from 1.2: * Expose global ColumnFamily metrics (CASSANDRA-7273) * cqlsh: Fix CompositeType columns in DESCRIBE TABLE output (CASSANDRA-7399) http://git-wip-us.apache.org/repos/asf/cassandra/blob/79a4dd58/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java -- diff --git a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java index 5464c23..642be66 100644 --- a/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java +++ b/src/java/org/apache/cassandra/cql3/SingleColumnRelation.java @@ -84,6 +84,16 @@ public class SingleColumnRelation extends Relation return false; } +public SingleColumnRelation withNonStrictOperator() +{ +switch (relationType) +{ +case GT: return new SingleColumnRelation(entity, Type.GTE, value); +case LT: return new SingleColumnRelation(entity, Type.LTE, value); +default: return this; +} +} + @Override public String toString() { http://git-wip-us.apache.org/repos/asf/cassandra/blob/79a4dd58/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java -- diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java index f106402..98bd99a 100644 --- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java +++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java @@ -1977,6 +1977,40 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache "thus may have unpredictable performance. If you want to execute " + "this query despite the performance unpredictability, use ALLOW FILTERING"); } + +// We don't internally support exclusive slice bounds on non-composite tables. To deal with it we do an +// inclusive slice and remove post-query the value that shouldn't be returned. One problem however is that +// if there is a user limit, that limit may make the query return before the end of the slice is reached, +// in which case, once we'll have removed bound post-query, we might end up with less results than +// requested which would be incorrect. For single-partition query, this is not a problem, we just ask for +// one more result (see updateLimitForQuery()) since that's enough to compensate for that problem. For key +// range however, each returned row may include one result that will have to be trimmed, so we would have +// to bump the query limit by N where N is the number of rows we will return, but we don't know that in +// advance. So, since we currently don't have a good way to handle such query, we refuse it (#7059) rather +// than answering with something that is wrong. +if (stmt.sliceRestriction != null && stmt.isKeyRange && limit != null) +{ +SingleColumnRelation rel = findInclusiveClusteringRelationForCompact(stmt.cfDef); +throw new InvalidRequestException(String.format("The query
[jira] [Commented] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044778#comment-14044778 ] graham sanderson commented on CASSANDRA-7449: - I guess using compositesToGroup, you could set count to 2, but that still isn't 1 ! > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 108 > (1 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point B > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 87 > 123 |2 | gibbon | this is large and should not be inserted > every time | 111 > (2 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > 123 > (2 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point C > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 67 > 123 |2 | gibbon | this is large and should not be inserted > every time | 91 > 123 |3 | firebucket | this is large and should not be inserted > every time | 110 > (3 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > 123 > 1
[jira] [Updated] (CASSANDRA-5345) Potential problem with GarbageCollectorMXBean
[ https://issues.apache.org/jira/browse/CASSANDRA-5345?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joshua McKenzie updated CASSANDRA-5345: --- Attachment: 5345_v1.txt Attaching a v1 that tries to gracefully check for GC MXBean validity on log and if an invalid GC is found, skip it and rebuild the list of cached MXBean's for next logging. Given that I can't find any info on *when* these things are getting recycled, I also put in some exception handling in-case the change happens in the middle of our logging process. This doesn't address the failure of nodetool info that Arya saw with nodetool info; that issue seems like a related but perhaps different (and less critical) effort than this ticket. > Potential problem with GarbageCollectorMXBean > - > > Key: CASSANDRA-5345 > URL: https://issues.apache.org/jira/browse/CASSANDRA-5345 > Project: Cassandra > Issue Type: Bug > Components: Core >Affects Versions: 1.0.7 > Environment: JVM:JVM vendor/version: Java HotSpot(TM) 64-Bit Server > VM/1.6.0_30 typical 6 node 2 availability zone Mutli DC cluster on linux vms > with > and mx4j-tools.jar and jna.jar both on path. Default configuration bar token > setup(equispaced), sensible cassandra-topology.properties file and use of > said snitch. >Reporter: Matt Byrd >Assignee: Joshua McKenzie > Attachments: 5345_v1.txt > > > I am not certain this is definitely a bug, but I thought it might be worth > posting to see if someone with more JVM//JMX knowledge could disprove my > reasoning. Apologies if I've failed to understand something. > We've seen an intermittent problem where there is an uncaught exception in > the scheduled task of logging gc results in GcInspector.java: > {code} > ... > ERROR [ScheduledTasks:1] 2013-03-08 01:09:06,335 > AbstractCassandraDaemon.java (line 139) Fatal exception in thread > Thread[ScheduledTasks:1,5,main] > java.lang.reflect.UndeclaredThrowableException > at $Proxy0.getName(Unknown Source) > at > org.apache.cassandra.service.GCInspector.logGCResults(GCInspector.java:95) > at > org.apache.cassandra.service.GCInspector.access$000(GCInspector.java:41) > at org.apache.cassandra.service.GCInspector$1.run(GCInspector.java:85) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:441) > at > java.util.concurrent.FutureTask$Sync.innerRunAndReset(FutureTask.java:317) > at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:150) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$101(ScheduledThreadPoolExecutor.java:98) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.runPeriodic(ScheduledThreadPoolExecutor.java:180) > at > java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:204) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) > at java.lang.Thread.run(Thread.java:662) > Caused by: javax.management.InstanceNotFoundException: > java.lang:name=ParNew,type=GarbageCollector > at > com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getMBean(DefaultMBeanServerInterceptor.java:1094) > at > com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.getAttribute(DefaultMBeanServerInterceptor.java:662) > at > com.sun.jmx.mbeanserver.JmxMBeanServer.getAttribute(JmxMBeanServer.java:638) > at > com.sun.jmx.mbeanserver.MXBeanProxy$GetHandler.invoke(MXBeanProxy.java:106) > at com.sun.jmx.mbeanserver.MXBeanProxy.invoke(MXBeanProxy.java:148) > at > javax.management.MBeanServerInvocationHandler.invoke(MBeanServerInvocationHandler.java:248) > ... 13 more > ... > {code} > I think the problem, may be caused by the following reasoning: > In GcInspector we populate a list of mxbeans when the GcInspector instance is > instantiated: > {code} > ... > List beans = new ArrayList(); > MBeanServer server = ManagementFactory.getPlatformMBeanServer(); > try > { > ObjectName gcName = new > ObjectName(ManagementFactory.GARBAGE_COLLECTOR_MXBEAN_DOMAIN_TYPE + ",*"); > for (ObjectName name : server.queryNames(gcName, null)) > { > GarbageCollectorMXBean gc = > ManagementFactory.newPlatformMXBeanProxy(server, name.getCanonicalName(), > GarbageCollectorMXBean.class); > beans.add(gc); > } > } > catch (Exception e) > { > throw new RuntimeException(e); > } > ... > {code} > Cassandra then periodically calls: > {code} > .
[jira] [Updated] (CASSANDRA-6621) STCS fallback is not optimal when bootstrapping
[ https://issues.apache.org/jira/browse/CASSANDRA-6621?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-6621: --- Attachment: 0001-option-to-disallow-L0-stcs.patch attaching patch against 2.0 to add an option to disallow stcs in L0 you would do something like {code} CREATE TABLE ... WITH compaction = {'class': 'LeveledCompactionStrategy', 'stcs_in_l0': 'false'}; {code} > STCS fallback is not optimal when bootstrapping > --- > > Key: CASSANDRA-6621 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6621 > Project: Cassandra > Issue Type: Improvement >Reporter: Bartłomiej Romański >Assignee: Marcus Eriksson >Priority: Minor > Labels: compaction, streaming > Fix For: 2.0.9 > > Attachments: 0001-option-to-disallow-L0-stcs.patch, > 0001-wip-keep-sstable-level-when-bootstrapping.patch > > > The initial discussion started in (closed) CASSANDRA-5371. I've rewritten my > last comment here... > After streaming (e.g. during boostrap) Cassandra places all sstables at L0. > At the end of the process we end up with huge number of sstables at the > lowest level. > Currently, Cassandra falls back to STCS until the number of sstables at L0 > reaches the reasonable level (32 or something). > I'm not sure if falling back to STCS is the best way to handle this > particular situation. I've read the comment in the code and I'm aware why it > is a good thing to do if we have to many sstables at L0 as a result of too > many random inserts. We have a lot of sstables, each of them covers the whole > ring, there's simply no better option. > However, after the bootstrap situation looks a bit different. The loaded > sstables already have very small ranges! We just have to tidy up a bit and > everything should be OK. STCS ignores that completely and after a while we > have a bit less sstables but each of them covers the whole ring instead of > just a small part. I believe that in that case letting LCS do the job is a > better option that allowing STCS mix everything up before. > Is there a way to disable STCS fallback? I'd like to test that scenario in > practice during our next bootstrap... > Does Cassandra really have to put streamed sstables at L0? The only thing we > have to assure is that sstables at any given level do not overlap. If we > stream different regions from different nodes how can we get any overlaps? -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7451) Launch scripts on Windows don't handle spaces gracefully
[ https://issues.apache.org/jira/browse/CASSANDRA-7451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joshua McKenzie updated CASSANDRA-7451: --- Reviewer: Jorge Bay (was: Philip Thompson) > Launch scripts on Windows don't handle spaces gracefully > > > Key: CASSANDRA-7451 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7451 > Project: Cassandra > Issue Type: Bug > Environment: 2.1.X >Reporter: Joshua McKenzie >Assignee: Joshua McKenzie >Priority: Minor > Labels: Windows > Fix For: 2.1.0 > > Attachments: 7451_v1.txt > > > There's also some .ps1 problems after we get past just the .bat. Should be > some trivial escaping to fix. > C:\src - Copy\cassandra\bin>cassandra.bat > Detected powershell execution permissions. Running with enhanced startup > scripts. > Processing -File 'C:\src' failed because the file does not have a '.ps1' > extension. Specify a valid PowerShell script file name, and then try again. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7451) Launch scripts on Windows don't handle spaces gracefully
[ https://issues.apache.org/jira/browse/CASSANDRA-7451?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joshua McKenzie updated CASSANDRA-7451: --- Attachment: 7451_v1.txt > Launch scripts on Windows don't handle spaces gracefully > > > Key: CASSANDRA-7451 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7451 > Project: Cassandra > Issue Type: Bug > Environment: 2.1.X >Reporter: Joshua McKenzie >Assignee: Joshua McKenzie >Priority: Minor > Labels: Windows > Fix For: 2.1.0 > > Attachments: 7451_v1.txt > > > There's also some .ps1 problems after we get past just the .bat. Should be > some trivial escaping to fix. > C:\src - Copy\cassandra\bin>cassandra.bat > Detected powershell execution permissions. Running with enhanced startup > scripts. > Processing -File 'C:\src' failed because the file does not have a '.ps1' > extension. Specify a valid PowerShell script file name, and then try again. -- This message was sent by Atlassian JIRA (v6.2#6252)
git commit: Expose global CF metrics
Repository: cassandra Updated Branches: refs/heads/trunk 7000bde6c -> ae0f63463 Expose global CF metrics Patch by Chris Lohfink, reviewed by brandonwilliams for CASSANDRA-7273 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/ae0f6346 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/ae0f6346 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/ae0f6346 Branch: refs/heads/trunk Commit: ae0f63463d5ff4e49b6dfa209d03dfaa98105d56 Parents: 7000bde Author: Brandon Williams Authored: Thu Jun 26 10:08:53 2014 -0500 Committer: Brandon Williams Committed: Thu Jun 26 10:08:53 2014 -0500 -- .../apache/cassandra/db/ColumnFamilyStore.java | 4 +- src/java/org/apache/cassandra/db/Keyspace.java | 2 +- .../cassandra/metrics/ColumnFamilyMetrics.java | 345 +++ .../cassandra/metrics/KeyspaceMetrics.java | 253 +++--- .../cassandra/metrics/LatencyMetrics.java | 28 +- .../org/apache/cassandra/db/KeyspaceTest.java | 16 +- 6 files changed, 439 insertions(+), 209 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/ae0f6346/src/java/org/apache/cassandra/db/ColumnFamilyStore.java -- diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java index 3975352..db3ed15 100644 --- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java +++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java @@ -2674,12 +2674,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean public double getTombstonesPerSlice() { -return metric.tombstoneScannedHistogram.getSnapshot().getMedian(); +return metric.tombstoneScannedHistogram.cf.getSnapshot().getMedian(); } public double getLiveCellsPerSlice() { -return metric.liveScannedHistogram.getSnapshot().getMedian(); +return metric.liveScannedHistogram.cf.getSnapshot().getMedian(); } // End JMX get/set. http://git-wip-us.apache.org/repos/asf/cassandra/blob/ae0f6346/src/java/org/apache/cassandra/db/Keyspace.java -- diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java index 60183db..9c686b7 100644 --- a/src/java/org/apache/cassandra/db/Keyspace.java +++ b/src/java/org/apache/cassandra/db/Keyspace.java @@ -273,12 +273,12 @@ public class Keyspace assert metadata != null : "Unknown keyspace " + keyspaceName; createReplicationStrategy(metadata); +this.metric = new KeyspaceMetrics(this); for (CFMetaData cfm : new ArrayList(metadata.cfMetaData().values())) { logger.debug("Initializing {}.{}", getName(), cfm.cfName); initCf(cfm.cfId, cfm.cfName, loadSSTables); } -this.metric = new KeyspaceMetrics(this); } public void createReplicationStrategy(KSMetaData ksm) http://git-wip-us.apache.org/repos/asf/cassandra/blob/ae0f6346/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java -- diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java index f4682da..75a21dc 100644 --- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java +++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java @@ -17,17 +17,23 @@ */ package org.apache.cassandra.metrics; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.ConcurrentMap; import java.util.concurrent.TimeUnit; -import com.yammer.metrics.Metrics; -import com.yammer.metrics.core.*; -import com.yammer.metrics.util.RatioGauge; - import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.io.sstable.SSTableReader; import org.apache.cassandra.io.sstable.metadata.MetadataCollector; import org.apache.cassandra.utils.EstimatedHistogram; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import com.yammer.metrics.Metrics; +import com.yammer.metrics.core.*; +import com.yammer.metrics.util.RatioGauge; + /** * Metrics for {@link ColumnFamilyStore}. */ @@ -56,7 +62,7 @@ public class ColumnFamilyMetrics /** Histogram of estimated number of columns. */ public final Gauge estimatedColumnCountHistogram; /** Histogram of the number of sstable data files accessed per read */ -public final Histogram sstablesPerReadHistogram; +public final ColumnFamilyHistogram sstablesPerReadHistogram; /** (Local) read metrics */
[jira] [Updated] (CASSANDRA-7411) Node enables vnodes when bounced
[ https://issues.apache.org/jira/browse/CASSANDRA-7411?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-7411: --- Reproduced In: 2.1 rc1, 2.0.8 (was: 2.0.8, 2.1 rc1) Fix Version/s: 2.1.1 Assignee: Philip Thompson > Node enables vnodes when bounced > > > Key: CASSANDRA-7411 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7411 > Project: Cassandra > Issue Type: Improvement > Environment: OSX 9 >Reporter: Philip Thompson >Assignee: Philip Thompson >Priority: Minor > Fix For: 2.1.1 > > Attachments: 7411.txt, system.log > > > According to cassandra.yaml, in the information for the num_tokens setting, > "Specifying initial_token will override this setting." So if exactly one > initial token is set, then vnodes are disabled, regardless of if or what > num_tokens are set to. This behavior is inconsistent when a node is started, > versus if it has been bounced. > From a fresh checkout of C*, if I build, then edit cassandra.yaml so that: > num_tokens: 256 > initial_token: -9223372036854775808 > then run bin/cassandra, C* will start correctly. I can run bin/nodetool ring > and see that the node has exactly one token and it is what I set in > initial_token. If I gracefully shutdown C*, then restart the node, running > bin/nodetool ring shows that the node now has vnodes enabled and has 256 > tokens. > I have been able to reproduce this locally on OSX using 2.0.8, 2.1 rc1, and > trunk. I have not yet tested in Linux or Windows to see if it occurs there. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7411) Node enables vnodes when bounced
[ https://issues.apache.org/jira/browse/CASSANDRA-7411?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-7411: --- Reviewer: Brandon Williams Reproduced In: 2.1 rc1, 2.0.8 (was: 2.0.8, 2.1 rc1) > Node enables vnodes when bounced > > > Key: CASSANDRA-7411 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7411 > Project: Cassandra > Issue Type: Improvement > Environment: OSX 9 >Reporter: Philip Thompson >Assignee: Philip Thompson >Priority: Minor > Fix For: 2.1.1 > > Attachments: 7411.txt, system.log > > > According to cassandra.yaml, in the information for the num_tokens setting, > "Specifying initial_token will override this setting." So if exactly one > initial token is set, then vnodes are disabled, regardless of if or what > num_tokens are set to. This behavior is inconsistent when a node is started, > versus if it has been bounced. > From a fresh checkout of C*, if I build, then edit cassandra.yaml so that: > num_tokens: 256 > initial_token: -9223372036854775808 > then run bin/cassandra, C* will start correctly. I can run bin/nodetool ring > and see that the node has exactly one token and it is what I set in > initial_token. If I gracefully shutdown C*, then restart the node, running > bin/nodetool ring shows that the node now has vnodes enabled and has 256 > tokens. > I have been able to reproduce this locally on OSX using 2.0.8, 2.1 rc1, and > trunk. I have not yet tested in Linux or Windows to see if it occurs there. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7451) Launch scripts on Windows don't handle spaces gracefully
Joshua McKenzie created CASSANDRA-7451: -- Summary: Launch scripts on Windows don't handle spaces gracefully Key: CASSANDRA-7451 URL: https://issues.apache.org/jira/browse/CASSANDRA-7451 Project: Cassandra Issue Type: Bug Environment: 2.1.X Reporter: Joshua McKenzie Assignee: Joshua McKenzie Priority: Minor Fix For: 2.1.0 There's also some .ps1 problems after we get past just the .bat. Should be some trivial escaping to fix. C:\src - Copy\cassandra\bin>cassandra.bat Detected powershell execution permissions. Running with enhanced startup scripts. Processing -File 'C:\src' failed because the file does not have a '.ps1' extension. Specify a valid PowerShell script file name, and then try again. -- This message was sent by Atlassian JIRA (v6.2#6252)
svn commit: r1605788 - in /cassandra/site: publish/download/index.html src/settings.py
Author: slebresne Date: Thu Jun 26 14:11:25 2014 New Revision: 1605788 URL: http://svn.apache.org/r1605788 Log: Update website for 2.1.0-rc2 release Modified: cassandra/site/publish/download/index.html cassandra/site/src/settings.py Modified: cassandra/site/publish/download/index.html URL: http://svn.apache.org/viewvc/cassandra/site/publish/download/index.html?rev=1605788&r1=1605787&r2=1605788&view=diff == --- cassandra/site/publish/download/index.html (original) +++ cassandra/site/publish/download/index.html Thu Jun 26 14:11:25 2014 @@ -98,16 +98,16 @@ Development Cassandra Server Releases (not production ready) - The latest development release is 2.1.0-rc1 (released on - 2014-06-02). + The latest development release is 2.1.0-rc2 (released on + 2014-06-26). -http://www.apache.org/dyn/closer.cgi?path=/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-bin.tar.gz";>apache-cassandra-2.1.0-rc1-bin.tar.gz -[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-bin.tar.gz.asc";>PGP] -[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-bin.tar.gz.md5";>MD5] -[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-bin.tar.gz.sha1";>SHA1] +http://www.apache.org/dyn/closer.cgi?path=/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-bin.tar.gz";>apache-cassandra-2.1.0-rc2-bin.tar.gz +[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-bin.tar.gz.asc";>PGP] +[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-bin.tar.gz.md5";>MD5] +[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-bin.tar.gz.sha1";>SHA1] @@ -186,10 +186,10 @@ -http://www.apache.org/dyn/closer.cgi?path=/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-src.tar.gz";>apache-cassandra-2.1.0-rc1-src.tar.gz -[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-src.tar.gz.asc";>PGP] -[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-src.tar.gz.md5";>MD5] -[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc1-src.tar.gz.sha1";>SHA1] +http://www.apache.org/dyn/closer.cgi?path=/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-src.tar.gz";>apache-cassandra-2.1.0-rc2-src.tar.gz +[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-src.tar.gz.asc";>PGP] +[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-src.tar.gz.md5";>MD5] +[http://www.apache.org/dist/cassandra/2.1.0/apache-cassandra-2.1.0-rc2-src.tar.gz.sha1";>SHA1] Modified: cassandra/site/src/settings.py URL: http://svn.apache.org/viewvc/cassandra/site/src/settings.py?rev=1605788&r1=1605787&r2=1605788&view=diff == --- cassandra/site/src/settings.py (original) +++ cassandra/site/src/settings.py Thu Jun 26 14:11:25 2014 @@ -100,8 +100,8 @@ class CassandraDef(object): veryoldstable_exists = True stable_version = '2.0.8' stable_release_date = '2014-05-29' -devel_version = '2.1.0-rc1' -devel_release_date = '2014-06-02' +devel_version = '2.1.0-rc2' +devel_release_date = '2014-06-26' devel_exists = True _apache_base_url = 'http://www.apache.org' _svn_base_url = 'https://svn.apache.org/repos/asf'
[jira] [Commented] (CASSANDRA-5263) Increase merkle tree depth as needed
[ https://issues.apache.org/jira/browse/CASSANDRA-5263?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044662#comment-14044662 ] Yuki Morishita commented on CASSANDRA-5263: --- bq. total number of unique partitions in stables containing the range using cardinality Stored cardinality is for whole SSTable so we cannot use it to estimate cardinality for arbitaly range. > Increase merkle tree depth as needed > > > Key: CASSANDRA-5263 > URL: https://issues.apache.org/jira/browse/CASSANDRA-5263 > Project: Cassandra > Issue Type: Improvement >Affects Versions: 1.1.9 >Reporter: Ahmed Bashir >Assignee: Yuki Morishita > Fix For: 2.1.1 > > Attachments: 5263-2.1-v1.txt > > > Currently, the maximum depth allowed for Merkle trees is hardcoded as 15. > This value should be configurable, just like phi_convict_treshold and other > properties. > Given a cluster with nodes responsible for a large number of row keys, Merkle > tree comparisons can result in a large amount of unnecessary row keys being > streamed. > Empirical testing indicates that reasonable changes to this depth (18, 20, > etc) don't affect the Merkle tree generation and differencing timings all > that much, and they can significantly reduce the amount of data being > streamed during repair. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7445) Support Thrift tables clustering columns on CqlPagingInputFormat
[ https://issues.apache.org/jira/browse/CASSANDRA-7445?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044656#comment-14044656 ] Paulo Motta commented on CASSANDRA-7445: nothing to fix for 2.0. cheers! > Support Thrift tables clustering columns on CqlPagingInputFormat > > > Key: CASSANDRA-7445 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7445 > Project: Cassandra > Issue Type: Bug > Components: Hadoop >Reporter: Paulo Motta >Assignee: Paulo Motta > Fix For: 1.2.18 > > Attachments: 1.2-CASSANDRA-7445-withPigTests.txt, > 1.2-CASSANDRA-7445.txt > > > CASSANDRA-5752 introduced support to thrift tables on CQLPagingInputFormat > via the retrieveKeysForThriftTables() method. > However, this method only retrieves partition keys from CFMetaData, so > clustering columns are ignored. > So, when the RowIterator tries to fetch the next page of a wide-row thrift > CF, it ignores the clustering column altogether, going to the next partition > token. So, only cassandra.input.page.row.size CQL rows are retrieved for each > partition key. > CqlRecordWriter had the same issue and was fixed on > bcfe352ea6ed3786f274b90191f988210360689d (CASSANDRA-5718). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7444) Performance drops when creating large amount of tables
[ https://issues.apache.org/jira/browse/CASSANDRA-7444?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044654#comment-14044654 ] Sylvain Lebresne commented on CASSANDRA-7444: - +1 (but may I suggest that it probably belongs to 2.1.1 at this point since it's really just a (minor) performance improvement, even if it's a simple one) > Performance drops when creating large amount of tables > --- > > Key: CASSANDRA-7444 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7444 > Project: Cassandra > Issue Type: Improvement > Components: Core > Environment: [cqlsh 3.1.8 | Cassandra 1.2.15.1 | CQL spec 3.0.0 | > Thrift protocol 19.36.2][cqlsh 4.1.1 | Cassandra 2.0.7.31 | CQL spec 3.1.1 | > Thrift protocol 19.39.0] >Reporter: Jose Martinez Poblete >Assignee: Aleksey Yeschenko >Priority: Minor > Labels: cassandra > Fix For: 2.1.0 > > Attachments: 7444.txt > > > We are creating 4000 tables from a script and using cqlsh to create the > tables. As the tables are being created, the time taken grows exponentially > and it becomes very slow and takes a lot of time. > We read a file get the keyspace append a random number and then create > keyspace with this new name example Airplane_12345678, Airplane_123575849... > then fed into cqlsh via script > Similarly each table is created via script use Airplane_12345678; create > table1...table25 , then use Airplane_123575849; create table1...create table25 > It is all done in singleton fashion, doing one after the other in a loop. > We tested using the following bash script > {noformat} > #!/bin/bash > SEED=0 > ITERATIONS=20 > while [ ${SEED} -lt ${ITERATIONS} ]; do >COUNT=0 >KEYSPACE=t10789_${SEED} >echo "CREATE KEYSPACE ${KEYSPACE} WITH replication = { 'class': > 'NetworkTopologyStrategy', 'Cassandra': '1' };" > ${KEYSPACE}.ddl >echo "USE ${KEYSPACE};" >> ${KEYSPACE}.ddl >while [ ${COUNT} -lt 25 ]; do > echo "CREATE TABLE user_colors${COUNT} (user_id int PRIMARY KEY, colors > list );" >> ${KEYSPACE}.ddl > ((COUNT++)) >done >((SEED++)) >time cat ${KEYSPACE}.ddl | cqlsh >if [ "$?" -gt 0 ]; then > echo "[ERROR] Failure at ${KEYSPACE}" > exit 1 >else > echo "[OK]Created ${KEYSPACE}" >fi >echo "===" >sleep 3 > done > #EOF > {noformat} > The timing we got on an otherwise idle system were inconsistent > {noformat} > real0m42.649s > user0m0.332s > sys 0m0.092s > [OK]Created t10789_0 > === > real1m22.211s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_1 > === > real2m45.907s > user0m0.304s > sys 0m0.124s > [OK]Created t10789_2 > === > real3m24.098s > user0m0.340s > sys 0m0.108s > [OK]Created t10789_3 > === > real2m38.930s > user0m0.324s > sys 0m0.116s > [OK]Created t10789_4 > === > real3m4.186s > user0m0.336s > sys 0m0.104s > [OK]Created t10789_5 > === > real2m55.391s > user0m0.344s > sys 0m0.092s > [OK]Created t10789_6 > === > real2m14.290s > user0m0.328s > sys 0m0.108s > [OK]Created t10789_7 > === > real2m44.880s > user0m0.344s > sys 0m0.092s > [OK]Created t10789_8 > === > real1m52.785s > user0m0.336s > sys 0m0.128s > [OK]Created t10789_9 > === > real1m18.404s > user0m0.344s > sys 0m0.108s > [OK]Created t10789_10 > === > real2m20.681s > user0m0.348s > sys 0m0.104s > [OK]Created t10789_11 > === > real1m11.860s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_12 > === > real1m37.887s > user0m0.324s > sys 0m0.100s > [OK]Created t10789_13 > === > real1m31.616s > user0m0.316s > sys 0m0.132s > [OK]Created t10789_14 > === > real1m12.103s > user0m0.360s > sys 0m0.088s > [OK]Created t10789_15 > === > real0m36.378s > user0m0.340s > sys 0m0.092s > [OK]Created t10789_16 > === > real0m40.883s > user0m0.352s > sys 0m0.096s > [OK]Created t10789_17 > === > real0m40.661s > user0m0.332s > sys 0m0.096s > [OK]Created t10789_18 > === > real0m44.943s > user0m0.32
[jira] [Commented] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044648#comment-14044648 ] graham sanderson commented on CASSANDRA-7449: - [~slebresne] given that static columns and selecting rows have some behavior in minor flux, I thought you might want to take a quick look at this > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 108 > (1 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point B > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 87 > 123 |2 | gibbon | this is large and should not be inserted > every time | 111 > (2 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > 123 > (2 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point C > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 67 > 123 |2 | gibbon | this is large and should not be inserted > every time | 91 > 123 |3 | firebucket | this is large and should not be inserted > every time | 110 > (3 rows) > cqlsh:test> SE
[jira] [Commented] (CASSANDRA-7449) Variation of SELECT DISTINCT to find clustering keys with only static columns
[ https://issues.apache.org/jira/browse/CASSANDRA-7449?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14044642#comment-14044642 ] graham sanderson commented on CASSANDRA-7449: - Note: patch predates CASSANDRA-7305 which I just noticed today with the 2.0.9 vote > Variation of SELECT DISTINCT to find clustering keys with only static columns > - > > Key: CASSANDRA-7449 > URL: https://issues.apache.org/jira/browse/CASSANDRA-7449 > Project: Cassandra > Issue Type: Improvement > Components: API, Core >Reporter: graham sanderson > Attachments: paging_broken_no_tests_v0.patch > > > A possible use case for static columns involves (per partition) multiple > small TTL time series data values combined with a potentially much larger > static piece of data. > While the TTL time series data will go away on its own, there is no way to > TTL the static data (and keep it updated with the latest TTL) without > re-inserting it every time to reset the TTL (which is undesirable since it is > large and unchanged) > The use case looks something like this: > {code} > CREATE KEYSPACE test WITH replication = { > 'class': 'SimpleStrategy', > 'replication_factor': '1' > }; > USE test; > CREATE TABLE expiring_series ( > id text, > series_order int, > small_data text, > large_data text static, > PRIMARY KEY (id, series_order) > ); > INSERT INTO expiring_series (id, large_data) VALUES ('123', 'this is large > and should not be inserted every time'); > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 1, > 'antelope') USING TTL 120; > // time passes (point A) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 2, > 'gibbon') USING TTL 120; > // time passes (point B) > INSERT INTO expiring_series (id, series_order, small_data) VALUES ('123', 3, > 'firebucket') USING TTL 120; > // time passes (point C) > // time passes and the first row expires (point D) > // more time passes and eventually all the "rows" expire (point E) > {code} > GIven the way the storage engine works, there is no trivial way to make the > static column expire when the last row expires, however if there was an easy > way to find partitions with no regular rows (just static columns), then that > would make manual clean up easy > The possible implementation of such a feature is very similar to SELECT > DISTINCT, so I'm suggesting SELECT STATICONLY > Looking at the points again > h4. Point A > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 108 > (1 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point B > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 87 > 123 |2 | gibbon | this is large and should not be inserted > every time | 111 > (2 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > 123 > (2 rows) > cqlsh:test> SELECT DISTINCT id FROM expiring_series; > id > - > 123 > (1 rows) > cqlsh:test> SELECT STATICONLY id FROM expiring_series; > (0 rows) > {code} > h4. Point C > {code} > cqlsh:test> SELECT id, series_order, small_data, large_data, ttl(small_data) > from expiring_series; > id | series_order | small_data | large_data > | ttl(small_data) > -+--++-+- > 123 |1 | antelope | this is large and should not be inserted > every time | 67 > 123 |2 | gibbon | this is large and should not be inserted > every time | 91 > 123 |3 | firebucket | this is large and should not be inserted > every time | 110 > (3 rows) > cqlsh:test> SELECT id FROM expiring_series; > id > - > 123 > 123 > 12