[jira] [Created] (CASSANDRA-13131) Update Cassandra service script to optionally protected it against the oom-killer
Edward Kuns created CASSANDRA-13131: --- Summary: Update Cassandra service script to optionally protected it against the oom-killer Key: CASSANDRA-13131 URL: https://issues.apache.org/jira/browse/CASSANDRA-13131 Project: Cassandra Issue Type: Improvement Components: Lifecycle Reporter: Edward Kuns Priority: Minor The Cassandra startup script for RedHat currently makes no attempt to optionally adjust the oom-killer tuning. It would be helpful if the startup scripts were modified to optionally echo a value to the special /proc file {{oom_score_adj}}. It could be triggered by an optional value that could be set in {{/etc/default/$NAME}}. Maybe something like the below where only a couple surrounding lines from {{/etc/init.d/cassandra}} are included for context: {noformat} [ $retval -eq 0 ] && touch $lock_file [ $retval -eq 0 -a ! -z "$OOM_SCORE_ADJ" ] && echo $OOM_SCORE_ADJ > /proc/$(cat $pid_file)/oom_score_adj echo "OK" {noformat} Of course, it could be cleaned up to avoid the duplication of checking the retval twice. This would make it trivial for those running Cassandra to protect it from the oom-killer, if they chose, by simply modifying one line of a configuration file, as opposed to modifying a script and re-modifying the script every time an upgrade is applied. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-13112) test failure in snitch_test.TestDynamicEndpointSnitch.test_multidatacenter_local_quorum
[ https://issues.apache.org/jira/browse/CASSANDRA-13112?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15827384#comment-15827384 ] Joel Knighton commented on CASSANDRA-13112: --- This should be a dtest only fix. I've PRed a fix at [https://github.com/riptano/cassandra-dtest/pull/1425]. > test failure in > snitch_test.TestDynamicEndpointSnitch.test_multidatacenter_local_quorum > --- > > Key: CASSANDRA-13112 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13112 > Project: Cassandra > Issue Type: Bug >Reporter: Sean McCarthy >Assignee: Joel Knighton > Labels: dtest, test-failure > Attachments: node1_debug.log, node1_gc.log, node1.log, > node2_debug.log, node2_gc.log, node2.log, node3_debug.log, node3_gc.log, > node3.log, node4_debug.log, node4_gc.log, node4.log, node5_debug.log, > node5_gc.log, node5.log, node6_debug.log, node6_gc.log, node6.log > > > example failure: > http://cassci.datastax.com/job/trunk_large_dtest/48/testReport/snitch_test/TestDynamicEndpointSnitch/test_multidatacenter_local_quorum > {code} > Error Message > 75 != 76 > {code}{code} > Stacktrace > File "/usr/lib/python2.7/unittest/case.py", line 329, in run > testMethod() > File "/home/automaton/cassandra-dtest/tools/decorators.py", line 48, in > wrapped > f(obj) > File "/home/automaton/cassandra-dtest/snitch_test.py", line 168, in > test_multidatacenter_local_quorum > bad_jmx.read_attribute(read_stage, 'Value')) > File "/usr/lib/python2.7/unittest/case.py", line 513, in assertEqual > assertion_func(first, second, msg=msg) > File "/usr/lib/python2.7/unittest/case.py", line 506, in _baseAssertEqual > raise self.failureException(msg) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-13115) Read repair is not blocking repair to finish in foreground repair
[ https://issues.apache.org/jira/browse/CASSANDRA-13115?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826926#comment-15826926 ] Xiaolong Jiang commented on CASSANDRA-13115: [~slebresne]Thanks for fixing this. I went through both changes and it looks good to me. Can you please merge both patches? Thanks!! > Read repair is not blocking repair to finish in foreground repair > - > > Key: CASSANDRA-13115 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13115 > Project: Cassandra > Issue Type: Bug > Environment: ccm on OSX >Reporter: Xiaolong Jiang >Assignee: Sylvain Lebresne > Fix For: 3.0.x, 3.x > > > The code trying to wait(block) for repair result to come back in 3.X is below: > {code:title= DataResolver.java|borderStyle=solid} > public void close() > { > try > { > FBUtilities.waitOnFutures(repairResults, > DatabaseDescriptor.getWriteRpcTimeout()); > } > catch (TimeoutException ex) > { > // We got all responses, but timed out while repairing > int blockFor = consistency.blockFor(keyspace); > if (Tracing.isTracing()) > Tracing.trace("Timed out while read-repairing after > receiving all {} data and digest responses", blockFor); > else > logger.debug("Timeout while read-repairing after > receiving all {} data and digest responses", blockFor); > throw new ReadTimeoutException(consistency, blockFor-1, > blockFor, true); > } > } > {code} > in DataResolver class, but this close method is never called and it's also > not auto close(RepairMergeListener is not extending from > AutoCloseable/CloseableIterator) which means we never wait for repair to > finish before returning final result. > The steps to reproduce: > 1. create some keyspace/table with RF = 2 > 2. start 2 nodes using ccm > 3. stop node2 > 4. disable node1 hinted hand off > 5. write some data to node1 with consistency level one > 6. start node2 > 7. query some data from node1 > This should trigger read repair. I put some log in above close method, and > can not see log print put. > So this bug will basically violate "monotonic quorum reads " guarantee. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-13126) native transport protocol corruption when using SSL
[ https://issues.apache.org/jira/browse/CASSANDRA-13126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826879#comment-15826879 ] Tom van der Woerdt edited comment on CASSANDRA-13126 at 1/17/17 9:43 PM: - Apparently I do! {code} ERROR [SharedPool-Worker-1] 2017-01-11 15:26:59,533 Message.java:617 - Unexpected exception during request; channel = [id: 0xc259e8df, /1.2.3.4:45232 => /5.6.7.8:9042] io.netty.handler.codec.DecoderException: java.lang.OutOfMemoryError: Direct buffer memory at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:153) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollSocketChannel$EpollSocketUnsafe.epollInReady(EpollSocketChannel.java:722) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:326) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:264) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_112] Caused by: java.lang.OutOfMemoryError: Direct buffer memory at java.nio.Bits.reserveMemory(Bits.java:693) ~[na:1.8.0_112] at java.nio.DirectByteBuffer.(DirectByteBuffer.java:123) ~[na:1.8.0_112] at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:311) ~[na:1.8.0_112] at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:434) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:179) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PoolArena.allocate(PoolArena.java:168) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PoolArena.reallocate(PoolArena.java:277) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PooledByteBuf.capacity(PooledByteBuf.java:108) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.ensureWritable(AbstractByteBuf.java:251) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:849) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:841) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:831) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:146) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] ... 9 common frames omitted {code} was (Author: tvdw): Apparently I do! {code} at java.lang.Thread.run(Thread.java:745) [na:1.8.0_112] ERROR [SharedPool-Worker-1] 2017-01-11 15:26:59,533 Message.java:617 - Unexpected exception during request; channel = [id: 0xc259e8df, /1.2.3.4:45232 => /5.6.7.8:9042] io.netty.handler.codec.DecoderException: java.lang.OutOfMemoryError: Direct buffer memory at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:153) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollSocketChannel$EpollSocketUnsafe.epollInReady(EpollSocketChannel.java:722) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:326) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:264) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at
[jira] [Commented] (CASSANDRA-13126) native transport protocol corruption when using SSL
[ https://issues.apache.org/jira/browse/CASSANDRA-13126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826879#comment-15826879 ] Tom van der Woerdt commented on CASSANDRA-13126: Apparently I do! {code} at java.lang.Thread.run(Thread.java:745) [na:1.8.0_112] ERROR [SharedPool-Worker-1] 2017-01-11 15:26:59,533 Message.java:617 - Unexpected exception during request; channel = [id: 0xc259e8df, /1.2.3.4:45232 => /5.6.7.8:9042] io.netty.handler.codec.DecoderException: java.lang.OutOfMemoryError: Direct buffer memory at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:153) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:333) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:319) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:787) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollSocketChannel$EpollSocketUnsafe.epollInReady(EpollSocketChannel.java:722) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:326) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:264) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:116) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_112] Caused by: java.lang.OutOfMemoryError: Direct buffer memory at java.nio.Bits.reserveMemory(Bits.java:693) ~[na:1.8.0_112] at java.nio.DirectByteBuffer.(DirectByteBuffer.java:123) ~[na:1.8.0_112] at java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:311) ~[na:1.8.0_112] at io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:434) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:179) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PoolArena.allocate(PoolArena.java:168) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PoolArena.reallocate(PoolArena.java:277) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.PooledByteBuf.capacity(PooledByteBuf.java:108) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.ensureWritable(AbstractByteBuf.java:251) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:849) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:841) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:831) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:146) ~[netty-all-4.0.23.Final.jar:4.0.23.Final] ... 9 common frames omitted {code} > native transport protocol corruption when using SSL > --- > > Key: CASSANDRA-13126 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13126 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tom van der Woerdt >Priority: Critical > > This is a series of conditions that can result in client connections becoming > unusable. > 1) Cassandra GC must be well-tuned, to have short GC pauses every minute or so > 2) *client* SSL must be enabled and transmitting a significant amount of data > 3) Cassandra must run with the default library versions > 4) disableexplicitgc must be set (this is the default in the current > cassandra-env.sh) > This ticket relates to CASSANDRA-13114 which is a possible workaround (but > not a fix) for the SSL requirement to trigger this bug. > * Netty allocates nio.ByteBuffers for every outgoing SSL message. > * ByteBuffers consist of two parts, the jvm object and the off-heap object. > The jvm object is small and goes with regular GC cycles, the off-heap object > gets freed only when the small jvm object is freed. To avoid exploding the > native memory use, the jvm defaults to limiting its allocation to the max > heap size. Allocating beyond that limit triggers a System.gc(), a retry, and > potentially an exception. > * System.gc is a no-op under disableexplicitgc > * This means ByteBuffers are likely to throw an exception
[jira] [Assigned] (CASSANDRA-13112) test failure in snitch_test.TestDynamicEndpointSnitch.test_multidatacenter_local_quorum
[ https://issues.apache.org/jira/browse/CASSANDRA-13112?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joel Knighton reassigned CASSANDRA-13112: - Assignee: Joel Knighton > test failure in > snitch_test.TestDynamicEndpointSnitch.test_multidatacenter_local_quorum > --- > > Key: CASSANDRA-13112 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13112 > Project: Cassandra > Issue Type: Bug >Reporter: Sean McCarthy >Assignee: Joel Knighton > Labels: dtest, test-failure > Attachments: node1_debug.log, node1_gc.log, node1.log, > node2_debug.log, node2_gc.log, node2.log, node3_debug.log, node3_gc.log, > node3.log, node4_debug.log, node4_gc.log, node4.log, node5_debug.log, > node5_gc.log, node5.log, node6_debug.log, node6_gc.log, node6.log > > > example failure: > http://cassci.datastax.com/job/trunk_large_dtest/48/testReport/snitch_test/TestDynamicEndpointSnitch/test_multidatacenter_local_quorum > {code} > Error Message > 75 != 76 > {code}{code} > Stacktrace > File "/usr/lib/python2.7/unittest/case.py", line 329, in run > testMethod() > File "/home/automaton/cassandra-dtest/tools/decorators.py", line 48, in > wrapped > f(obj) > File "/home/automaton/cassandra-dtest/snitch_test.py", line 168, in > test_multidatacenter_local_quorum > bad_jmx.read_attribute(read_stage, 'Value')) > File "/usr/lib/python2.7/unittest/case.py", line 513, in assertEqual > assertion_func(first, second, msg=msg) > File "/usr/lib/python2.7/unittest/case.py", line 506, in _baseAssertEqual > raise self.failureException(msg) > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-13126) native transport protocol corruption when using SSL
[ https://issues.apache.org/jira/browse/CASSANDRA-13126?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826727#comment-15826727 ] Jeff Jirsa commented on CASSANDRA-13126: Do you have the full stack trace for the {{DecoderException}} ? > native transport protocol corruption when using SSL > --- > > Key: CASSANDRA-13126 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13126 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tom van der Woerdt >Priority: Critical > > This is a series of conditions that can result in client connections becoming > unusable. > 1) Cassandra GC must be well-tuned, to have short GC pauses every minute or so > 2) *client* SSL must be enabled and transmitting a significant amount of data > 3) Cassandra must run with the default library versions > 4) disableexplicitgc must be set (this is the default in the current > cassandra-env.sh) > This ticket relates to CASSANDRA-13114 which is a possible workaround (but > not a fix) for the SSL requirement to trigger this bug. > * Netty allocates nio.ByteBuffers for every outgoing SSL message. > * ByteBuffers consist of two parts, the jvm object and the off-heap object. > The jvm object is small and goes with regular GC cycles, the off-heap object > gets freed only when the small jvm object is freed. To avoid exploding the > native memory use, the jvm defaults to limiting its allocation to the max > heap size. Allocating beyond that limit triggers a System.gc(), a retry, and > potentially an exception. > * System.gc is a no-op under disableexplicitgc > * This means ByteBuffers are likely to throw an exception when too many > objects are being allocated > * The netty version shipped in Cassandra is broken when using SSL (see > CASSANDRA-13114) and causes significantly too many bytebuffers to be > allocated. > This gets more complicated though. > When /some/ clients use SSL, and others don't, the clients not using SSL can > still be affected by this bug, as bytebuffer starvation caused by ssl will > leak to other users. > ByteBuffers are used very early on in the native protocol as well. Before > even being able to decode the network protocol, this error can be thrown : > {noformat} > io.netty.handler.codec.DecoderException: java.lang.OutOfMemoryError: Direct > buffer memory > {noformat} > Note that this comes back with stream_id 0, so clients end up waiting for the > client timeout before the query is considered failed and retried. > A few frames later on the same connection, this appears: > {noformat} > Provided frame does not appear to be Snappy compressed > {noformat} > And after that everything errors out with: > {noformat} > Invalid or unsupported protocol version (54); the lowest supported version is > 3 and the greatest is 4 > {noformat} > So this bug ultimately affects the binary protocol and the connection becomes > useless if not downright dangerous. > I think there are several things that need to be done here. > * CASSANDRA-13114 should be fixed (easy, and probably needs to land in 3.0.11 > anyway) > * Connections should be closed after a DecoderException > * DisableExplicitGC should be removed from the default JVM arguments > Any of these three would limit the impact to clients. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-13075) Indexer is not correctly invoked when building indexes over sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-13075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826571#comment-15826571 ] Alex Petrov commented on CASSANDRA-13075: - > Processing the collected range tombstones You're right. I have overlooked the tombstones processing. It was "functioning" correctly, although not passing elements where I expected them. I remember stepping into it with a debugger, so I made the wrong assumption. I've added proper tests for it now. I'll check the paging tomorrow once again to make sure range tombstones won't span across pages. This also allowed to address the other two points (additional begin/finish calls along with passing correct rows). The last point appears to be a github highlighting glitch, since that's all I have changed in my diff view locally: {code} private final DataLimits pageLimits; -private final DataLimits.Counter counter; +protected final DataLimits.Counter counter; private DecoratedKey currentKey; {code} > Indexer is not correctly invoked when building indexes over sstables > > > Key: CASSANDRA-13075 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13075 > Project: Cassandra > Issue Type: Bug >Reporter: Sergio Bossa >Assignee: Alex Petrov >Priority: Critical > Attachments: CustomIndexTest.java > > > Following CASSANDRA-12796, {{SecondaryIndexManager#indexPartition()}} calls > each {{Indexer}} {{begin}} and {{finish}} methods multiple times per > partition (depending on the page size), as > {{PartitionIterators#getOnlyElement()}} returns an empty partition even when > the iterator is exhausted. > This leads to bugs for {{Indexer}} implementations doing actual work in those > methods, but even worse, it provides the {{Indexer}} the same input of an > empty partition containing only a non-live partition deletion, as the > {{Indexer#partitionDelete()}} method is *not* actually called. > My proposed solution: > 1) Stop the iteration before the empty partition is returned and ingested > into the {{Indexer}}. > 2) Actually call the {{Indexer#partitionDelete()}} method inside > {{SecondaryIndexManager#indexPartition()}} (which requires to use a filtered > iterator so it actually contains the deletion info). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Comment Edited] (CASSANDRA-13075) Indexer is not correctly invoked when building indexes over sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-13075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826571#comment-15826571 ] Alex Petrov edited comment on CASSANDRA-13075 at 1/17/17 6:39 PM: -- bq. Processing the collected range tombstones You're right. I have overlooked the tombstones processing. It was "functioning" correctly, although not passing elements where I expected them. I remember stepping into it with a debugger, so I made the wrong assumption. I've added proper tests for it now. I'll check the paging tomorrow once again to make sure range tombstones won't span across pages. This also allowed to address the other two points (additional begin/finish calls along with passing correct rows). The last point appears to be a github highlighting glitch, since that's all I have changed in my diff view locally: {code} private final DataLimits pageLimits; -private final DataLimits.Counter counter; +protected final DataLimits.Counter counter; private DecoratedKey currentKey; {code} I've updated only trunk patch for now. was (Author: ifesdjeen): > Processing the collected range tombstones You're right. I have overlooked the tombstones processing. It was "functioning" correctly, although not passing elements where I expected them. I remember stepping into it with a debugger, so I made the wrong assumption. I've added proper tests for it now. I'll check the paging tomorrow once again to make sure range tombstones won't span across pages. This also allowed to address the other two points (additional begin/finish calls along with passing correct rows). The last point appears to be a github highlighting glitch, since that's all I have changed in my diff view locally: {code} private final DataLimits pageLimits; -private final DataLimits.Counter counter; +protected final DataLimits.Counter counter; private DecoratedKey currentKey; {code} > Indexer is not correctly invoked when building indexes over sstables > > > Key: CASSANDRA-13075 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13075 > Project: Cassandra > Issue Type: Bug >Reporter: Sergio Bossa >Assignee: Alex Petrov >Priority: Critical > Attachments: CustomIndexTest.java > > > Following CASSANDRA-12796, {{SecondaryIndexManager#indexPartition()}} calls > each {{Indexer}} {{begin}} and {{finish}} methods multiple times per > partition (depending on the page size), as > {{PartitionIterators#getOnlyElement()}} returns an empty partition even when > the iterator is exhausted. > This leads to bugs for {{Indexer}} implementations doing actual work in those > methods, but even worse, it provides the {{Indexer}} the same input of an > empty partition containing only a non-live partition deletion, as the > {{Indexer#partitionDelete()}} method is *not* actually called. > My proposed solution: > 1) Stop the iteration before the empty partition is returned and ingested > into the {{Indexer}}. > 2) Actually call the {{Indexer#partitionDelete()}} method inside > {{SecondaryIndexManager#indexPartition()}} (which requires to use a filtered > iterator so it actually contains the deletion info). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-9425) Make node-local schema fully immutable
[ https://issues.apache.org/jira/browse/CASSANDRA-9425?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826564#comment-15826564 ] Aleksey Yeschenko commented on CASSANDRA-9425: -- Alright. Fixed the failing tests ({{TableId}} patch broke Paxos, and index metadata patch altered custom index handling a bit - for the better - but broke {{CustomIndexTests}}), and addressed some of the comments. In addition got rid of {{DataResource}} in {{TableMetadata}}, and undid a tiny bit of of the related CASSANDRA-10410 optimisation I hope you don’t mind. bq. I'd rename {{TableMetadata.table}} to {{TableMetadata.name}} for consistency with other classes ({{KeyspaceMetadata}} and {{IndexMetadata}} at least). While at it, I'd also rename {{ksName}} to {{keyspace}} and {{viewName}} to {{name}} in {{ViewMetadata}}. Both done. bq. I'd also probably rename {{PartitionColumns}} to {{RegularAndStaticColumns}} now. Done. bq. Nit: In {{SSTableTxnWriter}}, it feels slightly inconsistent that {{createRangeAware}} takes a {{TableMetadata}}, but create takes a {{TableMetadataRef}}. I try to use {{TableMetadata}} directly whenever possible, only pass the Ref when unavoidable. In this case it’s no big deal, though, so done. bq. In {{Indexes.validate()}}, not sure it's worth doing the duplicate name check since we do it at the keyspace level already (and only doing it within a single table is a false sense of safety). Problem is, in announce for {{CREATE INDEX}} we do not call {{KSM.validate()}}, yet. So for the time being, until the next patch, where we always validate the entirety of the resulting schema before applying anything, it has to stay here, even if imperfect. Removing it will fail some tests. And some checks - for now - are still better than no checks. Verdict: left be until the next patch. bq. I'd rename {{addRegularColumn}} and {{alterColumnType}} in {{ViewMetadata}} as they are not modifying the {{ViewMetadata}} but creating copies (say to {{withAddedRegularColumn()}} and {{withAlteredColumnType}}). Done. bq. Is there a rational for notifying everything at the end, versus calling each notification in the "appropriate" sub-method (notifying table alter in {{alterTable()}}, )? The later would feels a tad more readable to me as it's slightly easier to check we haven't forgotten something. I prefer it stylistically. But I also want to get schema and related DB objects ({{Keyspace}}, {{CFS}}, and everything downstream) to get to the consistent state as fast as possible. For that reason I’m delaying potentially blocking change event handlers until the very end. On that note, we should probably eventually move their processing elsewhere, away from the sensitive path - just enqueue them into some queue and have something poll and process the queue from time to time. bq. I'd maybe suggest adding a private {{handleDiff(mapDiff, Function onDropped, Function onAdded, Function onUpdated)}} method to simplify a bit. It’s a very minor case of duplication that doesn’t worry me, I’d rather not factor it wouldn’t strictly speaking make things simpler. Also see the previous point. bq. There is a TODO FIXME in {{SchemaKeyspace}} (and not 100% sure what it's about). It’s an ugly pre-existing piece of code to avoid re-compiling the UDFs; It annoys me immensely that a method in {{SchemaKeyspace}} is referencing {{Schema.instance}}, hence the TODO. I want to get rid of it eventually, but it’s not important enough to spend my time on it atm. bq. There is TODO: FIXME in {{ColumnFamilyStore.setCompressionParameters}}. See CASSANDRA-12949. The method in its current implementation (1.1+) is broken and unsafe, so I’ve disabled it for safety reasons. Hopefully someone will get around to dealing with it some time, but I don’t have the time. Either way, committed a clarification for that TODO. bq. {{Keyspaces has both a {{get(String)}} that return {{Optional}} and a {{getNullable(String)}}. We should probably remove one. Not just {{Keyspaces}}. I prefer to have both options available, as there are multiple things that can tolerate null return. The alternative would be using {{get(name).orElse(null)}}, which I personally find unappealing. bq. The patch changes some system table options (compared to current trunk) I removed the explicit setting or {{readRepairChance}} to 0.0 as it was redundant. We made {{readRepairChance}} default to be 0.0 a while ago. Only {{dcLocalReadRepairChance}} needs resetting. {{system_distributed}} was switched to default gc gs because gc gs of 0 is a bug. {{view_build_status}} table accepts deletes, which can be compacted away before propagation with 0 gc gs. See CASSANDRA-12954. Other tables in that keyspace should be unaffected, as they don’t use TTL nor explicit deletes. As for {{system_auth}}, it was never intended to have a special memtable flush period setting, it was just
[jira] [Created] (CASSANDRA-13130) Strange result of several list updates in a single request
Mikhail Krupitskiy created CASSANDRA-13130: -- Summary: Strange result of several list updates in a single request Key: CASSANDRA-13130 URL: https://issues.apache.org/jira/browse/CASSANDRA-13130 Project: Cassandra Issue Type: Bug Reporter: Mikhail Krupitskiy Priority: Critical Let's assume that we have a row with the 'listColumn' column and value \{1,2,3,4\}. For me it looks logical to expect that the following two pieces of code will ends up with the same result but it isn't so. Code1: {code} UPDATE t SET listColumn[2] = 7, listColumn[2] = 8 WHERE id = 1; {code} Expected result: listColumn=\{1,2,8,4\} Actual result: listColumn=\{1,2,7,8,4\} Code2: {code} UPDATE t SET listColumn[2] = 7 WHERE id = 1; UPDATE t SET listColumn[2] = 8 WHERE id = 1; {code} Expected result: listColumn=\{1,2,8,4\} Actual result: listColumn=\{1,2,8,4\} So the question is why Code1 and Code2 give different results? Looks like Code1 should give the same result as Code2. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-13075) Indexer is not correctly invoked when building indexes over sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-13075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826423#comment-15826423 ] Sergio Bossa commented on CASSANDRA-13075: -- [~ifesdjeen], I believe there are a few problems with your patch: * [Passing|https://github.com/ifesdjeen/cassandra/commit/803d3c6ea6e6852103204c7d0bb46001be191ab2#diff-3f2c8994c4ff8748c3faf7e70958520dR550] the partition columns for the metadata definition is subtly different than passing the columns for the read partition, as the latter could be a subset of the former (at least according to javadocs and a brief code inspection); I'm not sure how much it matters in practice, but that could potentially lead to unwanted index calls. * [Processing|https://github.com/ifesdjeen/cassandra/commit/803d3c6ea6e6852103204c7d0bb46001be191ab2#diff-3f2c8994c4ff8748c3faf7e70958520dR567] the collected range tombstones inside the loop, piggybacking on the fact the loop itself will do one more iteration after exhaustion, doesn't actually work, that is, it seems range tombstones are never processed; this can be easily verified by checking range tombstones [here|https://github.com/ifesdjeen/cassandra/commit/803d3c6ea6e6852103204c7d0bb46001be191ab2#diff-13cb97758bcb11cce8fc6f4cb1990dd6R754] (as this test _should_ check for range tombstones, unless I'm missing something?). * Even if the last point worked, I believe that would lead to a duplicate {{begin}} and {{finish}} call for range tombstones. * [Unused|https://github.com/ifesdjeen/cassandra/commit/803d3c6ea6e6852103204c7d0bb46001be191ab2#diff-71513a85468b7cbc97f1e820b06a20a8R125]? > Indexer is not correctly invoked when building indexes over sstables > > > Key: CASSANDRA-13075 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13075 > Project: Cassandra > Issue Type: Bug >Reporter: Sergio Bossa >Assignee: Alex Petrov >Priority: Critical > Attachments: CustomIndexTest.java > > > Following CASSANDRA-12796, {{SecondaryIndexManager#indexPartition()}} calls > each {{Indexer}} {{begin}} and {{finish}} methods multiple times per > partition (depending on the page size), as > {{PartitionIterators#getOnlyElement()}} returns an empty partition even when > the iterator is exhausted. > This leads to bugs for {{Indexer}} implementations doing actual work in those > methods, but even worse, it provides the {{Indexer}} the same input of an > empty partition containing only a non-live partition deletion, as the > {{Indexer#partitionDelete()}} method is *not* actually called. > My proposed solution: > 1) Stop the iteration before the empty partition is returned and ingested > into the {{Indexer}}. > 2) Actually call the {{Indexer#partitionDelete()}} method inside > {{SecondaryIndexManager#indexPartition()}} (which requires to use a filtered > iterator so it actually contains the deletion info). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-12847) cqlsh DESCRIBE output doesn't properly quote index names
[ https://issues.apache.org/jira/browse/CASSANDRA-12847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826300#comment-15826300 ] Sam Tunnicliffe commented on CASSANDRA-12847: - I just need to sort out pulling the driver fix into the bundled version. I'll do my best to get that done this week. > cqlsh DESCRIBE output doesn't properly quote index names > > > Key: CASSANDRA-12847 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12847 > Project: Cassandra > Issue Type: Bug >Reporter: Sam Tunnicliffe >Assignee: Sam Tunnicliffe > Labels: cqlsh > Fix For: 2.2.x, 3.0.x, 3.x > > > CASSANDRA-8365 fixed the CQL grammar so that quoting index names preserves > case. The output of DESCRIBE in cqlsh wasn't updated however so this doesn't > round-trip properly. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Issue Comment Deleted] (CASSANDRA-12744) Randomness of stress distributions is not good
[ https://issues.apache.org/jira/browse/CASSANDRA-12744?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joshua McKenzie updated CASSANDRA-12744: Comment: was deleted (was: I am on leave until Monday 30 Jan. If you need an immediate response please contact [1]sa...@instaclustr.com or [2]supp...@instaclustr.com as appropriate. For less urgent queries, I will be checking email every couple of days and respond or redirect. Cheers Ben Slater Instaclustr -- Ben SlaterChief Product Officer[3]Instaclustr: Cassandra + Spark - Managed | Consulting | Support[4]www.instaclustr.com [1] mailto:sa...@instaclustr.com [2] mailto:supp...@instaclustr.com [3] https://www.instaclustr.com [4] http://www.instaclustr.com ) > Randomness of stress distributions is not good > -- > > Key: CASSANDRA-12744 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12744 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: T Jake Luciani >Assignee: T Jake Luciani >Priority: Minor > Labels: stress > Fix For: 3.0.x > > > The randomness of our distributions is pretty bad. We are using the > JDKRandomGenerator() but in testing of uniform(1..3) we see for 100 > iterations it's only outputting 3. If you bump it to 10k it hits all 3 > values. > I made a change to just use the default commons math random generator and now > see all 3 values for n=10 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-12847) cqlsh DESCRIBE output doesn't properly quote index names
[ https://issues.apache.org/jira/browse/CASSANDRA-12847?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826242#comment-15826242 ] Joshua McKenzie commented on CASSANDRA-12847: - [~ifesdjeen]: Is this Patch Available? > cqlsh DESCRIBE output doesn't properly quote index names > > > Key: CASSANDRA-12847 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12847 > Project: Cassandra > Issue Type: Bug >Reporter: Sam Tunnicliffe >Assignee: Sam Tunnicliffe > Labels: cqlsh > Fix For: 2.2.x, 3.0.x, 3.x > > > CASSANDRA-8365 fixed the CQL grammar so that quoting index names preserves > case. The output of DESCRIBE in cqlsh wasn't updated however so this doesn't > round-trip properly. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-12795) CASSANDRA-12127 changes ordering of empty buffers in reversed columns
[ https://issues.apache.org/jira/browse/CASSANDRA-12795?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joshua McKenzie updated CASSANDRA-12795: Assignee: (was: Benjamin Lerer) > CASSANDRA-12127 changes ordering of empty buffers in reversed columns > - > > Key: CASSANDRA-12795 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12795 > Project: Cassandra > Issue Type: Bug > Components: Core >Reporter: Tim Whittington > > A change to the ordering contract of empty buffers in reversed columns was > introduced for all maintained branches (2.1.16, 2.2.8, 3.0.9, 3.8) during the > fix for CASSANDRA-12127 by [~blerer]. > This change has wider consequences than those stated in the [comment > introducing the > change|https://issues.apache.org/jira/browse/CASSANDRA-12127?focusedCommentId=15368345=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15368345] > though, and the implications aren't really called out in the release notes > clearly, so rather than litigate it in the comments I'm creating a new issue > to get a clear public position on the change. > The change made was to alter the contract that the ordering of an empty byte > buffer is always < a non empty byte buffer, in both ASC and DESC contexts - > specifically the change introduced in 0.8 as the fix for CASSANDRA-3111 to > make empty buffers < valued buffers in ReversedType was reverted. > This changes the ordering of empty buffers in clustered columns with DESC > ordering, but also in DynamicCompositeType (and CompositeType) values with > DESC components. We have production code and data that relies on this > behaviour, and it's not clear from the release notes whether the updates to > scrub will also correct that data. > So the questions I have are: > * Was the existing behavior actually 'wrong', requiring a change? What was > the actual (functional/specification) bug that was fixed by the change? > * Is the new behavior the desired behavior in general? The new behavior is > more inline with 'typical' RDMS behaviour of ordering now, but then again > they tend to support per-query NULLS FIRST/NULLS LAST, so there's no strict > 'right' behaviour here. > * Did this change introduce a regression for CASSANDRA-3111? > * Will the scrub approach fix the ordering of empty buffer values in > DynamicCompositeType columns? If not, what do we do with production data > created pre-2.2.8 mixed with data created with 2.2.8? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-13128) test failure in counter_tests.TestCounters.upgrade_test
Sean McCarthy created CASSANDRA-13128: - Summary: test failure in counter_tests.TestCounters.upgrade_test Key: CASSANDRA-13128 URL: https://issues.apache.org/jira/browse/CASSANDRA-13128 Project: Cassandra Issue Type: Bug Components: Testing Reporter: Sean McCarthy Attachments: node1.log, node2.log example failure: http://cassci.datastax.com/job/cassandra-2.1_dtest_jdk8/262/testReport/counter_tests/TestCounters/upgrade_test {code} Error Message Unexpected error in log, see stdout {code}{code} Standard Output Unexpected error in node2 log, error: ERROR [CompactionExecutor:1] 2017-01-15 03:03:15,053 CassandraDaemon.java:231 - Exception in thread Thread[CompactionExecutor:1,1,main] java.util.concurrent.RejectedExecutionException: ThreadPoolExecutor has shut down at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor$1.rejectedExecution(DebuggableThreadPoolExecutor.java:61) ~[main/:na] at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:823) ~[na:1.8.0_45] at java.util.concurrent.ThreadPoolExecutor.execute(ThreadPoolExecutor.java:1364) ~[na:1.8.0_45] at org.apache.cassandra.concurrent.DebuggableThreadPoolExecutor.execute(DebuggableThreadPoolExecutor.java:150) ~[main/:na] at java.util.concurrent.AbstractExecutorService.submit(AbstractExecutorService.java:112) ~[na:1.8.0_45] at org.apache.cassandra.db.compaction.CompactionManager.submitBackground(CompactionManager.java:184) ~[main/:na] at org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionCandidate.run(CompactionManager.java:270) ~[main/:na] at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_45] at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[na:1.8.0_45] at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) ~[na:1.8.0_45] at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_45] at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45] {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (CASSANDRA-13129) test failure in upgrade_tests.upgrade_through_versions_test.ProtoV3Upgrade_AllVersions_RandomPartitioner_EndsAt_Trunk_HEAD.rolling_upgrade_with_internode_ssl_test
Sean McCarthy created CASSANDRA-13129: - Summary: test failure in upgrade_tests.upgrade_through_versions_test.ProtoV3Upgrade_AllVersions_RandomPartitioner_EndsAt_Trunk_HEAD.rolling_upgrade_with_internode_ssl_test Key: CASSANDRA-13129 URL: https://issues.apache.org/jira/browse/CASSANDRA-13129 Project: Cassandra Issue Type: Bug Components: Testing Reporter: Sean McCarthy Attachments: node1_debug.log, node1_gc.log, node1.log, node2_debug.log, node2_gc.log, node2.log, node3_debug.log, node3_gc.log, node3.log example failure: http://cassci.datastax.com/job/cassandra-3.X_large_dtest/20/testReport/upgrade_tests.upgrade_through_versions_test/ProtoV3Upgrade_AllVersions_RandomPartitioner_EndsAt_Trunk_HEAD/rolling_upgrade_with_internode_ssl_test {code} Unexpected error in node3 log, error: ERROR [MessagingService-Incoming-/127.0.0.2] 2017-01-15 04:26:30,486 CassandraDaemon.java:205 - Exception in thread Thread[MessagingService-Incoming-/127.0.0.2,5,main] java.io.IOError: java.io.EOFException: Stream ended prematurely at org.apache.cassandra.db.LegacyLayout$7.computeNext(LegacyLayout.java:1123) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.LegacyLayout$7.computeNext(LegacyLayout.java:1097) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.utils.MergeIterator$Candidate.advance(MergeIterator.java:369) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.utils.MergeIterator$ManyToOne.advance(MergeIterator.java:189) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:158) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.LegacyLayout.toUnfilteredRowIterator(LegacyLayout.java:608) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.LegacyLayout.onWireCellstoUnfilteredRowIterator(LegacyLayout.java:572) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.LegacyLayout.deserializeLegacyPartition(LegacyLayout.java:489) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserializePre30(PartitionUpdate.java:694) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.partitions.PartitionUpdate$PartitionUpdateSerializer.deserialize(PartitionUpdate.java:640) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:334) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:353) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.Mutation$MutationSerializer.deserialize(Mutation.java:290) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.net.MessageIn.read(MessageIn.java:98) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.net.IncomingTcpConnection.receiveMessage(IncomingTcpConnection.java:201) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.net.IncomingTcpConnection.receiveMessages(IncomingTcpConnection.java:178) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.net.IncomingTcpConnection.run(IncomingTcpConnection.java:92) ~[apache-cassandra-3.0.9.jar:3.0.9] Caused by: java.io.EOFException: Stream ended prematurely at net.jpountz.lz4.LZ4BlockInputStream.readFully(LZ4BlockInputStream.java:218) ~[lz4-1.3.0.jar:na] at net.jpountz.lz4.LZ4BlockInputStream.refill(LZ4BlockInputStream.java:150) ~[lz4-1.3.0.jar:na] at net.jpountz.lz4.LZ4BlockInputStream.read(LZ4BlockInputStream.java:117) ~[lz4-1.3.0.jar:na] at java.io.DataInputStream.readFully(DataInputStream.java:195) ~[na:1.8.0_51] at java.io.DataInputStream.readLong(DataInputStream.java:416) ~[na:1.8.0_51] at org.apache.cassandra.db.LegacyLayout.readLegacyCellBody(LegacyLayout.java:1072) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.LegacyLayout.readLegacyCell(LegacyLayout.java:1045) ~[apache-cassandra-3.0.9.jar:3.0.9] at org.apache.cassandra.db.LegacyLayout$7.computeNext(LegacyLayout.java:1108) ~[apache-cassandra-3.0.9.jar:3.0.9] ... 18 common frames omitted {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-12756) Duplicate (cql)rows for the same primary key
[ https://issues.apache.org/jira/browse/CASSANDRA-12756?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sylvain Lebresne resolved CASSANDRA-12756. -- Resolution: Duplicate > Duplicate (cql)rows for the same primary key > > > Key: CASSANDRA-12756 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12756 > Project: Cassandra > Issue Type: Bug > Components: Compaction, CQL > Environment: Linux, Cassandra 3.7 (upgraded at one point from 2.?). >Reporter: Andreas Wederbrand >Priority: Minor > > I observe what looks like duplicates when I run cql queries against a table. > It only show for rows written during a couple of hours on a specific date but > it shows for several partions and serveral clustering keys for each partition > during that time range. > We've loaded data in two ways. > 1) through a normal insert > 2) through sstableloader with sstables created using update-statements (to > append to the map) and an older version of SSTableWriter. During this > processes several months of data was re-loaded. > The table DDL is > {code:title=create statement|borderStyle=solid} > CREATE TABLE climate.climate_1510 ( > installation_id bigint, > node_id bigint, > time_bucket int, > gateway_time timestamp, > humidity map, > temperature map , > PRIMARY KEY ((installation_id, node_id, time_bucket), gateway_time) > ) WITH CLUSTERING ORDER BY (gateway_time DESC) > AND bloom_filter_fp_chance = 0.01 > AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'} > AND comment = '' > AND compaction = {'class': > 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', > 'max_threshold': '32', 'min_threshold': '4'} > AND compression = {'chunk_length_in_kb': '64', 'class': > 'org.apache.cassandra.io.compress.LZ4Compressor'} > AND crc_check_chance = 1.0 > AND dclocal_read_repair_chance = 0.1 > AND default_time_to_live = 0 > AND gc_grace_seconds = 864000 > 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 = '99PERCENTILE'; > {code} > and the result from the SELECT is > {code:title=cql output|borderStyle=solid} > > select * from climate.climate_1510 where installation_id = 133235 and > > node_id = 35453983 and time_bucket = 189 and gateway_time > '2016-08-10 > > 20:00:00' and gateway_time < '2016-08-10 21:00:00' ; > installation_id | node_id | time_bucket | gateway_time | > humidity | temperature > -+--+-+--+--+--- > 133235 | 35453983 | 189 | 20160810 20:23:28.00 | {0: > 51} | {0: 24.37891} > 133235 | 35453983 | 189 | 20160810 20:23:28.00 | {0: > 51} | {0: 24.37891} > 133235 | 35453983 | 189 | 20160810 20:23:28.00 | {0: > 51} | {0: 24.37891} > {code} > I've used Andrew Tolbert's sstable-tools to be able to dump the json for this > specific time and this is what I find. > {code:title=json dump|borderStyle=solid} > [133235:35453983:189] Row[info=[ts=1470878906618000] ]: > gateway_time=2016-08-10 22:23+0200 | > del(humidity)=deletedAt=1470878906617999, localDeletion=1470878906, > [humidity[0]=51.0 ts=1470878906618000], > del(temperature)=deletedAt=1470878906617999, localDeletion=1470878906, > [temperature[0]=24.378906 ts=1470878906618000] > [133235:35453983:189] Row[info=[ts=-9223372036854775808] > del=deletedAt=1470864506441999, localDeletion=1470864506 ]: > gateway_time=2016-08-10 22:23+0200 | , [humidity[0]=51.0 > ts=1470878906618000], , [temperature[0]=24.378906 ts=1470878906618000] > [133235:35453983:189] Row[info=[ts=-9223372036854775808] > del=deletedAt=1470868106489000, localDeletion=1470868106 ]: > gateway_time=2016-08-10 22:23+0200 | > [133235:35453983:189] Row[info=[ts=-9223372036854775808] > del=deletedAt=1470871706530999, localDeletion=1470871706 ]: > gateway_time=2016-08-10 22:23+0200 | > [133235:35453983:189] Row[info=[ts=-9223372036854775808] > del=deletedAt=1470878906617999, localDeletion=1470878906 ]: > gateway_time=2016-08-10 22:23+0200 | , [humidity[0]=51.0 > ts=1470878906618000], , [temperature[0]=24.378906 ts=1470878906618000] > {code} > From my understanding this should be impossible. Even if we have duplicates > in the sstables (which is normal) it should be filtered away before being > returned to the client. > I'm happy to add details to this bug if anything is missing. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Resolved] (CASSANDRA-12745) Cassandra Unknown exception caught while attempting to update MaterializedView!
[ https://issues.apache.org/jira/browse/CASSANDRA-12745?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko resolved CASSANDRA-12745. --- Resolution: Not A Problem No longer present in 3.X code after the refactor, closing. Thanks. > Cassandra Unknown exception caught while attempting to update > MaterializedView! > --- > > Key: CASSANDRA-12745 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12745 > Project: Cassandra > Issue Type: Bug > Environment: Cassandra 3.5.0 >Reporter: Serhat Rıfat Demircan >Priority: Critical > > Following error occured only on a node in cluster which have 9 nodes. > {noformat} > ERROR [SharedPool-Worker-7] 2016-10-01 00:11:15,839 Keyspace.java:521 - > Unknown exception caught while attempting to update MaterializedView! > keyspace.table_name > java.lang.AssertionError: Only one cell version can be marked Existing; > existingCell: TemporalCell{value=001e01631b2639cecf0d58cc00, > timestamp=1475171386448000, ttl=0, localDeletionTime=2147483647, > isNew=false}, cell: TemporalCell{value=001e01631b2639cecf0d58cc00, > timestamp=1475269876398001, ttl=0, localDeletionTime=2147483647, isNew=false} >at > org.apache.cassandra.db.view.TemporalRow$TemporalCell$Versions.setVersion(TemporalRow.java:225) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at > org.apache.cassandra.db.view.TemporalRow.addColumnValue(TemporalRow.java:371) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at > org.apache.cassandra.db.view.TemporalRow.addCell(TemporalRow.java:414) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at > org.apache.cassandra.db.view.TemporalRow$Set.addRow(TemporalRow.java:554) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at > org.apache.cassandra.db.view.View.createForDeletionInfo(View.java:518) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at org.apache.cassandra.db.view.View.createMutations(View.java:704) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at > org.apache.cassandra.db.view.ViewManager.pushViewReplicaUpdates(ViewManager.java:138) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:516) > [apache-cassandra-3.5.0.jar:3.5.0] >at org.apache.cassandra.db.Keyspace.lambda$apply$67(Keyspace.java:469) > [apache-cassandra-3.5.0.jar:3.5.0] >at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > ~[na:1.8.0_101] >at > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run(AbstractLocalAwareExecutorService.java:164) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at org.apache.cassandra.concurrent.SEPWorker.run(SEPWorker.java:105) > ~[apache-cassandra-3.5.0.jar:3.5.0] >at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_101] > {noformat} > Table and view definitions: > {noformat} > CREATE TABLE keyspace.table_name ( > a bigint, > b timestamp, > c bigint, > d decimal, > PRIMARY KEY (a, b, c) > ) WITH CLUSTERING ORDER BY (b ASC, c ASC) > CREATE MATERIALIZED VIEW keyspace.mv_table_name AS > SELECT * > FROM keyspace.table_name > WHERE b IS NOT NULL AND c IS NOT NULL > PRIMARY KEY (b, c, a) > WITH CLUSTERING ORDER BY (b ASC, a ASC) > {noformat} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-12744) Randomness of stress distributions is not good
[ https://issues.apache.org/jira/browse/CASSANDRA-12744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826181#comment-15826181 ] Ben Slater commented on CASSANDRA-12744: I am on leave until Monday 30 Jan. If you need an immediate response please contact [1]sa...@instaclustr.com or [2]supp...@instaclustr.com as appropriate. For less urgent queries, I will be checking email every couple of days and respond or redirect. Cheers Ben Slater Instaclustr -- Ben SlaterChief Product Officer[3]Instaclustr: Cassandra + Spark - Managed | Consulting | Support[4]www.instaclustr.com [1] mailto:sa...@instaclustr.com [2] mailto:supp...@instaclustr.com [3] https://www.instaclustr.com [4] http://www.instaclustr.com > Randomness of stress distributions is not good > -- > > Key: CASSANDRA-12744 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12744 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: T Jake Luciani >Assignee: T Jake Luciani >Priority: Minor > Labels: stress > Fix For: 3.0.x > > > The randomness of our distributions is pretty bad. We are using the > JDKRandomGenerator() but in testing of uniform(1..3) we see for 100 > iterations it's only outputting 3. If you bump it to 10k it hits all 3 > values. > I made a change to just use the default commons math random generator and now > see all 3 values for n=10 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-12744) Randomness of stress distributions is not good
[ https://issues.apache.org/jira/browse/CASSANDRA-12744?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15826180#comment-15826180 ] Joshua McKenzie commented on CASSANDRA-12744: - [~tjake] - is this patch available? > Randomness of stress distributions is not good > -- > > Key: CASSANDRA-12744 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12744 > Project: Cassandra > Issue Type: Bug > Components: Tools >Reporter: T Jake Luciani >Assignee: T Jake Luciani >Priority: Minor > Labels: stress > Fix For: 3.0.x > > > The randomness of our distributions is pretty bad. We are using the > JDKRandomGenerator() but in testing of uniform(1..3) we see for 100 > iterations it's only outputting 3. If you bump it to 10k it hits all 3 > values. > I made a change to just use the default commons math random generator and now > see all 3 values for n=10 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-12728) Handling partially written hint files
[ https://issues.apache.org/jira/browse/CASSANDRA-12728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joshua McKenzie updated CASSANDRA-12728: Reviewer: Aleksey Yeschenko > Handling partially written hint files > - > > Key: CASSANDRA-12728 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12728 > Project: Cassandra > Issue Type: Bug >Reporter: Sharvanath Pathak > Labels: lhf > Attachments: CASSANDRA-12728.patch > > > {noformat} > ERROR [HintsDispatcher:1] 2016-09-28 17:44:43,397 > HintsDispatchExecutor.java:225 - Failed to dispatch hints file > d5d7257c-9f81-49b2-8633-6f9bda6e3dea-1474892654160-1.hints: file is corrupted > ({}) > org.apache.cassandra.io.FSReadError: java.io.EOFException > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:282) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:252) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.sendHints(HintsDispatcher.java:156) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.sendHintsAndAwait(HintsDispatcher.java:137) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.dispatch(HintsDispatcher.java:119) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.dispatch(HintsDispatcher.java:91) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.deliver(HintsDispatchExecutor.java:259) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.dispatch(HintsDispatchExecutor.java:242) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.dispatch(HintsDispatchExecutor.java:220) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.run(HintsDispatchExecutor.java:199) > [apache-cassandra-3.0.6.jar:3.0.6] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_77] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > [na:1.8.0_77] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > [na:1.8.0_77] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > [na:1.8.0_77] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_77] > Caused by: java.io.EOFException: null > at > org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.ChecksummedDataInput.readFully(ChecksummedDataInput.java:126) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.readBuffer(HintsReader.java:310) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNextInternal(HintsReader.java:301) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:278) > ~[apache-cassandra-3.0.6.jar:3.0.6] > ... 15 common frames omitted > {noformat} > We've found out that the hint file was truncated because there was a hard > reboot around the time of last write to the file. I think we basically need > to handle partially written hint files. Also, the CRC file does not exist in > this case (probably because it crashed while writing the hints file). May be > ignoring and cleaning up such partially written hint files can be a way to > fix this? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-12728) Handling partially written hint files
[ https://issues.apache.org/jira/browse/CASSANDRA-12728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Joshua McKenzie updated CASSANDRA-12728: Reviewer: (was: Aleksey Yeschenko) > Handling partially written hint files > - > > Key: CASSANDRA-12728 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12728 > Project: Cassandra > Issue Type: Bug >Reporter: Sharvanath Pathak >Assignee: Aleksey Yeschenko > Labels: lhf > Attachments: CASSANDRA-12728.patch > > > {noformat} > ERROR [HintsDispatcher:1] 2016-09-28 17:44:43,397 > HintsDispatchExecutor.java:225 - Failed to dispatch hints file > d5d7257c-9f81-49b2-8633-6f9bda6e3dea-1474892654160-1.hints: file is corrupted > ({}) > org.apache.cassandra.io.FSReadError: java.io.EOFException > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:282) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:252) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.sendHints(HintsDispatcher.java:156) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.sendHintsAndAwait(HintsDispatcher.java:137) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.dispatch(HintsDispatcher.java:119) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.dispatch(HintsDispatcher.java:91) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.deliver(HintsDispatchExecutor.java:259) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.dispatch(HintsDispatchExecutor.java:242) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.dispatch(HintsDispatchExecutor.java:220) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.run(HintsDispatchExecutor.java:199) > [apache-cassandra-3.0.6.jar:3.0.6] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_77] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > [na:1.8.0_77] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > [na:1.8.0_77] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > [na:1.8.0_77] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_77] > Caused by: java.io.EOFException: null > at > org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.ChecksummedDataInput.readFully(ChecksummedDataInput.java:126) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.readBuffer(HintsReader.java:310) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNextInternal(HintsReader.java:301) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:278) > ~[apache-cassandra-3.0.6.jar:3.0.6] > ... 15 common frames omitted > {noformat} > We've found out that the hint file was truncated because there was a hard > reboot around the time of last write to the file. I think we basically need > to handle partially written hint files. Also, the CRC file does not exist in > this case (probably because it crashed while writing the hints file). May be > ignoring and cleaning up such partially written hint files can be a way to > fix this? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-12728) Handling partially written hint files
[ https://issues.apache.org/jira/browse/CASSANDRA-12728?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aleksey Yeschenko updated CASSANDRA-12728: -- Assignee: (was: Aleksey Yeschenko) > Handling partially written hint files > - > > Key: CASSANDRA-12728 > URL: https://issues.apache.org/jira/browse/CASSANDRA-12728 > Project: Cassandra > Issue Type: Bug >Reporter: Sharvanath Pathak > Labels: lhf > Attachments: CASSANDRA-12728.patch > > > {noformat} > ERROR [HintsDispatcher:1] 2016-09-28 17:44:43,397 > HintsDispatchExecutor.java:225 - Failed to dispatch hints file > d5d7257c-9f81-49b2-8633-6f9bda6e3dea-1474892654160-1.hints: file is corrupted > ({}) > org.apache.cassandra.io.FSReadError: java.io.EOFException > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:282) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:252) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.utils.AbstractIterator.hasNext(AbstractIterator.java:47) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.sendHints(HintsDispatcher.java:156) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.sendHintsAndAwait(HintsDispatcher.java:137) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.dispatch(HintsDispatcher.java:119) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatcher.dispatch(HintsDispatcher.java:91) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.deliver(HintsDispatchExecutor.java:259) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.dispatch(HintsDispatchExecutor.java:242) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.dispatch(HintsDispatchExecutor.java:220) > [apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsDispatchExecutor$DispatchHintsTask.run(HintsDispatchExecutor.java:199) > [apache-cassandra-3.0.6.jar:3.0.6] > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > [na:1.8.0_77] > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > [na:1.8.0_77] > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > [na:1.8.0_77] > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > [na:1.8.0_77] > at java.lang.Thread.run(Thread.java:745) [na:1.8.0_77] > Caused by: java.io.EOFException: null > at > org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:68) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.io.util.RebufferingInputStream.readFully(RebufferingInputStream.java:60) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.ChecksummedDataInput.readFully(ChecksummedDataInput.java:126) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:402) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.readBuffer(HintsReader.java:310) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNextInternal(HintsReader.java:301) > ~[apache-cassandra-3.0.6.jar:3.0.6] > at > org.apache.cassandra.hints.HintsReader$BuffersIterator.computeNext(HintsReader.java:278) > ~[apache-cassandra-3.0.6.jar:3.0.6] > ... 15 common frames omitted > {noformat} > We've found out that the hint file was truncated because there was a hard > reboot around the time of last write to the file. I think we basically need > to handle partially written hint files. Also, the CRC file does not exist in > this case (probably because it crashed while writing the hints file). May be > ignoring and cleaning up such partially written hint files can be a way to > fix this? -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-13124) Send ack when received hint is non-local
[ https://issues.apache.org/jira/browse/CASSANDRA-13124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15825832#comment-15825832 ] Stefan Podkowinski commented on CASSANDRA-13124: bq. Does this mean retry is broken on 3.0? Should we file a ticket for this? From what I understood CASSANDRA-11960 only affected 3.X so I guess retry should be working on 3.0? You can just test it by commenting out all code in {{HintVerbHandler}}. Afterwards the {{hintedhandoff_decom_test}} will fail due to data loss (that was supposed to be prevented by the transferred hints). Although no hints have been handled at all, decom will happily proceed and there won't be any exceptions or errors in the logs. bq. If my understanding above is correct we should probably also fix this on 3.0 to reinstate the same behavior before CASSANDRA-12905? Can you create a 3.0 patch and submit CI? This plus removing the retry loop in the dispatcher, due to the broken behaviour described in [my comment|https://issues.apache.org/jira/browse/CASSANDRA-13058?focusedCommentId=15818371=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15818371], which is what has been done for CASSANDRA-11960 as well. Afterwards retrying hints on page basis should happen just once as initiated by {{TransferHintsTask}} and periodically for regular hinting, executed by the HintsDispatcher executor. ||3.0|| |[branch|https://github.com/spodkowinski/cassandra/tree/CASSANDRA-13124-3.0]| |[dtest|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-13124-3.0-dtest/]| |[testall|http://cassci.datastax.com/view/Dev/view/spodkowinski/job/spodkowinski-CASSANDRA-13124-3.0-testall/]| > Send ack when received hint is non-local > > > Key: CASSANDRA-13124 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13124 > Project: Cassandra > Issue Type: Bug >Reporter: Paulo Motta >Priority: Minor > > CASSANDRA-12905 changed the hints path to be asynchronous in the normal case, > but when the hint is non-local and should be stored (ie. on decommission) an > ack is not sent so the operation does not complete successfully. > CASSANDRA-13058 fixed this for 3.0+, but for 3.11+ but there is an additional > concern which on 3.0 is that non-acked hints are being completed successfully > so we should probably look into this as well here. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (CASSANDRA-13075) Indexer is not correctly invoked when building indexes over sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-13075?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Alex Petrov updated CASSANDRA-13075: Status: Patch Available (was: Open) > Indexer is not correctly invoked when building indexes over sstables > > > Key: CASSANDRA-13075 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13075 > Project: Cassandra > Issue Type: Bug >Reporter: Sergio Bossa >Assignee: Alex Petrov >Priority: Critical > Attachments: CustomIndexTest.java > > > Following CASSANDRA-12796, {{SecondaryIndexManager#indexPartition()}} calls > each {{Indexer}} {{begin}} and {{finish}} methods multiple times per > partition (depending on the page size), as > {{PartitionIterators#getOnlyElement()}} returns an empty partition even when > the iterator is exhausted. > This leads to bugs for {{Indexer}} implementations doing actual work in those > methods, but even worse, it provides the {{Indexer}} the same input of an > empty partition containing only a non-live partition deletion, as the > {{Indexer#partitionDelete()}} method is *not* actually called. > My proposed solution: > 1) Stop the iteration before the empty partition is returned and ingested > into the {{Indexer}}. > 2) Actually call the {{Indexer#partitionDelete()}} method inside > {{SecondaryIndexManager#indexPartition()}} (which requires to use a filtered > iterator so it actually contains the deletion info). -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (CASSANDRA-13075) Indexer is not correctly invoked when building indexes over sstables
[ https://issues.apache.org/jira/browse/CASSANDRA-13075?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15825704#comment-15825704 ] Alex Petrov commented on CASSANDRA-13075: - I've prepared a patch which would address both issues you have described: * we use unfiltered pager now in order to allow seeing partition deletions and therefore notifying the client about the fact that partition was deleted * problem with empty extra page is addressed. It was surfacing itself only when the page size was exactly size of partition. In this case, it tried to fetch another page, was receiving an empty partition in return. We skip this corner case here. In other places it's used it doesn't seem to be a problem Also, it will help to handle range tombstones correctly for indexes as well. |[3.X|https://github.com/ifesdjeen/cassandra/tree/13075-3.X]|[utest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-13075-3.X-testall/]|[dtest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-13075-3.X-dtest/]| |[3.0|https://github.com/ifesdjeen/cassandra/tree/13075-3.0]|[utest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-13075-3.0-testall/]|[dtest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-13075-3.0-dtest/]| |[trunk|https://github.com/ifesdjeen/cassandra/tree/13075-trunk]|[utest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-13075-trunk-testall/]|[dtest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-13075-trunk-dtest/]| > Indexer is not correctly invoked when building indexes over sstables > > > Key: CASSANDRA-13075 > URL: https://issues.apache.org/jira/browse/CASSANDRA-13075 > Project: Cassandra > Issue Type: Bug >Reporter: Sergio Bossa >Assignee: Alex Petrov >Priority: Critical > Attachments: CustomIndexTest.java > > > Following CASSANDRA-12796, {{SecondaryIndexManager#indexPartition()}} calls > each {{Indexer}} {{begin}} and {{finish}} methods multiple times per > partition (depending on the page size), as > {{PartitionIterators#getOnlyElement()}} returns an empty partition even when > the iterator is exhausted. > This leads to bugs for {{Indexer}} implementations doing actual work in those > methods, but even worse, it provides the {{Indexer}} the same input of an > empty partition containing only a non-live partition deletion, as the > {{Indexer#partitionDelete()}} method is *not* actually called. > My proposed solution: > 1) Stop the iteration before the empty partition is returned and ingested > into the {{Indexer}}. > 2) Actually call the {{Indexer#partitionDelete()}} method inside > {{SecondaryIndexManager#indexPartition()}} (which requires to use a filtered > iterator so it actually contains the deletion info). -- This message was sent by Atlassian JIRA (v6.3.4#6332)