[jira] [Commented] (CASSANDRA-8630) Faster sequential IO (on compaction, streaming, etc)
[ https://issues.apache.org/jira/browse/CASSANDRA-8630?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532376#comment-14532376 ] Benedict commented on CASSANDRA-8630: - I'm in favour of simplifying this. Focusing on a small number of well designed and optimised paths for reads is the best route. I think we should also merge functionality with ByteBufferDataInput - if you look at it, you'll see for mmapped files we're actually incurring all of the CPU overhead for constructing the int/long values too. If we can tolerate this, we can instead tolerate a check before a read on if we need to move the buffer (so they can share the same implementation). In fact, this would at the same time permit us to eliminate the weirdness with multiple file segments, by having the mmap reader encapsulate that information and avoid it leaking into the rest of the codebase. If we can merge all of our readers into approximately one functional implementation of NIO reading, we're in a _much_ better position than we were. Obviously the main complexity is when a read spans two buffer offsets. The question then becomes what to do: ideally we want to read from the underlying file at page boundaries (although right now this is impossible in the common case of compression, so perhaps we shouldn't worry too much until CASSANDRA-8896 is delivered), but we also want to allocate page-aligned buffers (and CASSANDRA-8897 currently won't easily offer just slightly larger than page-aligned buffers). So: do we have a slow path for when crossing these boundaries? I don't like that either, as it will also likely slow down the common case. I think the best option is to have a buffer of size min(chunk-size + one page, 2 * chunk-size). This really requires CASSANDRA-8894, and even then probably requires an increase in the size of our buffer pool chunks in CASSANDRA-8897, which is quite achievable but may result in a higher watermark of memory use. We could make the default chunk size 256K (currently it is 64K), which would make it allocate _only_ page-aligned units, which would also simplify some of its logic but require that we complicate other bits, so that we don't discard 64K because we need a 68K allocation (i.e. we would need a queue of chunks we're currently able to allocate from). [~stef1927]: thoughts? Faster sequential IO (on compaction, streaming, etc) Key: CASSANDRA-8630 URL: https://issues.apache.org/jira/browse/CASSANDRA-8630 Project: Cassandra Issue Type: Improvement Components: Core, Tools Reporter: Oleg Anastasyev Assignee: Oleg Anastasyev Labels: performance Fix For: 3.x Attachments: 8630-FasterSequencialReadsAndWrites.txt, cpu_load.png When node is doing a lot of sequencial IO (streaming, compacting, etc) a lot of CPU is lost in calls to RAF's int read() and DataOutputStream's write(int). This is because default implementations of readShort,readLong, etc as well as their matching write* are implemented with numerous calls of byte by byte read and write. This makes a lot of syscalls as well. A quick microbench shows than just reimplementation of these methods in either way gives 8x speed increase. A patch attached implements RandomAccessReader.readType and SequencialWriter.writeType methods in more efficient way. I also eliminated some extra byte copies in CompositeType.split and ColumnNameHelper.maxComponents, which were on my profiler's hotspot method list during tests. A stress tests on my laptop show that this patch makes compaction 25-30% faster on uncompressed sstables and 15% faster for compressed ones. A deployment to production shows much less CPU load for compaction. (I attached a cpu load graph from one of our production, orange is niced CPU load - i.e. compaction; yellow is user - i.e. not compaction related tasks) -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9136) Improve error handling when table is queried before the schema has fully propagated
[ https://issues.apache.org/jira/browse/CASSANDRA-9136?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532183#comment-14532183 ] Sylvain Lebresne commented on CASSANDRA-9136: - +1 Improve error handling when table is queried before the schema has fully propagated --- Key: CASSANDRA-9136 URL: https://issues.apache.org/jira/browse/CASSANDRA-9136 Project: Cassandra Issue Type: Bug Components: Core Environment: 3 Nodes GCE, N1-Standard-2, Ubuntu 12, 1 Node on 2.1.4, 2 on 2.0.14 Reporter: Russell Alexander Spitzer Assignee: Tyler Hobbs Fix For: 2.1.x, 2.0.x Attachments: 9136-2.0-v2.txt, 9136-2.0.txt, 9136-2.1-v2.txt, 9136-2.1.txt This error occurs during a rolling upgrade between 2.0.14 and 2.1.4. h3. Repo With all the nodes on 2.0.14 make the following tables {code} CREATE KEYSPACE test WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; USE test; CREATE TABLE compact ( k int, c int, d int, PRIMARY KEY ((k), c) ) WITH COMPACT STORAGE; CREATE TABLE norm ( k int, c int, d int, PRIMARY KEY ((k), c) ) ; {code} Then load some data into these tables. I used the python driver {code} from cassandra.cluster import Cluster s = Cluster().connect() for x in range (1000): for y in range (1000): s.execute_async(INSERT INTO test.compact (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) s.execute_async(INSERT INTO test.norm (k,c,d) VALUES (%d,%d,%d)%(x,y,y)) {code} Upgrade one node from 2.0.14 - 2.1.4 From the 2.1.4 node, create a new table. Query that table On the 2.0.14 nodes you get these exceptions because the schema didn't propagate there. This exception kills the TCP connection between the nodes. {code} ERROR [Thread-19] 2015-04-08 18:48:45,337 CassandraDaemon.java (line 258) Exception in thread Thread[Thread-19,5,main] java.lang.NullPointerException at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247) at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156) at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74) {code} Run cqlsh on the upgraded node and queries will fail until the TCP connection is established again, easiest to repo with CL = ALL {code} cqlsh SELECT count(*) FROM test.norm where k = 22 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} cqlsh SELECT count(*) FROM test.norm where k = 21 ; ReadTimeout: code=1200 [Coordinator node timed out waiting for replica nodes' responses] message=Operation timed out - received only 1 responses. info={'received_responses': 1, 'required_responses': 2, 'consistency': 'ALL'} {code} So connection made: {code} DEBUG [Thread-227] 2015-04-09 05:09:02,718 IncomingTcpConnection.java (line 107) Set version for /10.240.14.115 to 8 (will use 7) {code} Connection broken by query of table before schema propagated: {code} ERROR [Thread-227] 2015-04-09 05:10:24,015 CassandraDaemon.java (line 258) Exception in thread Thread[Thread-227,5,main] java.lang.NullPointerException at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:247) at org.apache.cassandra.db.RangeSliceCommandSerializer.deserialize(RangeSliceCommand.java:156) at org.apache.cassandra.net.MessageIn.read(MessageIn.java:99) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:149) at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:131) at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:74) {code} All query to that node will fail with timeouts now until... Connection re-established {code} DEBUG [Thread-228] 2015-04-09 05:11:00,323 IncomingTcpConnection.java (line 107) Set version for /10.240.14.115 to 8 (will use 7) {code} Now queries work again. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14530782#comment-14530782 ] koray sariteke edited comment on CASSANDRA-9301 at 5/7/15 8:09 AM: --- Tried the same scenario on empty table. Then able to query missed data over cqlsh for a short period of time. Then data was disapeared(probably after compaction). was (Author: ksaritek): Observe one thing, for a while, able to query missed data for a short period of time after inserted data. Then data was disapeared. can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data(49 data) are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9295) Streaming not holding on to refs long enough.
[ https://issues.apache.org/jira/browse/CASSANDRA-9295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532319#comment-14532319 ] Benedict commented on CASSANDRA-9295: - Perhaps we could just ignore the last block for validation? Or alternatively we could switch this to using only complete files, as we have for nearly all other non-request-serving operations (this might be better for the receiving node, but would churn the page cache more on the sending node) Streaming not holding on to refs long enough. - Key: CASSANDRA-9295 URL: https://issues.apache.org/jira/browse/CASSANDRA-9295 Project: Cassandra Issue Type: Bug Reporter: Jeremiah Jordan Assignee: Yuki Morishita Fix For: 2.1.x Attachments: 0001-don-t-release-sstable-while-streaming.patch, 0002-demonstrate-checksum-failure-of-early-opened-SSTable.patch, 0003-skip-validation-of-early-opened-SSTable.patch, 9295.debug.txt While doing some testing around adding/removing nodes under load with cassandra-2.1 head as of a few days ago (after was 2.1.5 tagged) I am seeing stream out errors with file not found exceptions. The file in question just finished being compacted into a new file a few lines earlier in the log. Seems that streaming isn't holding onto Ref's correctly for the stuff in the stream plans. I also see a corrupt sstable exception for the file the missing file was compacted to. Trimmed logs with just the compaction/streaming related stuff: You can see the stream plan is initiated in between the compaction starting, and the compaction finishing. {noformat} INFO [MemtableFlushWriter:3] 2015-05-04 16:08:21,239 Memtable.java:380 - Completed flushing /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db (60666088 bytes) for commitlog position ReplayPosition(segmentId=1430755416941, position=32294797) INFO [CompactionExecutor:4] 2015-05-04 16:08:40,856 CompactionTask.java:140 - Compacting [SSTableReader(path='/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db'), SSTableReader(path='/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-3-Data.db')] INFO [STREAM-INIT-/10.240.213.56:53190] 2015-05-04 16:09:31,047 StreamResultFuture.java:109 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9 ID#0] Creating new streaming plan for Rebuild INFO [STREAM-INIT-/10.240.213.56:53190] 2015-05-04 16:09:31,238 StreamResultFuture.java:116 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9, ID#0] Received streaming plan for Rebuild INFO [STREAM-INIT-/10.240.213.56:53192] 2015-05-04 16:09:31,249 StreamResultFuture.java:116 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9, ID#0] Received streaming plan for Rebuild INFO [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:31,353 ColumnFamilyStore.java:882 - Enqueuing flush of standard1: 91768068 (19%) on-heap, 0 (0%) off-heap INFO [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:37,425 ColumnFamilyStore.java:882 - Enqueuing flush of solr: 10012689 (2%) on-heap, 0 (0%) off-heap INFO [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:38,073 StreamResultFuture.java:166 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9 ID#0] Prepare completed. Receiving 0 files(0 bytes), sending 6 files(284288285 bytes) INFO [CompactionExecutor:4] 2015-05-04 16:10:11,047 CompactionTask.java:270 - Compacted 2 sstables to [/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-5,/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-8,]. 182,162,816 bytes to 182,162,816 (~100% of original) in 90,188ms = 1.926243MB/s. 339,856 total partitions merged to 339,856. Partition merge counts were {1:339856, } ERROR [STREAM-OUT-/10.240.213.56] 2015-05-04 16:10:25,169 StreamSession.java:477 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9] Streaming error occurred java.io.IOException: Corrupted SSTable : /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-5-Data.db at org.apache.cassandra.io.util.DataIntegrityMetadata$ChecksumValidator.validate(DataIntegrityMetadata.java:79) ~[cassandra-all-2.1.5.426.jar:2.1.5.426] at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:149) ~[cassandra-all-2.1.5.426.jar:2.1.5.426] at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:102) ~[cassandra-all-2.1.5.426.jar:2.1.5.426] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:58)
[jira] [Comment Edited] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14530782#comment-14530782 ] koray sariteke edited comment on CASSANDRA-9301 at 5/7/15 8:10 AM: --- Tried the same scenario on empty table. Then able to query missed data over cqlsh for a short period of time. Then data was disapeared. was (Author: ksaritek): Tried the same scenario on empty table. Then able to query missed data over cqlsh for a short period of time. Then data was disapeared(probably after compaction). can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data(49 data) are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9321) Aggregate UDFs allow SFUNC return type to differ from STYPE if FFUNC specified
[ https://issues.apache.org/jira/browse/CASSANDRA-9321?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Robert Stupp updated CASSANDRA-9321: Attachment: 9321.txt The attached patch tackles both issues: * now always checks {{STYPE}} against {{SFUNC}} return type (argument type is implicitly checked using {{Functions.find()}}) * the new NPE, which is caused by the broken aggregate ({{UDAggregate}} could not be initialized - so {{stateFunction}} was {{null}}) * added new utest against the original issue (SFUNC return type and arg type differs) Aggregate UDFs allow SFUNC return type to differ from STYPE if FFUNC specified -- Key: CASSANDRA-9321 URL: https://issues.apache.org/jira/browse/CASSANDRA-9321 Project: Cassandra Issue Type: Bug Components: Core Reporter: Zachary Kurey Assignee: Robert Stupp Fix For: 3.0 Attachments: 9321.txt When a final function is specified in an aggregate C* allows the return type of the state function to not match the state type. Allowing the mismatch if a final function is specified seems to be intentional as if you don't provide a final function and you provide a state function with a return type that doesn't match the state type, then C* gives you an error that states that they must match unless a final function is specified. It seems incorrect regardless of whether or not a final function is present to allow the state functions return type to vary from state type. And indeed if you do so it produces an error when you try to use the aggregate. Here is a simple example that shows the problem: {code} CREATE OR REPLACE FUNCTION state_func(state int, p2 int) RETURNS double LANGUAGE java AS 'return Double.valueOf(1.0);'; CREATE OR REPLACE FUNCTION final_func(state int) RETURNS int LANGUAGE java AS 'return Integer.valueOf(1);'; CREATE OR REPLACE AGGREGATE my_aggregate( int ) SFUNC state_func STYPE int FINALFUNC final_func INITCOND 1; SELECT my_aggregate(column) FROM table; {code} The select produces the error: {noformat} Aggregate 'ks.my_aggregate : (int) - int' exists but hasn't been loaded successfully for the following reason: Referenced state function 'ks.state_func [double, int]' for aggregate 'ks.my_aggregate' does not exist. {noformat} I was reproducing this with 3.0 trunk, though now I just grabbed the latest and there is an NPE instead of the error above: {noformat} java.lang.NullPointerException: at index 1 at com.google.common.collect.ObjectArrays.checkElementNotNull(ObjectArrays.java:240) ~[guava-16.0.jar:na] at com.google.common.collect.ImmutableSet.construct(ImmutableSet.java:195) ~[guava-16.0.jar:na] at com.google.common.collect.ImmutableSet.of(ImmutableSet.java:116) ~[guava-16.0.jar:na] at org.apache.cassandra.cql3.functions.UDAggregate.getFunctions(UDAggregate.java:110) ~[main/:na] at org.apache.cassandra.cql3.selection.AbstractFunctionSelector$1.getFunctions(AbstractFunctionSelector.java:78) ~[main/:na] at org.apache.cassandra.cql3.selection.SelectorFactories.getFunctions(SelectorFactories.java:105) ~[main/:na] {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532262#comment-14532262 ] Benedict commented on CASSANDRA-9318: - I'm very much in favour of stopping C* keeling over, and doing so in 2.1, but I don't see smooth rate limiting as a realistic possibility (in the near-term), and nor do I think a coordinator managed approach will do the job. The problem with the coordinator is two-fold: # It has no idea how big its responses are going to be, so it cannot prevent new requests from starting that will not, combined, blow its constraints # It has no idea how many _of its own_ requests are still in flight in the rest of the cluster The problem can easily be illustrated by the fact that we can already bring down clusters trivially with a default stress workload. Stress packets are tiny, and only ~256 of them can be in-flight at any time with any coordinator. The combined request size is less than 1Mb, and yet the cluster can be destroyed. The reason for this is that many more requests are in-flight than the coordinator realises. The processing nodes need to have size-based load shedding, as was essentially proposed by CASSANDRA-8518. Or, as a simple initial mechanism, a bound on the size of the work queue as proposed by CASSANDRA-5039. This latter approach is probably the easiest to introduce in 2.1. Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 3.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9296) OHC fails to load in Zing 1.7 VM
[ https://issues.apache.org/jira/browse/CASSANDRA-9296?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532292#comment-14532292 ] Robert Stupp commented on CASSANDRA-9296: - [~aweisberg] can you check whether v0.3.5 fixes your issue with Zing? https://oss.sonatype.org/content/repositories/releases/org/caffinitas/ohc/ohc-core/0.3.5/ OHC fails to load in Zing 1.7 VM Key: CASSANDRA-9296 URL: https://issues.apache.org/jira/browse/CASSANDRA-9296 Project: Cassandra Issue Type: Bug Reporter: Ariel Weisberg Assignee: Robert Stupp Fix For: 3.x Error output is a disaster but I am including it here, will clean up later. I had to change code to get this error to appear properly with the invocation target exception unwrapped. 1.7 VM version {noformat} java version 1.7.0-zing_15.02.1.0 Zing Runtime Environment for Java Applications (build 1.7.0-zing_15.02.1.0-b2) Zing 64-Bit Tiered VM (build 1.7.0-zing_15.02.1.0-b2-product-azlinuxM-X86_64, mixed mode) {noformat} This does work with the 1.8 Zing VM. {noformat} [junit] java.lang.ExceptionInInitializerError [junit] at org.apache.cassandra.db.ColumnFamilyStore.init(ColumnFamilyStore.java:330) [junit] at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:468) [junit] at org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:444) [junit] at org.apache.cassandra.db.Keyspace.initCf(Keyspace.java:324) [junit] at org.apache.cassandra.db.Keyspace.init(Keyspace.java:275) [junit] at org.apache.cassandra.db.Keyspace.open(Keyspace.java:117) [junit] at org.apache.cassandra.db.Keyspace.open(Keyspace.java:94) [junit] at org.apache.cassandra.db.Keyspace$1.apply(Keyspace.java:81) [junit] at org.apache.cassandra.db.Keyspace$1.apply(Keyspace.java:78) [junit] at com.google.common.collect.Iterators$8.transform(Iterators.java:794) [junit] at com.google.common.collect.TransformedIterator.next(TransformedIterator.java:48) [junit] at org.apache.cassandra.db.ColumnFamilyStore.all(ColumnFamilyStore.java:2387) [junit] at org.apache.cassandra.config.CFMetaData.existingIndexNames(CFMetaData.java:1078) [junit] at org.apache.cassandra.config.CFMetaData.validate(CFMetaData.java:1035) [junit] at org.apache.cassandra.config.KSMetaData.validate(KSMetaData.java:180) [junit] at org.apache.cassandra.service.MigrationManager.announceNewKeyspace(MigrationManager.java:264) [junit] at org.apache.cassandra.service.MigrationManager.announceNewKeyspace(MigrationManager.java:259) [junit] at org.apache.cassandra.SchemaLoader.createKeyspace(SchemaLoader.java:340) [junit] at org.apache.cassandra.SchemaLoader.createKeyspace(SchemaLoader.java:328) [junit] at org.apache.cassandra.cache.AutoSavingCacheTest.defineSchema(AutoSavingCacheTest.java:49) [junit] at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) [junit] at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) [junit] at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) [junit] at java.lang.reflect.Method.invoke(Method.java:606) [junit] at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:44) [junit] at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:15) [junit] at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:41) [junit] at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:27) [junit] at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:31) [junit] at org.junit.runners.ParentRunner.run(ParentRunner.java:220) [junit] at junit.framework.JUnit4TestAdapter.run(JUnit4TestAdapter.java:39) [junit] at org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner.run(JUnitTestRunner.java:518) [junit] at org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner.launch(JUnitTestRunner.java:1052) [junit] at org.apache.tools.ant.taskdefs.optional.junit.JUnitTestRunner.main(JUnitTestRunner.java:906) [junit] Caused by: java.lang.RuntimeException: java.lang.reflect.InvocationTargetException [junit] at org.caffinitas.ohc.OHCacheBuilder.build(OHCacheBuilder.java:221) [junit] at org.apache.cassandra.cache.OHCProvider.create(OHCProvider.java:49) [junit] at org.apache.cassandra.service.CacheService.initRowCache(CacheService.java:151) [junit] at org.apache.cassandra.service.CacheService.init(CacheService.java:103) [junit] at
[jira] [Commented] (CASSANDRA-8584) Add strerror output on failed trySkipCache calls
[ https://issues.apache.org/jira/browse/CASSANDRA-8584?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532335#comment-14532335 ] Benedict commented on CASSANDRA-8584: - That looks fine. I have one more suggestion, though (not an important one): It would _sometimes_ be helpful for the NoSpamLogger to accept a second guard parameter. For instance, the original goal of this ticket (failed skip cache) could have different error reasons. It might be helpful to guard them independently, to avoid suppressing useful information. Not exactly required, given the non-essential nature of the call, but it could be helpful information for the operator we're suppressing. Add strerror output on failed trySkipCache calls Key: CASSANDRA-8584 URL: https://issues.apache.org/jira/browse/CASSANDRA-8584 Project: Cassandra Issue Type: Improvement Reporter: Joshua McKenzie Assignee: Ariel Weisberg Priority: Trivial Fix For: 2.1.x Attachments: 8584_v1.txt, NoSpamLogger.java, nospamlogger.txt Since trySkipCache returns an errno rather than -1 and setting errno like our other CLibrary calls, it's thread-safe and we could print out more helpful information if we failed to prompt the kernel to skip the page cache. That system call should always succeed unless we have an invalid fd as it's free to ignore us. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: StreamingTransferTest fails under test-compression due to bad assertion
Repository: cassandra Updated Branches: refs/heads/trunk 6d5b0b45a - 0ede9ec93 StreamingTransferTest fails under test-compression due to bad assertion patch by Ariel Weisberg; reviewed by tjake for CASSANDRA-9276 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/0ede9ec9 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/0ede9ec9 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/0ede9ec9 Branch: refs/heads/trunk Commit: 0ede9ec938e082f266bcebb56f1bdb5415dc798a Parents: 6d5b0b4 Author: T Jake Luciani j...@apache.org Authored: Thu May 7 14:08:00 2015 -0400 Committer: T Jake Luciani j...@apache.org Committed: Thu May 7 14:08:00 2015 -0400 -- .../cassandra/streaming/compress/CompressedStreamReader.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/0ede9ec9/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java -- diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java index 46f7d4f..89773ea 100644 --- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java +++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java @@ -82,7 +82,7 @@ public class CompressedStreamReader extends StreamReader { for (PairLong, Long section : sections) { -assert in.getBytesRead() totalSize; +assert cis.getTotalCompressedBytesRead() = totalSize; int sectionLength = (int) (section.right - section.left); // skip to beginning of section inside chunk
cassandra git commit: Delete processed sstables in sstablesplit/sstableupgrade. Adds option to keep originals in sstableupgrade. Removes unused verbose option in sstablesplit.
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 049a965bf - 09ccf60ed Delete processed sstables in sstablesplit/sstableupgrade. Adds option to keep originals in sstableupgrade. Removes unused verbose option in sstablesplit. patch by Branimir Lambov; reviewed by tjake for (CASSANDRA-8606) Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/09ccf60e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/09ccf60e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/09ccf60e Branch: refs/heads/cassandra-2.1 Commit: 09ccf60ed3372cc8378fb2f788b97e826a21d7e1 Parents: 049a965 Author: Branimir Lambov branimir.lam...@datastax.com Authored: Tue Mar 31 14:41:42 2015 +0100 Committer: T Jake Luciani j...@apache.org Committed: Thu May 7 14:19:39 2015 -0400 -- CHANGES.txt | 1 + .../org/apache/cassandra/tools/StandaloneSplitter.java | 8 .../org/apache/cassandra/tools/StandaloneUpgrader.java | 12 3 files changed, 17 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/09ccf60e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d512973..098cfe0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.6 + * Delete processed sstables in sstablesplit/sstableupgrade (CASSANDRA-8606) * Improve sstable exclusion from partition tombstones (CASSANDRA-9298) * Validate the indexed column rather than the cell's contents for 2i (CASSANDRA-9057) * Add support for top-k custom 2i queries (CASSANDRA-8717) http://git-wip-us.apache.org/repos/asf/cassandra/blob/09ccf60e/src/java/org/apache/cassandra/tools/StandaloneSplitter.java -- diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java index bc614cc..fd93f03 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java +++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java @@ -42,7 +42,6 @@ public class StandaloneSplitter public static final int DEFAULT_SSTABLE_SIZE = 50; private static final String TOOL_NAME = sstablessplit; -private static final String VERBOSE_OPTION = verbose; private static final String DEBUG_OPTION = debug; private static final String HELP_OPTION = help; private static final String NO_SNAPSHOT_OPTION = no-snapshot; @@ -152,6 +151,10 @@ public class StandaloneSplitter try { new SSTableSplitter(cfs, sstable, options.sizeInMB).split(); + +// Remove the sstable (it's been copied by split and snapshotted) +sstable.markObsolete(); +sstable.selfRef().release(); } catch (Exception e) { @@ -185,7 +188,6 @@ public class StandaloneSplitter public final ListString filenames; public boolean debug; -public boolean verbose; public boolean snapshot; public int sizeInMB; @@ -217,7 +219,6 @@ public class StandaloneSplitter } Options opts = new Options(Arrays.asList(args)); opts.debug = cmd.hasOption(DEBUG_OPTION); -opts.verbose = cmd.hasOption(VERBOSE_OPTION); opts.snapshot = !cmd.hasOption(NO_SNAPSHOT_OPTION); opts.sizeInMB = DEFAULT_SSTABLE_SIZE; @@ -244,7 +245,6 @@ public class StandaloneSplitter { CmdLineOptions options = new CmdLineOptions(); options.addOption(null, DEBUG_OPTION, display stack traces); -options.addOption(v, VERBOSE_OPTION,verbose output); options.addOption(h, HELP_OPTION, display this help message); options.addOption(null, NO_SNAPSHOT_OPTION,don't snapshot the sstables before splitting); options.addOption(s, SIZE_OPTION, size, maximum size in MB for the output sstables (default: + DEFAULT_SSTABLE_SIZE + )); http://git-wip-us.apache.org/repos/asf/cassandra/blob/09ccf60e/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java -- diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java index 8fa5b60..22c57f7 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java +++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java @@ -41,6 +41,7 @@ public class StandaloneUpgrader private static final String TOOL_NAME =
[1/2] cassandra git commit: Delete processed sstables in sstablesplit/sstableupgrade. Adds option to keep originals in sstableupgrade. Removes unused verbose option in sstablesplit.
Repository: cassandra Updated Branches: refs/heads/trunk 0ede9ec93 - 3cb00db3f Delete processed sstables in sstablesplit/sstableupgrade. Adds option to keep originals in sstableupgrade. Removes unused verbose option in sstablesplit. patch by Branimir Lambov; reviewed by tjake for (CASSANDRA-8606) Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/09ccf60e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/09ccf60e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/09ccf60e Branch: refs/heads/trunk Commit: 09ccf60ed3372cc8378fb2f788b97e826a21d7e1 Parents: 049a965 Author: Branimir Lambov branimir.lam...@datastax.com Authored: Tue Mar 31 14:41:42 2015 +0100 Committer: T Jake Luciani j...@apache.org Committed: Thu May 7 14:19:39 2015 -0400 -- CHANGES.txt | 1 + .../org/apache/cassandra/tools/StandaloneSplitter.java | 8 .../org/apache/cassandra/tools/StandaloneUpgrader.java | 12 3 files changed, 17 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/09ccf60e/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d512973..098cfe0 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.6 + * Delete processed sstables in sstablesplit/sstableupgrade (CASSANDRA-8606) * Improve sstable exclusion from partition tombstones (CASSANDRA-9298) * Validate the indexed column rather than the cell's contents for 2i (CASSANDRA-9057) * Add support for top-k custom 2i queries (CASSANDRA-8717) http://git-wip-us.apache.org/repos/asf/cassandra/blob/09ccf60e/src/java/org/apache/cassandra/tools/StandaloneSplitter.java -- diff --git a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java index bc614cc..fd93f03 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneSplitter.java +++ b/src/java/org/apache/cassandra/tools/StandaloneSplitter.java @@ -42,7 +42,6 @@ public class StandaloneSplitter public static final int DEFAULT_SSTABLE_SIZE = 50; private static final String TOOL_NAME = sstablessplit; -private static final String VERBOSE_OPTION = verbose; private static final String DEBUG_OPTION = debug; private static final String HELP_OPTION = help; private static final String NO_SNAPSHOT_OPTION = no-snapshot; @@ -152,6 +151,10 @@ public class StandaloneSplitter try { new SSTableSplitter(cfs, sstable, options.sizeInMB).split(); + +// Remove the sstable (it's been copied by split and snapshotted) +sstable.markObsolete(); +sstable.selfRef().release(); } catch (Exception e) { @@ -185,7 +188,6 @@ public class StandaloneSplitter public final ListString filenames; public boolean debug; -public boolean verbose; public boolean snapshot; public int sizeInMB; @@ -217,7 +219,6 @@ public class StandaloneSplitter } Options opts = new Options(Arrays.asList(args)); opts.debug = cmd.hasOption(DEBUG_OPTION); -opts.verbose = cmd.hasOption(VERBOSE_OPTION); opts.snapshot = !cmd.hasOption(NO_SNAPSHOT_OPTION); opts.sizeInMB = DEFAULT_SSTABLE_SIZE; @@ -244,7 +245,6 @@ public class StandaloneSplitter { CmdLineOptions options = new CmdLineOptions(); options.addOption(null, DEBUG_OPTION, display stack traces); -options.addOption(v, VERBOSE_OPTION,verbose output); options.addOption(h, HELP_OPTION, display this help message); options.addOption(null, NO_SNAPSHOT_OPTION,don't snapshot the sstables before splitting); options.addOption(s, SIZE_OPTION, size, maximum size in MB for the output sstables (default: + DEFAULT_SSTABLE_SIZE + )); http://git-wip-us.apache.org/repos/asf/cassandra/blob/09ccf60e/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java -- diff --git a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java index 8fa5b60..22c57f7 100644 --- a/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java +++ b/src/java/org/apache/cassandra/tools/StandaloneUpgrader.java @@ -41,6 +41,7 @@ public class StandaloneUpgrader private static final String TOOL_NAME = sstableupgrade;
[jira] [Commented] (CASSANDRA-9111) SSTables originated from the same incremental repair session have different repairedAt timestamps
[ https://issues.apache.org/jira/browse/CASSANDRA-9111?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533228#comment-14533228 ] Yuki Morishita commented on CASSANDRA-9111: --- Sorry for late reply. Can you create the patch for trunk to be released in 3.x? Since if we changed message format, repair can be hang if version is different in the cluster. SSTables originated from the same incremental repair session have different repairedAt timestamps - Key: CASSANDRA-9111 URL: https://issues.apache.org/jira/browse/CASSANDRA-9111 Project: Cassandra Issue Type: Bug Components: Core Reporter: prmg Attachments: CASSANDRA-9111-v0.txt, CASSANDRA-9111-v1.txt CASSANDRA-7168 optimizes QUORUM reads by skipping incrementally repaired SSTables on other replicas that were repaired on or before the maximum repairedAt timestamp of the coordinating replica's SSTables for the query partition. One assumption of that optimization is that SSTables originated from the same repair session in different nodes will have the same repairedAt timestamp, since the objective is to skip reading SSTables originated in the same repair session (or before). However, currently, each node timestamps independently SSTables originated from the same repair session, so they almost never have the same timestamp. Steps to reproduce the problem: {code} ccm create test ccm populate -n 3 ccm start ccm node1 cqlsh; {code} {code:sql} CREATE KEYSPACE foo WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 3}; CREATE TABLE foo.bar ( key int, col int, PRIMARY KEY (key) ) ; INSERT INTO foo.bar (key, col) VALUES (1, 1); exit; {code} {code} ccm node1 flush; ccm node2 flush; ccm node3 flush; nodetool -h 127.0.0.1 -p 7100 repair -par -inc foo bar [2015-04-02 21:56:07,726] Starting repair command #1, repairing 3 ranges for keyspace foo (parallelism=PARALLEL, full=false) [2015-04-02 21:56:07,816] Repair session 3655b670-d99c-11e4-b250-9107aba35569 for range (3074457345618258602,-9223372036854775808] finished [2015-04-02 21:56:07,816] Repair session 365a4a50-d99c-11e4-b250-9107aba35569 for range (-9223372036854775808,-3074457345618258603] finished [2015-04-02 21:56:07,818] Repair session 365bf800-d99c-11e4-b250-9107aba35569 for range (-3074457345618258603,3074457345618258602] finished [2015-04-02 21:56:07,995] Repair command #1 finished sstablemetadata ~/.ccm/test/node1/data/foo/bar-377b5540d99d11e49cc09107aba35569/foo-bar-ka-1-Statistics.db ~/.ccm/test/node2/data/foo/bar-377b5540d99d11e49cc09107aba35569/foo-bar-ka-1-Statistics.db ~/.ccm/test/node3/data/foo/bar-377b5540d99d11e49cc09107aba35569/foo-bar-ka-1-Statistics.db | grep Repaired Repaired at: 1428023050318 Repaired at: 1428023050322 Repaired at: 1428023050340 {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Failure detector detects and ignores local pauses
Repository: cassandra Updated Branches: refs/heads/trunk 3cb00db3f - 4047dd121 Failure detector detects and ignores local pauses Patch by brandonwilliams, reviewed by Richard Low for CASSANDRA-9183 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/4047dd12 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/4047dd12 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/4047dd12 Branch: refs/heads/trunk Commit: 4047dd1213ed99b3d7bec253b551b4cae911990a Parents: 3cb00db Author: Brandon Williams brandonwilli...@apache.org Authored: Thu May 7 14:15:51 2015 -0500 Committer: Brandon Williams brandonwilli...@apache.org Committed: Thu May 7 14:16:47 2015 -0500 -- CHANGES.txt | 1 + .../apache/cassandra/gms/FailureDetector.java | 29 2 files changed, 30 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/4047dd12/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 5a8ee93..9cfc772 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 3.0 + * Failure detector detects and ignores local pauses (CASSANDRA-9183) * Remove Thrift dependencies in bundled tools (CASSANDRA-8358) * Disable memory mapping of hsperfdata file for JVM statistics (CASSANDRA-9242) * Add pre-startup checks to detect potential incompatibilities (CASSANDRA-8049) http://git-wip-us.apache.org/repos/asf/cassandra/blob/4047dd12/src/java/org/apache/cassandra/gms/FailureDetector.java -- diff --git a/src/java/org/apache/cassandra/gms/FailureDetector.java b/src/java/org/apache/cassandra/gms/FailureDetector.java index fe825b6..b8c20d7 100644 --- a/src/java/org/apache/cassandra/gms/FailureDetector.java +++ b/src/java/org/apache/cassandra/gms/FailureDetector.java @@ -48,6 +48,22 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean public static final String MBEAN_NAME = org.apache.cassandra.net:type=FailureDetector; private static final int SAMPLE_SIZE = 1000; protected static final long INITIAL_VALUE_NANOS = TimeUnit.NANOSECONDS.convert(getInitialValue(), TimeUnit.MILLISECONDS); +private static final long DEFAULT_MAX_PAUSE = 5000L * 100L; // 5 seconds +private static final long MAX_LOCAL_PAUSE_IN_NANOS = getMaxLocalPause(); +private long lastInterpret = System.nanoTime(); +private boolean wasPaused = false; + +private static long getMaxLocalPause() +{ +if (System.getProperty(cassandra.max_local_pause_in_ms) != null) +{ +long pause = Long.parseLong(System.getProperty(cassandra.max_local_pause_in_ms)); +logger.warn(Overriding max local pause time to {}ms, pause); +return pause * 100L; +} +else +return DEFAULT_MAX_PAUSE; +} public static final IFailureDetector instance = new FailureDetector(); @@ -228,6 +244,19 @@ public class FailureDetector implements IFailureDetector, FailureDetectorMBean return; } long now = System.nanoTime(); +long diff = now - lastInterpret; +lastInterpret = now; +if (diff MAX_LOCAL_PAUSE_IN_NANOS) +{ +logger.warn(Not marking nodes down due to local pause of {} {}, diff, MAX_LOCAL_PAUSE_IN_NANOS); +wasPaused = true; +return; +} +if (wasPaused) +{ +wasPaused = false; +return; +} double phi = hbWnd.phi(now); if (logger.isTraceEnabled()) logger.trace(PHI for {} : {}, ep, phi);
[jira] [Updated] (CASSANDRA-9326) Seeing tombstone warning message
[ https://issues.apache.org/jira/browse/CASSANDRA-9326?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-9326: - Priority: Minor (was: Critical) Seeing tombstone warning message Key: CASSANDRA-9326 URL: https://issues.apache.org/jira/browse/CASSANDRA-9326 Project: Cassandra Issue Type: Bug Components: Core Reporter: srinivasu gottipati Priority: Minor Fix For: 2.0.x We deleted data for some of the rows in one of the column families. After that we ran repair on all nodes, and followed by reducing gc_grace_seconds that way compaction can remove all the tombstones upon expiry of gc_grace_seconds time. When we are querying the data now, seeing the following errors: WARN [ReadStage:1142] 2015-05-06 17:50:53,602 SliceQueryFilter.java (line 231) Read 1 live and 1487 tombstoned cells in (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} We deleted this data while back and for sure gc_grace_seconds is elapsed long time back and we use leveled compaction. In the above, errors, localDeletion points to some time in future (MAX INT VALUE) and that could be the reason these are n't being purged. Any help (or) workaround is appreciated. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9276) StreamingTransferTest fails under test-compression due to bad assertion
[ https://issues.apache.org/jira/browse/CASSANDRA-9276?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] T Jake Luciani updated CASSANDRA-9276: -- Fix Version/s: (was: 3.x) 3.0 StreamingTransferTest fails under test-compression due to bad assertion --- Key: CASSANDRA-9276 URL: https://issues.apache.org/jira/browse/CASSANDRA-9276 Project: Cassandra Issue Type: Test Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 3.0 https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java#L85 {noformat} assert in.getBytesRead() totalSize; {noformat} My guess is that total size is the compressed size, not the uncompressed size. Remove the assertion and the test passes. Total size is calculated with {noformat} long size = 0; // calculate total length of transferring chunks for (CompressionMetadata.Chunk chunk : compressionInfo.chunks) size += chunk.length + 4; // 4 bytes for CRC return size; {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9327) Tombstones are not being cleared up
[ https://issues.apache.org/jira/browse/CASSANDRA-9327?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9327: --- Reproduced In: 2.0.14 Fix Version/s: (was: 2.0.14) 2.0.x Tombstones are not being cleared up --- Key: CASSANDRA-9327 URL: https://issues.apache.org/jira/browse/CASSANDRA-9327 Project: Cassandra Issue Type: Bug Reporter: srinivasu gottipati Priority: Critical Fix For: 2.0.x We deleted part of data from one of column families. After that we ran repair, followed by reducing gc_grace_seconds to have tombstones space to be reclaimed. After this, we kept the compaction enabled to reclaim this space upon passing gc_grace_seconds. We have done this while back and for sure gc_grace_seconds has elapsed. Currently, when we run queries against this column family, we are seeing lot of tombstone errors like below: WARN [ReadStage:1113] 2015-05-06 17:48:22,556 SliceQueryFilter.java (line 231) Read 19 live and 1140 tombstoned cells in YYY (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} The localDeletion reported above is time in future (year:2038, I believe it is the default Integer.MAX_VALUE), and I believe this could be the reason it is not being reclaimed. May I know, how do we purge this tombstones and also, what could have set this date to time in future? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9327) Tombstones are not being cleared up
srinivasu gottipati created CASSANDRA-9327: -- Summary: Tombstones are not being cleared up Key: CASSANDRA-9327 URL: https://issues.apache.org/jira/browse/CASSANDRA-9327 Project: Cassandra Issue Type: Bug Reporter: srinivasu gottipati Priority: Critical Fix For: 2.0.14 We deleted part of data from one of column families. After that we ran repair, followed by reducing gc_grace_seconds to have tombstones space to be reclaimed. After this, we kept the compaction enabled to reclaim this space upon passing gc_grace_seconds. We have done this while back and for sure gc_grace_seconds has elapsed. Currently, when we run queries against this column family, we are seeing lot of tombstone errors like below: WARN [ReadStage:1113] 2015-05-06 17:48:22,556 SliceQueryFilter.java (line 231) Read 19 live and 1140 tombstoned cells in YYY (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} The localDeletion reported above is time in future (year:2038, I believe it is the default Integer.MAX_VALUE), and I believe this could be the reason it is not being reclaimed. May I know, how do we purge this tombstones and also, what could have set this date to time in future? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/2] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/3cb00db3 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/3cb00db3 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/3cb00db3 Branch: refs/heads/trunk Commit: 3cb00db3f9f49e452963be784b5f764878291b2f Parents: 0ede9ec 09ccf60 Author: T Jake Luciani j...@apache.org Authored: Thu May 7 14:21:17 2015 -0400 Committer: T Jake Luciani j...@apache.org Committed: Thu May 7 14:21:17 2015 -0400 -- CHANGES.txt | 1 + .../org/apache/cassandra/tools/StandaloneSplitter.java | 8 .../org/apache/cassandra/tools/StandaloneUpgrader.java | 12 3 files changed, 17 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/3cb00db3/CHANGES.txt -- diff --cc CHANGES.txt index de09e39,098cfe0..5a8ee93 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,104 -1,5 +1,105 @@@ +3.0 + * Remove Thrift dependencies in bundled tools (CASSANDRA-8358) + * Disable memory mapping of hsperfdata file for JVM statistics (CASSANDRA-9242) + * Add pre-startup checks to detect potential incompatibilities (CASSANDRA-8049) + * Distinguish between null and unset in protocol v4 (CASSANDRA-7304) + * Add user/role permissions for user-defined functions (CASSANDRA-7557) + * Allow cassandra config to be updated to restart daemon without unloading classes (CASSANDRA-9046) + * Don't initialize compaction writer before checking if iter is empty (CASSANDRA-9117) + * Don't execute any functions at prepare-time (CASSANDRA-9037) + * Share file handles between all instances of a SegmentedFile (CASSANDRA-8893) + * Make it possible to major compact LCS (CASSANDRA-7272) + * Make FunctionExecutionException extend RequestExecutionException + (CASSANDRA-9055) + * Add support for SELECT JSON, INSERT JSON syntax and new toJson(), fromJson() + functions (CASSANDRA-7970) + * Optimise max purgeable timestamp calculation in compaction (CASSANDRA-8920) + * Constrain internode message buffer sizes, and improve IO class hierarchy (CASSANDRA-8670) + * New tool added to validate all sstables in a node (CASSANDRA-5791) + * Push notification when tracing completes for an operation (CASSANDRA-7807) + * Delay node up and node added notifications until native protocol server is started (CASSANDRA-8236) + * Compressed Commit Log (CASSANDRA-6809) + * Optimise IntervalTree (CASSANDRA-8988) + * Add a key-value payload for third party usage (CASSANDRA-8553, 9212) + * Bump metrics-reporter-config dependency for metrics 3.0 (CASSANDRA-8149) + * Partition intra-cluster message streams by size, not type (CASSANDRA-8789) + * Add WriteFailureException to native protocol, notify coordinator of + write failures (CASSANDRA-8592) + * Convert SequentialWriter to nio (CASSANDRA-8709) + * Add role based access control (CASSANDRA-7653, 8650, 7216, 8760, 8849, 8761, 8850) + * Record client ip address in tracing sessions (CASSANDRA-8162) + * Indicate partition key columns in response metadata for prepared + statements (CASSANDRA-7660) + * Merge UUIDType and TimeUUIDType parse logic (CASSANDRA-8759) + * Avoid memory allocation when searching index summary (CASSANDRA-8793) + * Optimise (Time)?UUIDType Comparisons (CASSANDRA-8730) + * Make CRC32Ex into a separate maven dependency (CASSANDRA-8836) + * Use preloaded jemalloc w/ Unsafe (CASSANDRA-8714) + * Avoid accessing partitioner through StorageProxy (CASSANDRA-8244, 8268) + * Upgrade Metrics library and remove depricated metrics (CASSANDRA-5657) + * Serializing Row cache alternative, fully off heap (CASSANDRA-7438) + * Duplicate rows returned when in clause has repeated values (CASSANDRA-6707) + * Make CassandraException unchecked, extend RuntimeException (CASSANDRA-8560) + * Support direct buffer decompression for reads (CASSANDRA-8464) + * DirectByteBuffer compatible LZ4 methods (CASSANDRA-7039) + * Group sstables for anticompaction correctly (CASSANDRA-8578) + * Add ReadFailureException to native protocol, respond + immediately when replicas encounter errors while handling + a read request (CASSANDRA-7886) + * Switch CommitLogSegment from RandomAccessFile to nio (CASSANDRA-8308) + * Allow mixing token and partition key restrictions (CASSANDRA-7016) + * Support index key/value entries on map collections (CASSANDRA-8473) + * Modernize schema tables (CASSANDRA-8261) + * Support for user-defined aggregation functions (CASSANDRA-8053) + * Fix NPE in SelectStatement with empty IN values (CASSANDRA-8419) + * Refactor SelectStatement, return IN results in natural order instead + of IN value list order and
[jira] [Commented] (CASSANDRA-9327) Tombstones are not being cleared up
[ https://issues.apache.org/jira/browse/CASSANDRA-9327?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533218#comment-14533218 ] srinivasu gottipati commented on CASSANDRA-9327: Sorry, I didn't realize, I pressed submission times. closing this and keeping the other one open. Tombstones are not being cleared up --- Key: CASSANDRA-9327 URL: https://issues.apache.org/jira/browse/CASSANDRA-9327 Project: Cassandra Issue Type: Bug Reporter: srinivasu gottipati Priority: Critical Fix For: 2.0.x We deleted part of data from one of column families. After that we ran repair, followed by reducing gc_grace_seconds to have tombstones space to be reclaimed. After this, we kept the compaction enabled to reclaim this space upon passing gc_grace_seconds. We have done this while back and for sure gc_grace_seconds has elapsed. Currently, when we run queries against this column family, we are seeing lot of tombstone errors like below: WARN [ReadStage:1113] 2015-05-06 17:48:22,556 SliceQueryFilter.java (line 231) Read 19 live and 1140 tombstoned cells in YYY (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} The localDeletion reported above is time in future (year:2038, I believe it is the default Integer.MAX_VALUE), and I believe this could be the reason it is not being reclaimed. May I know, how do we purge this tombstones and also, what could have set this date to time in future? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9322) Possible overlap with LCS and including non-compacting sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-9322?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533123#comment-14533123 ] Yuki Morishita commented on CASSANDRA-9322: --- LGTM, +1. Possible overlap with LCS and including non-compacting sstables --- Key: CASSANDRA-9322 URL: https://issues.apache.org/jira/browse/CASSANDRA-9322 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Assignee: Marcus Eriksson Fix For: 2.0.x Attachments: 0001-9322.patch since CASSANDRA-7414 we are including high-level sstables in lower level compactions if we have not run compactions in the high level for a while. If the compaction candidates only contain a single partition this can cause overlap since first token in sstables == last token in sstables which we interpret as being entire ring. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9327) Tombstones are not being cleared up
[ https://issues.apache.org/jira/browse/CASSANDRA-9327?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533185#comment-14533185 ] Philip Thompson commented on CASSANDRA-9327: How is this different from CASSANDRA-9326? Tombstones are not being cleared up --- Key: CASSANDRA-9327 URL: https://issues.apache.org/jira/browse/CASSANDRA-9327 Project: Cassandra Issue Type: Bug Reporter: srinivasu gottipati Priority: Critical Fix For: 2.0.x We deleted part of data from one of column families. After that we ran repair, followed by reducing gc_grace_seconds to have tombstones space to be reclaimed. After this, we kept the compaction enabled to reclaim this space upon passing gc_grace_seconds. We have done this while back and for sure gc_grace_seconds has elapsed. Currently, when we run queries against this column family, we are seeing lot of tombstone errors like below: WARN [ReadStage:1113] 2015-05-06 17:48:22,556 SliceQueryFilter.java (line 231) Read 19 live and 1140 tombstoned cells in YYY (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} The localDeletion reported above is time in future (year:2038, I believe it is the default Integer.MAX_VALUE), and I believe this could be the reason it is not being reclaimed. May I know, how do we purge this tombstones and also, what could have set this date to time in future? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9326) Seeing tombstone warning message
[ https://issues.apache.org/jira/browse/CASSANDRA-9326?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9326: --- Reproduced In: 2.0.14 Fix Version/s: (was: 2.0.14) 2.0.x Seeing tombstone warning message Key: CASSANDRA-9326 URL: https://issues.apache.org/jira/browse/CASSANDRA-9326 Project: Cassandra Issue Type: Bug Components: Core Reporter: srinivasu gottipati Priority: Critical Fix For: 2.0.x We deleted data for some of the rows in one of the column families. After that we ran repair on all nodes, and followed by reducing gc_grace_seconds that way compaction can remove all the tombstones upon expiry of gc_grace_seconds time. When we are querying the data now, seeing the following errors: WARN [ReadStage:1142] 2015-05-06 17:50:53,602 SliceQueryFilter.java (line 231) Read 1 live and 1487 tombstoned cells in (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} We deleted this data while back and for sure gc_grace_seconds is elapsed long time back and we use leveled compaction. In the above, errors, localDeletion points to some time in future (MAX INT VALUE) and that could be the reason these are n't being purged. Any help (or) workaround is appreciated. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-9327) Tombstones are not being cleared up
[ https://issues.apache.org/jira/browse/CASSANDRA-9327?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] srinivasu gottipati resolved CASSANDRA-9327. Resolution: Duplicate Tombstones are not being cleared up --- Key: CASSANDRA-9327 URL: https://issues.apache.org/jira/browse/CASSANDRA-9327 Project: Cassandra Issue Type: Bug Reporter: srinivasu gottipati Priority: Critical Fix For: 2.0.x We deleted part of data from one of column families. After that we ran repair, followed by reducing gc_grace_seconds to have tombstones space to be reclaimed. After this, we kept the compaction enabled to reclaim this space upon passing gc_grace_seconds. We have done this while back and for sure gc_grace_seconds has elapsed. Currently, when we run queries against this column family, we are seeing lot of tombstone errors like below: WARN [ReadStage:1113] 2015-05-06 17:48:22,556 SliceQueryFilter.java (line 231) Read 19 live and 1140 tombstoned cells in YYY (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} The localDeletion reported above is time in future (year:2038, I believe it is the default Integer.MAX_VALUE), and I believe this could be the reason it is not being reclaimed. May I know, how do we purge this tombstones and also, what could have set this date to time in future? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9326) Seeing tombstone warning message
[ https://issues.apache.org/jira/browse/CASSANDRA-9326?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533286#comment-14533286 ] Aleksey Yeschenko commented on CASSANDRA-9326: -- {{localDeletion=2147483647}} ({{Integer.MAX_VALUE}}) here only means that there is no partition level deletion here. The reasons you see that warning are that 1) compaction hasn't yet compacted them all away and 2) the way the column counter works. CASSANDRA-9326 is open to handle the latter, and you'll have to deal with the former. Seeing tombstone warning message Key: CASSANDRA-9326 URL: https://issues.apache.org/jira/browse/CASSANDRA-9326 Project: Cassandra Issue Type: Bug Components: Core Reporter: srinivasu gottipati Priority: Minor Fix For: 2.0.x We deleted data for some of the rows in one of the column families. After that we ran repair on all nodes, and followed by reducing gc_grace_seconds that way compaction can remove all the tombstones upon expiry of gc_grace_seconds time. When we are querying the data now, seeing the following errors: WARN [ReadStage:1142] 2015-05-06 17:50:53,602 SliceQueryFilter.java (line 231) Read 1 live and 1487 tombstoned cells in (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} We deleted this data while back and for sure gc_grace_seconds is elapsed long time back and we use leveled compaction. In the above, errors, localDeletion points to some time in future (MAX INT VALUE) and that could be the reason these are n't being purged. Any help (or) workaround is appreciated. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9276) StreamingTransferTest fails under test-compression due to bad assertion
[ https://issues.apache.org/jira/browse/CASSANDRA-9276?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533107#comment-14533107 ] T Jake Luciani commented on CASSANDRA-9276: --- +1 StreamingTransferTest fails under test-compression due to bad assertion --- Key: CASSANDRA-9276 URL: https://issues.apache.org/jira/browse/CASSANDRA-9276 Project: Cassandra Issue Type: Test Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 3.x https://github.com/apache/cassandra/blob/trunk/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java#L85 {noformat} assert in.getBytesRead() totalSize; {noformat} My guess is that total size is the compressed size, not the uncompressed size. Remove the assertion and the test passes. Total size is calculated with {noformat} long size = 0; // calculate total length of transferring chunks for (CompressionMetadata.Chunk chunk : compressionInfo.chunks) size += chunk.length + 4; // 4 bytes for CRC return size; {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9326) Seeing tombstone warning message
[ https://issues.apache.org/jira/browse/CASSANDRA-9326?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533308#comment-14533308 ] srinivasu gottipati commented on CASSANDRA-9326: We did run enableAutoCompaction and we are using LCS. Is it that, even though gc_grace_seconds is passed, still these are not reclaimed due to these deletes being in higher levels in LCS and compactions didn't happen at those levels? I see nodetool compact runs only for size tiered. Is there any way, to force compaction to run reclaim this space in LCS case? We are seeing shot up in our read latencies and any workaround would be greatly appreciated. Seeing tombstone warning message Key: CASSANDRA-9326 URL: https://issues.apache.org/jira/browse/CASSANDRA-9326 Project: Cassandra Issue Type: Bug Components: Core Reporter: srinivasu gottipati Priority: Minor Fix For: 2.0.x We deleted data for some of the rows in one of the column families. After that we ran repair on all nodes, and followed by reducing gc_grace_seconds that way compaction can remove all the tombstones upon expiry of gc_grace_seconds time. When we are querying the data now, seeing the following errors: WARN [ReadStage:1142] 2015-05-06 17:50:53,602 SliceQueryFilter.java (line 231) Read 1 live and 1487 tombstoned cells in (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} We deleted this data while back and for sure gc_grace_seconds is elapsed long time back and we use leveled compaction. In the above, errors, localDeletion points to some time in future (MAX INT VALUE) and that could be the reason these are n't being purged. Any help (or) workaround is appreciated. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9326) Seeing tombstone warning message
srinivasu gottipati created CASSANDRA-9326: -- Summary: Seeing tombstone warning message Key: CASSANDRA-9326 URL: https://issues.apache.org/jira/browse/CASSANDRA-9326 Project: Cassandra Issue Type: Bug Components: Core Reporter: srinivasu gottipati Priority: Critical Fix For: 2.0.14 We deleted data for some of the rows in one of the column families. After that we ran repair on all nodes, and followed by reducing gc_grace_seconds that way compaction can remove all the tombstones upon expiry of gc_grace_seconds time. When we are querying the data now, seeing the following errors: WARN [ReadStage:1142] 2015-05-06 17:50:53,602 SliceQueryFilter.java (line 231) Read 1 live and 1487 tombstoned cells in (see tombstone_warn_threshold). 10001 columns was requested, slices=[-], delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} We deleted this data while back and for sure gc_grace_seconds is elapsed long time back and we use leveled compaction. In the above, errors, localDeletion points to some time in future (MAX INT VALUE) and that could be the reason these are n't being purged. Any help (or) workaround is appreciated. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9329) Make CAS retry logic configurable
[ https://issues.apache.org/jira/browse/CASSANDRA-9329?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9329: --- Fix Version/s: 3.x Make CAS retry logic configurable - Key: CASSANDRA-9329 URL: https://issues.apache.org/jira/browse/CASSANDRA-9329 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Aaron Whiteside Fix For: 3.x Make CAS retry logic configurable: One should be able to disable the internal CAS retry loop (when the condition is not met) and let the client choose how to do retries (so the client does not have to incur the server side random sleep of up to 100ms). Basically let the client handle all CAS retries in a manor it sees fit. Secondly the hardcoded sleep up to 100ms that happens when cassandra fails to meet the CAS condition should be configurable. - The max duration should be configurable - The algorithm used to choose the duration should be configurable (Random, Exponential, etc). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-9318: -- Fix Version/s: (was: 3.x) 2.1.x Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533657#comment-14533657 ] Benedict commented on CASSANDRA-9318: - CL=1 Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533679#comment-14533679 ] Jonathan Ellis commented on CASSANDRA-9318: --- We already keep the original request around until we either get acks from all replicas, or they time out (and we write a hint). Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8897) Remove FileCacheService, instead pooling the buffers
[ https://issues.apache.org/jira/browse/CASSANDRA-8897?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533732#comment-14533732 ] Stefania commented on CASSANDRA-8897: - Since CompressedPoolingSegmentedFile is gone we will need a sure way to retest CASSANDRA-9240. Remove FileCacheService, instead pooling the buffers Key: CASSANDRA-8897 URL: https://issues.apache.org/jira/browse/CASSANDRA-8897 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Stefania Fix For: 3.x After CASSANDRA-8893, a RAR will be a very lightweight object and will not need caching, so we can eliminate this cache entirely. Instead we should have a pool of buffers that are page-aligned. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9329) Make CAS retry logic configurable
[ https://issues.apache.org/jira/browse/CASSANDRA-9329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533614#comment-14533614 ] Aaron Whiteside commented on CASSANDRA-9329: Can we at least get something into the next 2.x release? Perhaps if cas_contention_timeout_in_ms is set to 0, then we never sleep and only try the condition once? Make CAS retry logic configurable - Key: CASSANDRA-9329 URL: https://issues.apache.org/jira/browse/CASSANDRA-9329 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Aaron Whiteside Fix For: 3.x Make CAS retry logic configurable: One should be able to disable the internal CAS retry loop (when the condition is not met) and let the client choose how to do retries (so the client does not have to incur the server side random sleep of up to 100ms). Basically let the client handle all CAS retries in a manor it sees fit. Secondly the hardcoded sleep up to 100ms that happens when cassandra fails to meet the CAS condition should be configurable. - The max duration should be configurable - The algorithm used to choose the duration should be configurable (Random, Exponential, etc). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9329) Make CAS retry logic configurable
[ https://issues.apache.org/jira/browse/CASSANDRA-9329?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533614#comment-14533614 ] Aaron Whiteside edited comment on CASSANDRA-9329 at 5/7/15 11:43 PM: - Can we at least get something into the next 2.x release? Perhaps if cas_contention_timeout_in_ms is set to 0, then we never sleep and only try the condition once? The configurable alg and timeout can come later. was (Author: aaronjwhiteside): Can we at least get something into the next 2.x release? Perhaps if cas_contention_timeout_in_ms is set to 0, then we never sleep and only try the condition once? Make CAS retry logic configurable - Key: CASSANDRA-9329 URL: https://issues.apache.org/jira/browse/CASSANDRA-9329 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Aaron Whiteside Fix For: 3.x Make CAS retry logic configurable: One should be able to disable the internal CAS retry loop (when the condition is not met) and let the client choose how to do retries (so the client does not have to incur the server side random sleep of up to 100ms). Basically let the client handle all CAS retries in a manor it sees fit. Secondly the hardcoded sleep up to 100ms that happens when cassandra fails to meet the CAS condition should be configurable. - The max duration should be configurable - The algorithm used to choose the duration should be configurable (Random, Exponential, etc). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533654#comment-14533654 ] Jonathan Ellis commented on CASSANDRA-9318: --- I'm having trouble understanding how 1MB of concurrent writes will make a cluster fall over. Where are we bounding that now? Perhaps the bounding code is buggy. Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Fixes ReadMessageTest.testNoCommitLog for test-compression
Repository: cassandra Updated Branches: refs/heads/trunk 4047dd121 - 36958f3ca Fixes ReadMessageTest.testNoCommitLog for test-compression Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/36958f3c Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/36958f3c Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/36958f3c Branch: refs/heads/trunk Commit: 36958f3cab74c9b7baca94b5e7d31de0cbd42d40 Parents: 4047dd1 Author: Branimir Lambov branimir.lam...@datastax.com Authored: Thu May 7 17:27:18 2015 -0700 Committer: Jonathan Ellis jbel...@apache.org Committed: Thu May 7 17:27:18 2015 -0700 -- .../apache/cassandra/db/ReadMessageTest.java| 84 .../db/commitlog/CommitLogTestReplayer.java | 79 ++ 2 files changed, 95 insertions(+), 68 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/36958f3c/test/unit/org/apache/cassandra/db/ReadMessageTest.java -- diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java index d32df49..34f25a1 100644 --- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java +++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java @@ -25,13 +25,14 @@ import java.nio.ByteBuffer; import java.util.SortedSet; import java.util.TreeSet; +import com.google.common.base.Predicate; import org.junit.BeforeClass; import org.junit.Test; import org.apache.cassandra.SchemaLoader; import org.apache.cassandra.Util; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.config.KSMetaData; +import org.apache.cassandra.db.commitlog.CommitLogTestReplayer; import org.apache.cassandra.db.composites.*; import org.apache.cassandra.db.filter.NamesQueryFilter; import org.apache.cassandra.db.filter.SliceQueryFilter; @@ -132,81 +133,28 @@ public class ReadMessageTest rm.add(Standard1, Util.cellname(commit2), ByteBufferUtil.bytes(abcd), 0); rm.apply(); -boolean commitLogMessageFound = false; -boolean noCommitLogMessageFound = false; - -File commitLogDir = new File(DatabaseDescriptor.getCommitLogLocation()); - -byte[] commitBytes = commit.getBytes(UTF-8); - -for(String filename : commitLogDir.list()) -{ -BufferedInputStream is = null; -try -{ -is = new BufferedInputStream(new FileInputStream(commitLogDir.getAbsolutePath()+File.separator+filename)); - -if (!isEmptyCommitLog(is)) -{ -while (findPatternInStream(commitBytes, is)) -{ -char c = (char)is.read(); - -if (c == '1') -commitLogMessageFound = true; -else if (c == '2') -noCommitLogMessageFound = true; -} -} -} -finally -{ -if (is != null) -is.close(); -} -} +Checker checker = new Checker(); +CommitLogTestReplayer.examineCommitLog(checker); -assertTrue(commitLogMessageFound); -assertFalse(noCommitLogMessageFound); +assertTrue(checker.commitLogMessageFound); +assertFalse(checker.noCommitLogMessageFound); } -private boolean isEmptyCommitLog(BufferedInputStream is) throws IOException +static class Checker implements PredicateMutation { -DataInputStream dis = new DataInputStream(is); -byte[] lookahead = new byte[100]; - -dis.mark(100); -dis.readFully(lookahead); -dis.reset(); - -for (int i = 0; i 100; i++) -{ -if (lookahead[i] != 0) -return false; -} - -return true; -} - -private boolean findPatternInStream(byte[] pattern, InputStream is) throws IOException -{ -int patternOffset = 0; +boolean commitLogMessageFound = false; +boolean noCommitLogMessageFound = false; -int b = is.read(); -while (b != -1) +public boolean apply(Mutation mutation) { -if (pattern[patternOffset] == ((byte) b)) +for (ColumnFamily cf : mutation.getColumnFamilies()) { -patternOffset++; -if (patternOffset == pattern.length) -return true; +if (cf.getColumn(Util.cellname(commit1)) != null) +commitLogMessageFound = true; +if (cf.getColumn(Util.cellname(commit2)) != null) +
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533662#comment-14533662 ] Ariel Weisberg commented on CASSANDRA-9318: --- Ah, I get it. So this make sense now and I think it's fixable in the context of bounding at the coordinator. CL=1 means we have to send the response back to the client, it doesn't mean we have to release the permit associated with in-flight writes. We could come up with a suitable definition of allowed concurrency for requests that continue to execute even after the response has gone back to the client. How this compares with other approaches to bounding utilization in terms of complexity and efficacy is still up for discussion. Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533665#comment-14533665 ] Benedict commented on CASSANDRA-9318: - Um... that sounds a lot like losing the A in CAP, leaving us too few letters Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9240) Performance issue after a restart
[ https://issues.apache.org/jira/browse/CASSANDRA-9240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533746#comment-14533746 ] Alan Boudreault commented on CASSANDRA-9240: [~Stefania] Use a low heap cassandra (ccm default in example: -Xms500M -Xmx500M -Xmn50M) and do some write, restart cassandra and do some read like: {code} ./tools/bin/cassandra-stress user profile=/home/jake/Downloads/issue.yaml ops\(insert=1\) n=200 -rate threads=50 ./tools/bin/cassandra-stress user profile=/home/jake/Downloads/issue.yaml ops\(read=1\) n=5 -rate threads=50 #restart cassandra ./tools/bin/cassandra-stress user profile=/home/jake/Downloads/issue.yaml ops\(read=1\) n=50 -rate threads=50 {code} Check my [^run_issue.sh] script to see the complete test. Performance issue after a restart - Key: CASSANDRA-9240 URL: https://issues.apache.org/jira/browse/CASSANDRA-9240 Project: Cassandra Issue Type: Bug Reporter: Alan Boudreault Assignee: Benedict Priority: Minor Fix For: 3.x Attachments: Cassandra.snapshots.zip, cassandra_2.1.4-clientrequest-read.log, cassandra_2.1.4.log, cassandra_2.1.5-clientrequest-read.log, cassandra_2.1.5.log, cassandra_trunk-clientrequest-read.log, cassandra_trunk.log, cassandra_trunk_no_restart-clientrequest-read.log, cassandra_trunk_no_restart.log, issue.yaml, run_issue.sh, runs.log, trace_query.cql I have noticed a performance issue while I was working on compaction perf tests for CASSANDRA-7409. The performance for my use case is very bad after a restart. It is mostly a read performance issue but not strictly. I have attached my use case (see run_issue.sh and issue.yaml) and all test logs for 2.1.4, 2.1.5 and trunk: * 2.1.* are OK (although 2.1.4 seems to be better than 2.1.5?): ~6-7k ops/second and ~2-2.5k of read latency. * trunk is NOT OK: ~1.5-2k ops/second and 25-30k of read latency. * trunk is OK without a restart: ~ same perf than 2.1.4 and 2.1.5. EDIT: branch cassandra-2.1 is OK. I can help to bisect and/or profile on Monday if needed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533657#comment-14533657 ] Benedict edited comment on CASSANDRA-9318 at 5/8/15 12:23 AM: -- CL=1 (or any CL ALL) was (Author: benedict): CL=1 Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9277) SSTableRewriterTest.testFileRemoval fails with test-compression
[ https://issues.apache.org/jira/browse/CASSANDRA-9277?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis updated CASSANDRA-9277: -- Reviewer: Joshua McKenzie [~JoshuaMcKenzie] to review SSTableRewriterTest.testFileRemoval fails with test-compression --- Key: CASSANDRA-9277 URL: https://issues.apache.org/jira/browse/CASSANDRA-9277 Project: Cassandra Issue Type: Test Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 3.x openEarly returns null with compression because not enough data has been written to trigger a flush. Solution is to write more data in the test case so it flushes. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-6702) Upgrading node uses the wrong port in gossiping
[ https://issues.apache.org/jira/browse/CASSANDRA-6702?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533689#comment-14533689 ] Ariel Weisberg commented on CASSANDRA-6702: --- I think I understand... +1. Upgrading node uses the wrong port in gossiping --- Key: CASSANDRA-6702 URL: https://issues.apache.org/jira/browse/CASSANDRA-6702 Project: Cassandra Issue Type: Bug Components: Core Environment: 1.1.7, AWS, Ec2MultiRegionSnitch Reporter: Minh Do Assignee: Blake Eggleston Priority: Minor Fix For: 2.0.x Attachments: C6702-1.2.txt, C6702-2.0.txt When upgrading a node in 1.1.7 (or 1.1.11) cluster to 1.2.15 and inspecting the gossip information on port/Ip, I could see that the upgrading node (1.2 version) communicates to one other node in the same region using Public IP and non-encrypted port. For the rest, the upgrading node uses the correct ports and IPs to communicate in this manner: Same region: private IP and non-encrypted port and Different region: public IP and encrypted port Because there is one node like this (or 2 out of 12 nodes cluster in which nodes are split equally on 2 AWS regions), we have to modify Security Group to allow the new traffics. Without modifying the SG, the 95th and 99th latencies for both reads and writes in the cluster are very bad (due to RPC timeout). Inspecting closer, that upgraded node (1.2 node) is contributing to all of the high latencies whenever it acts as a coordinator node. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533655#comment-14533655 ] Ariel Weisberg commented on CASSANDRA-9318: --- Benedict can you elaborate how 1 megabyte worth of mutations outstanding per node is blown up into enough load or memory to bring down nodes? Is it that the multiplicative factor is high, it's all routing to one place, or is it that the mutation stage acknowledges writes as completed when the work hasn't been done and resources are still being tied up? Maybe we have different definitions of in-flight. In my mind in-flight means that until the response is sent back to the client the request counts against the in-flight limit. Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533676#comment-14533676 ] Ariel Weisberg commented on CASSANDRA-9318: --- Can you get more specific? Is it that when things start failing you don't want to wait for the timeouts to continue coordinating requests? Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 2.1.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9240) Performance issue after a restart
[ https://issues.apache.org/jira/browse/CASSANDRA-9240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533736#comment-14533736 ] Stefania commented on CASSANDRA-9240: - [~tjake], [~benedict], [~aboudreault], this fix conflicted with CASSANDRA-8897 (no more CompressedPoolingSegmentedFile in favor or a buffer pool). Could you sum-up the easiest way to reproduce this issue so I can do some tests on the 8897 branch? Performance issue after a restart - Key: CASSANDRA-9240 URL: https://issues.apache.org/jira/browse/CASSANDRA-9240 Project: Cassandra Issue Type: Bug Reporter: Alan Boudreault Assignee: Benedict Priority: Minor Fix For: 3.x Attachments: Cassandra.snapshots.zip, cassandra_2.1.4-clientrequest-read.log, cassandra_2.1.4.log, cassandra_2.1.5-clientrequest-read.log, cassandra_2.1.5.log, cassandra_trunk-clientrequest-read.log, cassandra_trunk.log, cassandra_trunk_no_restart-clientrequest-read.log, cassandra_trunk_no_restart.log, issue.yaml, run_issue.sh, runs.log, trace_query.cql I have noticed a performance issue while I was working on compaction perf tests for CASSANDRA-7409. The performance for my use case is very bad after a restart. It is mostly a read performance issue but not strictly. I have attached my use case (see run_issue.sh and issue.yaml) and all test logs for 2.1.4, 2.1.5 and trunk: * 2.1.* are OK (although 2.1.4 seems to be better than 2.1.5?): ~6-7k ops/second and ~2-2.5k of read latency. * trunk is NOT OK: ~1.5-2k ops/second and 25-30k of read latency. * trunk is OK without a restart: ~ same perf than 2.1.4 and 2.1.5. EDIT: branch cassandra-2.1 is OK. I can help to bisect and/or profile on Monday if needed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9328) WriteTimeoutException thrown when LWT concurrency 1, despite the query duration taking MUCH less than cas_contention_timeout_in_ms
[ https://issues.apache.org/jira/browse/CASSANDRA-9328?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9328: --- Reproduced In: 2.1.5 Fix Version/s: 2.1.x Assignee: Benjamin Lerer WriteTimeoutException thrown when LWT concurrency 1, despite the query duration taking MUCH less than cas_contention_timeout_in_ms Key: CASSANDRA-9328 URL: https://issues.apache.org/jira/browse/CASSANDRA-9328 Project: Cassandra Issue Type: Bug Components: Core Reporter: Aaron Whiteside Assignee: Benjamin Lerer Priority: Critical Fix For: 2.1.x Attachments: CassandraLWTTest.java WriteTimeoutException thrown when LWT concurrency 1, despite the query duration taking MUCH less than cas_contention_timeout_in_ms. Unit test attached, run against a 3 node cluster running 2.1.5. If you reduce the threadCount to 1, you never see a WriteTimeoutException. If the WTE is due to not being able to communicate with other nodes, why does the concurrency 1 cause inter-node communication to fail? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9275) ReadMessageTest.testNoCommitLog fails with test-compression
[ https://issues.apache.org/jira/browse/CASSANDRA-9275?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Branimir Lambov updated CASSANDRA-9275: --- Attachment: 9275-readmessagetest.txt Sorry, this was meant to be fixed before I ask you to take another look. The helper is now in a separate utility class and I've reverted the irrelevant changes. [The branch|https://github.com/apache/cassandra/compare/trunk...blambov:9275-readmessagetest] is updated, patch is attached. Test results [here|http://cassci.datastax.com/view/Dev/view/blambov/]. ReadMessageTest.testNoCommitLog fails with test-compression --- Key: CASSANDRA-9275 URL: https://issues.apache.org/jira/browse/CASSANDRA-9275 Project: Cassandra Issue Type: Test Reporter: Ariel Weisberg Assignee: Branimir Lambov Fix For: 3.x Attachments: 9275-readmessagetest.txt It's reading the raw bytes of the commit log expecting to find an uncompressed log and instead finding a compressed log and now the test is not working if the log is compressed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9322) Possible overlap with LCS and including non-compacting sstables
Marcus Eriksson created CASSANDRA-9322: -- Summary: Possible overlap with LCS and including non-compacting sstables Key: CASSANDRA-9322 URL: https://issues.apache.org/jira/browse/CASSANDRA-9322 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Fix For: 2.0.x since CASSANDRA-7414 we are including high-level sstables in lower level compactions if we have not run compactions in the high level for a while. If the compaction candidates only contain a single partition this can cause overlap since first token in sstables == last token in sstables which we interpret as being entire ring. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8723) Cassandra 2.1.2 Memory issue - java process memory usage continuously increases until process is killed by OOM killer
[ https://issues.apache.org/jira/browse/CASSANDRA-8723?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532384#comment-14532384 ] Benedict commented on CASSANDRA-8723: - I'll try and arrange somewhere for you. Cassandra 2.1.2 Memory issue - java process memory usage continuously increases until process is killed by OOM killer - Key: CASSANDRA-8723 URL: https://issues.apache.org/jira/browse/CASSANDRA-8723 Project: Cassandra Issue Type: Bug Reporter: Jeff Liu Fix For: 2.1.x Attachments: cassandra.yaml Issue: We have an on-going issue with cassandra nodes running with continuously increasing memory until killed by OOM. {noformat} Jan 29 10:15:41 cass-chisel19 kernel: [24533109.783481] Out of memory: Kill process 13919 (java) score 911 or sacrifice child Jan 29 10:15:41 cass-chisel19 kernel: [24533109.783557] Killed process 13919 (java) total-vm:18366340kB, anon-rss:6461472kB, file-rss:6684kB {noformat} System Profile: cassandra version 2.1.2 system: aws c1.xlarge instance with 8 cores, 7.1G memory. cassandra jvm: -Xms1792M -Xmx1792M -Xmn400M -Xss256k {noformat} java -ea -javaagent:/usr/share/cassandra/lib/jamm-0.2.8.jar -XX:+UseThreadPriorities -XX:ThreadPriorityPolicy=42 -Xms1792M -Xmx1792M -Xmn400M -XX:+HeapDumpOnOutOfMemoryError -Xss256k -XX:StringTableSize=103 -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+CMSParallelRemarkEnabled -XX:SurvivorRatio=8 -XX:MaxTenuringThreshold=1 -XX:CMSInitiatingOccupancyFraction=75 -XX:+UseCMSInitiatingOccupancyOnly -XX:+UseTLAB -XX:+CMSClassUnloadingEnabled -XX:+UseCondCardMark -XX:+PrintGCDetails -XX:+PrintGCDateStamps -XX:+PrintHeapAtGC -XX:+PrintTenuringDistribution -XX:+PrintGCApplicationStoppedTime -XX:+PrintPromotionFailure -Xloggc:/var/log/cassandra/gc-1421511249.log -XX:+UseGCLogFileRotation -XX:NumberOfGCLogFiles=5 -XX:GCLogFileSize=48M -Djava.net.preferIPv4Stack=true -Dcom.sun.management.jmxremote.port=7199 -Dcom.sun.management.jmxremote.ssl=false -Dcom.sun.management.jmxremote.authenticate=false -javaagent:/usr/share/java/graphite-reporter-agent-1.0-SNAPSHOT.jar=graphiteServer=metrics-a.hq.nest.com;graphitePort=2003;graphitePollInt=60 -Dlogback.configurationFile=logback.xml -Dcassandra.logdir=/var/log/cassandra -Dcassandra.storagedir= -Dcassandra-pidfile=/var/run/cassandra/cassandra.pid -cp /etc/cassandra:/usr/share/cassandra/lib/airline-0.6.jar:/usr/share/cassandra/lib/antlr-runtime-3.5.2.jar:/usr/share/cassandra/lib/commons-cli-1.1.jar:/usr/share/cassandra/lib/commons-codec-1.2.jar:/usr/share/cassandra/lib/commons-lang3-3.1.jar:/usr/share/cassandra/lib/commons-math3-3.2.jar:/usr/share/cassandra/lib/compress-lzf-0.8.4.jar:/usr/share/cassandra/lib/concurrentlinkedhashmap-lru-1.4.jar:/usr/share/cassandra/lib/disruptor-3.0.1.jar:/usr/share/cassandra/lib/guava-16.0.jar:/usr/share/cassandra/lib/high-scale-lib-1.0.6.jar:/usr/share/cassandra/lib/jackson-core-asl-1.9.2.jar:/usr/share/cassandra/lib/jackson-mapper-asl-1.9.2.jar:/usr/share/cassandra/lib/jamm-0.2.8.jar:/usr/share/cassandra/lib/javax.inject.jar:/usr/share/cassandra/lib/jbcrypt-0.3m.jar:/usr/share/cassandra/lib/jline-1.0.jar:/usr/share/cassandra/lib/jna-4.0.0.jar:/usr/share/cassandra/lib/json-simple-1.1.jar:/usr/share/cassandra/lib/libthrift-0.9.1.jar:/usr/share/cassandra/lib/logback-classic-1.1.2.jar:/usr/share/cassandra/lib/logback-core-1.1.2.jar:/usr/share/cassandra/lib/lz4-1.2.0.jar:/usr/share/cassandra/lib/metrics-core-2.2.0.jar:/usr/share/cassandra/lib/metrics-graphite-2.2.0.jar:/usr/share/cassandra/lib/mx4j-tools.jar:/usr/share/cassandra/lib/netty-all-4.0.23.Final.jar:/usr/share/cassandra/lib/reporter-config-2.1.0.jar:/usr/share/cassandra/lib/slf4j-api-1.7.2.jar:/usr/share/cassandra/lib/snakeyaml-1.11.jar:/usr/share/cassandra/lib/snappy-java-1.0.5.2.jar:/usr/share/cassandra/lib/stream-2.5.2.jar:/usr/share/cassandra/lib/stringtemplate-4.0.2.jar:/usr/share/cassandra/lib/super-csv-2.1.0.jar:/usr/share/cassandra/lib/thrift-server-0.3.7.jar:/usr/share/cassandra/apache-cassandra-2.1.2.jar:/usr/share/cassandra/apache-cassandra-thrift-2.1.2.jar:/usr/share/cassandra/apache-cassandra.jar:/usr/share/cassandra/cassandra-driver-core-2.0.5.jar:/usr/share/cassandra/netty-3.9.0.Final.jar:/usr/share/cassandra/stress.jar: -XX:HeapDumpPath=/var/lib/cassandra/java_1421511248.hprof -XX:ErrorFile=/var/lib/cassandra/hs_err_1421511248.log org.apache.cassandra.service.CassandraDaemon {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/3] cassandra git commit: Improve sstable exclusion from partition tombstones
Improve sstable exclusion from partition tombstones patch by benedict; reviewed by aleksey for CASSANDRA-9298 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/b14236f9 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/b14236f9 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/b14236f9 Branch: refs/heads/trunk Commit: b14236f946337cab5f78407253bd296a6f3fbb22 Parents: ce3ce44 Author: Benedict Elliott Smith bened...@apache.org Authored: Thu May 7 11:24:50 2015 +0100 Committer: Benedict Elliott Smith bened...@apache.org Committed: Thu May 7 11:24:50 2015 +0100 -- CHANGES.txt | 1 + .../org/apache/cassandra/db/CollationController.java| 12 +++- 2 files changed, 4 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/b14236f9/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index 09bdfb1..6363974 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 2.1.6 + * Improve sstable exclusion from partition tombstones (CASSANDRA-9298) * Validate the indexed column rather than the cell's contents for 2i (CASSANDRA-9057) * Add support for top-k custom 2i queries (CASSANDRA-8717) * Fix error when dropping table during compaction (CASSANDRA-9251) http://git-wip-us.apache.org/repos/asf/cassandra/blob/b14236f9/src/java/org/apache/cassandra/db/CollationController.java -- diff --git a/src/java/org/apache/cassandra/db/CollationController.java b/src/java/org/apache/cassandra/db/CollationController.java index 4efcd9c..5c6a3db 100644 --- a/src/java/org/apache/cassandra/db/CollationController.java +++ b/src/java/org/apache/cassandra/db/CollationController.java @@ -77,11 +77,11 @@ public class CollationController boolean isEmpty = true; Tracing.trace(Acquiring sstable references); ColumnFamilyStore.ViewFragment view = cfs.select(cfs.viewFilter(filter.key)); +DeletionInfo returnDeletionInfo = container.deletionInfo(); try { Tracing.trace(Merging memtable contents); -long mostRecentRowTombstone = Long.MIN_VALUE; for (Memtable memtable : view.memtables) { ColumnFamily cf = memtable.getColumnFamily(filter.key); @@ -98,7 +98,6 @@ public class CollationController container.addColumn(cell); } } -mostRecentRowTombstone = container.deletionInfo().getTopLevelDeletion().markedForDeleteAt; } // avoid changing the filter columns of the original filter @@ -116,7 +115,7 @@ public class CollationController // if we've already seen a row tombstone with a timestamp greater // than the most recent update to this sstable, we're done, since the rest of the sstables // will also be older -if (sstable.getMaxTimestamp() mostRecentRowTombstone) +if (sstable.getMaxTimestamp() returnDeletionInfo.getTopLevelDeletion().markedForDeleteAt) break; long currentMaxTs = sstable.getMaxTimestamp(); @@ -136,7 +135,6 @@ public class CollationController while (iter.hasNext()) container.addAtom(iter.next()); } -mostRecentRowTombstone = container.deletionInfo().getTopLevelDeletion().markedForDeleteAt; } // we need to distinguish between there is no data at all for this row (BF will let us rebuild that efficiently) @@ -244,7 +242,6 @@ public class CollationController */ Collections.sort(view.sstables, SSTableReader.maxTimestampComparator); ListSSTableReader skippedSSTables = null; -long mostRecentRowTombstone = Long.MIN_VALUE; long minTimestamp = Long.MAX_VALUE; int nonIntersectingSSTables = 0; @@ -253,7 +250,7 @@ public class CollationController minTimestamp = Math.min(minTimestamp, sstable.getMinTimestamp()); // if we've already seen a row tombstone with a timestamp greater // than the most recent update to this sstable, we can skip it -if (sstable.getMaxTimestamp() mostRecentRowTombstone) +if (sstable.getMaxTimestamp() returnDeletionInfo.getTopLevelDeletion().markedForDeleteAt) break; if (!filter.shouldInclude(sstable)) @@ -275,9 +272,6 @@ public class CollationController if (iter.getColumnFamily() != null)
[5/6] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Conflicts: CHANGES.txt src/java/org/apache/cassandra/db/index/SecondaryIndex.java test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/8c5ce66f Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/8c5ce66f Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/8c5ce66f Branch: refs/heads/cassandra-2.1 Commit: 8c5ce66fe32c8bfdc7712b70164948e82592f91e Parents: b14236f9 6ab0c30 Author: Benedict Elliott Smith bened...@apache.org Authored: Thu May 7 11:57:52 2015 +0100 Committer: Benedict Elliott Smith bened...@apache.org Committed: Thu May 7 11:57:52 2015 +0100 -- CHANGES.txt | 2 ++ .../cassandra/db/index/SecondaryIndex.java | 13 .../db/index/SecondaryIndexManager.java | 4 ++-- .../cassandra/db/ColumnFamilyStoreTest.java | 21 +--- .../db/index/PerRowSecondaryIndexTest.java | 20 +-- 5 files changed, 40 insertions(+), 20 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5ce66f/CHANGES.txt -- diff --cc CHANGES.txt index 6363974,d76606c..41e0340 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,20 -1,5 +1,22 @@@ +2.1.6 + * Improve sstable exclusion from partition tombstones (CASSANDRA-9298) + * Validate the indexed column rather than the cell's contents for 2i (CASSANDRA-9057) + * Add support for top-k custom 2i queries (CASSANDRA-8717) + * Fix error when dropping table during compaction (CASSANDRA-9251) + * cassandra-stress supports validation operations over user profiles (CASSANDRA-8773) + * Add support for rate limiting log messages (CASSANDRA-9029) + * Log the partition key with tombstone warnings (CASSANDRA-8561) + * Reduce runWithCompactionsDisabled poll interval to 1ms (CASSANDRA-9271) + * Fix PITR commitlog replay (CASSANDRA-9195) + * GCInspector logs very different times (CASSANDRA-9124) + * Fix deleting from an empty list (CASSANDRA-9198) + * Update tuple and collection types that use a user-defined type when that UDT + is modified (CASSANDRA-9148, CASSANDRA-9192) + * Use higher timeout for prepair and snapshot in repair (CASSANDRA-9261) + * Fix anticompaction blocking ANTI_ENTROPY stage (CASSANDRA-9151) +Merged from 2.0: + 2.0.15: + * Overload SecondaryIndex#indexes to accept the column definition (CASSANDRA-9314) * (cqlsh) Add SERIAL and LOCAL_SERIAL consistency levels (CASSANDRA-8051) * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281) * Fix partition-level-delete-only workload accounting (CASSANDRA-9194) http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5ce66f/src/java/org/apache/cassandra/db/index/SecondaryIndex.java -- diff --cc src/java/org/apache/cassandra/db/index/SecondaryIndex.java index 20fdcec,789cc29..2dc0a22 --- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java +++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java @@@ -304,11 -290,32 +304,24 @@@ public abstract class SecondaryInde } /** - * Returns true if the provided column name is indexed by this secondary index. + * Returns true if the provided cell name is indexed by this secondary index. + * + * The default implementation checks whether the name is one the columnDef name, + * but this should be overriden but subclass if needed. */ -public boolean indexes(ByteBuffer name) -{ -for (ColumnDefinition columnDef : columnDefs) -{ -if (baseCfs.getComparator().compare(columnDef.name, name) == 0) -return true; -} -return false; -} - +public abstract boolean indexes(CellName name); + /** + * Returns true if the provided column definition is indexed by this secondary index. + * + * The default implementation checks whether it is contained in this index column definitions set. + */ + public boolean indexes(ColumnDefinition cdef) + { + return columnDefs.contains(cdef); + } + + /** * This is the primary way to create a secondary index instance for a CF column. * It will validate the index_options before initializing. * http://git-wip-us.apache.org/repos/asf/cassandra/blob/8c5ce66f/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java --
cassandra git commit: Fix regression with compressed reader performance due to no pooling and excessive mapping/unmapping
Repository: cassandra Updated Branches: refs/heads/trunk ea2ee3703 - aedce5fc6 Fix regression with compressed reader performance due to no pooling and excessive mapping/unmapping patch by benedict; reviewed by tjake for CASSANDRA-9240 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/aedce5fc Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/aedce5fc Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/aedce5fc Branch: refs/heads/trunk Commit: aedce5fc6ba46ca734e91190cfaaeb23ba47a846 Parents: ea2ee37 Author: Benedict Elliott Smith bened...@apache.org Authored: Thu May 7 11:31:08 2015 +0100 Committer: Benedict Elliott Smith bened...@apache.org Committed: Thu May 7 11:31:08 2015 +0100 -- .../compress/CompressedRandomAccessReader.java | 94 ++-- .../io/compress/CompressedThrottledReader.java | 9 +- .../io/compress/DeflateCompressor.java | 7 +- .../cassandra/io/compress/LZ4Compressor.java| 6 +- .../cassandra/io/compress/SnappyCompressor.java | 2 + .../io/util/CompressedPoolingSegmentedFile.java | 39 ++-- .../io/util/CompressedSegmentedFile.java| 71 ++- .../cassandra/io/util/ICompressedFile.java | 5 ++ .../cassandra/io/util/RandomAccessReader.java | 4 +- .../apache/cassandra/io/util/SegmentedFile.java | 2 +- 10 files changed, 152 insertions(+), 87 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/aedce5fc/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java -- diff --git a/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java b/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java index edf8c68..1febe37 100644 --- a/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java +++ b/src/java/org/apache/cassandra/io/compress/CompressedRandomAccessReader.java @@ -42,17 +42,23 @@ import org.apache.cassandra.utils.FBUtilities; */ public class CompressedRandomAccessReader extends RandomAccessReader { -private static final boolean useMmap = DatabaseDescriptor.getDiskAccessMode() == Config.DiskAccessMode.mmap; - public static CompressedRandomAccessReader open(ChannelProxy channel, CompressionMetadata metadata) { -return open(channel, metadata, null); +try +{ +return new CompressedRandomAccessReader(channel, metadata, null); +} +catch (FileNotFoundException e) +{ +throw new RuntimeException(e); +} } -public static CompressedRandomAccessReader open(ChannelProxy channel, CompressionMetadata metadata, CompressedPoolingSegmentedFile owner) + +public static CompressedRandomAccessReader open(ICompressedFile file) { try { -return new CompressedRandomAccessReader(channel, metadata, owner); +return new CompressedRandomAccessReader(file.channel(), file.getMetadata(), file); } catch (FileNotFoundException e) { @@ -60,9 +66,7 @@ public class CompressedRandomAccessReader extends RandomAccessReader } } - -private TreeMapLong, MappedByteBuffer chunkSegments; -private int MAX_SEGMENT_SIZE = Integer.MAX_VALUE; +private final TreeMapLong, MappedByteBuffer chunkSegments; private final CompressionMetadata metadata; @@ -75,61 +79,24 @@ public class CompressedRandomAccessReader extends RandomAccessReader // raw checksum bytes private ByteBuffer checksumBytes; -protected CompressedRandomAccessReader(ChannelProxy channel, CompressionMetadata metadata, PoolingSegmentedFile owner) throws FileNotFoundException +protected CompressedRandomAccessReader(ChannelProxy channel, CompressionMetadata metadata, ICompressedFile file) throws FileNotFoundException { -super(channel, metadata.chunkLength(), metadata.compressedFileLength, metadata.compressor().useDirectOutputByteBuffers(), owner); +super(channel, metadata.chunkLength(), metadata.compressedFileLength, metadata.compressor().useDirectOutputByteBuffers(), file instanceof PoolingSegmentedFile ? (PoolingSegmentedFile) file : null); this.metadata = metadata; checksum = new Adler32(); -if (!useMmap) +chunkSegments = file == null ? null : file.chunkSegments(); +if (chunkSegments == null) { -compressed = ByteBuffer.wrap(new byte[metadata.compressor().initialCompressedBufferLength(metadata.chunkLength())]); +compressed = super.allocateBuffer(metadata.compressor().initialCompressedBufferLength(metadata.chunkLength()), metadata.compressor().useDirectOutputByteBuffers());
[jira] [Updated] (CASSANDRA-9322) Possible overlap with LCS and including non-compacting sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-9322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Marcus Eriksson updated CASSANDRA-9322: --- Attachment: 0001-9322.patch check if max == min and if so dont add a starved sstable Possible overlap with LCS and including non-compacting sstables --- Key: CASSANDRA-9322 URL: https://issues.apache.org/jira/browse/CASSANDRA-9322 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Assignee: Marcus Eriksson Fix For: 2.0.x Attachments: 0001-9322.patch since CASSANDRA-7414 we are including high-level sstables in lower level compactions if we have not run compactions in the high level for a while. If the compaction candidates only contain a single partition this can cause overlap since first token in sstables == last token in sstables which we interpret as being entire ring. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9240) Performance issue after a restart
[ https://issues.apache.org/jira/browse/CASSANDRA-9240?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533781#comment-14533781 ] Stefania commented on CASSANDRA-9240: - Thanks for this. I see the mean latency on read doubles after restart on trunk, is this expected? On the 8897 branch it's even worse, it's already double that of trunk on the first read and it degrades further after a restart so I definitely did something wrong which I'll have to work out but I'll carry on the discussion on that ticket. Here are the results in case anyone is interested. {code} Mean latency (one none running in foreground with -Xms500M -Xmx500M -Xmn50M, inserting 200k and reading back 50k): TRUNK insert 7.2 TRUNK read 8.7 TRUNK read after restart 16.4 8897 insert 6.8 8897 read 16.6 8897 read after restart 22.3 {code} Performance issue after a restart - Key: CASSANDRA-9240 URL: https://issues.apache.org/jira/browse/CASSANDRA-9240 Project: Cassandra Issue Type: Bug Reporter: Alan Boudreault Assignee: Benedict Priority: Minor Fix For: 3.x Attachments: Cassandra.snapshots.zip, cassandra_2.1.4-clientrequest-read.log, cassandra_2.1.4.log, cassandra_2.1.5-clientrequest-read.log, cassandra_2.1.5.log, cassandra_trunk-clientrequest-read.log, cassandra_trunk.log, cassandra_trunk_no_restart-clientrequest-read.log, cassandra_trunk_no_restart.log, issue.yaml, run_issue.sh, runs.log, trace_query.cql I have noticed a performance issue while I was working on compaction perf tests for CASSANDRA-7409. The performance for my use case is very bad after a restart. It is mostly a read performance issue but not strictly. I have attached my use case (see run_issue.sh and issue.yaml) and all test logs for 2.1.4, 2.1.5 and trunk: * 2.1.* are OK (although 2.1.4 seems to be better than 2.1.5?): ~6-7k ops/second and ~2-2.5k of read latency. * trunk is NOT OK: ~1.5-2k ops/second and 25-30k of read latency. * trunk is OK without a restart: ~ same perf than 2.1.4 and 2.1.5. EDIT: branch cassandra-2.1 is OK. I can help to bisect and/or profile on Monday if needed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-8897) Remove FileCacheService, instead pooling the buffers
[ https://issues.apache.org/jira/browse/CASSANDRA-8897?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533732#comment-14533732 ] Stefania edited comment on CASSANDRA-8897 at 5/8/15 3:03 AM: - Since CompressedPoolingSegmentedFile is gone we will need a sure way to retest CASSANDRA-9240. The results just after a rebase are not that good: {code} Mean latency (one none running in foreground with -Xms500M -Xmx500M -Xmn50M, inserting 200k and reading back 50k): TRUNK insert 7.2 TRUNK read 8.7 TRUNK read after restart 16.4 8897 insert 6.8 8897 read 16.6 8897 read after restart 22.3 {code} was (Author: stefania): Since CompressedPoolingSegmentedFile is gone we will need a sure way to retest CASSANDRA-9240. Remove FileCacheService, instead pooling the buffers Key: CASSANDRA-8897 URL: https://issues.apache.org/jira/browse/CASSANDRA-8897 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Benedict Assignee: Stefania Fix For: 3.x After CASSANDRA-8893, a RAR will be a very lightweight object and will not need caching, so we can eliminate this cache entirely. Instead we should have a pool of buffers that are page-aligned. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9331) Too many Compaction pending Tasks
Paddy Krishnamoorthy created CASSANDRA-9331: --- Summary: Too many Compaction pending Tasks Key: CASSANDRA-9331 URL: https://issues.apache.org/jira/browse/CASSANDRA-9331 Project: Cassandra Issue Type: Bug Components: Core Environment: Ubuntu Reporter: Paddy Krishnamoorthy Priority: Critical Fix For: 2.1.x We have a 9 node cluster that has been running for about 8 months now. Lately the selects seems to take longer. Also I noticed that the compaction jobs are always running. Last time when I checked every node has average 600+ pending compaction tasks. Two days ago I posted this issue on IRC chat and someone recommended to upgrade from 2.1.2 to current release. I updated one of the nodes to 2.1.4 and I still noticed that newly added node has about 600+ compaction tasks. We also noticed that nodetool rebuild takes more than 24 hours to complete. Wondering if there is an issue with our configuration or is it a bug? Hardware and other config details: The current storage size per node is approximately 100GB. All our disks are spinning disks. The servers are hosted in out own data centre. We use VNODES and num_tokens set to 256 Here is the out from desc table daily_index command for one of our table that has 10s of millions of records. CREATE TABLE daily_indexes.daily_index ( indexday int, pid text, docid text, author text, e mapint, decimal, hit boolean, p decimal, s int, PRIMARY KEY ((indexday, pid), docid) ) WITH CLUSTERING ORDER BY (docid ASC) AND bloom_filter_fp_chance = 0.01 AND caching = '{keys:ALL, rows_per_partition:NONE}' AND comment = '' AND compaction = {'min_threshold': '4', 'class': 'org.apache.cassandra.db.compaction.LeveledCompactionStrategy', 'max_threshold': '32'} AND compression = {'sstable_compression': 'org.apache.cassandra.io.compress.LZ4Compressor'} AND dclocal_read_repair_chance = 0.1 AND default_time_to_live = 0 AND gc_grace_seconds = 86400 AND max_index_interval = 2048 AND memtable_flush_period_in_ms = 0 AND min_index_interval = 128 AND read_repair_chance = 0.0 AND speculative_retry = '99.0PERCENTILE'; -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9332) NPE when creating column family via thrift
[ https://issues.apache.org/jira/browse/CASSANDRA-9332?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Colin Kuo updated CASSANDRA-9332: - Summary: NPE when creating column family via thrift (was: NPE when adding column family via thrift) NPE when creating column family via thrift -- Key: CASSANDRA-9332 URL: https://issues.apache.org/jira/browse/CASSANDRA-9332 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle JDK 1.7.0_79 Casandra 2.0.6 in single node Ubuntu 14.04 Reporter: Colin Kuo Labels: thrift Fix For: 2.0.x When triggering unit test testAddDropColumnFamily() in https://github.com/hector-client/hector/blob/master/core/src/test/java/me/prettyprint/cassandra/service/CassandraClusterTest.java It occurs NPE when using *Cassandra 2.0.6* or later version. {noformat} 11:42:39,173 [Thrift:1] ERROR CustomTThreadPoolServer:212 - Error occurred during processing of message. java.lang.NullPointerException at org.apache.cassandra.db.RowMutation.add(RowMutation.java:112) at org.apache.cassandra.service.MigrationManager.addSerializedKeyspace(MigrationManager.java:265) at org.apache.cassandra.service.MigrationManager.announceNewColumnFamily(MigrationManager.java:213) at org.apache.cassandra.thrift.CassandraServer.system_add_column_family(CassandraServer.java:1521) at org.apache.cassandra.thrift.Cassandra$Processor$system_add_column_family.getResult(Cassandra.java:4300) at org.apache.cassandra.thrift.Cassandra$Processor$system_add_column_family.getResult(Cassandra.java:4284) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:194) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} It seems that was introduced by fix of CASSANDRA-5631. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9333) Edge case - Empty of blank password for JMX authentication not handled properly in nodetool commands
Sumod Pawgi created CASSANDRA-9333: -- Summary: Edge case - Empty of blank password for JMX authentication not handled properly in nodetool commands Key: CASSANDRA-9333 URL: https://issues.apache.org/jira/browse/CASSANDRA-9333 Project: Cassandra Issue Type: Bug Components: Core, Tools Environment: Apache Cassandra 2.1.2 Reporter: Sumod Pawgi Priority: Minor While setting up JMX authentication for Apache Cassandra, if we set the password blank (in the file - jmxremote.password), nodetool commands do not work example creds are cassandra cassandra. In this case, for a secured cluster, we run the nodetool command as - nodetool -u cassandra -pw cassandra status But if the password is kept as blank then we cannot execute nodetool command. However, I believe that if a third party software used JMX authentication via API, then they can use blank password for the operations. So this behavior needs to be clarified and be consistent for this edge case scenario. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9332) NPE when adding column family via thrift
Colin Kuo created CASSANDRA-9332: Summary: NPE when adding column family via thrift Key: CASSANDRA-9332 URL: https://issues.apache.org/jira/browse/CASSANDRA-9332 Project: Cassandra Issue Type: Bug Components: Core Environment: Oracle JDK 1.7.0_79 Casandra 2.0.6 in single node Ubuntu 14.04 Reporter: Colin Kuo Fix For: 2.0.x When triggering unit test testAddDropColumnFamily() in https://github.com/hector-client/hector/blob/master/core/src/test/java/me/prettyprint/cassandra/service/CassandraClusterTest.java It occurs NPE when using *Cassandra 2.0.6* or later version. {noformat} 11:42:39,173 [Thrift:1] ERROR CustomTThreadPoolServer:212 - Error occurred during processing of message. java.lang.NullPointerException at org.apache.cassandra.db.RowMutation.add(RowMutation.java:112) at org.apache.cassandra.service.MigrationManager.addSerializedKeyspace(MigrationManager.java:265) at org.apache.cassandra.service.MigrationManager.announceNewColumnFamily(MigrationManager.java:213) at org.apache.cassandra.thrift.CassandraServer.system_add_column_family(CassandraServer.java:1521) at org.apache.cassandra.thrift.Cassandra$Processor$system_add_column_family.getResult(Cassandra.java:4300) at org.apache.cassandra.thrift.Cassandra$Processor$system_add_column_family.getResult(Cassandra.java:4284) at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:194) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:745) {noformat} It seems that was introduced by fix of CASSANDRA-5631. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koray sariteke updated CASSANDRA-9301: -- Description: Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing was: Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data(49 data) are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532614#comment-14532614 ] koray sariteke edited comment on CASSANDRA-9301 at 5/7/15 2:12 PM: --- Try 3 scenarios and get same missing data result below, scenario 1 - three cluster and replication factor 2, consistency level one scenario 2 - three cluster and replication factor 2, consistency level ALL scenario 3 - one node Also observe nothing wrong at logs symbol orderno timestamp DOAS 124976 27042015-110001 TMSN 124977 27042015-110001 TMSN 124978 27042015-110001 TMSN 124979 27042015-110001 SODA 124980 27042015-110002 BJKAS 124981 27042015-110002 GLYHO 124982 27042015-110002 GLYHO 124983 27042015-110002 GLYHO 124984 27042015-110002 GLYHO 124985 27042015-110002 SODA 124986 27042015-110002 TMSN 124987 27042015-110002 NTHOL 124988 27042015-110002 ALCTL 124989 27042015-110002 SANFM 389692 27042015-170004 SANFM 389693 27042015-170004 SANFM 389694 27042015-170004 SANFM 389695 27042015-170004 SANFM 389696 27042015-170004 SANFM 389697 27042015-170004 SANFM 389698 27042015-170004 SANFM 389699 27042015-170004 VKGYO 389700 27042015-170004 EUHOL 389701 27042015-170004 VKGYO 389702 27042015-170004 YKBNK 389703 27042015-170004 YKBNK 389704 27042015-170004 YKBNK 389705 27042015-170004 YKBNK 389706 27042015-170004 YKBNK 389707 27042015-170004 YKBNK 389708 27042015-170004 YKBNK 389709 27042015-170004 YKBNK 389710 27042015-170004 YKBNK 389711 27042015-170004 YKBNK 389712 27042015-170004 YKBNK 389713 27042015-170004 YKBNK 389714 27042015-170004 YKBNK 389715 27042015-170004 YKBNK 389716 27042015-170004 YKBNK 389717 27042015-170004 YKBNK 389718 27042015-170004 YKBNK 389719 27042015-170004 VADYTV 389720 27042015-170004 BOYNR 389721 27042015-170004 BOYNR 389722 27042015-170004 KARTN 389723 27042015-170004 KARTN 389724 27042015-170004 was (Author: ksaritek): Try 3 scenarios and get same missing data result below, scenario 1 - three cluster and replication factor 2, consistency level one scenario 2 - three cluster and replication factor 2, consistency level ALL scenario 3 - one node Also observe nothing wrong at logs symbol orderno timestamp ALCTL 124989 27042015-110002 VKGYO 389702 27042015-170004 YKBNK 389712 27042015-170004 GLYHO 124985 27042015-110002 VKGYO 389700 27042015-170004 YKBNK 389719 27042015-170004 BJKAS 124981 27042015-110002 SANFM 389697 27042015-170004 YKBNK 389706 27042015-170004 BOYNR 389721 27042015-170004 TMSN 124978 27042015-110001 GLYHO 124982 27042015-110002 SANFM 389698 27042015-170004 YKBNK 389709 27042015-170004 YKBNK 389710 27042015-170004 TMSN 124987 27042015-110002 ODAS 114602 27042015-105048 YKBNK 389708 27042015-170004 YKBNK 389716 27042015-170004 ODAS 109575 27042015-104700 SODA 124980 27042015-110002 SANFM 389692 27042015-170004 YKBNK 389714 27042015-170004 TMSN 124979 27042015-110001 GLYHO 124983 27042015-110002 EUHOL 389701 27042015-170004 SODA 124986 27042015-110002 SANFM 389699 27042015-170004 YKBNK 389715 27042015-170004 YKBNK 389707 27042015-170004 KARTN 389724 27042015-170004 SANFM 389695 27042015-170004 YKBNK 389703 27042015-170004 VADYTV 389720 27042015-170004 TMSN 124977 27042015-110001 YKBNK 389704 27042015-170004 BOYNR 389722 27042015-170004 NTHOL 124988 27042015-110002 SANFM 389693 27042015-170004 YKBNK 389713 27042015-170004 YKBNK 389717 27042015-170004 YKBNK 389718 27042015-170004 SANFM 389694 27042015-170004 SANFM 389696 27042015-170004 DOAS 124976 27042015-110001 GLYHO 124984 27042015-110002 YKBNK 389705 27042015-170004 YKBNK 389711 27042015-170004 KARTN 389723 27042015-170004 can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data(49 data) are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532614#comment-14532614 ] koray sariteke commented on CASSANDRA-9301: --- Try 3 scenarios and get same missing data result below, scenario 1 - three cluster and replication factor 2, consistency level one scenario 2 - three cluster and replication factor 2, consistency level ALL scenario 3 - one node ALCTL 124989 27042015-110002 VKGYO 389702 27042015-170004 YKBNK 389712 27042015-170004 GLYHO 124985 27042015-110002 VKGYO 389700 27042015-170004 YKBNK 389719 27042015-170004 BJKAS 124981 27042015-110002 SANFM 389697 27042015-170004 YKBNK 389706 27042015-170004 BOYNR 389721 27042015-170004 TMSN 124978 27042015-110001 GLYHO 124982 27042015-110002 SANFM 389698 27042015-170004 YKBNK 389709 27042015-170004 YKBNK 389710 27042015-170004 TMSN 124987 27042015-110002 ODAS 114602 27042015-105048 YKBNK 389708 27042015-170004 YKBNK 389716 27042015-170004 ODAS 109575 27042015-104700 SODA 124980 27042015-110002 SANFM 389692 27042015-170004 YKBNK 389714 27042015-170004 TMSN 124979 27042015-110001 GLYHO 124983 27042015-110002 EUHOL 389701 27042015-170004 SODA 124986 27042015-110002 SANFM 389699 27042015-170004 YKBNK 389715 27042015-170004 YKBNK 389707 27042015-170004 KARTN 389724 27042015-170004 SANFM 389695 27042015-170004 YKBNK 389703 27042015-170004 VADYTV 389720 27042015-170004 TMSN 124977 27042015-110001 YKBNK 389704 27042015-170004 BOYNR 389722 27042015-170004 NTHOL 124988 27042015-110002 SANFM 389693 27042015-170004 YKBNK 389713 27042015-170004 YKBNK 389717 27042015-170004 YKBNK 389718 27042015-170004 SANFM 389694 27042015-170004 SANFM 389696 27042015-170004 DOAS 124976 27042015-110001 GLYHO 124984 27042015-110002 YKBNK 389705 27042015-170004 YKBNK 389711 27042015-170004 KARTN 389723 27042015-170004 can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data(49 data) are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532614#comment-14532614 ] koray sariteke edited comment on CASSANDRA-9301 at 5/7/15 1:34 PM: --- Try 3 scenarios and get same missing data result below, scenario 1 - three cluster and replication factor 2, consistency level one scenario 2 - three cluster and replication factor 2, consistency level ALL scenario 3 - one node Also observe nothing wrong at logs ALCTL 124989 27042015-110002 VKGYO 389702 27042015-170004 YKBNK 389712 27042015-170004 GLYHO 124985 27042015-110002 VKGYO 389700 27042015-170004 YKBNK 389719 27042015-170004 BJKAS 124981 27042015-110002 SANFM 389697 27042015-170004 YKBNK 389706 27042015-170004 BOYNR 389721 27042015-170004 TMSN 124978 27042015-110001 GLYHO 124982 27042015-110002 SANFM 389698 27042015-170004 YKBNK 389709 27042015-170004 YKBNK 389710 27042015-170004 TMSN 124987 27042015-110002 ODAS 114602 27042015-105048 YKBNK 389708 27042015-170004 YKBNK 389716 27042015-170004 ODAS 109575 27042015-104700 SODA 124980 27042015-110002 SANFM 389692 27042015-170004 YKBNK 389714 27042015-170004 TMSN 124979 27042015-110001 GLYHO 124983 27042015-110002 EUHOL 389701 27042015-170004 SODA 124986 27042015-110002 SANFM 389699 27042015-170004 YKBNK 389715 27042015-170004 YKBNK 389707 27042015-170004 KARTN 389724 27042015-170004 SANFM 389695 27042015-170004 YKBNK 389703 27042015-170004 VADYTV 389720 27042015-170004 TMSN 124977 27042015-110001 YKBNK 389704 27042015-170004 BOYNR 389722 27042015-170004 NTHOL 124988 27042015-110002 SANFM 389693 27042015-170004 YKBNK 389713 27042015-170004 YKBNK 389717 27042015-170004 YKBNK 389718 27042015-170004 SANFM 389694 27042015-170004 SANFM 389696 27042015-170004 DOAS 124976 27042015-110001 GLYHO 124984 27042015-110002 YKBNK 389705 27042015-170004 YKBNK 389711 27042015-170004 KARTN 389723 27042015-170004 was (Author: ksaritek): Try 3 scenarios and get same missing data result below, scenario 1 - three cluster and replication factor 2, consistency level one scenario 2 - three cluster and replication factor 2, consistency level ALL scenario 3 - one node ALCTL 124989 27042015-110002 VKGYO 389702 27042015-170004 YKBNK 389712 27042015-170004 GLYHO 124985 27042015-110002 VKGYO 389700 27042015-170004 YKBNK 389719 27042015-170004 BJKAS 124981 27042015-110002 SANFM 389697 27042015-170004 YKBNK 389706 27042015-170004 BOYNR 389721 27042015-170004 TMSN 124978 27042015-110001 GLYHO 124982 27042015-110002 SANFM 389698 27042015-170004 YKBNK 389709 27042015-170004 YKBNK 389710 27042015-170004 TMSN 124987 27042015-110002 ODAS 114602 27042015-105048 YKBNK 389708 27042015-170004 YKBNK 389716 27042015-170004 ODAS 109575 27042015-104700 SODA 124980 27042015-110002 SANFM 389692 27042015-170004 YKBNK 389714 27042015-170004 TMSN 124979 27042015-110001 GLYHO 124983 27042015-110002 EUHOL 389701 27042015-170004 SODA 124986 27042015-110002 SANFM 389699 27042015-170004 YKBNK 389715 27042015-170004 YKBNK 389707 27042015-170004 KARTN 389724 27042015-170004 SANFM 389695 27042015-170004 YKBNK 389703 27042015-170004 VADYTV 389720 27042015-170004 TMSN 124977 27042015-110001 YKBNK 389704 27042015-170004 BOYNR 389722 27042015-170004 NTHOL 124988 27042015-110002 SANFM 389693 27042015-170004 YKBNK 389713 27042015-170004 YKBNK 389717 27042015-170004 YKBNK 389718 27042015-170004 SANFM 389694 27042015-170004 SANFM 389696 27042015-170004 DOAS 124976 27042015-110001 GLYHO 124984 27042015-110002 YKBNK 389705 27042015-170004 YKBNK 389711 27042015-170004 KARTN 389723 27042015-170004 can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data(49 data) are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[2/3] cassandra git commit: Ninja - remove section to create directories below /var
Ninja - remove section to create directories below /var Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2c8e810b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2c8e810b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2c8e810b Branch: refs/heads/trunk Commit: 2c8e810be4810129c1bee4266b17bf9f88552fed Parents: 8c5ce66 Author: Robert Stupp sn...@snazy.de Authored: Thu May 7 16:15:34 2015 +0200 Committer: Robert Stupp sn...@snazy.de Committed: Thu May 7 16:18:02 2015 +0200 -- README.asc | 9 + 1 file changed, 1 insertion(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c8e810b/README.asc -- diff --git a/README.asc b/README.asc index 2e0a0e6..6360fe9 100644 --- a/README.asc +++ b/README.asc @@ -25,14 +25,7 @@ First, we'll unpack our archive: $ tar -zxvf apache-cassandra-$VERSION.tar.gz $ cd apache-cassandra-$VERSION -and create the log and data directories. These correspond to the defaults from conf/ and may be adjusted to suit your own environment: - - $ sudo mkdir -p /var/log/cassandra - $ sudo chown -R `whoami` /var/log/cassandra - $ sudo mkdir -p /var/lib/cassandra - $ sudo chown -R `whoami` /var/lib/cassandra - -Finally, we start the server. Running the startup script with the -f argument will cause +After that we start the server. Running the startup script with the -f argument will cause Cassandra to remain in the foreground and log to standard out; it can be stopped with ctrl-C. $ bin/cassandra -f
[1/3] cassandra git commit: Ninja - remove section to create directories below /var
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 8c5ce66fe - 2c8e810be refs/heads/trunk 2499baeb3 - 7954965eb Ninja - remove section to create directories below /var Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/2c8e810b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/2c8e810b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/2c8e810b Branch: refs/heads/cassandra-2.1 Commit: 2c8e810be4810129c1bee4266b17bf9f88552fed Parents: 8c5ce66 Author: Robert Stupp sn...@snazy.de Authored: Thu May 7 16:15:34 2015 +0200 Committer: Robert Stupp sn...@snazy.de Committed: Thu May 7 16:18:02 2015 +0200 -- README.asc | 9 + 1 file changed, 1 insertion(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/2c8e810b/README.asc -- diff --git a/README.asc b/README.asc index 2e0a0e6..6360fe9 100644 --- a/README.asc +++ b/README.asc @@ -25,14 +25,7 @@ First, we'll unpack our archive: $ tar -zxvf apache-cassandra-$VERSION.tar.gz $ cd apache-cassandra-$VERSION -and create the log and data directories. These correspond to the defaults from conf/ and may be adjusted to suit your own environment: - - $ sudo mkdir -p /var/log/cassandra - $ sudo chown -R `whoami` /var/log/cassandra - $ sudo mkdir -p /var/lib/cassandra - $ sudo chown -R `whoami` /var/lib/cassandra - -Finally, we start the server. Running the startup script with the -f argument will cause +After that we start the server. Running the startup script with the -f argument will cause Cassandra to remain in the foreground and log to standard out; it can be stopped with ctrl-C. $ bin/cassandra -f
[3/3] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/7954965e Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/7954965e Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/7954965e Branch: refs/heads/trunk Commit: 7954965ebc4bd1f8cb7933c3216055975cd0f21c Parents: 2499bae 2c8e810 Author: Robert Stupp sn...@snazy.de Authored: Thu May 7 16:20:49 2015 +0200 Committer: Robert Stupp sn...@snazy.de Committed: Thu May 7 16:20:49 2015 +0200 -- README.asc | 9 + 1 file changed, 1 insertion(+), 8 deletions(-) --
[jira] [Comment Edited] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532614#comment-14532614 ] koray sariteke edited comment on CASSANDRA-9301 at 5/7/15 1:35 PM: --- Try 3 scenarios and get same missing data result below, scenario 1 - three cluster and replication factor 2, consistency level one scenario 2 - three cluster and replication factor 2, consistency level ALL scenario 3 - one node Also observe nothing wrong at logs symbol orderno timestamp ALCTL 124989 27042015-110002 VKGYO 389702 27042015-170004 YKBNK 389712 27042015-170004 GLYHO 124985 27042015-110002 VKGYO 389700 27042015-170004 YKBNK 389719 27042015-170004 BJKAS 124981 27042015-110002 SANFM 389697 27042015-170004 YKBNK 389706 27042015-170004 BOYNR 389721 27042015-170004 TMSN 124978 27042015-110001 GLYHO 124982 27042015-110002 SANFM 389698 27042015-170004 YKBNK 389709 27042015-170004 YKBNK 389710 27042015-170004 TMSN 124987 27042015-110002 ODAS 114602 27042015-105048 YKBNK 389708 27042015-170004 YKBNK 389716 27042015-170004 ODAS 109575 27042015-104700 SODA 124980 27042015-110002 SANFM 389692 27042015-170004 YKBNK 389714 27042015-170004 TMSN 124979 27042015-110001 GLYHO 124983 27042015-110002 EUHOL 389701 27042015-170004 SODA 124986 27042015-110002 SANFM 389699 27042015-170004 YKBNK 389715 27042015-170004 YKBNK 389707 27042015-170004 KARTN 389724 27042015-170004 SANFM 389695 27042015-170004 YKBNK 389703 27042015-170004 VADYTV 389720 27042015-170004 TMSN 124977 27042015-110001 YKBNK 389704 27042015-170004 BOYNR 389722 27042015-170004 NTHOL 124988 27042015-110002 SANFM 389693 27042015-170004 YKBNK 389713 27042015-170004 YKBNK 389717 27042015-170004 YKBNK 389718 27042015-170004 SANFM 389694 27042015-170004 SANFM 389696 27042015-170004 DOAS 124976 27042015-110001 GLYHO 124984 27042015-110002 YKBNK 389705 27042015-170004 YKBNK 389711 27042015-170004 KARTN 389723 27042015-170004 was (Author: ksaritek): Try 3 scenarios and get same missing data result below, scenario 1 - three cluster and replication factor 2, consistency level one scenario 2 - three cluster and replication factor 2, consistency level ALL scenario 3 - one node Also observe nothing wrong at logs ALCTL 124989 27042015-110002 VKGYO 389702 27042015-170004 YKBNK 389712 27042015-170004 GLYHO 124985 27042015-110002 VKGYO 389700 27042015-170004 YKBNK 389719 27042015-170004 BJKAS 124981 27042015-110002 SANFM 389697 27042015-170004 YKBNK 389706 27042015-170004 BOYNR 389721 27042015-170004 TMSN 124978 27042015-110001 GLYHO 124982 27042015-110002 SANFM 389698 27042015-170004 YKBNK 389709 27042015-170004 YKBNK 389710 27042015-170004 TMSN 124987 27042015-110002 ODAS 114602 27042015-105048 YKBNK 389708 27042015-170004 YKBNK 389716 27042015-170004 ODAS 109575 27042015-104700 SODA 124980 27042015-110002 SANFM 389692 27042015-170004 YKBNK 389714 27042015-170004 TMSN 124979 27042015-110001 GLYHO 124983 27042015-110002 EUHOL 389701 27042015-170004 SODA 124986 27042015-110002 SANFM 389699 27042015-170004 YKBNK 389715 27042015-170004 YKBNK 389707 27042015-170004 KARTN 389724 27042015-170004 SANFM 389695 27042015-170004 YKBNK 389703 27042015-170004 VADYTV 389720 27042015-170004 TMSN 124977 27042015-110001 YKBNK 389704 27042015-170004 BOYNR 389722 27042015-170004 NTHOL 124988 27042015-110002 SANFM 389693 27042015-170004 YKBNK 389713 27042015-170004 YKBNK 389717 27042015-170004 YKBNK 389718 27042015-170004 SANFM 389694 27042015-170004 SANFM 389696 27042015-170004 DOAS 124976 27042015-110001 GLYHO 124984 27042015-110002 YKBNK 389705 27042015-170004 YKBNK 389711 27042015-170004 KARTN 389723 27042015-170004 can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data(49 data) are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532683#comment-14532683 ] Ariel Weisberg commented on CASSANDRA-9318: --- It definitely does almost nothing for large reads. For writes it might work OK if everything is distributed evenly. Do you think it doesn't help for writes either? Even if the coordinator doesn't know about how much load each remote participant has in aggregate if load is even it does place a bound. What does happen that is unfortunate is that over time all requests end up being for the slowest node given enough time which can bring it down and prevent progress at other nodes. I agree that other approaches that can shed load where it actually happens based on actual load metrics and not admission control guesses will be more effective in the long run. Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 3.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9275) ReadMessageTest.testNoCommitLog fails with test-compression
[ https://issues.apache.org/jira/browse/CASSANDRA-9275?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533386#comment-14533386 ] Ariel Weisberg commented on CASSANDRA-9275: --- Thanks. +1 ReadMessageTest.testNoCommitLog fails with test-compression --- Key: CASSANDRA-9275 URL: https://issues.apache.org/jira/browse/CASSANDRA-9275 Project: Cassandra Issue Type: Test Reporter: Ariel Weisberg Assignee: Branimir Lambov Fix For: 3.x Attachments: 9275-readmessagetest.txt It's reading the raw bytes of the commit log expecting to find an uncompressed log and instead finding a compressed log and now the test is not working if the log is compressed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9328) WriteTimeoutException thrown when LWT concurrency 1, despite the query duration taking MUCH less than cas_contention_timeout_in_ms
Aaron Whiteside created CASSANDRA-9328: -- Summary: WriteTimeoutException thrown when LWT concurrency 1, despite the query duration taking MUCH less than cas_contention_timeout_in_ms Key: CASSANDRA-9328 URL: https://issues.apache.org/jira/browse/CASSANDRA-9328 Project: Cassandra Issue Type: Bug Components: Core Reporter: Aaron Whiteside Priority: Critical Attachments: CassandraLWTTest.java WriteTimeoutException thrown when LWT concurrency 1, despite the query duration taking MUCH less than cas_contention_timeout_in_ms. Unit test attached, run against a 3 node cluster running 2.1.5. If you reduce the threadCount to 1, you never see a WriteTimeoutException. If the WTE is due to not being able to communicate with other nodes, why does the concurrency 1 cause inter-node communication to fail? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-8593) Test for leap second related bugs
[ https://issues.apache.org/jira/browse/CASSANDRA-8593?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ryan McGuire resolved CASSANDRA-8593. - Resolution: Not A Problem CASSANDRA-9131 captures everything left to do, so closing as not a problem. Test for leap second related bugs - Key: CASSANDRA-8593 URL: https://issues.apache.org/jira/browse/CASSANDRA-8593 Project: Cassandra Issue Type: Test Reporter: Ryan McGuire Assignee: Jim Witschey http://www.datastax.com/dev/blog/linux-cassandra-and-saturdays-leap-second-problem Another leap second is being added in June, we need to find an old system/platform that does still have this issue, create a test that exercises it. Then we can use the test to create a list of any still affected platforms. Ideally, we can include a list of the affected platforms/configs inside C* and it will issue a warning in the logs. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-8517) Make client-side Token Aware Balancing possible with Global Indexes
[ https://issues.apache.org/jira/browse/CASSANDRA-8517?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Jonathan Ellis resolved CASSANDRA-8517. --- Resolution: Not A Problem Switching CASSANDRA-6477 syntax from GI to MV will make this a non-problem. Make client-side Token Aware Balancing possible with Global Indexes --- Key: CASSANDRA-8517 URL: https://issues.apache.org/jira/browse/CASSANDRA-8517 Project: Cassandra Issue Type: Improvement Reporter: Michaël Figuière Fix For: 3.x This is a follow up ticket for CASSANDRA-6477, to consider enabling client-side Token Awareness Balancing with Global Indexes, a it would be a pity to loose this efficient optimization when using these new indexes. My original comment was: {quote} Something that hasn't been mentioned so far in this thread is client side's Token Aware Balancing. So far when a query relies on a Secondary Index, it doesn't have any partition key specified in its where clause which means that the driver will fall back on the underlying balancing policy, which is DC aware round robin by default in the DS Java Driver. That's the appropriate behavior as there's no node that can be better than another as a coordinator in this situation. With Global Indexes, in order for the Driver to still be able to perform Token Aware Balancing, it'll need to be able to figure out which index will be used, which doesn't seems to be always trivial in the above examples, especially if several columns and/or indexes are involved in the WHERE clause. So here we might need to include an extra information about it in the Prepare response message of the Native Protocol, if the indexes to be involved can be figured out at Prepare-time. {quote} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9330) CAS timeout errors should use a different exception than WriteTimeoutException as WTE can happen when nodes fail to respond.
Aaron Whiteside created CASSANDRA-9330: -- Summary: CAS timeout errors should use a different exception than WriteTimeoutException as WTE can happen when nodes fail to respond. Key: CASSANDRA-9330 URL: https://issues.apache.org/jira/browse/CASSANDRA-9330 Project: Cassandra Issue Type: Improvement Components: Core, Drivers (now out of tree) Reporter: Aaron Whiteside Perhaps a CASContentionTimeoutException? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-7622) Implement virtual tables
[ https://issues.apache.org/jira/browse/CASSANDRA-7622?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533578#comment-14533578 ] Tupshin Harper commented on CASSANDRA-7622: --- The correct decision to make JMX bind to localhost only for security reasons creates additional importance and urgency for this as a feature. I'd like to promote it from hand-wavey 3.x to more concrete 3.1 in hopes that it wouldn't slip from there. We really need to simplify the access patterns and reduce the surface area. Implement virtual tables Key: CASSANDRA-7622 URL: https://issues.apache.org/jira/browse/CASSANDRA-7622 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Tupshin Harper Assignee: Benjamin Lerer Fix For: 3.x There are a variety of reasons to want virtual tables, which would be any table that would be backed by an API, rather than data explicitly managed and stored as sstables. One possible use case would be to expose JMX data through CQL as a resurrection of CASSANDRA-3527. Another is a more general framework to implement the ability to expose yaml configuration information. So it would be an alternate approach to CASSANDRA-7370. A possible implementation would be in terms of CASSANDRA-7443, but I am not presupposing. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-8593) Test for leap second related bugs
[ https://issues.apache.org/jira/browse/CASSANDRA-8593?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14533480#comment-14533480 ] Jim Witschey commented on CASSANDRA-8593: - [Here|http://www.datastax.com/dev/blog/preparing-for-the-leap-second] is the blog post that came out of this work. Are there any objections to closing this? Test for leap second related bugs - Key: CASSANDRA-8593 URL: https://issues.apache.org/jira/browse/CASSANDRA-8593 Project: Cassandra Issue Type: Test Reporter: Ryan McGuire Assignee: Jim Witschey http://www.datastax.com/dev/blog/linux-cassandra-and-saturdays-leap-second-problem Another leap second is being added in June, we need to find an old system/platform that does still have this issue, create a test that exercises it. Then we can use the test to create a list of any still affected platforms. Ideally, we can include a list of the affected platforms/configs inside C* and it will issue a warning in the logs. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9329) Make CAS retry logic configurable
Aaron Whiteside created CASSANDRA-9329: -- Summary: Make CAS retry logic configurable Key: CASSANDRA-9329 URL: https://issues.apache.org/jira/browse/CASSANDRA-9329 Project: Cassandra Issue Type: Improvement Components: Core Reporter: Aaron Whiteside Make CAS retry logic configurable: One should be able to disable the internal CAS retry loop (when the condition is not met) and let the client choose how to do retries (so the client does not have to incur the server side random sleep of up to 100ms). Basically let the client handle all CAS retries in a manor it sees fit. Secondly the hardcoded sleep up to 100ms that happens when cassandra fails to meet the CAS condition should be configurable. - The max duration should be configurable - The algorithm used to choose the duration should be configurable (Random, Exponential, etc). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9323) Bulk upload is slow
[ https://issues.apache.org/jira/browse/CASSANDRA-9323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pierre N. updated CASSANDRA-9323: - Description: When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? was: Hi, When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? Bulk upload is slow --- Key: CASSANDRA-9323 URL: https://issues.apache.org/jira/browse/CASSANDRA-9323 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Attachments: App.java When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-9301) can not insert properly for compound primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-9301?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] koray sariteke resolved CASSANDRA-9301. --- Resolution: Invalid Although i am not totally sure, i encountered some problems in my logs. I am closing the issue considering that there is a big chance that issue is not related with cassandra. can not insert properly for compound primary key Key: CASSANDRA-9301 URL: https://issues.apache.org/jira/browse/CASSANDRA-9301 Project: Cassandra Issue Type: Bug Components: Core Environment: cassandra 2.1.3, oracle jdk 1.7, linux Reporter: koray sariteke Schema: {code} CREATE KEYSPACE tick WITH replication = { 'class': 'SimpleStrategy', 'replication_factor': '2' }; CREATE TABLE trade ( date_symbol text, symbol text, signal_time bigint, signal_time_text text, bidask int, price double, quantity bigint, order_no bigint, PRIMARY KEY (date_symbol, signal_time) ) WITH compaction={'sstable_size_in_mb': '160', 'class': 'LeveledCompactionStrategy'} AND compression={'sstable_compression': 'LZ4Compressor'}; {code} Tried many times for 3 million data and same data are not inserted. Increase data frequency but result dont change. sample data at https://drive.google.com/file/d/0BzHZ-rv0D5JxSDRpblFQcUdRaVU/view?usp=sharing -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9323) Bulk loading is slow
[ https://issues.apache.org/jira/browse/CASSANDRA-9323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pierre N. updated CASSANDRA-9323: - Summary: Bulk loading is slow (was: Bulk upload is slow) Bulk loading is slow Key: CASSANDRA-9323 URL: https://issues.apache.org/jira/browse/CASSANDRA-9323 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Attachments: App.java When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9323) Bulk loading is slow
[ https://issues.apache.org/jira/browse/CASSANDRA-9323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pierre N. updated CASSANDRA-9323: - Description: When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable using sstableloader I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? was: When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? Bulk loading is slow Key: CASSANDRA-9323 URL: https://issues.apache.org/jira/browse/CASSANDRA-9323 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Attachments: App.java When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable using sstableloader I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
cassandra git commit: Log warning on unrecognized table during message deser
Repository: cassandra Updated Branches: refs/heads/cassandra-2.0 6ab0c301e - 58de86bff Log warning on unrecognized table during message deser Patch by Tyler Hobbs; reviewed by Sylvain Lebresne for CASSANDRA-9136 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/58de86bf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/58de86bf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/58de86bf Branch: refs/heads/cassandra-2.0 Commit: 58de86bff5e9383b679d9bb48cc95ec444275ca7 Parents: 6ab0c30 Author: Tyler Hobbs tylerho...@apache.org Authored: Thu May 7 10:30:01 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Thu May 7 10:30:01 2015 -0500 -- CHANGES.txt | 2 ++ src/java/org/apache/cassandra/db/PagedRangeCommand.java | 10 ++ src/java/org/apache/cassandra/db/RangeSliceCommand.java | 7 +++ .../org/apache/cassandra/db/SliceByNamesReadCommand.java | 8 .../org/apache/cassandra/db/SliceFromReadCommand.java | 8 5 files changed, 35 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d76606c..8473777 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 2.0.15: + * Log warning message when a table is queried before the schema has fully + propagated (CASSANDRA-9136) * Overload SecondaryIndex#indexes to accept the column definition (CASSANDRA-9314) * (cqlsh) Add SERIAL and LOCAL_SERIAL consistency levels (CASSANDRA-8051) * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281) http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/PagedRangeCommand.java -- diff --git a/src/java/org/apache/cassandra/db/PagedRangeCommand.java b/src/java/org/apache/cassandra/db/PagedRangeCommand.java index a8d7f49..3bc6539 100644 --- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java +++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import org.apache.cassandra.config.Schema; import org.apache.cassandra.db.filter.*; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.io.IVersionedSerializer; @@ -153,6 +154,15 @@ public class PagedRangeCommand extends AbstractRangeCommand { String keyspace = in.readUTF(); String columnFamily = in.readUTF(); + +if (Schema.instance.getCFMetaData(keyspace, columnFamily) == null) +{ +String message = String.format(Got paged range command for nonexistent table %s.%s. If the table was just + +created, this is likely due to the schema not being fully propagated. Please wait for schema + +agreement on table creation. , keyspace, columnFamily); +throw new UnknownColumnFamilyException(message, null); +} + long timestamp = in.readLong(); AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/RangeSliceCommand.java -- diff --git a/src/java/org/apache/cassandra/db/RangeSliceCommand.java b/src/java/org/apache/cassandra/db/RangeSliceCommand.java index 4aa1595..553f111 100644 --- a/src/java/org/apache/cassandra/db/RangeSliceCommand.java +++ b/src/java/org/apache/cassandra/db/RangeSliceCommand.java @@ -213,6 +213,13 @@ class RangeSliceCommandSerializer implements IVersionedSerializerRangeSliceComm long timestamp = version MessagingService.VERSION_20 ? System.currentTimeMillis() : in.readLong(); CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily); +if (metadata == null) +{ +String message = String.format(Got range slice command for nonexistent table %s.%s. If the table was just + +created, this is likely due to the schema not being fully propagated. Please wait for schema + +agreement on table creation. , keyspace, columnFamily); +throw new UnknownColumnFamilyException(message, null); +} IDiskAtomFilter predicate; if (version MessagingService.VERSION_20) http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java
[2/2] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/049a965b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/049a965b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/049a965b Branch: refs/heads/cassandra-2.1 Commit: 049a965bf1e57d30cac13141bdf228268765ebeb Parents: 2c8e810 58de86b Author: Tyler Hobbs tylerho...@apache.org Authored: Thu May 7 10:31:52 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Thu May 7 10:31:52 2015 -0500 -- CHANGES.txt | 3 ++- src/java/org/apache/cassandra/db/PagedRangeCommand.java | 7 +++ src/java/org/apache/cassandra/db/RangeSliceCommand.java | 7 +++ src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java | 7 +++ src/java/org/apache/cassandra/db/SliceFromReadCommand.java| 7 +++ 5 files changed, 30 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/049a965b/CHANGES.txt -- diff --cc CHANGES.txt index 41e0340,8473777..d512973 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,21 -1,6 +1,22 @@@ -2.0.15: +2.1.6 + * Improve sstable exclusion from partition tombstones (CASSANDRA-9298) + * Validate the indexed column rather than the cell's contents for 2i (CASSANDRA-9057) + * Add support for top-k custom 2i queries (CASSANDRA-8717) + * Fix error when dropping table during compaction (CASSANDRA-9251) + * cassandra-stress supports validation operations over user profiles (CASSANDRA-8773) + * Add support for rate limiting log messages (CASSANDRA-9029) + * Log the partition key with tombstone warnings (CASSANDRA-8561) + * Reduce runWithCompactionsDisabled poll interval to 1ms (CASSANDRA-9271) + * Fix PITR commitlog replay (CASSANDRA-9195) + * GCInspector logs very different times (CASSANDRA-9124) + * Fix deleting from an empty list (CASSANDRA-9198) + * Update tuple and collection types that use a user-defined type when that UDT + is modified (CASSANDRA-9148, CASSANDRA-9192) + * Use higher timeout for prepair and snapshot in repair (CASSANDRA-9261) + * Fix anticompaction blocking ANTI_ENTROPY stage (CASSANDRA-9151) +Merged from 2.0: - 2.0.15: + * Log warning message when a table is queried before the schema has fully +propagated (CASSANDRA-9136) * Overload SecondaryIndex#indexes to accept the column definition (CASSANDRA-9314) * (cqlsh) Add SERIAL and LOCAL_SERIAL consistency levels (CASSANDRA-8051) * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281) http://git-wip-us.apache.org/repos/asf/cassandra/blob/049a965b/src/java/org/apache/cassandra/db/PagedRangeCommand.java -- diff --cc src/java/org/apache/cassandra/db/PagedRangeCommand.java index 614f0f7,3bc6539..70325a8 --- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java +++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java @@@ -156,16 -154,23 +156,23 @@@ public class PagedRangeCommand extends { String keyspace = in.readUTF(); String columnFamily = in.readUTF(); +long timestamp = in.readLong(); -if (Schema.instance.getCFMetaData(keyspace, columnFamily) == null) +AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); + +CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily); ++if (metadata == null) + { + String message = String.format(Got paged range command for nonexistent table %s.%s. If the table was just + + created, this is likely due to the schema not being fully propagated. Please wait for schema + + agreement on table creation. , keyspace, columnFamily); + throw new UnknownColumnFamilyException(message, null); + } -long timestamp = in.readLong(); +SliceQueryFilter predicate = metadata.comparator.sliceQueryFilterSerializer().deserialize(in, version); -AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); - -SliceQueryFilter predicate = SliceQueryFilter.serializer.deserialize(in, version); - -ByteBuffer start = ByteBufferUtil.readWithShortLength(in); -ByteBuffer stop = ByteBufferUtil.readWithShortLength(in); +Composite start = metadata.comparator.serializer().deserialize(in); +Composite stop = metadata.comparator.serializer().deserialize(in);
[jira] [Commented] (CASSANDRA-9318) Bound the number of in-flight requests at the coordinator
[ https://issues.apache.org/jira/browse/CASSANDRA-9318?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532930#comment-14532930 ] Jonathan Ellis commented on CASSANDRA-9318: --- I'm sure someone out there has made C* OOM with huge read requests but it's almost always with writes that I see the problem in the wild. So I'm totally fine with adding a coordinator bound on writes for 2.1 while recognizing that it doesn't help much for reads. That will definitely be better than nothing, and we can revisit for 3.x with something more sophisticated. Bound the number of in-flight requests at the coordinator - Key: CASSANDRA-9318 URL: https://issues.apache.org/jira/browse/CASSANDRA-9318 Project: Cassandra Issue Type: Improvement Reporter: Ariel Weisberg Assignee: Ariel Weisberg Fix For: 3.x It's possible to somewhat bound the amount of load accepted into the cluster by bounding the number of in-flight requests and request bytes. An implementation might do something like track the number of outstanding bytes and requests and if it reaches a high watermark disable read on client connections until it goes back below some low watermark. Need to make sure that disabling read on the client connection won't introduce other issues. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9325) cassandra-stress requires keystore for SSL but provides no way to configure it
[ https://issues.apache.org/jira/browse/CASSANDRA-9325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9325: --- Fix Version/s: 2.1.x cassandra-stress requires keystore for SSL but provides no way to configure it -- Key: CASSANDRA-9325 URL: https://issues.apache.org/jira/browse/CASSANDRA-9325 Project: Cassandra Issue Type: Bug Reporter: J.B. Langston Fix For: 2.1.x Even though it shouldn't be required unless client certificate authentication is enabled, the stress tool is looking for a keystore in the default location of conf/.keystore with the default password of cassandra. There is no command line option to override these defaults so you have to provide a keystore that satisfies the default. It looks for conf/.keystore in the working directory, so you need to create this in the directory you are running cassandra-stress from.It doesn't really matter what's in the keystore; it just needs to exist in the expected location and have a password of cassandra. Since the keystore might be required if client certificate authentication is enabled, we need to add -transport parameters for keystore and keystore-password. Ideally, these should be optional and stress shouldn't require the keystore unless client certificate authentication is enabled on the server. In case it wasn't apparent, this is for Cassandra 2.1 and later's stress tool. I actually had even more problems getting Cassandra 2.0's stress tool working with SSL and gave up on it. We probably don't need to fix 2.0; we can just document that it doesn't support SSL and recommend using 2.1 instead. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9322) Possible overlap with LCS and including non-compacting sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-9322?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Yuki Morishita updated CASSANDRA-9322: -- Reviewer: Yuki Morishita Possible overlap with LCS and including non-compacting sstables --- Key: CASSANDRA-9322 URL: https://issues.apache.org/jira/browse/CASSANDRA-9322 Project: Cassandra Issue Type: Improvement Reporter: Marcus Eriksson Assignee: Marcus Eriksson Fix For: 2.0.x Attachments: 0001-9322.patch since CASSANDRA-7414 we are including high-level sstables in lower level compactions if we have not run compactions in the high level for a while. If the compaction candidates only contain a single partition this can cause overlap since first token in sstables == last token in sstables which we interpret as being entire ring. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9325) cassandra-stress requires keystore but provides no way to configure it
J.B. Langston created CASSANDRA-9325: Summary: cassandra-stress requires keystore but provides no way to configure it Key: CASSANDRA-9325 URL: https://issues.apache.org/jira/browse/CASSANDRA-9325 Project: Cassandra Issue Type: Bug Reporter: J.B. Langston Even though it shouldn't be required unless client certificate authentication is enabled, the stress tool is looking for a keystore in the default location of conf/.keystore with the default password of cassandra. There is no command line option to override these defaults so you have to provide a keystore that satisfies the default. It looks for conf/.keystore in the working directory, so you need to create this in the directory you are running cassandra-stress from.It doesn't really matter what's in the keystore; it just needs to exist in the expected location and have a password of cassandra. Since the keystore might be required if client certificate authentication is enabled, we need to add -transport parameters for keystore and keystore-password. These should be optional unless client certificate authentication is enabled on the server. In case it wasn't apparent, this is for Cassandra 2.1 and later's stress tool. I actually had even more problems getting Cassandra 2.0's stress tool working with SSL and gave up on it. We probably don't need to fix 2.0; we can just document that it doesn't support SSL and recommend using 2.1 instead. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9324) Map Mutation rejected by Cassandra: IllegalArgumentException
[ https://issues.apache.org/jira/browse/CASSANDRA-9324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9324: --- Description: We use a collection (mapascii,ascii) in a CQL3 table. We write into that cql3 table using thrift mutations, from a c++ application. We are prototyping migrating from our current Cassandra (2.0.7) to 2.1.5, and are unable to write rows to this cql3 table. We have no problems when we remove the writes to the map column, and all other writes succeed in this case. Cassandra is rejecting our writes and we are catching a TTransportException (no more data to read). The below call stack is from the Cassandra instance that is rejecting the write. {code} ERROR 14:08:10 Error occurred during processing of message. java.lang.IllegalArgumentException: null at java.nio.Buffer.limit(Unknown Source) ~[na:1.7.0_71] at org.apache.cassandra.utils.ByteBufferUtil.readBytes(ByteBufferUtil.java:543) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.readValue(CollectionSerializer.java:124) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.MapSerializer.validateForNativeProtocol(MapSerializer.java:80) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.validate(CollectionSerializer.java:61) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:97) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnData(ThriftValidation.java:449) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnOrSuperColumn(ThriftValidation.java:318) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateMutation(ThriftValidation.java:385) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.CassandraServer.createMutationList(CassandraServer.java:861) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:976) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3996) ~[apache-cassandra-thrift-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3980) ~[apache-cassandra-thrift-2.1.5.jar:2.1.5] at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[libthrift-0.9.2.jar:0.9.2] at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[libthrift-0.9.2.jar:0.9.2] at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205) ~[apache-cassandra-2.1.5.jar:2.1.5] at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) [na:1.7.0_71] at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) [na:1.7.0_71] at java.lang.Thread.run(Unknown Source) [na:1.7.0_71]{code} was: We use a collection (mapascii,ascii) in a CQL3 table. We write into that cql3 table using thrift mutations, from a c++ application. We are prototyping migrating from our current Cassandra (2.0.7) to 2.1.5, and are unable to write rows to this cql3 table. We have no problems when we remove the writes to the map column, and all other writes succeed in this case. Cassandra is rejecting our writes and we are catching a TTransportException (no more data to read). The below call stack is from the Cassandra instance that is rejecting the write. ERROR 14:08:10 Error occurred during processing of message. java.lang.IllegalArgumentException: null at java.nio.Buffer.limit(Unknown Source) ~[na:1.7.0_71] at org.apache.cassandra.utils.ByteBufferUtil.readBytes(ByteBufferUtil.java:543) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.readValue(CollectionSerializer.java:124) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.MapSerializer.validateForNativeProtocol(MapSerializer.java:80) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.validate(CollectionSerializer.java:61) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:97) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnData(ThriftValidation.java:449) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnOrSuperColumn(ThriftValidation.java:318) ~[apache-cassandra-2.1.5.jar:2.1.5] at
[jira] [Updated] (CASSANDRA-9324) Map Mutation rejected by Cassandra: IllegalArgumentException
[ https://issues.apache.org/jira/browse/CASSANDRA-9324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9324: --- Fix Version/s: 2.1.x Map Mutation rejected by Cassandra: IllegalArgumentException Key: CASSANDRA-9324 URL: https://issues.apache.org/jira/browse/CASSANDRA-9324 Project: Cassandra Issue Type: Bug Components: API Environment: Windows 7, Cassandra 2.1.5 Reporter: Mark Wick Priority: Minor Fix For: 2.1.x We use a collection (mapascii,ascii) in a CQL3 table. We write into that cql3 table using thrift mutations, from a c++ application. We are prototyping migrating from our current Cassandra (2.0.7) to 2.1.5, and are unable to write rows to this cql3 table. We have no problems when we remove the writes to the map column, and all other writes succeed in this case. Cassandra is rejecting our writes and we are catching a TTransportException (no more data to read). The below call stack is from the Cassandra instance that is rejecting the write. {code} ERROR 14:08:10 Error occurred during processing of message. java.lang.IllegalArgumentException: null at java.nio.Buffer.limit(Unknown Source) ~[na:1.7.0_71] at org.apache.cassandra.utils.ByteBufferUtil.readBytes(ByteBufferUtil.java:543) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.readValue(CollectionSerializer.java:124) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.MapSerializer.validateForNativeProtocol(MapSerializer.java:80) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.validate(CollectionSerializer.java:61) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:97) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnData(ThriftValidation.java:449) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnOrSuperColumn(ThriftValidation.java:318) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateMutation(ThriftValidation.java:385) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.CassandraServer.createMutationList(CassandraServer.java:861) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:976) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3996) ~[apache-cassandra-thrift-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3980) ~[apache-cassandra-thrift-2.1.5.jar:2.1.5] at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[libthrift-0.9.2.jar:0.9.2] at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[libthrift-0.9.2.jar:0.9.2] at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205) ~[apache-cassandra-2.1.5.jar:2.1.5] at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) [na:1.7.0_71] at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) [na:1.7.0_71] at java.lang.Thread.run(Unknown Source) [na:1.7.0_71]{code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-9323) Bulk upload is slow
Pierre N. created CASSANDRA-9323: Summary: Bulk upload is slow Key: CASSANDRA-9323 URL: https://issues.apache.org/jira/browse/CASSANDRA-9323 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Attachments: App.java Hi, When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-9323) Bulk loading is slow
[ https://issues.apache.org/jira/browse/CASSANDRA-9323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Pierre N. updated CASSANDRA-9323: - Description: When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable using sstableloader I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserialize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? was: When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable using sstableloader I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserailize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? Bulk loading is slow Key: CASSANDRA-9323 URL: https://issues.apache.org/jira/browse/CASSANDRA-9323 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Attachments: App.java When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable using sstableloader I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserialize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9295) Streaming not holding on to refs long enough.
[ https://issues.apache.org/jira/browse/CASSANDRA-9295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532797#comment-14532797 ] Joshua McKenzie commented on CASSANDRA-9295: +1 for streaming complete files only. I think the risks associated with doing so (as illustrated here) outweigh potential benefits. Streaming not holding on to refs long enough. - Key: CASSANDRA-9295 URL: https://issues.apache.org/jira/browse/CASSANDRA-9295 Project: Cassandra Issue Type: Bug Reporter: Jeremiah Jordan Assignee: Yuki Morishita Fix For: 2.1.x Attachments: 0001-don-t-release-sstable-while-streaming.patch, 0002-demonstrate-checksum-failure-of-early-opened-SSTable.patch, 0003-skip-validation-of-early-opened-SSTable.patch, 9295.debug.txt While doing some testing around adding/removing nodes under load with cassandra-2.1 head as of a few days ago (after was 2.1.5 tagged) I am seeing stream out errors with file not found exceptions. The file in question just finished being compacted into a new file a few lines earlier in the log. Seems that streaming isn't holding onto Ref's correctly for the stuff in the stream plans. I also see a corrupt sstable exception for the file the missing file was compacted to. Trimmed logs with just the compaction/streaming related stuff: You can see the stream plan is initiated in between the compaction starting, and the compaction finishing. {noformat} INFO [MemtableFlushWriter:3] 2015-05-04 16:08:21,239 Memtable.java:380 - Completed flushing /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db (60666088 bytes) for commitlog position ReplayPosition(segmentId=1430755416941, position=32294797) INFO [CompactionExecutor:4] 2015-05-04 16:08:40,856 CompactionTask.java:140 - Compacting [SSTableReader(path='/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-4-Data.db'), SSTableReader(path='/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-3-Data.db')] INFO [STREAM-INIT-/10.240.213.56:53190] 2015-05-04 16:09:31,047 StreamResultFuture.java:109 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9 ID#0] Creating new streaming plan for Rebuild INFO [STREAM-INIT-/10.240.213.56:53190] 2015-05-04 16:09:31,238 StreamResultFuture.java:116 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9, ID#0] Received streaming plan for Rebuild INFO [STREAM-INIT-/10.240.213.56:53192] 2015-05-04 16:09:31,249 StreamResultFuture.java:116 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9, ID#0] Received streaming plan for Rebuild INFO [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:31,353 ColumnFamilyStore.java:882 - Enqueuing flush of standard1: 91768068 (19%) on-heap, 0 (0%) off-heap INFO [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:37,425 ColumnFamilyStore.java:882 - Enqueuing flush of solr: 10012689 (2%) on-heap, 0 (0%) off-heap INFO [STREAM-IN-/10.240.213.56] 2015-05-04 16:09:38,073 StreamResultFuture.java:166 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9 ID#0] Prepare completed. Receiving 0 files(0 bytes), sending 6 files(284288285 bytes) INFO [CompactionExecutor:4] 2015-05-04 16:10:11,047 CompactionTask.java:270 - Compacted 2 sstables to [/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-5,/mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-8,]. 182,162,816 bytes to 182,162,816 (~100% of original) in 90,188ms = 1.926243MB/s. 339,856 total partitions merged to 339,856. Partition merge counts were {1:339856, } ERROR [STREAM-OUT-/10.240.213.56] 2015-05-04 16:10:25,169 StreamSession.java:477 - [Stream #f261c040-f277-11e4-a070-d126f0416bc9] Streaming error occurred java.io.IOException: Corrupted SSTable : /mnt/cass_data_disks/data1/keyspace1/standard1-49f17b30f27711e4a438775021e2cd7f/keyspace1-standard1-ka-5-Data.db at org.apache.cassandra.io.util.DataIntegrityMetadata$ChecksumValidator.validate(DataIntegrityMetadata.java:79) ~[cassandra-all-2.1.5.426.jar:2.1.5.426] at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:149) ~[cassandra-all-2.1.5.426.jar:2.1.5.426] at org.apache.cassandra.streaming.StreamWriter.write(StreamWriter.java:102) ~[cassandra-all-2.1.5.426.jar:2.1.5.426] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:58) ~[cassandra-all-2.1.5.426.jar:2.1.5.426] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:42)
[jira] [Updated] (CASSANDRA-9323) Bulk loading is slow
[ https://issues.apache.org/jira/browse/CASSANDRA-9323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Philip Thompson updated CASSANDRA-9323: --- Fix Version/s: 2.1.x Bulk loading is slow Key: CASSANDRA-9323 URL: https://issues.apache.org/jira/browse/CASSANDRA-9323 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Fix For: 2.1.x Attachments: App.java When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable using sstableloader I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserialize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[1/2] cassandra git commit: Log warning on unrecognized table during message deser
Repository: cassandra Updated Branches: refs/heads/cassandra-2.1 2c8e810be - 049a965bf Log warning on unrecognized table during message deser Patch by Tyler Hobbs; reviewed by Sylvain Lebresne for CASSANDRA-9136 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/58de86bf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/58de86bf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/58de86bf Branch: refs/heads/cassandra-2.1 Commit: 58de86bff5e9383b679d9bb48cc95ec444275ca7 Parents: 6ab0c30 Author: Tyler Hobbs tylerho...@apache.org Authored: Thu May 7 10:30:01 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Thu May 7 10:30:01 2015 -0500 -- CHANGES.txt | 2 ++ src/java/org/apache/cassandra/db/PagedRangeCommand.java | 10 ++ src/java/org/apache/cassandra/db/RangeSliceCommand.java | 7 +++ .../org/apache/cassandra/db/SliceByNamesReadCommand.java | 8 .../org/apache/cassandra/db/SliceFromReadCommand.java | 8 5 files changed, 35 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d76606c..8473777 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 2.0.15: + * Log warning message when a table is queried before the schema has fully + propagated (CASSANDRA-9136) * Overload SecondaryIndex#indexes to accept the column definition (CASSANDRA-9314) * (cqlsh) Add SERIAL and LOCAL_SERIAL consistency levels (CASSANDRA-8051) * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281) http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/PagedRangeCommand.java -- diff --git a/src/java/org/apache/cassandra/db/PagedRangeCommand.java b/src/java/org/apache/cassandra/db/PagedRangeCommand.java index a8d7f49..3bc6539 100644 --- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java +++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import org.apache.cassandra.config.Schema; import org.apache.cassandra.db.filter.*; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.io.IVersionedSerializer; @@ -153,6 +154,15 @@ public class PagedRangeCommand extends AbstractRangeCommand { String keyspace = in.readUTF(); String columnFamily = in.readUTF(); + +if (Schema.instance.getCFMetaData(keyspace, columnFamily) == null) +{ +String message = String.format(Got paged range command for nonexistent table %s.%s. If the table was just + +created, this is likely due to the schema not being fully propagated. Please wait for schema + +agreement on table creation. , keyspace, columnFamily); +throw new UnknownColumnFamilyException(message, null); +} + long timestamp = in.readLong(); AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/RangeSliceCommand.java -- diff --git a/src/java/org/apache/cassandra/db/RangeSliceCommand.java b/src/java/org/apache/cassandra/db/RangeSliceCommand.java index 4aa1595..553f111 100644 --- a/src/java/org/apache/cassandra/db/RangeSliceCommand.java +++ b/src/java/org/apache/cassandra/db/RangeSliceCommand.java @@ -213,6 +213,13 @@ class RangeSliceCommandSerializer implements IVersionedSerializerRangeSliceComm long timestamp = version MessagingService.VERSION_20 ? System.currentTimeMillis() : in.readLong(); CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily); +if (metadata == null) +{ +String message = String.format(Got range slice command for nonexistent table %s.%s. If the table was just + +created, this is likely due to the schema not being fully propagated. Please wait for schema + +agreement on table creation. , keyspace, columnFamily); +throw new UnknownColumnFamilyException(message, null); +} IDiskAtomFilter predicate; if (version MessagingService.VERSION_20) http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java
[jira] [Created] (CASSANDRA-9324) Map Mutation rejected by Cassandra: IllegalArgumentException
Mark Wick created CASSANDRA-9324: Summary: Map Mutation rejected by Cassandra: IllegalArgumentException Key: CASSANDRA-9324 URL: https://issues.apache.org/jira/browse/CASSANDRA-9324 Project: Cassandra Issue Type: Bug Components: API Environment: Windows 7, Cassandra 2.1.5 Reporter: Mark Wick Priority: Minor We use a collection (mapascii,ascii) in a CQL3 table. We write into that cql3 table using thrift mutations, from a c++ application. We are prototyping migrating from our current Cassandra (2.0.7) to 2.1.5, and are unable to write rows to this cql3 table. We have no problems when we remove the writes to the map column, and all other writes succeed in this case. Cassandra is rejecting our writes and we are catching a TTransportException (no more data to read). The below call stack is from the Cassandra instance that is rejecting the write. ERROR 14:08:10 Error occurred during processing of message. java.lang.IllegalArgumentException: null at java.nio.Buffer.limit(Unknown Source) ~[na:1.7.0_71] at org.apache.cassandra.utils.ByteBufferUtil.readBytes(ByteBufferUtil.java:543) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.readValue(CollectionSerializer.java:124) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.MapSerializer.validateForNativeProtocol(MapSerializer.java:80) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.serializers.CollectionSerializer.validate(CollectionSerializer.java:61) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.db.marshal.AbstractType.validate(AbstractType.java:97) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnData(ThriftValidation.java:449) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateColumnOrSuperColumn(ThriftValidation.java:318) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.ThriftValidation.validateMutation(ThriftValidation.java:385) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.CassandraServer.createMutationList(CassandraServer.java:861) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.CassandraServer.batch_mutate(CassandraServer.java:976) ~[apache-cassandra-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3996) ~[apache-cassandra-thrift-2.1.5.jar:2.1.5] at org.apache.cassandra.thrift.Cassandra$Processor$batch_mutate.getResult(Cassandra.java:3980) ~[apache-cassandra-thrift-2.1.5.jar:2.1.5] at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:39) ~[libthrift-0.9.2.jar:0.9.2] at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39) ~[libthrift-0.9.2.jar:0.9.2] at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:205) ~[apache-cassandra-2.1.5.jar:2.1.5] at java.util.concurrent.ThreadPoolExecutor.runWorker(Unknown Source) [na:1.7.0_71] at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source) [na:1.7.0_71] at java.lang.Thread.run(Unknown Source) [na:1.7.0_71] -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[1/3] cassandra git commit: Log warning on unrecognized table during message deser
Repository: cassandra Updated Branches: refs/heads/trunk 7954965eb - 6d5b0b45a Log warning on unrecognized table during message deser Patch by Tyler Hobbs; reviewed by Sylvain Lebresne for CASSANDRA-9136 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/58de86bf Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/58de86bf Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/58de86bf Branch: refs/heads/trunk Commit: 58de86bff5e9383b679d9bb48cc95ec444275ca7 Parents: 6ab0c30 Author: Tyler Hobbs tylerho...@apache.org Authored: Thu May 7 10:30:01 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Thu May 7 10:30:01 2015 -0500 -- CHANGES.txt | 2 ++ src/java/org/apache/cassandra/db/PagedRangeCommand.java | 10 ++ src/java/org/apache/cassandra/db/RangeSliceCommand.java | 7 +++ .../org/apache/cassandra/db/SliceByNamesReadCommand.java | 8 .../org/apache/cassandra/db/SliceFromReadCommand.java | 8 5 files changed, 35 insertions(+) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/CHANGES.txt -- diff --git a/CHANGES.txt b/CHANGES.txt index d76606c..8473777 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,6 @@ 2.0.15: + * Log warning message when a table is queried before the schema has fully + propagated (CASSANDRA-9136) * Overload SecondaryIndex#indexes to accept the column definition (CASSANDRA-9314) * (cqlsh) Add SERIAL and LOCAL_SERIAL consistency levels (CASSANDRA-8051) * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281) http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/PagedRangeCommand.java -- diff --git a/src/java/org/apache/cassandra/db/PagedRangeCommand.java b/src/java/org/apache/cassandra/db/PagedRangeCommand.java index a8d7f49..3bc6539 100644 --- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java +++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java @@ -24,6 +24,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import org.apache.cassandra.config.Schema; import org.apache.cassandra.db.filter.*; import org.apache.cassandra.dht.AbstractBounds; import org.apache.cassandra.io.IVersionedSerializer; @@ -153,6 +154,15 @@ public class PagedRangeCommand extends AbstractRangeCommand { String keyspace = in.readUTF(); String columnFamily = in.readUTF(); + +if (Schema.instance.getCFMetaData(keyspace, columnFamily) == null) +{ +String message = String.format(Got paged range command for nonexistent table %s.%s. If the table was just + +created, this is likely due to the schema not being fully propagated. Please wait for schema + +agreement on table creation. , keyspace, columnFamily); +throw new UnknownColumnFamilyException(message, null); +} + long timestamp = in.readLong(); AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/RangeSliceCommand.java -- diff --git a/src/java/org/apache/cassandra/db/RangeSliceCommand.java b/src/java/org/apache/cassandra/db/RangeSliceCommand.java index 4aa1595..553f111 100644 --- a/src/java/org/apache/cassandra/db/RangeSliceCommand.java +++ b/src/java/org/apache/cassandra/db/RangeSliceCommand.java @@ -213,6 +213,13 @@ class RangeSliceCommandSerializer implements IVersionedSerializerRangeSliceComm long timestamp = version MessagingService.VERSION_20 ? System.currentTimeMillis() : in.readLong(); CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily); +if (metadata == null) +{ +String message = String.format(Got range slice command for nonexistent table %s.%s. If the table was just + +created, this is likely due to the schema not being fully propagated. Please wait for schema + +agreement on table creation. , keyspace, columnFamily); +throw new UnknownColumnFamilyException(message, null); +} IDiskAtomFilter predicate; if (version MessagingService.VERSION_20) http://git-wip-us.apache.org/repos/asf/cassandra/blob/58de86bf/src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java
[2/3] cassandra git commit: Merge branch 'cassandra-2.0' into cassandra-2.1
Merge branch 'cassandra-2.0' into cassandra-2.1 Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/049a965b Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/049a965b Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/049a965b Branch: refs/heads/trunk Commit: 049a965bf1e57d30cac13141bdf228268765ebeb Parents: 2c8e810 58de86b Author: Tyler Hobbs tylerho...@apache.org Authored: Thu May 7 10:31:52 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Thu May 7 10:31:52 2015 -0500 -- CHANGES.txt | 3 ++- src/java/org/apache/cassandra/db/PagedRangeCommand.java | 7 +++ src/java/org/apache/cassandra/db/RangeSliceCommand.java | 7 +++ src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java | 7 +++ src/java/org/apache/cassandra/db/SliceFromReadCommand.java| 7 +++ 5 files changed, 30 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/049a965b/CHANGES.txt -- diff --cc CHANGES.txt index 41e0340,8473777..d512973 --- a/CHANGES.txt +++ b/CHANGES.txt @@@ -1,21 -1,6 +1,22 @@@ -2.0.15: +2.1.6 + * Improve sstable exclusion from partition tombstones (CASSANDRA-9298) + * Validate the indexed column rather than the cell's contents for 2i (CASSANDRA-9057) + * Add support for top-k custom 2i queries (CASSANDRA-8717) + * Fix error when dropping table during compaction (CASSANDRA-9251) + * cassandra-stress supports validation operations over user profiles (CASSANDRA-8773) + * Add support for rate limiting log messages (CASSANDRA-9029) + * Log the partition key with tombstone warnings (CASSANDRA-8561) + * Reduce runWithCompactionsDisabled poll interval to 1ms (CASSANDRA-9271) + * Fix PITR commitlog replay (CASSANDRA-9195) + * GCInspector logs very different times (CASSANDRA-9124) + * Fix deleting from an empty list (CASSANDRA-9198) + * Update tuple and collection types that use a user-defined type when that UDT + is modified (CASSANDRA-9148, CASSANDRA-9192) + * Use higher timeout for prepair and snapshot in repair (CASSANDRA-9261) + * Fix anticompaction blocking ANTI_ENTROPY stage (CASSANDRA-9151) +Merged from 2.0: - 2.0.15: + * Log warning message when a table is queried before the schema has fully +propagated (CASSANDRA-9136) * Overload SecondaryIndex#indexes to accept the column definition (CASSANDRA-9314) * (cqlsh) Add SERIAL and LOCAL_SERIAL consistency levels (CASSANDRA-8051) * Fix index selection during rebuild with certain table layouts (CASSANDRA-9281) http://git-wip-us.apache.org/repos/asf/cassandra/blob/049a965b/src/java/org/apache/cassandra/db/PagedRangeCommand.java -- diff --cc src/java/org/apache/cassandra/db/PagedRangeCommand.java index 614f0f7,3bc6539..70325a8 --- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java +++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java @@@ -156,16 -154,23 +156,23 @@@ public class PagedRangeCommand extends { String keyspace = in.readUTF(); String columnFamily = in.readUTF(); +long timestamp = in.readLong(); -if (Schema.instance.getCFMetaData(keyspace, columnFamily) == null) +AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); + +CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily); ++if (metadata == null) + { + String message = String.format(Got paged range command for nonexistent table %s.%s. If the table was just + + created, this is likely due to the schema not being fully propagated. Please wait for schema + + agreement on table creation. , keyspace, columnFamily); + throw new UnknownColumnFamilyException(message, null); + } -long timestamp = in.readLong(); +SliceQueryFilter predicate = metadata.comparator.sliceQueryFilterSerializer().deserialize(in, version); -AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); - -SliceQueryFilter predicate = SliceQueryFilter.serializer.deserialize(in, version); - -ByteBuffer start = ByteBufferUtil.readWithShortLength(in); -ByteBuffer stop = ByteBufferUtil.readWithShortLength(in); +Composite start = metadata.comparator.serializer().deserialize(in); +Composite stop = metadata.comparator.serializer().deserialize(in); int
[3/3] cassandra git commit: Merge branch 'cassandra-2.1' into trunk
Merge branch 'cassandra-2.1' into trunk Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/6d5b0b45 Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/6d5b0b45 Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/6d5b0b45 Branch: refs/heads/trunk Commit: 6d5b0b45ad86be7568ab15dc61c75fa50143082a Parents: 7954965 049a965 Author: Tyler Hobbs tylerho...@apache.org Authored: Thu May 7 10:32:26 2015 -0500 Committer: Tyler Hobbs tylerho...@apache.org Committed: Thu May 7 10:32:26 2015 -0500 -- CHANGES.txt | 3 ++- src/java/org/apache/cassandra/db/PagedRangeCommand.java | 7 +++ src/java/org/apache/cassandra/db/RangeSliceCommand.java | 7 +++ src/java/org/apache/cassandra/db/SliceByNamesReadCommand.java | 7 +++ src/java/org/apache/cassandra/db/SliceFromReadCommand.java| 7 +++ 5 files changed, 30 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d5b0b45/CHANGES.txt -- http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d5b0b45/src/java/org/apache/cassandra/db/PagedRangeCommand.java -- diff --cc src/java/org/apache/cassandra/db/PagedRangeCommand.java index ab988f1,70325a8..402e711 --- a/src/java/org/apache/cassandra/db/PagedRangeCommand.java +++ b/src/java/org/apache/cassandra/db/PagedRangeCommand.java @@@ -159,10 -158,16 +159,17 @@@ public class PagedRangeCommand extends String columnFamily = in.readUTF(); long timestamp = in.readLong(); -AbstractBoundsRowPosition keyRange = AbstractBounds.serializer.deserialize(in, version).toRowBounds(); +AbstractBoundsRowPosition keyRange = +AbstractBounds.rowPositionSerializer.deserialize(in, MessagingService.globalPartitioner(), version); CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily); + if (metadata == null) + { + String message = String.format(Got paged range command for nonexistent table %s.%s. If the table was just + + created, this is likely due to the schema not being fully propagated. Please wait for schema + + agreement on table creation. , keyspace, columnFamily); + throw new UnknownColumnFamilyException(message, null); + } SliceQueryFilter predicate = metadata.comparator.sliceQueryFilterSerializer().deserialize(in, version); http://git-wip-us.apache.org/repos/asf/cassandra/blob/6d5b0b45/src/java/org/apache/cassandra/db/RangeSliceCommand.java --
[jira] [Updated] (CASSANDRA-9325) cassandra-stress requires keystore for SSL but provides no way to configure it
[ https://issues.apache.org/jira/browse/CASSANDRA-9325?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] J.B. Langston updated CASSANDRA-9325: - Description: Even though it shouldn't be required unless client certificate authentication is enabled, the stress tool is looking for a keystore in the default location of conf/.keystore with the default password of cassandra. There is no command line option to override these defaults so you have to provide a keystore that satisfies the default. It looks for conf/.keystore in the working directory, so you need to create this in the directory you are running cassandra-stress from.It doesn't really matter what's in the keystore; it just needs to exist in the expected location and have a password of cassandra. Since the keystore might be required if client certificate authentication is enabled, we need to add -transport parameters for keystore and keystore-password. Ideally, these should be optional and stress shouldn't require the keystore unless client certificate authentication is enabled on the server. In case it wasn't apparent, this is for Cassandra 2.1 and later's stress tool. I actually had even more problems getting Cassandra 2.0's stress tool working with SSL and gave up on it. We probably don't need to fix 2.0; we can just document that it doesn't support SSL and recommend using 2.1 instead. was: Even though it shouldn't be required unless client certificate authentication is enabled, the stress tool is looking for a keystore in the default location of conf/.keystore with the default password of cassandra. There is no command line option to override these defaults so you have to provide a keystore that satisfies the default. It looks for conf/.keystore in the working directory, so you need to create this in the directory you are running cassandra-stress from.It doesn't really matter what's in the keystore; it just needs to exist in the expected location and have a password of cassandra. Since the keystore might be required if client certificate authentication is enabled, we need to add -transport parameters for keystore and keystore-password. These should be optional unless client certificate authentication is enabled on the server. In case it wasn't apparent, this is for Cassandra 2.1 and later's stress tool. I actually had even more problems getting Cassandra 2.0's stress tool working with SSL and gave up on it. We probably don't need to fix 2.0; we can just document that it doesn't support SSL and recommend using 2.1 instead. cassandra-stress requires keystore for SSL but provides no way to configure it -- Key: CASSANDRA-9325 URL: https://issues.apache.org/jira/browse/CASSANDRA-9325 Project: Cassandra Issue Type: Bug Reporter: J.B. Langston Even though it shouldn't be required unless client certificate authentication is enabled, the stress tool is looking for a keystore in the default location of conf/.keystore with the default password of cassandra. There is no command line option to override these defaults so you have to provide a keystore that satisfies the default. It looks for conf/.keystore in the working directory, so you need to create this in the directory you are running cassandra-stress from.It doesn't really matter what's in the keystore; it just needs to exist in the expected location and have a password of cassandra. Since the keystore might be required if client certificate authentication is enabled, we need to add -transport parameters for keystore and keystore-password. Ideally, these should be optional and stress shouldn't require the keystore unless client certificate authentication is enabled on the server. In case it wasn't apparent, this is for Cassandra 2.1 and later's stress tool. I actually had even more problems getting Cassandra 2.0's stress tool working with SSL and gave up on it. We probably don't need to fix 2.0; we can just document that it doesn't support SSL and recommend using 2.1 instead. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9323) Bulk loading is slow
[ https://issues.apache.org/jira/browse/CASSANDRA-9323?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14532839#comment-14532839 ] T Jake Luciani commented on CASSANDRA-9323: --- bq. I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter It needs un-serialize the data to build the bloom-filters, index and other components. Bulk loading is slow Key: CASSANDRA-9323 URL: https://issues.apache.org/jira/browse/CASSANDRA-9323 Project: Cassandra Issue Type: Bug Reporter: Pierre N. Fix For: 2.1.x Attachments: App.java When I bulk upload sstable created with CQLSSTableWriter, it's very slow. I tested on a fresh cassandra node (nothing in keyspace, nor tables) with good hardware (8x2.8ghz, 32G ram), but with classic hard disk (performance won't be improved with SSD in this case I think). When I upload from a different server an sstable using sstableloader I get an average of 3 MB/sec, in the attached example I managed to get 5 MB/sec, which is still slow. During the streaming process I noticed that one core of the server is full CPU, so I think the operation is CPU bound server side. I quickly attached a sample profiler to the cassandra instance and got the following output : https://i.imgur.com/IfLc2Ip.png So, I think, but I may be wrong because it's inaccurate sampling, during streaming the table is unserialized and reserialized to another sstable, and that's this unserialize/serialize process which is taking a big amount of CPU, slowing down the insert speed. Can someone confirm the bulk load is slow ? I tested also on my computer and barely reach 1MB/sec I don't understand the point of totally unserializing the table I just did build using the CQLSStableWriter (because it's already a long process to build and sort the table), couldn't it just copy the table from offset X to offset Y (using index information by example) without unserializing/reserializing it ? -- This message was sent by Atlassian JIRA (v6.3.4#6332)