[jira] [Commented] (CASSANDRA-7366) Use node's hostId instead of generating counterId-s
[ https://issues.apache.org/jira/browse/CASSANDRA-7366?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026179#comment-14026179 ] Sylvain Lebresne commented on CASSANDRA-7366: - The one downside I see might be that, unless I'm wrong, this will roughly double the size of the counters for all existing ones and I wonder if the code saved is worth that disagreement for our users. I'm slightly ambivalent here on what is the best choice. On the one side if we ever do this, better do it sooner than later. On the other side, we've already made many chances to counters, so keeping a small amount of code to save some trouble to upgrades feels nicer. Anyway, not feeling extremely strongly here though slightly leaning towards keeping things as they are to be nice to upgraders if I'd have to pick. But with that caveat, +1 on the patch technically speaking. Use node's hostId instead of generating counterId-s --- Key: CASSANDRA-7366 URL: https://issues.apache.org/jira/browse/CASSANDRA-7366 Project: Cassandra Issue Type: Improvement Reporter: Aleksey Yeschenko Assignee: Aleksey Yeschenko Priority: Minor Fix For: 2.1.0 Now that we no longer renew, or have to renew, counter ids - we can/should simply reuse the node's hostId. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7346) Explicitly set deletion timestamp to Long.MAX_VALUE for counter deletions
[ https://issues.apache.org/jira/browse/CASSANDRA-7346?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026185#comment-14026185 ] Sylvain Lebresne commented on CASSANDRA-7346: - +1 Explicitly set deletion timestamp to Long.MAX_VALUE for counter deletions - Key: CASSANDRA-7346 URL: https://issues.apache.org/jira/browse/CASSANDRA-7346 Project: Cassandra Issue Type: Improvement Reporter: Richard Low Assignee: Aleksey Yeschenko Priority: Minor Fix For: 2.1.0 For counters, timestamps are automatically computed to be milliseconds since the epoch. For everything else, when not specified manually, they are microseconds since the epoch. This means if you delete a counter row, subsequent updates are lost unexpectedly. I know that deleting counters is not recommended, but that's only because deletes and incs don't commute. If you know you have stopped incs, then delete, then start again (with some external synchronization) then deleting is fine. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Resolved] (CASSANDRA-4501) Gossip the ip and port used by the native protocol
[ https://issues.apache.org/jira/browse/CASSANDRA-4501?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne resolved CASSANDRA-4501. - Resolution: Later Fix Version/s: (was: 2.1.0) I've changed my mind a bit her. So far I don't think the reuse of rpc_address has been a problem so let's not change just for the sake of change. Closing for now, we can revisit later if we feel there is renewed reasons to do this. Gossip the ip and port used by the native protocol -- Key: CASSANDRA-4501 URL: https://issues.apache.org/jira/browse/CASSANDRA-4501 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Sylvain Lebresne Priority: Minor Attachments: 4501.txt Same as we gossip the rpc_address, we should add gossipping of the native transport address (including the port). CASSANDRA-4480 has one reason why we would want to do that. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7372) Exception when querying a composite-keyed table with a collection index
[ https://issues.apache.org/jira/browse/CASSANDRA-7372?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026201#comment-14026201 ] Sylvain Lebresne commented on CASSANDRA-7372: - Why do we end up with an empty composite there? That feels wrong and so I'm not sure creating an index entry for it is the right fix. Will look more closely when I'm not on a phone. Exception when querying a composite-keyed table with a collection index --- Key: CASSANDRA-7372 URL: https://issues.apache.org/jira/browse/CASSANDRA-7372 Project: Cassandra Issue Type: Bug Reporter: Ghais Issa Fix For: 2.1.0 Attachments: CASSANDRA-2.1-7372-v2.patch Given the following schema: {code} CREATE TABLE products ( account text, id int, categories settext, PRIMARY KEY (account, id) ); CREATE INDEX cat_index ON products(categories); {code} The following query fails with an exception {code} SELECT * FROM products WHERE account = 'xyz' AND categories CONTAINS 'lmn'; errors={}, last_host=127.0.0.1 {code} The exception in cassandra's log is: {code} WARN 17:01:49 Uncaught exception on thread Thread[SharedPool-Worker-2,5,main]: {} java.lang.RuntimeException: java.lang.IndexOutOfBoundsException at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2015) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_25] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.lang.Thread.run(Thread.java:724) ~[na:1.7.0_25] Caused by: java.lang.IndexOutOfBoundsException: null at org.apache.cassandra.db.composites.Composites$EmptyComposite.get(Composites.java:60) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesIndexOnCollectionKey.makeIndexColumnPrefix(CompositesIndexOnCollectionKey.java:78) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.makePrefix(CompositesSearcher.java:82) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.getIndexedIterator(CompositesSearcher.java:116) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.search(CompositesSearcher.java:68) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:589) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:2060) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:131) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$LocalRangeSliceRunnable.runMayThrow(StorageProxy.java:1368) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2011) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] ... 4 common frames omitted {code} The following query however works {code} SELECT * FROM products WHERE categories CONTAINS 'lmn'; {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7371) DELETEs get lost
[ https://issues.apache.org/jira/browse/CASSANDRA-7371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026203#comment-14026203 ] Robert Stupp commented on CASSANDRA-7371: - Regarding the failed tests: The CQL DELETE gets through - but it is never executed - not just delayed (verified in cqlsh some seconds later) I'll check that patch later this evening (CEST - 6/7 hours before EST). What I did right now is this. {noformat} $ uname -a Darwin macbook-retina.local 13.2.0 Darwin Kernel Version 13.2.0: Thu Apr 17 23:03:13 PDT 2014; root:xnu-2422.100.13~1/RELEASE_X86_64 x86_64 (System is a MacBook Pro/Retina - 16G RAM - Core i7) $ cd (directory for cassandra source) $ rm -rf (everything inside the directory) $ git clone https://github.com/apache/cassandra.git . $ git checkout -b cassandra-2.1 --track origin/cassandra-2.1 $ cat .git/config ... [branch cassandra-2.1] remote = origin merge = refs/heads/cassandra-2.1 $ pwd /Users/snazy/devel/cassandra/cassandra/ $ ant publish $ java -version java version 1.7.0_55 $ l ~/.ccm/repository lrwxr-xr-x 1 snazy staff 39 9 Jun 20:21 2.1.0-rc1-snapshot@ - /Users/snazy/devel/cassandra/cassandra/ $ ccm create -v 2.1.0-rc1-snapshot -n 1 -s --vnodes test Current cluster is now: test {noformat} Run test... (with 1.7.0_55) 31/500 failed {noformat} $ ccm remove test $ git reset -q --hard b29d882df82c1b0aa2c1878c0ba704ac814c69d3 $ rm -rf build $ ant publish $ ccm create -v 2.1.0-rc1-snapshot -n 1 -s --vnodes test Current cluster is now: test {noformat} Run test... (with 1.7.0_55) 0/500 failed DELETEs get lost Key: CASSANDRA-7371 URL: https://issues.apache.org/jira/browse/CASSANDRA-7371 Project: Cassandra Issue Type: Bug Components: Core Environment: 2.1 git branch since merge commit 4722fe70aa9ae1b62772cfa1a1de58ef289445d5 (RefCount native frames from netty to avoid corruption bugs) Reporter: Robert Stupp Assignee: T Jake Luciani Priority: Blocker Fix For: 2.1.0 Attachments: Cassandra7371.java The mentioned commit introduced a bug which is not easy to reproduce: Workload description: - One INSERT into a table - multiple concurrent SELECTs against different tables (one select returns a result) - One UPDATE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables (one select returns a result) - One DELETE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables Expected is that the last bunch of SELECTs returns no result. But since commit SHA the DELETE gets not processed. To clarify - the DELETE is not delayed - it is not executed at all. Checked against a single node C* cluster. Does only affect unreleased 2.1 - not 2.0 nor 1.2. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6458) nodetool getendpoints doesn't validate key arity
[ https://issues.apache.org/jira/browse/CASSANDRA-6458?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026250#comment-14026250 ] Mihai Suteu commented on CASSANDRA-6458: Hello! I started working on this issue, but would like to have some feedback on how to proceed. I am quite new to Cassandra and might be making some wrong assumptions about it's internals. In order for NodeTool to check the arity, it has to get the number of columns in the partition key of a cf. As far as I understand it, I would have to get that from CFMetaData. The problem is that NodeTool can only connect through NodeProbe, which has the proxies for the MBeans. The closest I got was through the StorageServerProxy and ColumnFamilyStore proxy, but none of them specify a method in their interface to access a cfMetaData. The options I see so far are to either add a method to ColumnFamilyStoreMBean or try to validate the key arity somewhere lower. Or is there an easier way? Thanks! nodetool getendpoints doesn't validate key arity - Key: CASSANDRA-6458 URL: https://issues.apache.org/jira/browse/CASSANDRA-6458 Project: Cassandra Issue Type: Bug Components: Tools Reporter: Daneel Yaitskov Priority: Trivial Labels: lhf I have a complex row key. $ create table b (x int, s text, ((x,s)) primary key); In cqlsh I cannot fill row key partially: {noformat} $ insert into b (x) values(4); Bad Request: Missing mandatory PRIMARY KEY part s {noformat} But nodetool can find hosts by incomplete key {noformat} $ nodetool -h cas3 getendpoints anti_portal b 12 192.168.4.4 192.168.4.5 192.168.4.6 {noformat} No error is reported. I found that columns are separated by :. And If I pass to many elements then the error happens. {noformat} $ nodetool -h cas3 getendpoints anit_portal b 12:dd:dd Exception in thread main org.apache.cassandra.serializers.MarshalException: unable to make int from '12:dd:dd' at org.apache.cassandra.db.marshal.Int32Type.fromString(Int32Type.java:69) at org.apache.cassandra.service.StorageService.getNaturalEndpoints(StorageService.java:2495) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.reflect.misc.Trampoline.invoke(MethodUtil.java:75) at sun.reflect.GeneratedMethodAccessor7.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.reflect.misc.MethodUtil.invoke(MethodUtil.java:279) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:112) at com.sun.jmx.mbeanserver.StandardMBeanIntrospector.invokeM2(StandardMBeanIntrospector.java:46) at com.sun.jmx.mbeanserver.MBeanIntrospector.invokeM(MBeanIntrospector.java:237) at com.sun.jmx.mbeanserver.PerInterface.invoke(PerInterface.java:138) at com.sun.jmx.mbeanserver.MBeanSupport.invoke(MBeanSupport.java:252) at com.sun.jmx.interceptor.DefaultMBeanServerInterceptor.invoke(DefaultMBeanServerInterceptor.java:819) at com.sun.jmx.mbeanserver.JmxMBeanServer.invoke(JmxMBeanServer.java:801) at javax.management.remote.rmi.RMIConnectionImpl.doOperation(RMIConnectionImpl.java:1487) at javax.management.remote.rmi.RMIConnectionImpl.access$300(RMIConnectionImpl.java:97) at javax.management.remote.rmi.RMIConnectionImpl$PrivilegedOperation.run(RMIConnectionImpl.java:1328) at javax.management.remote.rmi.RMIConnectionImpl.doPrivilegedOperation(RMIConnectionImpl.java:1420) at javax.management.remote.rmi.RMIConnectionImpl.invoke(RMIConnectionImpl.java:848) at sun.reflect.GeneratedMethodAccessor12.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at sun.rmi.server.UnicastServerRef.dispatch(UnicastServerRef.java:322) at sun.rmi.transport.Transport$1.run(Transport.java:177) at sun.rmi.transport.Transport$1.run(Transport.java:174) at java.security.AccessController.doPrivileged(Native Method) at sun.rmi.transport.Transport.serviceCall(Transport.java:173) at sun.rmi.transport.tcp.TCPTransport.handleMessages(TCPTransport.java:556) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run0(TCPTransport.java:811) at sun.rmi.transport.tcp.TCPTransport$ConnectionHandler.run(TCPTransport.java:670) at
[jira] [Commented] (CASSANDRA-6890) Standardize on a single read path
[ https://issues.apache.org/jira/browse/CASSANDRA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026475#comment-14026475 ] Jason Brown commented on CASSANDRA-6890: Hmm, are we sure we really want to do this before CASSANDRA-5863 is proven (or even implemented)? The v1 patch is simple enough, but we are taking a known performance hit against an unknown future payoff. How much simpler would removing the mmap path make implementing CASSANDRA-5863? I can't imagine it'd make that much of a difference? Furthermore, if anyone is truly interested, I'll be talking about direct i/o (O_DIRECT) at the 2014 NGCC. Standardize on a single read path - Key: CASSANDRA-6890 URL: https://issues.apache.org/jira/browse/CASSANDRA-6890 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Joshua McKenzie Assignee: Joshua McKenzie Labels: performance Fix For: 3.0 Attachments: 6890_v1.txt, mmap_gc.jpg, mmap_jstat.txt, mmap_perf.txt, nommap_gc.jpg, nommap_jstat.txt Since we actively unmap unreferenced SSTR's and also copy data out of those readers on the read path, the current memory mapped i/o is a lot of complexity for very little payoff. Clean out the mmapp'ed i/o on the read path. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6890) Standardize on a single read path
[ https://issues.apache.org/jira/browse/CASSANDRA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026490#comment-14026490 ] Joshua McKenzie commented on CASSANDRA-6890: That's a fair point - just pulling out the memory mapped file path is a pretty simple effort that could easily be integrated into CASSANDRA-5863. For the 3.0 release and Windows stability, we have it hard-coded in our DatabaseDescriptor to not allow memory-mapped file I/O on Windows so there's really no functional difference as far as platform stability on that front. Standardize on a single read path - Key: CASSANDRA-6890 URL: https://issues.apache.org/jira/browse/CASSANDRA-6890 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Joshua McKenzie Assignee: Joshua McKenzie Labels: performance Fix For: 3.0 Attachments: 6890_v1.txt, mmap_gc.jpg, mmap_jstat.txt, mmap_perf.txt, nommap_gc.jpg, nommap_jstat.txt Since we actively unmap unreferenced SSTR's and also copy data out of those readers on the read path, the current memory mapped i/o is a lot of complexity for very little payoff. Clean out the mmapp'ed i/o on the read path. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7365) some compactions do not works under windows (file in use during rename)
[ https://issues.apache.org/jira/browse/CASSANDRA-7365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026497#comment-14026497 ] Joshua McKenzie commented on CASSANDRA-7365: Thanks Radim - I was specifically looking to see what your indexAccessMode was set to on your node - looks like it was standard so not memory-mapped where we have known issues on Windows (see CASSANDRA-6993 and CASSANDRA-4050). I've reproduced this error on rc-1 with a simple write stress, forcing disk_access_mode to standard: {code:title=reproduction} ERROR [CompactionExecutor:1] 2014-06-10 09:22:41,405 CassandraDaemon.java:166 - Exception in thread Thread[CompactionExecutor:1,1,main] java.lang.RuntimeException: Failed to rename \var\lib\cassandra\data\Keyspace1\Standard1-654f28c0f0aa11e39dd391f9d8b074f2\Keyspace1-Standard1-tmp-ka-7-Index.db to \var\lib\cassandra\data\Keyspace1\Standard1-654f28c0f0aa11e39dd391f9d8b074f2\Keyspace1-Standard1-ka-7-Index.db at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:167) ~[main/:na] at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.java:151) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.java:512) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.java:504) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.java:479) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SSTableWriter.java:427) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SSTableWriter.java:422) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewriter.java:312) ~[main/:na] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewriter.java:306) ~[main/:na] at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:188) ~[main/:na] ... Caused by: java.nio.file.FileSystemException: \var\lib\cassandra\data\Keyspace1\Standard1-654f28c0f0aa11e39dd391f9d8b074f2\Keyspace1-Standard1-tmp-ka-7-Index.db - \var\lib\cassandra\data\Keyspace1\Standard1-654f28c0f0aa11e39dd391f9d8b074f2\Keyspace1-Standard1-ka-7-Index.db: The process cannot access the file because it is being used by another process. {code} I haven't seen this issue prior to rc1 so I'll dig back a bit and try to find out when a regression was introduced. some compactions do not works under windows (file in use during rename) --- Key: CASSANDRA-7365 URL: https://issues.apache.org/jira/browse/CASSANDRA-7365 Project: Cassandra Issue Type: Bug Components: Core Environment: jdk7, cassandra-2.1rc1, os windows 32 bit Reporter: Radim Kolar Assignee: Joshua McKenzie Labels: Windows Fix For: 2.1.1 Attachments: cassandra.yaml, system.log compaction do not works under windows due to file rename fails: (Pro es nemß p°Ýstup k souboru, neboŁ jej prßvý vyu×Ývß jinř proces = process can not access file because its in use by another process). Not all compactions are broken. compactions done during server startup on system tables works fine. INFO 18:30:27 Completed flushing c:\cassandra-2.1\data\system\compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b\system-compactions_in_progress-ka-6-Dat.db (42 bytes) for commitlog position ReplayPosition(segmentId=1402165543361, psition=8024611) ERROR 18:30:27 Exception in thread hread[CompactionExecutor:5,1,RMI Runtime] java.lang.RuntimeException: Failed to rename c:\cassandra-2.1\data\test\sipdb-5 f51090ee6511e3815625991ef2b954\test-sipdb-tmp-ka-7-Index.db to c:\cassandra-2.1 data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka-7-Index.db at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:167) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:151) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:512) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:504) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.ja a:479) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:427) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:422)
[jira] [Commented] (CASSANDRA-6666) Avoid accumulating tombstones after partial hint replay
[ https://issues.apache.org/jira/browse/CASSANDRA-?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026537#comment-14026537 ] Walid Darwish commented on CASSANDRA-: -- Just wondering why compaction is disabled on the hints table by default (and only on that table across the system keyspace)? is that for performance reasons? Can this be turned on safely? Also, can this be changed safely to levelled compaction? compaction= {'enabled': 'false', 'class': 'SizeTieredCompactionStrategy'} I am using 2.0.7 Avoid accumulating tombstones after partial hint replay --- Key: CASSANDRA- URL: https://issues.apache.org/jira/browse/CASSANDRA- Project: Cassandra Issue Type: Bug Components: Core Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Labels: hintedhandoff Fix For: 1.2.17, 2.0.9 Attachments: .txt, cassandra_system.log.debug.gz -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6666) Avoid accumulating tombstones after partial hint replay
[ https://issues.apache.org/jira/browse/CASSANDRA-?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026545#comment-14026545 ] Jonathan Ellis commented on CASSANDRA-: --- Because the hints manager does compactions manually, only when it knows it will be useful (after hint delivery). The generic strategies would waste a ton of effort, especially LCS. Avoid accumulating tombstones after partial hint replay --- Key: CASSANDRA- URL: https://issues.apache.org/jira/browse/CASSANDRA- Project: Cassandra Issue Type: Bug Components: Core Reporter: Jonathan Ellis Assignee: Jonathan Ellis Priority: Minor Labels: hintedhandoff Fix For: 1.2.17, 2.0.9 Attachments: .txt, cassandra_system.log.debug.gz -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7363) PropertyFileSnitch should allow name address that does not yet exist
[ https://issues.apache.org/jira/browse/CASSANDRA-7363?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026551#comment-14026551 ] Jeremiah Jordan commented on CASSANDRA-7363: [~cywjackson] the issue is those IP's won't be known later. We only lookup the hostnames at startup or when we detect the file changed. So if you add those DNS entries later, without updating the file, the server will not magically start knowing about them. And since you have to update the file anyway to get it to reload, you might as well update it after you create the DNS entries. PropertyFileSnitch should allow name address that does not yet exist Key: CASSANDRA-7363 URL: https://issues.apache.org/jira/browse/CASSANDRA-7363 Project: Cassandra Issue Type: Bug Reporter: Jackson Chung when starting a new node with PropertyFileSnitch with cassandra-topology.properties contains an unknown host, it fails with: {noformat} ERROR [main] 2014-06-06 17:48:38,233 DatabaseDescriptor.java (line 116) Fatal configuration error org.apache.cassandra.exceptions.ConfigurationException: Error instantiating snitch class 'org.apache.cassandra.locator.PropertyFileSnitch'. at org.apache.cassandra.utils.FBUtilities.construct(FBUtilities.java:503) at org.apache.cassandra.config.DatabaseDescriptor.createEndpointSnitch(DatabaseDescriptor.java:506) at org.apache.cassandra.config.DatabaseDescriptor.applyConfig(DatabaseDescriptor.java:341) at org.apache.cassandra.config.DatabaseDescriptor.clinit(DatabaseDescriptor.java:111) at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:155) at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:480) at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:569) Caused by: org.apache.cassandra.exceptions.ConfigurationException: Unknown host cassandra11-staging.amz.relateiq.com at org.apache.cassandra.locator.PropertyFileSnitch.reloadConfiguration(PropertyFileSnitch.java:174) at org.apache.cassandra.locator.PropertyFileSnitch.init(PropertyFileSnitch.java:60) at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:408) at java.lang.Class.newInstance(Class.java:433) at org.apache.cassandra.utils.FBUtilities.construct(FBUtilities.java:488) ... 6 more Caused by: java.net.UnknownHostException: cassandra11-staging.amz.relateiq.com: unknown error at java.net.Inet4AddressImpl.lookupAllHostAddr(Native Method) at java.net.InetAddress$2.lookupAllHostAddr(InetAddress.java:907) at java.net.InetAddress.getAddressesFromNameService(InetAddress.java:1302) at java.net.InetAddress.getAllByName0(InetAddress.java:1255) at java.net.InetAddress.getAllByName(InetAddress.java:1171) at java.net.InetAddress.getAllByName(InetAddress.java:1105) at java.net.InetAddress.getByName(InetAddress.java:1055) at org.apache.cassandra.locator.PropertyFileSnitch.reloadConfiguration(PropertyFileSnitch.java:170) ... 13 more {noformat} The real impact here is we are trying to launch a number of new nodes (via chef) with pre-configured hostname (and among other variables). The additional hostname (but not yet alive) made no impact to the existing nodes, which is good (looks like we only catch the ConfigurationException in the watcher thread, but not on the initial start); but it causes new node fail to start. (Pretty sure if we restart an existing one, it will fail too). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7364) assert error in StorageProxy.submitHint
[ https://issues.apache.org/jira/browse/CASSANDRA-7364?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-7364: -- Fix Version/s: (was: 2.1 rc1) 2.1.0 Assignee: Aleksey Yeschenko assert error in StorageProxy.submitHint --- Key: CASSANDRA-7364 URL: https://issues.apache.org/jira/browse/CASSANDRA-7364 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1-rc1, os windows, 32 bit Reporter: Radim Kolar Assignee: Aleksey Yeschenko Priority: Blocker Fix For: 2.1.0 in 2.1-rc1. assert error and hector based client ends with all nodes down message (its single node cluster). I assume that client connection got closed. INFO 18:28:33 Compacting [SSTableReader(path='c:\cassandra-2.1\data\test\sipdb- 58f51090ee6511e3815625991ef2b954\test-sipdb-ka-3-Data.db'), SSTableReader(path=' c:\cassandra-2.1\data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka- 1-Data.db'), SSTableReader(path='c:\cassandra-2.1\data\test\sipdb-58f51090ee6511 e3815625991ef2b954\test-sipdb-ka-4-Data.db'), SSTableReader(path='c:\cassandra-2 .1\data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka-2-Data.db'), S STableReader(path='c:\cassandra-2.1\data\test\sipdb-58f51090ee6511e3815625991ef2 b954\test-sipdb-ka-6-Data.db')] ERROR 18:29:50 Exception in thread Thread[Thrift:16,5,main] java.lang.AssertionError: localhost/127.0.0.1 at org.apache.cassandra.service.StorageProxy.submitHint(StorageProxy.jav a:870) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy.mutate(StorageProxy.java:49 3) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy.mutateWithTriggers(StorageP roxy.java:537) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer. java:1095) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.thrift.CassandraServer.doInsert(CassandraServer. java:1077) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraSer ver.java:970) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResul t(Cassandra.java:3996) ~[apache-cassandra-thrift-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResul t(Cassandra.java:3980) ~[apache-cassandra-thrift-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[ libthrift-0.9.1.jar:0.9.1] at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[li bthrift-0.9.1.jar:0.9.1] at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run (CustomTThreadPoolServer.java:201) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor. java:1145) ~[na:1.7.0_60] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor .java:615) ~[na:1.7.0_60] at java.lang.Thread.run(Thread.java:745) ~[na:1.7.0_60] INFO 18:29:55 1 MUTATION messages dropped in last 5000ms -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-6890) Standardize on a single read path
[ https://issues.apache.org/jira/browse/CASSANDRA-6890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026475#comment-14026475 ] Jason Brown edited comment on CASSANDRA-6890 at 6/10/14 3:35 PM: - Hmm, are we sure we really want to do this before CASSANDRA-5863 is proven (or even implemented)? The v1 patch is simple enough, but we are taking a known performance hit against an unknown future payoff. How much simpler would removing the mmap path make implementing CASSANDRA-5863? I can't imagine it'd make that much of a difference? Furthermore, if anyone is truly interested, I'll be talking about direct i/o (O_DIRECT) at the 2014 NGCC this week. was (Author: jasobrown): Hmm, are we sure we really want to do this before CASSANDRA-5863 is proven (or even implemented)? The v1 patch is simple enough, but we are taking a known performance hit against an unknown future payoff. How much simpler would removing the mmap path make implementing CASSANDRA-5863? I can't imagine it'd make that much of a difference? Furthermore, if anyone is truly interested, I'll be talking about direct i/o (O_DIRECT) at the 2014 NGCC. Standardize on a single read path - Key: CASSANDRA-6890 URL: https://issues.apache.org/jira/browse/CASSANDRA-6890 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Joshua McKenzie Assignee: Joshua McKenzie Labels: performance Fix For: 3.0 Attachments: 6890_v1.txt, mmap_gc.jpg, mmap_jstat.txt, mmap_perf.txt, nommap_gc.jpg, nommap_jstat.txt Since we actively unmap unreferenced SSTR's and also copy data out of those readers on the read path, the current memory mapped i/o is a lot of complexity for very little payoff. Clean out the mmapp'ed i/o on the read path. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7046) Update nodetool commands to output the date and time they were run on
[ https://issues.apache.org/jira/browse/CASSANDRA-7046?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026657#comment-14026657 ] Johnny Miller commented on CASSANDRA-7046: -- [~clardeur] Do you know if this is making it into a release? Update nodetool commands to output the date and time they were run on - Key: CASSANDRA-7046 URL: https://issues.apache.org/jira/browse/CASSANDRA-7046 Project: Cassandra Issue Type: Improvement Reporter: Johnny Miller Assignee: Clément Lardeur Priority: Trivial Labels: lhf Attachments: trunk-7046-v1.patch It would help if the various nodetool commands also outputted the system date time they were run. Often these commands are executed and then we look at the cassandra log files to try and find out what was happening at that time. This is certainly just a convenience feature, but it would be nice to have the information in there to aid with diagnostics. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7374) Support SSL and non-SSL applications to connect to same Cassandra cluster
Paresh created CASSANDRA-7374: - Summary: Support SSL and non-SSL applications to connect to same Cassandra cluster Key: CASSANDRA-7374 URL: https://issues.apache.org/jira/browse/CASSANDRA-7374 Project: Cassandra Issue Type: Bug Components: Config Environment: [root@stage2gce027 bin]# ./nodetool version xss = -ea -javaagent:/opt/dse-ebay-4.0.0/resources/cassandra/lib/jamm-0.2.5.jar -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms8G -Xmx8G -Xmn800M -XX:+HeapDumpOnOutOfMemoryError -Xss180k ReleaseVersion: 2.0.5-SNAPSHOT Reporter: Paresh We would like to see one Cassandra cluster works with SSL and non-SSL applications. This is more of a where application stack is placed in our infrastructure. If application pool is running in same network zone as database then there is no need for SSL but if application pool is running in a different network zone then we need to have SSL enabled between application pool and database cluster. We do have use cases like this. This will also be useful in testing/development environments where you can share one cluster between SSL enabled applications and non-SSL enabled applications. Please refer Request #10144 : SSL connectivity between application and Cassandra node Thanks, Paresh -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7125) Fail to start by default if Commit Log fails to validate any messages
[ https://issues.apache.org/jira/browse/CASSANDRA-7125?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026686#comment-14026686 ] sankalp kohli commented on CASSANDRA-7125: -- something like -Dcassandra.commitlog.skipbrokenentries=true +1 on this and also to fail the startup. Also what if it fails to load an sstable on startup specially if stable is created recently? Do we have something for it? Fail to start by default if Commit Log fails to validate any messages - Key: CASSANDRA-7125 URL: https://issues.apache.org/jira/browse/CASSANDRA-7125 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Benedict Priority: Minor Labels: correctness Fix For: 2.1.1 Current behaviour can be pretty dangerous, and also has a tendency to mask bugs during development. We should change the behaviour to default to failure if anything unexpected happens, and introduce a cassandra.yaml option that permits overriding the default behaviour. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6539) Track metrics at a keyspace level as well as column family level
[ https://issues.apache.org/jira/browse/CASSANDRA-6539?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026740#comment-14026740 ] Yuki Morishita commented on CASSANDRA-6539: --- You need to instantiate KeyspaceMetrics object at Keyspace creation and discard(release) when it closed. Otherwise metrics won't show up. I honestly don't get the point of trying to aggregate all metrics from CF. Total memtable/SSTable/BF sizes are fine, but I don't think others like max row size/latency/BF fp ratio are not so much. Track metrics at a keyspace level as well as column family level Key: CASSANDRA-6539 URL: https://issues.apache.org/jira/browse/CASSANDRA-6539 Project: Cassandra Issue Type: Improvement Reporter: Nick Bailey Assignee: Brandon Williams Priority: Minor Labels: lhf Fix For: 1.2.17, 2.0.9 Attachments: 6539-1.2.txt, 6539-2.0.txt It would be useful to be able to see aggregated metrics (write/read count/latency) at a keyspace level as well as at the individual column family level. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Created] (CASSANDRA-7375) nodetool units wrong for streamthroughput
Mike Heffner created CASSANDRA-7375: --- Summary: nodetool units wrong for streamthroughput Key: CASSANDRA-7375 URL: https://issues.apache.org/jira/browse/CASSANDRA-7375 Project: Cassandra Issue Type: Bug Components: Core Reporter: Mike Heffner Priority: Minor Stream throughput is measured in megabits (Mbps) in cassandray.yaml: # When unset, the default is 200 Mbps or 25 MB/s. # stream_throughput_outbound_megabits_per_sec: 200 However, the nodetool command uses the unit MB/s which implies megabytes/sec: getstreamthroughput- Print the MB/s throughput cap for streaming in the system setstreamthroughput value_in_mb - Set the MB/s throughput cap for streaming in the system, or 0 to disable throttling. $ nodetool getstreamthroughput Current stream throughput: 200 MB/s Fix references in nodetool to use Mbps -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7375) nodetool units wrong for streamthroughput
[ https://issues.apache.org/jira/browse/CASSANDRA-7375?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mike Heffner updated CASSANDRA-7375: Description: Stream throughput is measured in megabits (Mbps) in cassandray.yaml: {code} # When unset, the default is 200 Mbps or 25 MB/s. # stream_throughput_outbound_megabits_per_sec: 200 {code} However, the nodetool command uses the unit MB/s which implies megabytes/sec: getstreamthroughput- Print the MB/s throughput cap for streaming in the system setstreamthroughput value_in_mb - Set the MB/s throughput cap for streaming in the system, or 0 to disable throttling. $ nodetool getstreamthroughput Current stream throughput: 200 MB/s Fix references in nodetool to use Mbps was: Stream throughput is measured in megabits (Mbps) in cassandray.yaml: # When unset, the default is 200 Mbps or 25 MB/s. # stream_throughput_outbound_megabits_per_sec: 200 However, the nodetool command uses the unit MB/s which implies megabytes/sec: getstreamthroughput- Print the MB/s throughput cap for streaming in the system setstreamthroughput value_in_mb - Set the MB/s throughput cap for streaming in the system, or 0 to disable throttling. $ nodetool getstreamthroughput Current stream throughput: 200 MB/s Fix references in nodetool to use Mbps nodetool units wrong for streamthroughput - Key: CASSANDRA-7375 URL: https://issues.apache.org/jira/browse/CASSANDRA-7375 Project: Cassandra Issue Type: Bug Components: Core Reporter: Mike Heffner Priority: Minor Stream throughput is measured in megabits (Mbps) in cassandray.yaml: {code} # When unset, the default is 200 Mbps or 25 MB/s. # stream_throughput_outbound_megabits_per_sec: 200 {code} However, the nodetool command uses the unit MB/s which implies megabytes/sec: getstreamthroughput- Print the MB/s throughput cap for streaming in the system setstreamthroughput value_in_mb - Set the MB/s throughput cap for streaming in the system, or 0 to disable throttling. $ nodetool getstreamthroughput Current stream throughput: 200 MB/s Fix references in nodetool to use Mbps -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7046) Update nodetool commands to output the date and time they were run on
[ https://issues.apache.org/jira/browse/CASSANDRA-7046?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026800#comment-14026800 ] Clément Lardeur commented on CASSANDRA-7046: Maybe [~brandon.williams] could review the patch or someone else but for now I can't tell you when it will be commited, sorry :( Update nodetool commands to output the date and time they were run on - Key: CASSANDRA-7046 URL: https://issues.apache.org/jira/browse/CASSANDRA-7046 Project: Cassandra Issue Type: Improvement Reporter: Johnny Miller Assignee: Clément Lardeur Priority: Trivial Labels: lhf Attachments: trunk-7046-v1.patch It would help if the various nodetool commands also outputted the system date time they were run. Often these commands are executed and then we look at the cassandra log files to try and find out what was happening at that time. This is certainly just a convenience feature, but it would be nice to have the information in there to aid with diagnostics. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7046) Update nodetool commands to output the date and time they were run on
[ https://issues.apache.org/jira/browse/CASSANDRA-7046?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026826#comment-14026826 ] Brandon Williams commented on CASSANDRA-7046: - I'm still not sure I see the utility in this vs just checking ~/.cassandra/nodetool.history instead. Update nodetool commands to output the date and time they were run on - Key: CASSANDRA-7046 URL: https://issues.apache.org/jira/browse/CASSANDRA-7046 Project: Cassandra Issue Type: Improvement Reporter: Johnny Miller Assignee: Clément Lardeur Priority: Trivial Labels: lhf Attachments: trunk-7046-v1.patch It would help if the various nodetool commands also outputted the system date time they were run. Often these commands are executed and then we look at the cassandra log files to try and find out what was happening at that time. This is certainly just a convenience feature, but it would be nice to have the information in there to aid with diagnostics. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7368) Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException
[ https://issues.apache.org/jira/browse/CASSANDRA-7368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14026867#comment-14026867 ] Francois Richard commented on CASSANDRA-7368: - The scrub of the sstables completed and the problems does not seem to happen again. Coming back to the original problem, why did compaction totally stop on this node even if we have the following setting: concurrent_compactors: 26. Should there be more resilience at the compaction level to keep on going when some SSTABLE are corrupted? Compaction stops after org.apache.cassandra.io.sstable.CorruptSSTableException -- Key: CASSANDRA-7368 URL: https://issues.apache.org/jira/browse/CASSANDRA-7368 Project: Cassandra Issue Type: Bug Components: Core Environment: OS: RHEL 6.5 Cassandra version: 1.2.16 Reporter: Francois Richard Hi, We are getting a case where compaction stops totally on a node after an exception related to: org.apache.cassandra.io.sstable.CorruptSSTableException. nodetool compactionstats remains at the same level for hours: {code} pending tasks: 1451 compaction typekeyspace column family completed total unit progress CompactionSyncCoreContactPrefixBytesIndex 257799931 376785179 bytes68.42% Active compaction remaining time :n/a {code} Here is the exception log: {code} ERROR [Deserialize SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db')] 2014-06-09 06:39:37,570 CassandraDaemon.java (line 191) Exception in thread Thread[Deserialize SSTableReader(path='/home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db'),1,main] org.apache.cassandra.io.sstable.CorruptSSTableException: java.io.IOException: dataSize of 7421941880990663551 starting at 257836699 would be larger than file /home/y/var/cassandra/data/SyncCore/ContactPrefixBytesIndex/SyncCore-ContactPrefixBytesIndex-ic-116118-Data.db length 376785179 at org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:167) at org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:83) at org.apache.cassandra.io.sstable.SSTableIdentityIterator.init(SSTableIdentityIterator.java:69) at org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:180) at org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:155) at org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:142) at org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38) at org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:238) at org.apache.cassandra.db.compaction.LeveledCompactionStrategy$LeveledScanner.computeNext(LeveledCompactionStrategy.java:207) at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:143) at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:138) -- {code} -- 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-tabpanelfocusedCommentId=14026916#comment-14026916 ] Rick Branson commented on CASSANDRA-6621: - This is definitely sub-optimal for us as well, just in terms of time spent compacting after a bootstrap/rebuild. We never get behind in L0 during normal operation. In addition, it causes one of the nice things about LCS to be invalidated as well, which is that you never have to worry about having double the disk space to compact. Bootstraps cause large compactions (~50% of the size of the CF), which means we need a ton of extra disk on bootstrap just to build nodes. 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 Priority: Minor 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-tabpanelfocusedCommentId=14027006#comment-14027006 ] Jonathan Ellis commented on CASSANDRA-6621: --- What do you think we could do to mitigate this, [~krummas]? 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 Priority: Minor 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-tabpanelfocusedCommentId=14027029#comment-14027029 ] sankalp kohli commented on CASSANDRA-6621: -- If we stream different regions from different nodes how can we get any overlaps? If a node from which it is streaming has the same row in two stable in different levels, it will come in as 2 stables. The only place we can put it is L0. We might want to also stream the stable level and can put stables coming from same level in one level on the bootstrapping node. The problem with this will be that we might end up with very few stable in higher levels violating the constrain that only last level can be less than limit. 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 Priority: Minor 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-tabpanelfocusedCommentId=14027035#comment-14027035 ] Jonathan Ellis commented on CASSANDRA-6621: --- What if we just special cased LCS during bootstrap to just put streamed data into the first level it doesn't overlap, up to X levels where X is calculated from total dataset being streamed? This won't necessarily be optimal but it will always do better than the status quo and never worse. 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 Priority: Minor 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-7267) Embedded sets in user defined data-types are not updating
[ https://issues.apache.org/jira/browse/CASSANDRA-7267?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027060#comment-14027060 ] Thomas Zimmer commented on CASSANDRA-7267: -- Working fine with 2.1.0-rc1. Thanks :) Embedded sets in user defined data-types are not updating - Key: CASSANDRA-7267 URL: https://issues.apache.org/jira/browse/CASSANDRA-7267 Project: Cassandra Issue Type: Bug Components: Core Reporter: Thomas Zimmer Assignee: Mikhail Stepura Fix For: 2.1 rc1 Attachments: CASSANDRA-2.1-7267.patch Hi, i just played around with Cassandra 2.1.0 beta2 and i might have found an issue with embedded Sets in User Defined Data Types. Here is how i can reproduce it: 1.) Create a keyspace test 2.) Create a table like this: {{create table songs (title varchar PRIMARY KEY, band varchar, tags Setvarchar);}} 3.) Create a udt like this: {{create type band_info_type (founded timestamp, members Setvarchar, description text);}} 4.) Try to insert data: {code} insert into songs (title, band, band_info, tags) values ('The trooper', 'Iron Maiden', {founded:188694000, members: {'Bruce Dickinson', 'Dave Murray', 'Adrian Smith', 'Janick Gers', 'Steve Harris', 'Nicko McBrain'}, description: 'Pure evil metal'}, {'metal', 'england'}); {code} 5.) Select the data: {{select * from songs;}} Returns this: {code} The trooper | Iron Maiden | {founded: '1970-01-03 05:24:54+0100', members: {}, description: 'Pure evil metal'} | {'england', 'metal'} {code} The embedded data-set seems to empty. I also tried updating a row which also does not seem to work. Regards, Thomas -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7371) DELETEs get lost
[ https://issues.apache.org/jira/browse/CASSANDRA-7371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027077#comment-14027077 ] Robert Stupp commented on CASSANDRA-7371: - It is not reproducible when connecting from Mac to Linux box. It is not reproducible when connecting from Linux box to Mac. It is not reproducible when connecting from Linux box to Linux box. It is not reproducible when connecting from Mac to Linux VM on Mac. It is not reproducible when connecting from Linux VM on Mac to Mac. It is not reproducible when connecting from Linux VM on Mac to Linux VM on Mac. It is only reproducible when both client and server run on OSX. It does not depend on the Java version (checked with 1.7.0_55, 1.7.0_60, 1.8.0_05, 1.9.0-ea-b16). Running test from IDE or command line makes no difference. Can anyone cross check it? I'm a bit frustrated. I see that there is an issue - but cannot isolate the cause. {noformat} Darwin macbook-retina.local 13.2.0 Darwin Kernel Version 13.2.0: Thu Apr 17 23:03:13 PDT 2014; root:xnu-2422.100.13~1/RELEASE_X86_64 x86_64 2.6 GHz Intel Core i7 {noformat} DELETEs get lost Key: CASSANDRA-7371 URL: https://issues.apache.org/jira/browse/CASSANDRA-7371 Project: Cassandra Issue Type: Bug Components: Core Environment: 2.1 git branch since merge commit 4722fe70aa9ae1b62772cfa1a1de58ef289445d5 (RefCount native frames from netty to avoid corruption bugs) Reporter: Robert Stupp Assignee: T Jake Luciani Priority: Blocker Fix For: 2.1.0 Attachments: Cassandra7371.java The mentioned commit introduced a bug which is not easy to reproduce: Workload description: - One INSERT into a table - multiple concurrent SELECTs against different tables (one select returns a result) - One UPDATE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables (one select returns a result) - One DELETE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables Expected is that the last bunch of SELECTs returns no result. But since commit SHA the DELETE gets not processed. To clarify - the DELETE is not delayed - it is not executed at all. Checked against a single node C* cluster. Does only affect unreleased 2.1 - not 2.0 nor 1.2. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7371) DELETEs get lost
[ https://issues.apache.org/jira/browse/CASSANDRA-7371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027100#comment-14027100 ] Jonathan Ellis commented on CASSANDRA-7371: --- Crazy! Do you have an OS X box to test on, [~mshuler]? DELETEs get lost Key: CASSANDRA-7371 URL: https://issues.apache.org/jira/browse/CASSANDRA-7371 Project: Cassandra Issue Type: Bug Components: Core Environment: 2.1 git branch since merge commit 4722fe70aa9ae1b62772cfa1a1de58ef289445d5 (RefCount native frames from netty to avoid corruption bugs) Reporter: Robert Stupp Assignee: T Jake Luciani Priority: Blocker Fix For: 2.1.0 Attachments: Cassandra7371.java The mentioned commit introduced a bug which is not easy to reproduce: Workload description: - One INSERT into a table - multiple concurrent SELECTs against different tables (one select returns a result) - One UPDATE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables (one select returns a result) - One DELETE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables Expected is that the last bunch of SELECTs returns no result. But since commit SHA the DELETE gets not processed. To clarify - the DELETE is not delayed - it is not executed at all. Checked against a single node C* cluster. Does only affect unreleased 2.1 - not 2.0 nor 1.2. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-5483) Repair tracing
[ https://issues.apache.org/jira/browse/CASSANDRA-5483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027102#comment-14027102 ] Jonathan Ellis commented on CASSANDRA-5483: --- Some comments having applied the patch: # Why is command part of events instead of sessions? Also: should use an enum internally. Logging as string representation is fine. # It makes people grumpy when we break JMX signatures. Can we add a new overload instead, preserving the old? This should cut down on some of the code churn in StorageService as well. # It's a minor thing to get hung up on, but I'm not wild about all the work needed to propagate TTLs around. Is it really super important to persist repair traces much longer than queries? If so, what if we used a separate table and just allowed users to modify the default ttl? (The original trace code predates default TTLs, I think, or we would have made use of it there.) Repair tracing -- Key: CASSANDRA-5483 URL: https://issues.apache.org/jira/browse/CASSANDRA-5483 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Yuki Morishita Assignee: Ben Chan Priority: Minor Labels: repair Attachments: 5483-full-trunk.txt, 5483-v06-04-Allow-tracing-ttl-to-be-configured.patch, 5483-v06-05-Add-a-command-column-to-system_traces.events.patch, 5483-v06-06-Fix-interruption-in-tracestate-propagation.patch, 5483-v07-07-Better-constructor-parameters-for-DebuggableThreadPoolExecutor.patch, 5483-v07-08-Fix-brace-style.patch, 5483-v07-09-Add-trace-option-to-a-more-complete-set-of-repair-functions.patch, 5483-v07-10-Correct-name-of-boolean-repairedAt-to-fullRepair.patch, 5483-v08-11-Shorten-trace-messages.-Use-Tracing-begin.patch, 5483-v08-12-Trace-streaming-in-Differencer-StreamingRepairTask.patch, 5483-v08-13-sendNotification-of-local-traces-back-to-nodetool.patch, 5483-v08-14-Poll-system_traces.events.patch, 5483-v08-15-Limit-trace-notifications.-Add-exponential-backoff.patch, 5483-v09-16-Fix-hang-caused-by-incorrect-exit-code.patch, 5483-v10-17-minor-bugfixes-and-changes.patch, 5483-v10-rebased-and-squashed-471f5cc.patch, 5483-v11-01-squashed.patch, 5483-v11-squashed-nits.patch, 5483-v12-02-cassandra-yaml-ttl-doc.patch, 5483-v13-608fb03-May-14-trace-formatting-changes.patch, 5483-v14-01-squashed.patch, ccm-repair-test, cqlsh-left-justify-text-columns.patch, prerepair-vs-postbuggedrepair.diff, test-5483-system_traces-events.txt, trunk@4620823-5483-v02-0001-Trace-filtering-and-tracestate-propagation.patch, trunk@4620823-5483-v02-0002-Put-a-few-traces-parallel-to-the-repair-logging.patch, tr...@8ebeee1-5483-v01-001-trace-filtering-and-tracestate-propagation.txt, tr...@8ebeee1-5483-v01-002-simple-repair-tracing.txt, v02p02-5483-v03-0003-Make-repair-tracing-controllable-via-nodetool.patch, v02p02-5483-v04-0003-This-time-use-an-EnumSet-to-pass-boolean-repair-options.patch, v02p02-5483-v05-0003-Use-long-instead-of-EnumSet-to-work-with-JMX.patch I think it would be nice to log repair stats and results like query tracing stores traces to system keyspace. With it, you don't have to lookup each log file to see what was the status and how it performed the repair you invoked. Instead, you can query the repair log with session ID to see the state and stats of all nodes involved in that repair session. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7371) DELETEs get lost
[ https://issues.apache.org/jira/browse/CASSANDRA-7371?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027106#comment-14027106 ] T Jake Luciani commented on CASSANDRA-7371: --- Yes I'm not able to repro on my linux box I can try osx. I do think I can fix the bug but I really really want to find a way to reproduce it consistently as a unit test DELETEs get lost Key: CASSANDRA-7371 URL: https://issues.apache.org/jira/browse/CASSANDRA-7371 Project: Cassandra Issue Type: Bug Components: Core Environment: 2.1 git branch since merge commit 4722fe70aa9ae1b62772cfa1a1de58ef289445d5 (RefCount native frames from netty to avoid corruption bugs) Reporter: Robert Stupp Assignee: T Jake Luciani Priority: Blocker Fix For: 2.1.0 Attachments: Cassandra7371.java The mentioned commit introduced a bug which is not easy to reproduce: Workload description: - One INSERT into a table - multiple concurrent SELECTs against different tables (one select returns a result) - One UPDATE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables (one select returns a result) - One DELETE against the same table as the INSERT - (same) multiple concurrent SELECTs against different tables Expected is that the last bunch of SELECTs returns no result. But since commit SHA the DELETE gets not processed. To clarify - the DELETE is not delayed - it is not executed at all. Checked against a single node C* cluster. Does only affect unreleased 2.1 - not 2.0 nor 1.2. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-5483) Repair tracing
[ https://issues.apache.org/jira/browse/CASSANDRA-5483?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027108#comment-14027108 ] Jonathan Ellis commented on CASSANDRA-5483: --- Also have a nagging feeling that the notify/wait logic in StorageService + TraceState is more complex than necessary. WDYT [~benedict]? Repair tracing -- Key: CASSANDRA-5483 URL: https://issues.apache.org/jira/browse/CASSANDRA-5483 Project: Cassandra Issue Type: Improvement Components: Tools Reporter: Yuki Morishita Assignee: Ben Chan Priority: Minor Labels: repair Attachments: 5483-full-trunk.txt, 5483-v06-04-Allow-tracing-ttl-to-be-configured.patch, 5483-v06-05-Add-a-command-column-to-system_traces.events.patch, 5483-v06-06-Fix-interruption-in-tracestate-propagation.patch, 5483-v07-07-Better-constructor-parameters-for-DebuggableThreadPoolExecutor.patch, 5483-v07-08-Fix-brace-style.patch, 5483-v07-09-Add-trace-option-to-a-more-complete-set-of-repair-functions.patch, 5483-v07-10-Correct-name-of-boolean-repairedAt-to-fullRepair.patch, 5483-v08-11-Shorten-trace-messages.-Use-Tracing-begin.patch, 5483-v08-12-Trace-streaming-in-Differencer-StreamingRepairTask.patch, 5483-v08-13-sendNotification-of-local-traces-back-to-nodetool.patch, 5483-v08-14-Poll-system_traces.events.patch, 5483-v08-15-Limit-trace-notifications.-Add-exponential-backoff.patch, 5483-v09-16-Fix-hang-caused-by-incorrect-exit-code.patch, 5483-v10-17-minor-bugfixes-and-changes.patch, 5483-v10-rebased-and-squashed-471f5cc.patch, 5483-v11-01-squashed.patch, 5483-v11-squashed-nits.patch, 5483-v12-02-cassandra-yaml-ttl-doc.patch, 5483-v13-608fb03-May-14-trace-formatting-changes.patch, 5483-v14-01-squashed.patch, ccm-repair-test, cqlsh-left-justify-text-columns.patch, prerepair-vs-postbuggedrepair.diff, test-5483-system_traces-events.txt, trunk@4620823-5483-v02-0001-Trace-filtering-and-tracestate-propagation.patch, trunk@4620823-5483-v02-0002-Put-a-few-traces-parallel-to-the-repair-logging.patch, tr...@8ebeee1-5483-v01-001-trace-filtering-and-tracestate-propagation.txt, tr...@8ebeee1-5483-v01-002-simple-repair-tracing.txt, v02p02-5483-v03-0003-Make-repair-tracing-controllable-via-nodetool.patch, v02p02-5483-v04-0003-This-time-use-an-EnumSet-to-pass-boolean-repair-options.patch, v02p02-5483-v05-0003-Use-long-instead-of-EnumSet-to-work-with-JMX.patch I think it would be nice to log repair stats and results like query tracing stores traces to system keyspace. With it, you don't have to lookup each log file to see what was the status and how it performed the repair you invoked. Instead, you can query the repair log with session ID to see the state and stats of all nodes involved in that repair session. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7372) Exception when querying a composite-keyed table with a collection index
[ https://issues.apache.org/jira/browse/CASSANDRA-7372?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027114#comment-14027114 ] Mikhail Stepura commented on CASSANDRA-7372: bq. Why do we end up with an empty composite there? Well, we have no {{columnRestrictions}} (in SelectStatement) in that case, so {{CompositesSearcher.makePrefix()}} has to deal with {{SliceQueryFilter(EMPTY, EMPTY)}}. {{makePrefix}} handles only the situation where {{key}} is empty, returning an empty composite, and dtests cover exactly this use-case (no restriction on a partition key) : https://github.com/riptano/cassandra-dtest/blob/master/cql_tests.py#L3521 {code} //1. works because CompositesSearcher.makePrefix handles the empty key SELECT * FROM products WHERE categories CONTAINS 'lmn'; //2. IndexOutOfBoundsException. Fails to create the startPrefix at CompositesSearcher.getIndexedIterator SELECT * FROM products WHERE account = 'test' AND categories CONTAINS 'lmn'; //3. IndexOutOfBoundsException. Fails to create the endPrefix at CompositesSearcher.getIndexedIterator SELECT * FROM products WHERE account = 'test' AND id 6 AND categories CONTAINS 'lmn'; //4. Works fine SELECT * FROM products WHERE account = 'test' AND id 6 AND id 1000 AND categories CONTAINS 'lmn'; {code} Exception when querying a composite-keyed table with a collection index --- Key: CASSANDRA-7372 URL: https://issues.apache.org/jira/browse/CASSANDRA-7372 Project: Cassandra Issue Type: Bug Reporter: Ghais Issa Fix For: 2.1.0 Attachments: CASSANDRA-2.1-7372-v2.patch Given the following schema: {code} CREATE TABLE products ( account text, id int, categories settext, PRIMARY KEY (account, id) ); CREATE INDEX cat_index ON products(categories); {code} The following query fails with an exception {code} SELECT * FROM products WHERE account = 'xyz' AND categories CONTAINS 'lmn'; errors={}, last_host=127.0.0.1 {code} The exception in cassandra's log is: {code} WARN 17:01:49 Uncaught exception on thread Thread[SharedPool-Worker-2,5,main]: {} java.lang.RuntimeException: java.lang.IndexOutOfBoundsException at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2015) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_25] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.lang.Thread.run(Thread.java:724) ~[na:1.7.0_25] Caused by: java.lang.IndexOutOfBoundsException: null at org.apache.cassandra.db.composites.Composites$EmptyComposite.get(Composites.java:60) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesIndexOnCollectionKey.makeIndexColumnPrefix(CompositesIndexOnCollectionKey.java:78) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.makePrefix(CompositesSearcher.java:82) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.getIndexedIterator(CompositesSearcher.java:116) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.search(CompositesSearcher.java:68) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:589) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:2060) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:131) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$LocalRangeSliceRunnable.runMayThrow(StorageProxy.java:1368) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2011) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] ... 4 common frames omitted {code} The following query however works {code} SELECT * FROM products WHERE categories CONTAINS 'lmn'; {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Comment Edited] (CASSANDRA-7372) Exception when querying a composite-keyed table with a collection index
[ https://issues.apache.org/jira/browse/CASSANDRA-7372?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027114#comment-14027114 ] Mikhail Stepura edited comment on CASSANDRA-7372 at 6/10/14 10:08 PM: -- bq. Why do we end up with an empty composite there? Well, we have no {{columnRestrictions}} (in SelectStatement) in that case, so {{CompositesSearcher.makePrefix()}} has to deal with {{SliceQueryFilter(EMPTY, EMPTY)}}. {{makePrefix}} handles only the situation where {{key}} is empty, returning an empty composite, and dtests cover exactly this use-case (no restriction on a partition key) : https://github.com/riptano/cassandra-dtest/blob/master/cql_tests.py#L3521 {code} //1. works because CompositesSearcher.makePrefix handles the empty key. This is covered by dtests SELECT * FROM products WHERE categories CONTAINS 'lmn'; //2. IndexOutOfBoundsException. Fails to create the startPrefix at CompositesSearcher.getIndexedIterator SELECT * FROM products WHERE account = 'test' AND categories CONTAINS 'lmn'; //3. IndexOutOfBoundsException. Fails to create the endPrefix at CompositesSearcher.getIndexedIterator SELECT * FROM products WHERE account = 'test' AND id 6 AND categories CONTAINS 'lmn'; //4. Works fine SELECT * FROM products WHERE account = 'test' AND id 6 AND id 1000 AND categories CONTAINS 'lmn'; {code} was (Author: mishail): bq. Why do we end up with an empty composite there? Well, we have no {{columnRestrictions}} (in SelectStatement) in that case, so {{CompositesSearcher.makePrefix()}} has to deal with {{SliceQueryFilter(EMPTY, EMPTY)}}. {{makePrefix}} handles only the situation where {{key}} is empty, returning an empty composite, and dtests cover exactly this use-case (no restriction on a partition key) : https://github.com/riptano/cassandra-dtest/blob/master/cql_tests.py#L3521 {code} //1. works because CompositesSearcher.makePrefix handles the empty key SELECT * FROM products WHERE categories CONTAINS 'lmn'; //2. IndexOutOfBoundsException. Fails to create the startPrefix at CompositesSearcher.getIndexedIterator SELECT * FROM products WHERE account = 'test' AND categories CONTAINS 'lmn'; //3. IndexOutOfBoundsException. Fails to create the endPrefix at CompositesSearcher.getIndexedIterator SELECT * FROM products WHERE account = 'test' AND id 6 AND categories CONTAINS 'lmn'; //4. Works fine SELECT * FROM products WHERE account = 'test' AND id 6 AND id 1000 AND categories CONTAINS 'lmn'; {code} Exception when querying a composite-keyed table with a collection index --- Key: CASSANDRA-7372 URL: https://issues.apache.org/jira/browse/CASSANDRA-7372 Project: Cassandra Issue Type: Bug Reporter: Ghais Issa Fix For: 2.1.0 Attachments: CASSANDRA-2.1-7372-v2.patch Given the following schema: {code} CREATE TABLE products ( account text, id int, categories settext, PRIMARY KEY (account, id) ); CREATE INDEX cat_index ON products(categories); {code} The following query fails with an exception {code} SELECT * FROM products WHERE account = 'xyz' AND categories CONTAINS 'lmn'; errors={}, last_host=127.0.0.1 {code} The exception in cassandra's log is: {code} WARN 17:01:49 Uncaught exception on thread Thread[SharedPool-Worker-2,5,main]: {} java.lang.RuntimeException: java.lang.IndexOutOfBoundsException at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2015) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_25] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.lang.Thread.run(Thread.java:724) ~[na:1.7.0_25] Caused by: java.lang.IndexOutOfBoundsException: null at org.apache.cassandra.db.composites.Composites$EmptyComposite.get(Composites.java:60) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesIndexOnCollectionKey.makeIndexColumnPrefix(CompositesIndexOnCollectionKey.java:78) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.makePrefix(CompositesSearcher.java:82) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.getIndexedIterator(CompositesSearcher.java:116) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at
[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-tabpanelfocusedCommentId=14027113#comment-14027113 ] sankalp kohli commented on CASSANDRA-6621: -- What if we just special cased LCS during bootstrap to just put streamed data into the first level it doesn't overlap I agree this will be better but here is another optional improvement which can minimize stables in L0. Stream stables from the source by sorting them by level which will cause streaming of stables in following order L1 to Lx and then finally L0. Here is why this will help. 1) If we stream an stable from higher level first, it will take a plot in L1 and will kick other stables to higher levels or to even L0. 2) If L0 of the streaming node is backed up and has 20-30 stables, it might end up in filling X levels and will kick other stables to L0 due to overlapping. Streaming L0 in the end will help in this case. Also I find it cleaner just to visualize that Level Z stables will go in Level Z on the node being bootstrapped. where X is calculated from total dataset being streamed Also I am not sure whether doing the sort based improvement which I am proposing will result in limited number of levels in the bootstrapping node. If node is bootstrapping from node A and B and A has 5 levels and B has 3 levels. The bootstrap node will have 5 levels. So we might not need to calculate X() 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 Priority: Minor 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-6839) Support non equal conditions (for LWT)
[ https://issues.apache.org/jira/browse/CASSANDRA-6839?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Tyler Hobbs updated CASSANDRA-6839: --- Attachment: 6839.txt The attached patch (and [branch|https://github.com/thobbs/cassandra/tree/CASSANDRA-6839]) adds support for , =, , =, and != in CAS operations. I've also created some [dtests|https://github.com/thobbs/cassandra-dtest/tree/CASSANDRA-6839]. It seems like we should also add support for {{IN}}. I can add do that in this ticket, if we'd like, or I can create a new ticket for that. Support non equal conditions (for LWT) -- Key: CASSANDRA-6839 URL: https://issues.apache.org/jira/browse/CASSANDRA-6839 Project: Cassandra Issue Type: Improvement Reporter: Sylvain Lebresne Assignee: Tyler Hobbs Priority: Minor Fix For: 2.0.9 Attachments: 6839.txt We currently only support equal conditions in conditional updates, but it would be relatively trivial to support non-equal ones as well. At the very least we should support '', '=', '' and '=', though it would probably also make sense to add a non-equal relation too ('!='). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7234) Post-compaction cache preheating can result in FileNotFoundExceptions when tables are dropped
[ https://issues.apache.org/jira/browse/CASSANDRA-7234?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027163#comment-14027163 ] Tyler Hobbs commented on CASSANDRA-7234: bq. I couldn't really reproduce this, could you elaborate when this happens? - If we drop a CF, we always interrupt the ongoing compactions. The timing has to be just right. It only checks to see if the compaction has been interrupted before writing each row. If the drop happens after the final row is written but before {{replaceCompactedSSTables()}} is run (basically, the {{finally}} block), you'll see this problem. I saw it occasionally while trying to reproduce CASSANDRA-6525, where a lot of compactions were backed up when the table was dropped. +1 on the patch, though. Post-compaction cache preheating can result in FileNotFoundExceptions when tables are dropped - Key: CASSANDRA-7234 URL: https://issues.apache.org/jira/browse/CASSANDRA-7234 Project: Cassandra Issue Type: Bug Components: Core Reporter: Tyler Hobbs Assignee: Marcus Eriksson Priority: Minor Labels: compaction Fix For: 2.0.9 Attachments: 0001-reference-sstable-before-populating-keycache.patch In CompactionTask.java, after a compaction finishes, we do this: {code} replaceCompactedSSTables(toCompact, sstables); // TODO: this doesn't belong here, it should be part of the reader to load when the tracker is wired up for (SSTableReader sstable : sstables) sstable.preheat(cachedKeyMap.get(sstable.descriptor)); {code} The problem is that if the table was dropped, {{replaceCompactedSSTables}} will release its references on the new {{sstables}}, resulting in them being closed. -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6968) Reduce Unit Test Times Due to Schema Loading
[ https://issues.apache.org/jira/browse/CASSANDRA-6968?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027164#comment-14027164 ] Tyler Hobbs commented on CASSANDRA-6968: [~lyubent] 7327 has committed, so you should be good to update this now. Reduce Unit Test Times Due to Schema Loading Key: CASSANDRA-6968 URL: https://issues.apache.org/jira/browse/CASSANDRA-6968 Project: Cassandra Issue Type: Test Components: Tests Reporter: Tyler Hobbs Assignee: Lyuben Todorov Priority: Minor Fix For: 2.1.1 Attachments: trunk-6968-speedup-unittests.patch Unit tests which extend SchemaLoader take about 6s longer to run than the others, on average. We could greatly reduce the time it takes to run the tests by improving this. None of the tests require everything that SchemaLoader does. We should change SchemaLoader into a set of test utilities that are run as needed in {{\@BeforeClass}} and {{\@AfterClass}} methods. Additionally, instead of running a full cleanup, most tests could simply use a keyspace or column family with a unique name (class/method name or perhaps class/method name + timestamp). -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-7365) some compactions do not works under windows (file in use during rename)
[ https://issues.apache.org/jira/browse/CASSANDRA-7365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027347#comment-14027347 ] Joshua McKenzie commented on CASSANDRA-7365: Tracked down error introduction to commit 4e95953f29d89a441dfe06d3f0393ed7dd8586df, CASSANDRA-6916. [~benedict] - This isn't causing any issues on trunk (3.x versions) with Windows but on 2.X, renaming files is only safe if all references / handles to it are closed out. I'm going to have to chew on the new logic of the SSTableRewriter a bit to see how best to work around it on Windows - have any immediate ideas? some compactions do not works under windows (file in use during rename) --- Key: CASSANDRA-7365 URL: https://issues.apache.org/jira/browse/CASSANDRA-7365 Project: Cassandra Issue Type: Bug Components: Core Environment: jdk7, cassandra-2.1rc1, os windows 32 bit Reporter: Radim Kolar Assignee: Joshua McKenzie Labels: Windows Fix For: 2.1.1 Attachments: cassandra.yaml, system.log compaction do not works under windows due to file rename fails: (Pro es nemß p°Ýstup k souboru, neboŁ jej prßvý vyu×Ývß jinř proces = process can not access file because its in use by another process). Not all compactions are broken. compactions done during server startup on system tables works fine. INFO 18:30:27 Completed flushing c:\cassandra-2.1\data\system\compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b\system-compactions_in_progress-ka-6-Dat.db (42 bytes) for commitlog position ReplayPosition(segmentId=1402165543361, psition=8024611) ERROR 18:30:27 Exception in thread hread[CompactionExecutor:5,1,RMI Runtime] java.lang.RuntimeException: Failed to rename c:\cassandra-2.1\data\test\sipdb-5 f51090ee6511e3815625991ef2b954\test-sipdb-tmp-ka-7-Index.db to c:\cassandra-2.1 data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka-7-Index.db at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:167) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:151) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:512) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:504) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.ja a:479) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:427) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:422) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewrit r.java:312) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewrit r.java:306) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionTask.runWith(Compaction ask.java:188) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAware unnable.java:48) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java: 8) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(Co pactionTask.java:74) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(Ab tractCompactionTask.java:59) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompa tionTask.run(CompactionManager.java:235) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0 rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:4 1) ~[na:1.7.0_60] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_ 0] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor java:1145) ~[na:1.7.0_60] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecuto .java:615) [na:1.7.0_60] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_60] Caused by: java.nio.file.FileSystemException: c:\cassandra-2.1\data\test\sipdb- 8f51090ee6511e3815625991ef2b954\test-sipdb-tmp-ka-7-Index.db - c:\cassandra-2.
[jira] [Commented] (CASSANDRA-7365) some compactions do not works under windows (file in use during rename)
[ https://issues.apache.org/jira/browse/CASSANDRA-7365?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027350#comment-14027350 ] Benedict commented on CASSANDRA-7365: - Simplest thing is to disable the feature on Windows some compactions do not works under windows (file in use during rename) --- Key: CASSANDRA-7365 URL: https://issues.apache.org/jira/browse/CASSANDRA-7365 Project: Cassandra Issue Type: Bug Components: Core Environment: jdk7, cassandra-2.1rc1, os windows 32 bit Reporter: Radim Kolar Assignee: Joshua McKenzie Labels: Windows Fix For: 2.1.1 Attachments: cassandra.yaml, system.log compaction do not works under windows due to file rename fails: (Pro es nemß p°Ýstup k souboru, neboŁ jej prßvý vyu×Ývß jinř proces = process can not access file because its in use by another process). Not all compactions are broken. compactions done during server startup on system tables works fine. INFO 18:30:27 Completed flushing c:\cassandra-2.1\data\system\compactions_in_progress-55080ab05d9c388690a4acb25fe1f77b\system-compactions_in_progress-ka-6-Dat.db (42 bytes) for commitlog position ReplayPosition(segmentId=1402165543361, psition=8024611) ERROR 18:30:27 Exception in thread hread[CompactionExecutor:5,1,RMI Runtime] java.lang.RuntimeException: Failed to rename c:\cassandra-2.1\data\test\sipdb-5 f51090ee6511e3815625991ef2b954\test-sipdb-tmp-ka-7-Index.db to c:\cassandra-2.1 data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka-7-Index.db at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:167) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.util.FileUtils.renameWithConfirm(FileUtils.j va:151) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:512) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.rename(SSTableWriter.j va:504) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.close(SSTableWriter.ja a:479) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:427) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableWriter.closeAndOpenReader(SST bleWriter.java:422) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewrit r.java:312) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.sstable.SSTableRewriter.finish(SSTableRewrit r.java:306) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionTask.runWith(Compaction ask.java:188) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAware unnable.java:48) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java: 8) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionTask.executeInternal(Co pactionTask.java:74) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(Ab tractCompactionTask.java:59) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompa tionTask.run(CompactionManager.java:235) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0 rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:4 1) ~[na:1.7.0_60] at java.util.concurrent.FutureTask.run(FutureTask.java:262) ~[na:1.7.0_ 0] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor java:1145) ~[na:1.7.0_60] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecuto .java:615) [na:1.7.0_60] at java.lang.Thread.run(Thread.java:745) [na:1.7.0_60] Caused by: java.nio.file.FileSystemException: c:\cassandra-2.1\data\test\sipdb- 8f51090ee6511e3815625991ef2b954\test-sipdb-tmp-ka-7-Index.db - c:\cassandra-2. \data\test\sipdb-58f51090ee6511e3815625991ef2b954\test-sipdb-ka-7-Index.db: Pro es nemß p°Ýstup k souboru, neboŁ jej prßvý vyu×Ývß jinř proces. at sun.nio.fs.WindowsException.translateToIOException(WindowsException. ava:86) ~[na:1.7.0_60] at sun.nio.fs.WindowsException.rethrowAsIOException(WindowsException.ja a:97) ~[na:1.7.0_60] at
[jira] [Updated] (CASSANDRA-7372) Exception when querying a composite-keyed table with a collection index
[ https://issues.apache.org/jira/browse/CASSANDRA-7372?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mikhail Stepura updated CASSANDRA-7372: --- Attachment: (was: CASSANDRA-2.1-7372-v2.patch) Exception when querying a composite-keyed table with a collection index --- Key: CASSANDRA-7372 URL: https://issues.apache.org/jira/browse/CASSANDRA-7372 Project: Cassandra Issue Type: Bug Reporter: Ghais Issa Fix For: 2.1.0 Given the following schema: {code} CREATE TABLE products ( account text, id int, categories settext, PRIMARY KEY (account, id) ); CREATE INDEX cat_index ON products(categories); {code} The following query fails with an exception {code} SELECT * FROM products WHERE account = 'xyz' AND categories CONTAINS 'lmn'; errors={}, last_host=127.0.0.1 {code} The exception in cassandra's log is: {code} WARN 17:01:49 Uncaught exception on thread Thread[SharedPool-Worker-2,5,main]: {} java.lang.RuntimeException: java.lang.IndexOutOfBoundsException at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2015) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_25] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.lang.Thread.run(Thread.java:724) ~[na:1.7.0_25] Caused by: java.lang.IndexOutOfBoundsException: null at org.apache.cassandra.db.composites.Composites$EmptyComposite.get(Composites.java:60) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesIndexOnCollectionKey.makeIndexColumnPrefix(CompositesIndexOnCollectionKey.java:78) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.makePrefix(CompositesSearcher.java:82) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.getIndexedIterator(CompositesSearcher.java:116) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.search(CompositesSearcher.java:68) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:589) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:2060) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:131) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$LocalRangeSliceRunnable.runMayThrow(StorageProxy.java:1368) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2011) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] ... 4 common frames omitted {code} The following query however works {code} SELECT * FROM products WHERE categories CONTAINS 'lmn'; {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Updated] (CASSANDRA-7372) Exception when querying a composite-keyed table with a collection index
[ https://issues.apache.org/jira/browse/CASSANDRA-7372?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Mikhail Stepura updated CASSANDRA-7372: --- Attachment: CASSANDRA-2.1-7372-v3.patch Attaching v3 of the patch with changes for {{CompositesSearcher.makePrefix}} only Exception when querying a composite-keyed table with a collection index --- Key: CASSANDRA-7372 URL: https://issues.apache.org/jira/browse/CASSANDRA-7372 Project: Cassandra Issue Type: Bug Reporter: Ghais Issa Fix For: 2.1.0 Attachments: CASSANDRA-2.1-7372-v3.patch Given the following schema: {code} CREATE TABLE products ( account text, id int, categories settext, PRIMARY KEY (account, id) ); CREATE INDEX cat_index ON products(categories); {code} The following query fails with an exception {code} SELECT * FROM products WHERE account = 'xyz' AND categories CONTAINS 'lmn'; errors={}, last_host=127.0.0.1 {code} The exception in cassandra's log is: {code} WARN 17:01:49 Uncaught exception on thread Thread[SharedPool-Worker-2,5,main]: {} java.lang.RuntimeException: java.lang.IndexOutOfBoundsException at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2015) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) ~[na:1.7.0_25] at org.apache.cassandra.concurrent.AbstractTracingAwareExecutorService$FutureTask.run(AbstractTracingAwareExecutorService.java:162) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:103) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at java.lang.Thread.run(Thread.java:724) ~[na:1.7.0_25] Caused by: java.lang.IndexOutOfBoundsException: null at org.apache.cassandra.db.composites.Composites$EmptyComposite.get(Composites.java:60) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesIndexOnCollectionKey.makeIndexColumnPrefix(CompositesIndexOnCollectionKey.java:78) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.makePrefix(CompositesSearcher.java:82) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.getIndexedIterator(CompositesSearcher.java:116) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.composites.CompositesSearcher.search(CompositesSearcher.java:68) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.index.SecondaryIndexManager.search(SecondaryIndexManager.java:589) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.ColumnFamilyStore.search(ColumnFamilyStore.java:2060) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.db.RangeSliceCommand.executeLocally(RangeSliceCommand.java:131) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$LocalRangeSliceRunnable.runMayThrow(StorageProxy.java:1368) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] at org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2011) ~[apache-cassandra-2.1.0-rc1.jar:2.1.0-rc1] ... 4 common frames omitted {code} The following query however works {code} SELECT * FROM products WHERE categories CONTAINS 'lmn'; {code} -- This message was sent by Atlassian JIRA (v6.2#6252)
[jira] [Commented] (CASSANDRA-6602) Compaction improvements to optimize time series data
[ https://issues.apache.org/jira/browse/CASSANDRA-6602?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14027378#comment-14027378 ] Jonathan Ellis commented on CASSANDRA-6602: --- Is this on your list, [~krummas]? Compaction improvements to optimize time series data Key: CASSANDRA-6602 URL: https://issues.apache.org/jira/browse/CASSANDRA-6602 Project: Cassandra Issue Type: New Feature Components: Core Reporter: Tupshin Harper Assignee: Björn Hegerfors Labels: compaction, performance Fix For: 3.0 Attachments: cassandra-2.0-CASSANDRA-6602-DateTieredCompactionStrategy.txt, cassandra-2.0-CASSANDRA-6602-DateTieredCompactionStrategy_v2.txt There are some unique characteristics of many/most time series use cases that both provide challenges, as well as provide unique opportunities for optimizations. One of the major challenges is in compaction. The existing compaction strategies will tend to re-compact data on disk at least a few times over the lifespan of each data point, greatly increasing the cpu and IO costs of that write. Compaction exists to 1) ensure that there aren't too many files on disk 2) ensure that data that should be contiguous (part of the same partition) is laid out contiguously 3) deleting data due to ttls or tombstones The special characteristics of time series data allow us to optimize away all three. Time series data 1) tends to be delivered in time order, with relatively constrained exceptions 2) often has a pre-determined and fixed expiration date 3) Never gets deleted prior to TTL 4) Has relatively predictable ingestion rates Note that I filed CASSANDRA-5561 and this ticket potentially replaces or lowers the need for it. In that ticket, jbellis reasonably asks, how that compaction strategy is better than disabling compaction. Taking that to heart, here is a compaction-strategy-less approach that could be extremely efficient for time-series use cases that follow the above pattern. (For context, I'm thinking of an example use case involving lots of streams of time-series data with a 5GB per day ingestion rate, and a 1000 day retention with TTL, resulting in an eventual steady state of 5TB per node) 1) You have an extremely large memtable (preferably off heap, if/when doable) for the table, and that memtable is sized to be able to hold a lengthy window of time. A typical period might be one day. At the end of that period, you flush the contents of the memtable to an sstable and move to the next one. This is basically identical to current behaviour, but with thresholds adjusted so that you can ensure flushing at predictable intervals. (Open question is whether predictable intervals is actually necessary, or whether just waiting until the huge memtable is nearly full is sufficient) 2) Combine the behaviour with CASSANDRA-5228 so that sstables will be efficiently dropped once all of the columns have. (Another side note, it might be valuable to have a modified version of CASSANDRA-3974 that doesn't bother storing per-column TTL since it is required that all columns have the same TTL) 3) Be able to mark column families as read/write only (no explicit deletes), so no tombstones. 4) Optionally add back an additional type of delete that would delete all data earlier than a particular timestamp, resulting in immediate dropping of obsoleted sstables. The result is that for in-order delivered data, Every cell will be laid out optimally on disk on the first pass, and over the course of 1000 days and 5TB of data, there will only be 1000 5GB sstables, so the number of filehandles will be reasonable. For exceptions (out-of-order delivery), most cases will be caught by the extended (24 hour+) memtable flush times and merged correctly automatically. For those that were slightly askew at flush time, or were delivered so far out of order that they go in the wrong sstable, there is relatively low overhead to reading from two sstables for a time slice, instead of one, and that overhead would be incurred relatively rarely unless out-of-order delivery was the common case, in which case, this strategy should not be used. Another possible optimization to address out-of-order would be to maintain more than one time-centric memtables in memory at a time (e.g. two 12 hour ones), and then you always insert into whichever one of the two owns the appropriate range of time. By delaying flushing the ahead one until we are ready to roll writes over to a third one, we are able to avoid any fragmentation as long as all deliveries come in no more than 12 hours late (in this example, presumably tunable). Anything that triggers compactions will have to