Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1BF862009F4 for ; Thu, 26 May 2016 13:20:13 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 1A940160A10; Thu, 26 May 2016 11:20:13 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 972BC160939 for ; Thu, 26 May 2016 13:20:11 +0200 (CEST) Received: (qmail 35454 invoked by uid 500); 26 May 2016 11:20:10 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 35443 invoked by uid 99); 26 May 2016 11:20:10 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 26 May 2016 11:20:10 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id A8543C155A for ; Thu, 26 May 2016 11:20:09 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.179 X-Spam-Level: * X-Spam-Status: No, score=1.179 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_LOW=-0.7, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, SPF_PASS=-0.001] autolearn=disabled Authentication-Results: spamd1-us-west.apache.org (amavisd-new); dkim=pass (1024-bit key) header.d=textkernel.nl Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id glon2oXFMBNA for ; Thu, 26 May 2016 11:20:07 +0000 (UTC) Received: from mail-vk0-f42.google.com (mail-vk0-f42.google.com [209.85.213.42]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id 613A25F2C2 for ; Thu, 26 May 2016 11:20:06 +0000 (UTC) Received: by mail-vk0-f42.google.com with SMTP id r140so98143129vkf.0 for ; Thu, 26 May 2016 04:20:06 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=textkernel.nl; s=google; h=mime-version:in-reply-to:references:date:message-id:subject:from:to; bh=KMJXXQ1pPuWPJFPkDY/92HTOnnSBp6e/tJuNdooZvto=; b=DUYCOC942/FF5uuNEpdw86xl8isCsAGorsL44Dr8M78B2UqO/Jxbz1NQfp9MAI9ZOL CHEnt37XW8ewTrXQkwZqoz6VmBijIko923qy7kObhnFKf+r2a8Sh5qyA/UF15GGMp/Xu okKF3M5br+hlir7PP5RTeisG8VvymvVineDiA= X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20130820; h=x-gm-message-state:mime-version:in-reply-to:references:date :message-id:subject:from:to; bh=KMJXXQ1pPuWPJFPkDY/92HTOnnSBp6e/tJuNdooZvto=; b=kHe5lDyMz+ra0CQhjXlOkpHEwJ15XJchyfXQFT0uxxmfOW5Fy4mVFQwyKD9Z0qjFr0 RSLIauV6bTVwrN/aW91uIpO4IPcqKMLVnf55ZXC0atTcx6F2Z1cF6yRqAuuUPsUNbAv1 +tJgUfwueL9ZlrpVdXq/p/M/m8nhWYwOxlSJFGLmOyG16mWbto3blQzf7WD7IX99AdZ8 h2pBOyv/IOvPEoWYeMp1Okc+amEKZjuW43VqbyprMKIxCu6mvsLkmEH1j7/u4X/RWPRR T6v9DSLE3AeUFbYHDTJDN6VZ601sX3Pv/v7lIqZCxpscRKg0iD3fzQUDkkOO2/iAi7rJ zLJA== X-Gm-Message-State: ALyK8tJLGkXQBbx2S+3EPsWNWpl8iqUY7mk93bvAmqiPgAl8H81J2b0t/nQi+v8GDB185gVmj8EGRspspt0qQ8t5 MIME-Version: 1.0 X-Received: by 10.31.137.68 with SMTP id l65mr5293463vkd.103.1464261605160; Thu, 26 May 2016 04:20:05 -0700 (PDT) Received: by 10.31.165.67 with HTTP; Thu, 26 May 2016 04:20:05 -0700 (PDT) In-Reply-To: References: Date: Thu, 26 May 2016 13:20:05 +0200 Message-ID: Subject: Re: Error while rebuilding a node: Stream failed From: George Sigletos To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=001a11441656d58f1d0533bcf89e archived-at: Thu, 26 May 2016 11:20:13 -0000 --001a11441656d58f1d0533bcf89e Content-Type: text/plain; charset=UTF-8 I tried again with setting streaming_socket_timeout_in_ms to 1 day on all nodes and after having upgraded to 2.1.14. My tcp_keep_alive_time is set to 2 hours and tcp_keepalive_probes to 9. That should be ok I would believe. I get streaming error again, shortly after starting the rebuild process. This is from the destination node: ERROR [STREAM-IN-/192.168.1.141] 2016-05-26 09:08:05,027 StreamSession.java:505 - [Stream #74c57bc0-231a-11e6-a698-1b05ac77baf9] Streaming error occurred java.lang.RuntimeException: Outgoing stream handler has been closed at org.apache.cassandra.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:138) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.StreamSession.receive(StreamSession.java:568) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:457) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:263) ~[apache-cassandra-2.1.14.jar:2.1.14] at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] And this is from the source node: ERROR [STREAM-OUT-/172.31.22.104] 2016-05-26 11:08:05,097 StreamSession.java:505 - [Stream #74c57bc0-231a-11e6-a698-1b05ac77baf9] Streaming error occurred java.io.IOException: Broken pipe at sun.nio.ch.FileChannelImpl.transferTo0(Native Method) ~[na:1.7.0_79] at sun.nio.ch.FileChannelImpl.transferToDirectly(Unknown Source) ~[na:1.7.0_79] at sun.nio.ch.FileChannelImpl.transferTo(Unknown Source) ~[na:1.7.0_79] at org.apache.cassandra.streaming.compress.CompressedStreamWriter.write(CompressedStreamWriter.java:84) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:88) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:49) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:41) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:45) ~[apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:358) [apache-cassandra-2.1.14.jar:2.1.14] at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:330) [apache-cassandra-2.1.14.jar:2.1.14] at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] INFO [STREAM-OUT-/172.31.22.104] 2016-05-26 11:08:05,111 StreamResultFuture.java:180 - [Stream #74c57bc0-231a-11e6-a698-1b05ac77baf9] Session with /172.31.22.104 is complete WARN [STREAM-OUT-/172.31.22.104] 2016-05-26 11:08:05,114 StreamResultFuture.java:207 - [Stream #74c57bc0-231a-11e6-a698-1b05ac77baf9] Stream failed Streaming does not seem to be resumed again from this node. Shall I just kill again the entire rebuild process? On Thu, May 26, 2016 at 12:17 AM, Paulo Motta wrote: > If increasing or disabling streaming_socket_timeout_in_ms on the source > node does not fix it, you may want to have a look on your tcp keep alive > settings on the source and destination nodes as intermediate > routers/firewalls may be killing the connections due to inactivity. See > this for more information: > https://docs.datastax.com/en/cassandra/2.0/cassandra/troubleshooting/trblshootIdleFirewall.html > > This will ultimately fixed by CASSANDRA-11841 by adding keep-alive to the > streaming protocol. > > 2016-05-25 18:09 GMT-03:00 George Sigletos : > >> Thanks a lot for your help. I will try that tomorrow. The first time that >> I tried to rebuild, streaming_socket_timeout_in_ms was 0 and still failed. >> Below is the directly previous error on the source node: >> >> ERROR [STREAM-IN-/172.31.22.104] 2016-05-24 22:32:20,437 >> StreamSession.java:505 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] >> Streaming error occurred >> java.io.IOException: Connection timed out >> at sun.nio.ch.FileDispatcherImpl.read0(Native Method) >> ~[na:1.7.0_79] >> at sun.nio.ch.SocketDispatcher.read(Unknown Source) ~[na:1.7.0_79] >> at sun.nio.ch.IOUtil.readIntoNativeBuffer(Unknown Source) >> ~[na:1.7.0_79] >> at sun.nio.ch.IOUtil.read(Unknown Source) ~[na:1.7.0_79] >> at sun.nio.ch.SocketChannelImpl.read(Unknown Source) >> ~[na:1.7.0_79] >> at >> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:51) >> ~[apache-cassandra-2.1.13.jar:2.1.13] >> at >> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:250) >> ~[apache-cassandra-2.1.13.jar:2.1.13] >> at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] >> >> On Wed, May 25, 2016 at 10:28 PM, Paulo Motta >> wrote: >> >>> > Workaround is to set to a larger streaming_socket_timeout_in_ms **on >>> the source node**., the new default will be 86400000ms (1 day). >>> >>> 2016-05-25 17:23 GMT-03:00 Paulo Motta : >>> >>>> Was there any other ERROR preceding this on this node (in particular >>>> the last few lines of [STREAM-IN-/172.31.22.104])? If it's a >>>> SocketTimeoutException, then what is happening is that the default >>>> streaming socket timeout of 1 hour is not sufficient to stream a single >>>> file and the stream session is failed. Workaround is to set to a larger >>>> streaming_socket_timeout_in_ms, the new default will be 86400000ms (1 >>>> day). >>>> >>>> We are addressing this on >>>> https://issues.apache.org/jira/browse/CASSANDRA-11839. >>>> >>>> 2016-05-25 16:42 GMT-03:00 George Sigletos : >>>> >>>>> Hello again, >>>>> >>>>> Here is the error message from the source >>>>> >>>>> INFO [STREAM-IN-/172.31.22.104] 2016-05-25 00:44:57,275 >>>>> StreamResultFuture.java:180 - [Stream >>>>> #2c290460-20d4-11e6-930f-1b05ac77baf9] Session with /172.31.22.104 is >>>>> complete >>>>> WARN [STREAM-IN-/172.31.22.104] 2016-05-25 00:44:57,276 >>>>> StreamResultFuture.java:207 - [Stream >>>>> #2c290460-20d4-11e6-930f-1b05ac77baf9] Stream failed >>>>> ERROR [STREAM-OUT-/172.31.22.104] 2016-05-25 00:44:57,353 >>>>> StreamSession.java:505 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] >>>>> Streaming error occurred >>>>> java.lang.AssertionError: Memory was freed >>>>> at >>>>> org.apache.cassandra.io.util.SafeMemory.checkBounds(SafeMemory.java:97) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.io.util.Memory.getLong(Memory.java:249) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.io.compress.CompressionMetadata.getTotalSizeForSections(CompressionMetadata.java:247) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.streaming.messages.FileMessageHeader.size(FileMessageHeader.java:112) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.streaming.StreamSession.fileSent(StreamSession.java:546) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:50) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:41) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:45) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:351) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at >>>>> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:331) >>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>> at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] >>>>> >>>>> On Wed, May 25, 2016 at 8:49 PM, Paulo Motta >>>> > wrote: >>>>> >>>>>> This is the log of the destination/rebuilding node, you need to check >>>>>> what is the error message on the stream source node (192.168.1.140). >>>>>> >>>>>> >>>>>> 2016-05-25 15:22 GMT-03:00 George Sigletos : >>>>>> >>>>>>> Hello, >>>>>>> >>>>>>> Here is additional stack trace from system.log: >>>>>>> >>>>>>> ERROR [STREAM-IN-/192.168.1.140] 2016-05-24 22:44:57,704 >>>>>>> StreamSession.java:620 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] >>>>>>> Remote peer 192.168.1.140 failed stream session. >>>>>>> ERROR [STREAM-OUT-/192.168.1.140] 2016-05-24 22:44:57,705 >>>>>>> StreamSession.java:505 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] >>>>>>> Streaming error occurred >>>>>>> java.io.IOException: Connection timed out >>>>>>> at sun.nio.ch.FileDispatcherImpl.write0(Native Method) >>>>>>> ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.SocketDispatcher.write(Unknown Source) >>>>>>> ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.IOUtil.writeFromNativeBuffer(Unknown Source) >>>>>>> ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.IOUtil.write(Unknown Source) ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.SocketChannelImpl.write(Unknown Source) >>>>>>> ~[na:1.7.0_79] >>>>>>> at >>>>>>> org.apache.cassandra.io.util.DataOutputStreamAndChannel.write(DataOutputStreamAndChannel.java:48) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:351) >>>>>>> [apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:323) >>>>>>> [apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] >>>>>>> INFO [STREAM-IN-/192.168.1.140] 2016-05-24 22:44:58,625 >>>>>>> StreamResultFuture.java:180 - [Stream >>>>>>> #2c290460-20d4-11e6-930f-1b05ac77baf9] Session with /192.168.1.140 >>>>>>> is complete >>>>>>> WARN [STREAM-IN-/192.168.1.140] 2016-05-24 22:44:58,627 >>>>>>> StreamResultFuture.java:207 - [Stream >>>>>>> #2c290460-20d4-11e6-930f-1b05ac77baf9] Stream failed >>>>>>> ERROR [RMI TCP Connection(24)-127.0.0.1] 2016-05-24 22:44:58,628 >>>>>>> StorageService.java:1075 - Error while rebuilding node >>>>>>> org.apache.cassandra.streaming.StreamException: Stream failed >>>>>>> at >>>>>>> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> com.google.common.util.concurrent.Futures$4.run(Futures.java:1172) >>>>>>> ~[guava-16.0.jar:na] >>>>>>> at >>>>>>> com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297) >>>>>>> ~[guava-16.0.jar:na] >>>>>>> at >>>>>>> com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156) >>>>>>> ~[guava-16.0.jar:na] >>>>>>> at >>>>>>> com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145) >>>>>>> ~[guava-16.0.jar:na] >>>>>>> at >>>>>>> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202) >>>>>>> ~[guava-16.0.jar:na] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:208) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:184) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:415) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.StreamSession.sessionFailed(StreamSession.java:621) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:475) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:256) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at java.lang.Thread.run(Unknown Source) ~[na:1.7.0_79] >>>>>>> ERROR [STREAM-OUT-/192.168.1.140] 2016-05-24 22:44:58,629 >>>>>>> StreamSession.java:505 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] >>>>>>> Streaming error occurred >>>>>>> java.io.IOException: Broken pipe >>>>>>> at sun.nio.ch.FileDispatcherImpl.write0(Native Method) >>>>>>> ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.SocketDispatcher.write(Unknown Source) >>>>>>> ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.IOUtil.writeFromNativeBuffer(Unknown Source) >>>>>>> ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.IOUtil.write(Unknown Source) ~[na:1.7.0_79] >>>>>>> at sun.nio.ch.SocketChannelImpl.write(Unknown Source) >>>>>>> ~[na:1.7.0_79] >>>>>>> at >>>>>>> org.apache.cassandra.io.util.DataOutputStreamAndChannel.write(DataOutputStreamAndChannel.java:48) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44) >>>>>>> ~[apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:351) >>>>>>> [apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at >>>>>>> org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.java:331) >>>>>>> [apache-cassandra-2.1.13.jar:2.1.13] >>>>>>> at java.lang.Thread.run(Unknown Source) [na:1.7.0_79] >>>>>>> >>>>>>> >>>>>>> On Wed, May 25, 2016 at 5:23 PM, Paulo Motta < >>>>>>> pauloricardomg@gmail.com> wrote: >>>>>>> >>>>>>>> The stack trace from the rebuild command not show the root cause of >>>>>>>> the rebuild stream error. Can you check the system.log for ERROR logs >>>>>>>> during streaming and paste here? >>>>>>>> >>>>>>> >>>>>>> >>>>>> >>>>> >>>> >>> >> > --001a11441656d58f1d0533bcf89e Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
I tried again with setting stream= ing_socket_timeout_in_ms to 1 day on all nodes and after having upgraded to= 2.1.14.

My tcp_keep_alive_time is set to 2 hours and tcp_kee= palive_probes to 9. That should be ok I would believe.

<= /div>I get streaming error again, shortly after starting the rebuild proces= s. This is from the destination node:

ERROR [STREAM-IN-/192.168.1.141] 2016-05-26 09:08:05,027 StreamSessi= on.java:505 - [Stream #74c57bc0-231a-11e6-a698-1b05ac77baf9] Streaming erro= r occurred
java.lang.RuntimeException: Outgoing stream handler has been = closed
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandr= a.streaming.ConnectionHandler.sendMessage(ConnectionHandler.java:138) ~[apa= che-cassandra-2.1.14.jar:2.1.14]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0 at org.apache.cassandra.streaming.StreamSession.receive(StreamSession.j= ava:568) ~[apache-cassandra-2.1.14.jar:2.1.14]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.StreamSession.messageR= eceived(StreamSession.java:457) ~[apache-cassandra-2.1.14.jar:2.1.14]
= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streamin= g.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:263) = ~[apache-cassandra-2.1.14.jar:2.1.14]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 at java.lang.Thread.run(Unknown Source) [na:1.7.0_79]

And this is from the source node:

ERROR [STREAM-OUT-/172.31.22.104] 2016-05-26 11:08:05,097 StreamSessio= n.java:505 - [Stream #74c57bc0-231a-11e6-a698-1b05ac77baf9] Streaming error= occurred
java.io.IOException: Broken pipe
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at sun.nio.ch.FileChannelImpl.transferTo0(Native Method)= ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch= .FileChannelImpl.transferToDirectly(Unknown Source) ~[na:1.7.0_79]
=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.FileChannelImpl.trans= ferTo(Unknown Source) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 at org.apache.cassandra.streaming.compress.CompressedStreamWriter= .write(CompressedStreamWriter.java:84) ~[apache-cassandra-2.1.14.jar:2.1.14= ]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.str= eaming.messages.OutgoingFileMessage.serialize(OutgoingFileMessage.java:88) = ~[apache-cassandra-2.1.14.jar:2.1.14]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 at org.apache.cassandra.streaming.messages.OutgoingFileMessage$1.= serialize(OutgoingFileMessage.java:49) ~[apache-cassandra-2.1.14.jar:2.1.14= ]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.str= eaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.java:41= ) ~[apache-cassandra-2.1.14.jar:2.1.14]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0 at org.apache.cassandra.streaming.messages.StreamMessage.seria= lize(StreamMessage.java:45) ~[apache-cassandra-2.1.14.jar:2.1.14]
=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.Conn= ectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.java:358= ) [apache-cassandra-2.1.14.jar:2.1.14]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessa= geHandler.run(ConnectionHandler.java:330) [apache-cassandra-2.1.14.jar:2.1.= 14]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at java.lang.Thread.run(U= nknown Source) [na:1.7.0_79]
INFO=C2=A0 [STREAM-OUT-/172.31.22.104] 2016-05-26 11:08:05,111 StreamResultFuture= .java:180 - [Stream #74c57bc0-231a-11e6-a698-1b05ac77baf9] Session with /172.31.22.104 is complete
WARN=C2=A0 = [STREAM-OUT-/172.31.22.104] 2016-05-26= 11:08:05,114 StreamResultFuture.java:207 - [Stream #74c57bc0-231a-11e6-a69= 8-1b05ac77baf9] Stream failed


Streaming does not = seem to be resumed again from this node. Shall I just kill again the entire= rebuild process?
On Thu, May 26, 2016 at 12:17 AM, Paulo Motta <= span dir=3D"ltr"><pauloricardomg@gmail.com> wrote:
If increasing or disabling streami= ng_socket_timeout_in_ms on the source node does not fix it, you may want to= have a look on your tcp keep alive settings on the source and destination = nodes as intermediate routers/firewalls may be killing the connections due = to inactivity. See this for more information: https://docs.datastax.com/en/cassandra/2.0/cassandra/= troubleshooting/trblshootIdleFirewall.html

This will = ultimately fixed by CASSANDRA-11841 by adding keep-alive to the streaming p= rotocol.

2016-05-25 18:09 GMT-0= 3:00 George Sigletos <sigletos@textkernel.nl>:
Thanks a lot for your help. I wi= ll try that tomorrow. The first time that I tried to rebuild, streaming_soc= ket_timeout_in_ms was 0 and still failed. Below is the directly previous er= ror on the source node:

ERROR=20 [STREAM-IN-/172.31.22.10= 4] 2016-05-24 22:32:20,437=20 StreamSession.java:505 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9]=20 Streaming error occurred
java.io.IOException: Connection timed out
= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.FileDispatcherImpl= .read0(Native Method) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 at sun.nio.ch.SocketDispatcher.read(Unknown Source) ~[na:1.7.0_79= ]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.IOUtil.readIn= toNativeBuffer(Unknown Source) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at sun.nio.ch.IOUtil.read(Unknown Source) ~[na:1.7.0_79]=
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.SocketChannelI= mpl.read(Unknown Source) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0 at=20 org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMes= sage.java:51) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 at=20 org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run= (ConnectionHandler.java:250) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0 at java.lang.Thread.run(Unknown Source) [na:1.7.0_79]

On Wed, May 25, 2016 at 10:28 PM, Paulo Motta <pauloricardomg@g= mail.com> wrote:
> Workaround is to set to a larger streaming_socket_timeout_in_= ms **on the source node**., the new default will be 8640= 0000ms (1 day).

2016-05-25 17:23 GMT-03:00 Paulo Motta <= pauloricardomg@gmail.com>:
=
Was there any other ERROR preceding this on this node= (in particular the last few lines of [STREAM-IN-/172.31.22.104])? If it's a SocketTimeoutE= xception, then what is happening is that the default streaming socket timeo= ut of 1 hour is not sufficient to stream a single file and the stream sessi= on is failed. Workaround is to set to a larger streaming_socket_timeout_in_= ms, the new default will be 86400000ms (1 day).

W= e are addressing this on https://issues.apache.org/jira/browse/CAS= SANDRA-11839.

2016-05-25 16:42 GMT-03:00 George Sigletos <sigle= tos@textkernel.nl>:
Hello again,

Here is the error message from the= source

INFO=C2=A0 [STREAM-IN-/172.31.22.104] 2016-05-25 00:44:57,275 StreamResultFuture= .java:180 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] Session with /172.31.22.104 is compl= ete
WARN=C2=A0 [STREAM-IN-/172.31.22.104] 2016-05-25 00:44:57,276 StreamResultFuture.java:20= 7 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] Stream failed
ERROR [= STREAM-OUT-/172.31.22.10= 4] 2016-05-25 00:44:57,353 StreamSession.java:505 - [Stream #2c290460-2= 0d4-11e6-930f-1b05ac77baf9] Streaming error occurred
java.lang.Assertion= Error: Memory was freed
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at or= g.apache.cassandra.io.util.SafeMemory.checkBounds(SafeMemory.java:97) ~[apa= che-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0 at org.apache.cassandra.io.util.Memory.getLong(Memory.java:249) ~[apach= e-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0 at org.apache.cassandra.io.compress.CompressionMetadata.getTotalSizeFor= Sections(CompressionMetadata.java:247) ~[apache-cassandra-2.1.13.jar:2.1.13= ]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.str= eaming.messages.FileMessageHeader.size(FileMessageHeader.java:112) ~[apache= -cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0= at org.apache.cassandra.streaming.StreamSession.fileSent(StreamSession.jav= a:546) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.messages.OutgoingFileMess= age$1.serialize(OutgoingFileMessage.java:50) ~[apache-cassandra-2.1.13.jar:= 2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassand= ra.streaming.messages.OutgoingFileMessage$1.serialize(OutgoingFileMessage.j= ava:41) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.messages.StreamMessage= .serialize(StreamMessage.java:45) ~[apache-cassandra-2.1.13.jar:2.1.13]
= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streamin= g.ConnectionHandler$OutgoingMessageHandler.sendMessage(ConnectionHandler.ja= va:351) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.ConnectionHandler$Outg= oingMessageHandler.run(ConnectionHandler.java:331) ~[apache-cassandra-2.1.1= 3.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at java.l= ang.Thread.run(Unknown Source) [na:1.7.0_79]

On Wed, May 25, 2016 = at 8:49 PM, Paulo Motta <pauloricardomg@gmail.com> wr= ote:
This is the log of = the destination/rebuilding node, you need to check what is the error messag= e on the stream source node (192.168.1.140).


2016-05-25 15:22 GMT-03:00 George = Sigletos <sigletos@textkernel.nl>:
Hello,

Here is additional = stack trace from system.log:

ERROR [STREAM-IN-/192.168.1.140] 2016-05-24 22:44:57,704 St= reamSession.java:620 - [Stream #2c290460-20d4-11e6-930f-1b05ac77baf9] Remot= e peer 192.168.1.140 failed stream session.
ERROR [STREAM-OUT-/192.168.1.140] 2016-05-24 2= 2:44:57,705 StreamSession.java:505 - [Stream #2c290460-20d4-11e6-930f-1b05a= c77baf9] Streaming error occurred
java.io.IOException: Connection timed = out
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.FileDispatc= herImpl.write0(Native Method) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0 at sun.nio.ch.SocketDispatcher.write(Unknown Source) ~[na:1= .7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.IOUtil= .writeFromNativeBuffer(Unknown Source) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.IOUtil.write(Unknown Source) ~[na:1.= 7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.SocketC= hannelImpl.write(Unknown Source) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at org.apache.cassandra.io.util.DataOutputStreamAndChann= el.write(DataOutputStreamAndChannel.java:48) ~[apache-cassandra-2.1.13.jar:= 2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassand= ra.streaming.messages.StreamMessage.serialize(StreamMessage.java:44) ~[apac= he-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0 at org.apache.cassandra.streaming.ConnectionHandler$OutgoingMessageHand= ler.sendMessage(ConnectionHandler.java:351) [apache-cassandra-2.1.13.jar:2.= 1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra= .streaming.ConnectionHandler$OutgoingMessageHandler.run(ConnectionHandler.j= ava:323) [apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at java.lang.Thread.run(Unknown Source) [na:1.7.0_79]INFO=C2=A0 [STREAM-IN-/= 192.168.1.140] 2016-05-24 22:44:58,625 StreamResultFuture.java:180 - [S= tream #2c290460-20d4-11e6-930f-1b05ac77baf9] Session with /192.168.1.140 is complete
WARN=C2= =A0 [STREAM-IN-/192.168.= 1.140] 2016-05-24 22:44:58,627 StreamResultFuture.java:207 - [Stream #2= c290460-20d4-11e6-930f-1b05ac77baf9] Stream failed
ERROR [RMI TCP Connec= tion(24)-127.0.0.1] 2016-05-24 22:44:58,628 StorageService.java:1075 - Erro= r while rebuilding node
org.apache.cassandra.streaming.StreamException: = Stream failed
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.c= assandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJ= MXNotifier.java:85) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at com.google.common.util.concurrent.Futures= $4.run(Futures.java:1172) ~[guava-16.0.jar:na]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at com.google.common.util.concurrent.MoreExecutors$SameT= hreadExecutorService.execute(MoreExecutors.java:297) ~[guava-16.0.jar:na]=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at com.google.common.util.conc= urrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guava-16.0.j= ar:na]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at com.google.common.u= til.concurrent.ExecutionList.execute(ExecutionList.java:145) ~[guava-16.0.j= ar:na]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at com.google.common.u= til.concurrent.AbstractFuture.setException(AbstractFuture.java:202) ~[guava= -16.0.jar:na]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.c= assandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java= :208) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.StreamResultFuture.handle= SessionComplete(StreamResultFuture.java:184) ~[apache-cassandra-2.1.13.jar:= 2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassand= ra.streaming.StreamSession.closeSession(StreamSession.java:415) ~[apache-ca= ssandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at= org.apache.cassandra.streaming.StreamSession.sessionFailed(StreamSession.j= ava:621) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.StreamSession.messageR= eceived(StreamSession.java:475) ~[apache-cassandra-2.1.13.jar:2.1.13]
= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.streamin= g.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:256) = ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0 at java.lang.Thread.run(Unknown Source) ~[na:1.7.0_79]
ERROR [= STREAM-OUT-/192.168.1.14= 0] 2016-05-24 22:44:58,629 StreamSession.java:505 - [Stream #2c290460-2= 0d4-11e6-930f-1b05ac77baf9] Streaming error occurred
java.io.IOException= : Broken pipe
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.F= ileDispatcherImpl.write0(Native Method) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.SocketDispatcher.write(Unknown So= urce) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.n= io.ch.IOUtil.writeFromNativeBuffer(Unknown Source) ~[na:1.7.0_79]
=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.nio.ch.IOUtil.write(Unknown Sou= rce) ~[na:1.7.0_79]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at sun.ni= o.ch.SocketChannelImpl.write(Unknown Source) ~[na:1.7.0_79]
=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apache.cassandra.io.util.DataOutputSt= reamAndChannel.write(DataOutputStreamAndChannel.java:48) ~[apache-cassandra= -2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.ap= ache.cassandra.streaming.messages.StreamMessage.serialize(StreamMessage.jav= a:44) ~[apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2= =A0=C2=A0=C2=A0 at org.apache.cassandra.streaming.ConnectionHandler$Outgoin= gMessageHandler.sendMessage(ConnectionHandler.java:351) [apache-cassandra-2= .1.13.jar:2.1.13]
=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at org.apac= he.cassandra.streaming.ConnectionHandler$OutgoingMessageHandler.run(Connect= ionHandler.java:331) [apache-cassandra-2.1.13.jar:2.1.13]
=C2=A0=C2=A0= =C2=A0=C2=A0=C2=A0=C2=A0=C2=A0 at java.lang.Thread.run(Unknown Source) [na:= 1.7.0_79]


On Wed, May 25, 2016 at 5:23 PM, Paulo Motta <paul= oricardomg@gmail.com> wrote:
The stack trace from the rebuild command not show the ro= ot cause of the rebuild stream error. Can you check the system.log for ERRO= R logs during streaming and paste here?








--001a11441656d58f1d0533bcf89e--