cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Alex Petrov (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (CASSANDRA-11838) dtest failure in largecolumn_test:TestLargeColumn.cleanup_test
Date Wed, 01 Jun 2016 07:03:12 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-11838?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15309434#comment-15309434
] 

Alex Petrov edited comment on CASSANDRA-11838 at 6/1/16 7:02 AM:
-----------------------------------------------------------------

I've been trying to get rid of the {{DataOutputBuffer}} instances, although there always was
a reference hanging somewhere. [~tjake] pointed out that it may be happening when the instance
is recycled once but not again. 

I've opened a [PR|https://github.com/netty/netty/pull/5333] (already merged thanks to [~norman]).
In short: large / grown instances were not recycled by us, but were hanging in the array backing
the {{Stack}}. 

I'd suggest let's trim buffers to {{128}} bytes (default initial value) when the instances
are > 1mb until we upgrade to the version where we can throw away the recycled instances.
And create a follow-up ticket to make sure we remove custom code that trims buffers and can
throw the instances away. The second point (with {{fill}}) is addressed, too. I've triggered
tests (including one with running the failing test many times):

|[trunk|https://github.com/ifesdjeen/cassandra/tree/11838-trunk] |[utest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-11838-trunk-testall/]|[dtest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-11838-trunk-dtest/]|[multiplexed|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-11838-trunk-multiplexed/]|
 


was (Author: ifesdjeen):
I've been trying to get rid of the `DataOutputBuffer` instances, although there always was
a reference hanging somewhere. [~tjake] pointed out that it may be happening when the instance
is recycled once but not again. 

I've opened a [PR|https://github.com/netty/netty/pull/5333] (already merged thanks to [~norman]).
In short: large / grown instances were not recycled by us, but were hanging in the array backing
the {{Stack}}. 

I'd suggest let's trim buffers to {128} bytes (default initial value) when the instances are
> 1mb until we upgrade to the version where we can throw away the recycled instances. And
create a follow-up ticket to make sure we remove custom code that trims buffers and can throw
the instances away. The second point (with {{fill}}) is addressed, too. I've triggered tests
(including one with running the failing test many times):

|[trunk|https://github.com/ifesdjeen/cassandra/tree/11838-trunk] |[utest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-11838-trunk-testall/]|[dtest|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-11838-trunk-dtest/]|[multiplexed|https://cassci.datastax.com/view/Dev/view/ifesdjeen/job/ifesdjeen-11838-trunk-multiplexed/]|
 

> dtest failure in largecolumn_test:TestLargeColumn.cleanup_test
> --------------------------------------------------------------
>
>                 Key: CASSANDRA-11838
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-11838
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Philip Thompson
>            Assignee: Alex Petrov
>              Labels: dtest
>             Fix For: 3.x
>
>         Attachments: node1.log, node1_debug.log, node2.log, node2_debug.log
>
>
> Example failure at:
> http://cassci.datastax.com/job/trunk_offheap_dtest/200/testReport/largecolumn_test/TestLargeColumn/cleanup_test/
> node 1 contains the following OOM in its log:
> {code}
> ERROR [SharedPool-Worker-1] 2016-05-16 22:54:10,112 Message.java:611 - Unexpected exception
during request; channel = [id: 0xb97f2640, L:/127.0.0.1:9042 - R:/127.0.0.1:48190]
> java.lang.OutOfMemoryError: Java heap space
> 	at org.apache.cassandra.transport.CBUtil.readRawBytes(CBUtil.java:533) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readBoundValue(CBUtil.java:407) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readValueList(CBUtil.java:462) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:417) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:365) ~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:45)
~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:41)
~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:280) ~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:261) ~[main/:na]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:89)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:277)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:264)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:962)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:879)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:360) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:276) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> ERROR [SharedPool-Worker-1] 2016-05-16 22:54:10,756 Message.java:611 - Unexpected exception
during request; channel = [id: 0xba0be401, L:/127.0.0.1:9042 - R:/127.0.0.1:48191]
> java.lang.OutOfMemoryError: Java heap space
> 	at org.apache.cassandra.transport.CBUtil.readRawBytes(CBUtil.java:533) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readBoundValue(CBUtil.java:407) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readValueList(CBUtil.java:462) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:417) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:365) ~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:45)
~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:41)
~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:280) ~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:261) ~[main/:na]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:89)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:277)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:264)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:962)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:879)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:360) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:276) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> ERROR [SharedPool-Worker-1] 2016-05-16 22:54:11,397 Message.java:611 - Unexpected exception
during request; channel = [id: 0x6a690f39, L:/127.0.0.1:9042 - R:/127.0.0.1:48193]
> java.lang.OutOfMemoryError: Java heap space
> 	at org.apache.cassandra.transport.CBUtil.readRawBytes(CBUtil.java:533) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readBoundValue(CBUtil.java:407) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readValueList(CBUtil.java:462) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:417) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:365) ~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:45)
~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:41)
~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:280) ~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:261) ~[main/:na]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:89)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:277)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:264)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:962)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:879)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:360) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:276) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> ERROR [SharedPool-Worker-1] 2016-05-16 22:54:12,006 Message.java:611 - Unexpected exception
during request; channel = [id: 0xc05601f1, L:/127.0.0.1:9042 - R:/127.0.0.1:48192]
> java.lang.OutOfMemoryError: Java heap space
> 	at org.apache.cassandra.transport.CBUtil.readRawBytes(CBUtil.java:533) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readBoundValue(CBUtil.java:407) ~[main/:na]
> 	at org.apache.cassandra.transport.CBUtil.readValueList(CBUtil.java:462) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:417) ~[main/:na]
> 	at org.apache.cassandra.cql3.QueryOptions$Codec.decode(QueryOptions.java:365) ~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:45)
~[main/:na]
> 	at org.apache.cassandra.transport.messages.ExecuteMessage$1.decode(ExecuteMessage.java:41)
~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:280) ~[main/:na]
> 	at org.apache.cassandra.transport.Message$ProtocolDecoder.decode(Message.java:261) ~[main/:na]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:89)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:103)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.fireChannelRead(ByteToMessageDecoder.java:277)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.handler.codec.ByteToMessageDecoder.channelRead(ByteToMessageDecoder.java:264)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:292)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:278)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:962)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:879)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:360) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:276) ~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:137)
~[netty-all-4.0.36.Final.jar:4.0.36.Final]
> 	at java.lang.Thread.run(Thread.java:745) [na:1.8.0_45]
> {code}
> Given that this test has been stable for a long time, only runs two nodes, and we haven't
made any recent changes to the environment, I'm filing this directly as a bug. Node logs are
attached. Please take note that this was run with offheap_memtables, as that may be relevant.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message