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 9D1F6200BC7 for ; Fri, 25 Nov 2016 20:32:12 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 9BA6B160B1C; Fri, 25 Nov 2016 19:32:12 +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 4E772160AFA for ; Fri, 25 Nov 2016 20:32:11 +0100 (CET) Received: (qmail 89122 invoked by uid 500); 25 Nov 2016 19:32:09 -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 89112 invoked by uid 99); 25 Nov 2016 19:32:09 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Nov 2016 19:32:09 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 0C1771AB46F for ; Fri, 25 Nov 2016 19:32:09 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 4.822 X-Spam-Level: **** X-Spam-Status: No, score=4.822 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, HTML_MESSAGE=2, HTML_OBFUSCATE_20_30=2.441, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RCVD_IN_SORBS_SPAM=0.5, SPF_PASS=-0.001, URIBL_BLOCKED=0.001, WEIRD_PORT=0.001] autolearn=disabled Authentication-Results: spamd2-us-west.apache.org (amavisd-new); dkim=pass (1024-bit key) header.d=jaumo.com Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id w1gsD2O-F8LF for ; Fri, 25 Nov 2016 19:32:04 +0000 (UTC) Received: from mail-ua0-f181.google.com (mail-ua0-f181.google.com [209.85.217.181]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id 03AE15F4A8 for ; Fri, 25 Nov 2016 19:32:03 +0000 (UTC) Received: by mail-ua0-f181.google.com with SMTP id 20so86209383uak.0 for ; Fri, 25 Nov 2016 11:32:03 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=jaumo.com; s=google; h=mime-version:in-reply-to:references:from:date:message-id:subject:to; bh=TEyK9QN0kFeAziGhjzKfTyOPgC8AuiSGzGzZmIBUNOI=; b=jjRG6IEQo3I+63IUU0LO7Np/lmVVbalSDuGC+U071VoSOd1LRYqE0n4kP/SXA85F5F iBZNKGYp0fWSan8fS9eiKn9ALmq0Y/pMVxOI11gUVqDqXF5Tg0n+qDHsqxm6K/kbMYqV 1L1AIt6iDdxwgAJA8dSS6vlVo8En4wSUFPjb4= 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:from:date :message-id:subject:to; bh=TEyK9QN0kFeAziGhjzKfTyOPgC8AuiSGzGzZmIBUNOI=; b=D39z4Et4wYuXkZVC7Uo/+aseMc7xbLM38WR2JGgL6BmCKA3UpXOn96Dkpo+tQo6bxe toyuj3E8HaqSZ8H9O3lWcWWxUG19NyxjmWz6snPicJSVGorg37i+xvThgKpwErmdLIfA 1TPny2nt2/FX8ZcC3XkpaMKxZ6FDSokPTwLCifwKqdg6tA3XFrI9hwW13B8ZX2Q4a7f5 GpTJwWBoLaotvShpzhJYuD/qK8BZDo6OUVi/13scLsWDl+Y2NkWY0/19q7rTEEUsAkWE 8z5BOhKyJE5ZBx/KpudyJztemTwrpjOnJy2QzkrzjPpbcWYqi7eJS1pVQpvA1+E0/s7i zjqw== X-Gm-Message-State: AKaTC01Y2eq2jqdzeD6H3m+bZC7qLobpmV7Qgbz5ofDorWbIr8Fxm9/+89NcVdU1pskX98/O8aiHXijalf3Hlbd2 X-Received: by 10.176.1.17 with SMTP id 17mr5805795uak.143.1480102322861; Fri, 25 Nov 2016 11:32:02 -0800 (PST) MIME-Version: 1.0 Received: by 10.159.37.203 with HTTP; Fri, 25 Nov 2016 11:32:02 -0800 (PST) In-Reply-To: References: From: Benjamin Roth Date: Fri, 25 Nov 2016 20:32:02 +0100 Message-ID: Subject: Re: Bootstrap fails on 3.10 To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=001a113cfa702f9b780542252d56 archived-at: Fri, 25 Nov 2016 19:32:12 -0000 --001a113cfa702f9b780542252d56 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: quoted-printable I proposed a quite simple fix for https://issues.apache.org/jira/browse/CASSANDRA-12905 Sorry that I don't supply a patch. I am good at analysing code but totally unexperienced with the workflows here. 2016-11-25 19:57 GMT+01:00 Benjamin Roth : > Yes, I have MVs. > > Interesting is also that in the middle of bootstrapping (cannot tell when > exactly) it seemed like other nodes started to send hints to the > bootstrapping node. When that happened, it seems that every single HintVe= rb > fails also with a WTE. At least the logs are completely flooded with WTE. > When I paused hints on all other nodes, logs were quiet again. > > I completely restarted the bootstrap (deleted /var/lib/cassandra) - this > time with hints paused from the beginning. We will see if that changes > anything. > > I find it also quite weird that other nodes have hints for a bootstrappin= g > node. Is that intended behaviour? > And is it possible that streaming locks the whole CF? I looked like > absolutely no hint could be delivered successfully. > > 2016-11-25 19:43 GMT+01:00 Paulo Motta : > >> If you have an MV table It seems you're hitting https://issues.apache. >> org/jira/browse/CASSANDRA-12905. I will bump it's priority to critical >> since it can prevent or difficult bootstrap. >> >> Did you try resuming bootstrap with "nodetool bootstrap resume" after th= e >> failure? It may eventually succeed, since this is an MV lock contention >> problem. >> >> 2016-11-25 15:59 GMT-02:00 Benjamin Roth : >> >>> Hi! >>> >>> Today I wanted a new node to join the cluster. >>> When looking at netstats on all the old nodes, it seemed like the >>> streaming sessions did complete. >>> They all said that all files have been transferred. But looking at the >>> debug.log the stream sessions finished with an error. >>> Also after all streams have been done the node remains in state >>> "JOINING". >>> >>> See logs: >>> >>> debug.log, last words >>> =3D=3D=3D=3D=3D=3D=3D=3D >>> ERROR [StreamReceiveTask:94] 2016-11-25 17:50:51,712 >>> StreamSession.java:593 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8d45c] >>> Streaming error occurred on session with peer 10.23.71.6 >>> org.apache.cassandra.exceptions.WriteTimeoutException: Operation timed >>> out - received only 0 responses. >>> at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:525) >>> ~[apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.db.Keyspace.applyNotDeferrable(Keyspace.java:44= 0) >>> ~[apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.db.Mutation.apply(Mutation.java:223) >>> ~[apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.db.Mutation.applyUnsafe(Mutation.java:242) >>> ~[apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.streaming.StreamReceiveTask$OnCompletio >>> nRunnable.run(StreamReceiveTask.java:205) ~[apache-cassandra-3.10.jar:3= . >>> 10] >>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:5= 11) >>> [na:1.8.0_102] >>> at java.util.concurrent.FutureTask.run(FutureTask.java:266) >>> [na:1.8.0_102] >>> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor= .java:1142) >>> [na:1.8.0_102] >>> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecuto= r.java:617) >>> [na:1.8.0_102] >>> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102] >>> DEBUG [STREAM-OUT-/10.23.71.6:7000] 2016-11-25 17:50:51,713 >>> ConnectionHandler.java:388 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8d= 45c] >>> Sending Session Failed >>> DEBUG [StreamReceiveTask:94] 2016-11-25 17:50:51,713 >>> StreamSession.java:472 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8d45c] >>> Finishing keep-alive task. >>> DEBUG [StreamReceiveTask:94] 2016-11-25 17:50:51,713 >>> ConnectionHandler.java:120 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8d= 45c] >>> Closing stream connection handler on /10.23.71.6 >>> INFO [StreamReceiveTask:94] 2016-11-25 17:50:51,719 >>> StreamResultFuture.java:187 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8= d45c] >>> Session with /10.23.71.6 is complete >>> DEBUG [StreamReceiveTask:94] 2016-11-25 17:50:51,719 >>> StreamCoordinator.java:146 - Finished connecting all sessions >>> WARN [StreamReceiveTask:94] 2016-11-25 17:50:51,723 >>> StreamResultFuture.java:214 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8= d45c] >>> Stream failed >>> ERROR [main] 2016-11-25 17:50:51,724 StorageService.java:1493 - Error >>> while waiting on bootstrap to complete. Bootstrap will have to be resta= rted. >>> java.util.concurrent.ExecutionException: org.apache.cassandra.streaming= .StreamException: >>> Stream failed >>> at com.google.common.util.concurrent.AbstractFuture$Sync.getValue(Abstr= actFuture.java:299) >>> ~[guava-18.0.jar:na] >>> at com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFu= ture.java:286) >>> ~[guava-18.0.jar:na] >>> at com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.= java:116) >>> ~[guava-18.0.jar:na] >>> at org.apache.cassandra.service.StorageService.bootstrap(StorageService= .java:1488) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.service.StorageService.joinTokenRing(StorageSer= vice.java:948) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.service.StorageService.initServer(StorageServic= e.java:667) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.service.StorageService.initServer(StorageServic= e.java:598) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.j= ava:394) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemo= n.java:601) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.ja= va:735) >>> [apache-cassandra-3.10.jar:3.10] >>> Caused by: org.apache.cassandra.streaming.StreamException: Stream faile= d >>> at org.apache.cassandra.streaming.StreamResultFuture.maybeCompl >>> ete(StreamResultFuture.java:215) ~[apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.streaming.StreamResultFuture.handleSess >>> ionComplete(StreamResultFuture.java:191) ~[apache-cassandra-3.10.jar:3. >>> 10] >>> at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSess= ion.java:481) >>> ~[apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.j= ava:571) >>> ~[apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.streaming.StreamReceiveTask$OnCompletio >>> nRunnable.run(StreamReceiveTask.java:251) ~[apache-cassandra-3.10.jar:3= . >>> 10] >>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:5= 11) >>> ~[na:1.8.0_102] >>> at java.util.concurrent.FutureTask.run(FutureTask.java:266) >>> ~[na:1.8.0_102] >>> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor= .java:1142) >>> ~[na:1.8.0_102] >>> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecuto= r.java:617) >>> ~[na:1.8.0_102] >>> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_102] >>> WARN [StreamReceiveTask:94] 2016-11-25 17:50:51,731 >>> StorageService.java:1483 - Error during bootstrap. >>> org.apache.cassandra.streaming.StreamException: Stream failed >>> at org.apache.cassandra.streaming.management.StreamEventJMXNoti >>> fier.onFailure(StreamEventJMXNotifier.java:88) >>> ~[apache-cassandra-3.10.jar:3.10] >>> at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) >>> [guava-18.0.jar:na] >>> at com.google.common.util.concurrent.MoreExecutors$DirectExecut >>> or.execute(MoreExecutors.java:457) [guava-18.0.jar:na] >>> at com.google.common.util.concurrent.ExecutionList.executeListener(Exec= utionList.java:156) >>> [guava-18.0.jar:na] >>> at com.google.common.util.concurrent.ExecutionList.execute(ExecutionLis= t.java:145) >>> [guava-18.0.jar:na] >>> at com.google.common.util.concurrent.AbstractFuture.setException(Abstra= ctFuture.java:202) >>> [guava-18.0.jar:na] >>> at org.apache.cassandra.streaming.StreamResultFuture.maybeCompl >>> ete(StreamResultFuture.java:215) [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.streaming.StreamResultFuture.handleSess >>> ionComplete(StreamResultFuture.java:191) [apache-cassandra-3.10.jar:3.1 >>> 0] >>> at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSess= ion.java:481) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.j= ava:571) >>> [apache-cassandra-3.10.jar:3.10] >>> at org.apache.cassandra.streaming.StreamReceiveTask$OnCompletio >>> nRunnable.run(StreamReceiveTask.java:251) [apache-cassandra-3.10.jar:3.= 1 >>> 0] >>> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:5= 11) >>> [na:1.8.0_102] >>> at java.util.concurrent.FutureTask.run(FutureTask.java:266) >>> [na:1.8.0_102] >>> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor= .java:1142) >>> [na:1.8.0_102] >>> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecuto= r.java:617) >>> [na:1.8.0_102] >>> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102] >>> >>> netstat output of 10.23.71.6, the node mentioned above in the debug.log= , >>> obviously all files + bytes have been transferred >>> =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D >>> Mode: NORMAL >>> Bootstrap b998aec0-b2fd-11e6-a63d-75828fa8d45c >>> /10.23.71.8 >>> Sending 1598 files, 60610896516 bytes total. Already sent 1598 files, >>> 60610896516 bytes total >>> /var/lib/cassandra/data/log/log_fake-b130c05070e611e6986e29a4f0eae2e7/m= c-97218-big-Data.db >>> 29425392/29425392 bytes(100%) sent to idx:0/10.23.71.8 >>> ... >>> >>> nodetool info on new node says >>> =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D >>> ID : 9dedcc9a-d951-4c7a-b794-434db1af960f >>> Gossip active : true >>> Thrift active : true >>> Native Transport active: true >>> Load : 185.2 GiB >>> Generation No : 1480071319 >>> Uptime (seconds) : 25082 >>> Heap Memory (MB) : 3607.60 / 15974.44 >>> Off Heap Memory (MB) : 405.70 >>> Data Center : DC1 >>> Rack : RAC1 >>> Exceptions : 0 >>> Key Cache : entries 1312994, size 100 MiB, capacity 100 >>> MiB, 126413327 hits, 162672698 requests, 0.777 recent hit rate, 14400 s= ave >>> period in seconds >>> Row Cache : entries 0, size 0 bytes, capacity 0 bytes, 0 >>> hits, 0 requests, NaN recent hit rate, 0 save period in seconds >>> Counter Cache : entries 0, size 0 bytes, capacity 50 MiB, 0 >>> hits, 0 requests, NaN recent hit rate, 7200 save period in seconds >>> Chunk Cache : entries 7680, size 480 MiB, capacity 480 MiB, >>> 8277584 misses, 147262566 requests, 0.944 recent hit rate, 835.412 >>> microseconds miss latency >>> Percent Repaired : 4.232661592656687% >>> Token : (node is not joined to the cluster) >>> >>> Any idea whats going wrong? >>> >>> Same situation was when I bootstrapped a node last time. In the end I >>> just started the node with auto_bootstrap=3Dfalse to get it up and runn= ing >>> and I ran repair afterwards. I'd like to avoid that repair and all the >>> inconsistencies this time. >>> >>> -- >>> Benjamin Roth >>> Prokurist >>> >>> Jaumo GmbH =C2=B7 www.jaumo.com >>> Wehrstra=C3=9Fe 46 =C2=B7 73035 G=C3=B6ppingen =C2=B7 Germany >>> Phone +49 7161 304880-6 =C2=B7 Fax +49 7161 304880-1 >>> AG Ulm =C2=B7 HRB 731058 =C2=B7 Managing Director: Jens Kammerer >>> >> >> > > > -- > Benjamin Roth > Prokurist > > Jaumo GmbH =C2=B7 www.jaumo.com > Wehrstra=C3=9Fe 46 =C2=B7 73035 G=C3=B6ppingen =C2=B7 Germany > Phone +49 7161 304880-6 =C2=B7 Fax +49 7161 304880-1 > AG Ulm =C2=B7 HRB 731058 =C2=B7 Managing Director: Jens Kammerer > --=20 Benjamin Roth Prokurist Jaumo GmbH =C2=B7 www.jaumo.com Wehrstra=C3=9Fe 46 =C2=B7 73035 G=C3=B6ppingen =C2=B7 Germany Phone +49 7161 304880-6 =C2=B7 Fax +49 7161 304880-1 AG Ulm =C2=B7 HRB 731058 =C2=B7 Managing Director: Jens Kammerer --001a113cfa702f9b780542252d56 Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
I proposed a quite simple fix for=C2=A0https://issues.apache.org/ji= ra/browse/CASSANDRA-12905

Sorry that I don't sup= ply a patch. I am good at analysing code but totally unexperienced with the= workflows here.

2016-11-25 19:57 GMT+01:00 Benjamin Roth <<= a href=3D"mailto:benjamin.roth@jaumo.com" target=3D"_blank">benjamin.roth@j= aumo.com>:
Yes, I have MVs.

Interesting is also that in the = middle of bootstrapping (cannot tell when exactly) it seemed like other nod= es started to send hints to the bootstrapping node. When that happened, it = seems that every single HintVerb fails also with a WTE. At least the logs a= re completely flooded with WTE. When I paused hints on all other nodes, log= s were quiet again.

I completely restarted the bootstrap= (deleted /var/lib/cassandra) - this time with hints paused from the beginn= ing. We will see if that changes anything.

I find = it also quite weird that other nodes have hints for a bootstrapping node. I= s that intended behaviour?
And is it possible that streaming lock= s the whole CF? I looked like absolutely no hint could be delivered success= fully.

2016-11-25 19:43 GMT+01:00 Paulo M= otta <pauloricardomg@gmail.com>:
If you have an MV table It seems you're = hitting=C2=A0https://issues.apache.org/jira/browse/CASSANDRA-= 12905. I will bump it's priority to critical since it can prev= ent or difficult bootstrap.

Did you try resuming bootstr= ap with "nodetool bootstrap resume" after the failure? It may eve= ntually succeed, since this is an MV lock contention problem.

2016-11-25 1= 5:59 GMT-02:00 Benjamin Roth <benjamin.roth@jaumo.com>= :
Hi!

Today I wanted a new node to join the cluster.
When looking at n= etstats on all the old nodes, it seemed like the streaming sessions did com= plete.
They all said that all files have been transferred. But lo= oking at the debug.log the stream sessions finished with an error.
Also = after all streams have been done the node remains in state "JOINING&qu= ot;.

See logs:

debug.log, last words
=3D=3D=3D=3D= =3D=3D=3D=3D
ERROR [StreamReceiveTask:94] 2016-11-25 17:50:51,712 Stream= Session.java:593 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8d45c] Stre= aming error occurred on session with peer 10.23.71.6
org.apache.cassandr= a.exceptions.WriteTimeoutException: Operation timed out - received onl= y 0 responses.
at org.apache.cassandra.db.Keyspace.apply(Keyspace.j= ava:525) ~[apache-cassandra-3.10.jar:3.10]
at org.apache.cassandra.= db.Keyspace.applyNotDeferrable(Keyspace.java:440) ~[apache-cassan= dra-3.10.jar:3.10]
at org.apache.cassandra.db.Mutation.apply(M= utation.java:223) ~[apache-cassandra-3.10.jar:3.10]
at org.apache.c= assandra.db.Mutation.applyUnsafe(Mutation.java:242) ~[apache-cass= andra-3.10.jar:3.10]
at org.apache.cassandra.streaming.StreamR= eceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:205) ~= [apache-cassandra-3.10.jar:3.10]
at java.util.concurrent.Executors<= wbr>$RunnableAdapter.call(Executors.java:511) [na:1.8.0_102]
at jav= a.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_102]at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool= Executor.java:1142) [na:1.8.0_102]
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_102]
a= t java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]
DEBUG [STREA= M-OUT-/10.23.71.6:7000= ] 2016-11-25 17:50:51,713 ConnectionHandler.java:388 - [Stream #b998aec= 0-b2fd-11e6-a63d-75828fa8d45c] Sending Session Failed
DEBUG [Stream= ReceiveTask:94] 2016-11-25 17:50:51,713 StreamSession.java:472 - [Stream #b= 998aec0-b2fd-11e6-a63d-75828fa8d45c] Finishing keep-alive task.
DEB= UG [StreamReceiveTask:94] 2016-11-25 17:50:51,713 ConnectionHandler.java:12= 0 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8d45c] Closing stream conn= ection handler on /10.23.71= .6
INFO =C2=A0[StreamReceiveTask:94] 2016-11-25 17:50:51,719 StreamR= esultFuture.java:187 - [Stream #b998aec0-b2fd-11e6-a63d-75828fa8d45c] = Session with /10.23.71.6 is complete
DEBUG [StreamReceiveTask:94] 2016-11-25 17:50:51,719 Strea= mCoordinator.java:146 - Finished connecting all sessions
WARN =C2=A0[Str= eamReceiveTask:94] 2016-11-25 17:50:51,723 StreamResultFuture.java:214 - [S= tream #b998aec0-b2fd-11e6-a63d-75828fa8d45c] Stream failed
ERROR [m= ain] 2016-11-25 17:50:51,724 StorageService.java:1493 - Error while waiting= on bootstrap to complete. Bootstrap will have to be restarted.
java.uti= l.concurrent.ExecutionException: org.apache.cassandra.streaming.S= treamException: Stream failed
at com.google.common.util.concurrent.= AbstractFuture$Sync.getValue(AbstractFuture.java:299) ~[guava-18.0.jar= :na]
at com.google.common.util.concurrent.AbstractFuture$Sync.get(A= bstractFuture.java:286) ~[guava-18.0.jar:na]
at com.google.common.u= til.concurrent.AbstractFuture.get(AbstractFuture.java:116) ~[guav= a-18.0.jar:na]
at org.apache.cassandra.service.StorageService.boots= trap(StorageService.java:1488) [apache-cassandra-3.10.jar:3.10]at org.apache.cassandra.service.StorageService.joinTokenRing(St= orageService.java:948) [apache-cassandra-3.10.jar:3.10]
at org.apac= he.cassandra.service.StorageService.initServer(StorageService.jav= a:667) [apache-cassandra-3.10.jar:3.10]
at org.apache.cassandra.ser= vice.StorageService.initServer(StorageService.java:598) [apache-c= assandra-3.10.jar:3.10]
at org.apache.cassandra.service.Cassan= draDaemon.setup(CassandraDaemon.java:394) [apache-cassandra-3.10.jar:3= .10]
at org.apache.cassandra.service.CassandraDaemon.activate(= CassandraDaemon.java:601) [apache-cassandra-3.10.jar:3.10]
at = org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon= .java:735) [apache-cassandra-3.10.jar:3.10]
Caused by: org.apache.c= assandra.streaming.StreamException: Stream failed
at org.apache.cas= sandra.streaming.StreamResultFuture.maybeComplete(StreamResultFut= ure.java:215) ~[apache-cassandra-3.10.jar:3.10]
at org.apache.= cassandra.streaming.StreamResultFuture.handleSessionComplete(Stre= amResultFuture.java:191) ~[apache-cassandra-3.10.jar:3.10]
at = org.apache.cassandra.streaming.StreamSession.closeSession(StreamS= ession.java:481) ~[apache-cassandra-3.10.jar:3.10]
at org.apache.ca= ssandra.streaming.StreamSession.onError(StreamSession.java:571) ~= [apache-cassandra-3.10.jar:3.10]
at org.apache.cassandra.streaming<= wbr>.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask= .java:251) ~[apache-cassandra-3.10.jar:3.10]
at java.util.concurren= t.Executors$RunnableAdapter.call(Executors.java:511) ~[na:1.8.0_1= 02]
at java.util.concurrent.FutureTask.run(FutureTask.java:266) ~[n= a:1.8.0_102]
at java.util.concurrent.ThreadPoolExecutor.runWorker(T= hreadPoolExecutor.java:1142) ~[na:1.8.0_102]
at java.util.concurren= t.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) ~[na= :1.8.0_102]
at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_102= ]
WARN =C2=A0[StreamReceiveTask:94] 2016-11-25 17:50:51,731 StorageServi= ce.java:1483 - Error during bootstrap.
org.apache.cassandra.streaming.StreamException: Stream failed
at org.apache.cassandra.streaming= .management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifi= er.java:88) ~[apache-cassandra-3.10.jar:3.10]
at com.google.common.= util.concurrent.Futures$6.run(Futures.java:1310) [guava-18.0.jar:= na]
at com.google.common.util.concurrent.MoreExecutors$DirectExecut= or.execute(MoreExecutors.java:457) [guava-18.0.jar:na]
at com.= google.common.util.concurrent.ExecutionList.executeListener(Execu= tionList.java:156) [guava-18.0.jar:na]
at com.google.common.util.concurr= ent.ExecutionList.execute(ExecutionList.java:145) [guava-18.0.jar= :na]
at com.google.common.util.concurrent.AbstractFuture.setExcepti= on(AbstractFuture.java:202) [guava-18.0.jar:na]
at org.apache.cassa= ndra.streaming.StreamResultFuture.maybeComplete(StreamResultFutur= e.java:215) [apache-cassandra-3.10.jar:3.10]
at org.apache.cas= sandra.streaming.StreamResultFuture.handleSessionComplete(StreamR= esultFuture.java:191) [apache-cassandra-3.10.jar:3.10]
at org.= apache.cassandra.streaming.StreamSession.closeSession(StreamSessi= on.java:481) [apache-cassandra-3.10.jar:3.10]
at org.apache.cassand= ra.streaming.StreamSession.onError(StreamSession.java:571) [apach= e-cassandra-3.10.jar:3.10]
at org.apache.cassandra.streaming.S= treamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:= 251) [apache-cassandra-3.10.jar:3.10]
at java.util.concurrent.Execu= tors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_102]
a= t java.util.concurrent.FutureTask.run(FutureTask.java:266) [na:1.8.0_1= 02]
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPool= Executor.java:1142) [na:1.8.0_102]
at java.util.concurrent.ThreadPo= olExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_102]=
at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]

net= stat output of 10.23.71.6, the node mentioned above in the debug.log, obvio= usly all files + bytes have been transferred
=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D
Mode: NORMAL
Bootstrap b998aec0-b2fd-11e6-a63d-75828fa8d45c
/
10.23.71.8=
Sending 1598 files, 60610896516 bytes total. Already sent 1598 files, 6= 0610896516 bytes total
/var/lib/cassandra/data/log/log_fake-b130c05= 070e611e6986e29a4f0eae2e7/mc-97218-big-Data.db 29425392/29425392 bytes= (100%) sent to idx:0/10.23.= 71.8
...

nodetool info on new node says
=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D
ID =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2= =A0 =C2=A0 =C2=A0 =C2=A0 : 9dedcc9a-d951-4c7a-b794-434db1af960f
Gos= sip active =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0: true
Thrift active =C2=A0= =C2=A0 =C2=A0 =C2=A0 =C2=A0: true
Native Transport active: true
Load= =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 : 185.2 GiB=
Generation No =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0: 1480071319
Uptime = (seconds) =C2=A0 =C2=A0 =C2=A0 : 25082
Heap Memory (MB) =C2=A0 =C2=A0 = =C2=A0 : 3607.60 / 15974.44
Off Heap Memory (MB) =C2=A0 : 405.70
Data= Center =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0: DC1
Rack =C2=A0 =C2= =A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 : RAC1
Exceptions = =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 : 0
Key Cache =C2=A0 =C2=A0 = =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0: entries 1312994, size 100 MiB, capacity= 100 MiB, 126413327 hits, 162672698 requests, 0.777 recent hit rate, 14400 = save period in seconds
Row Cache =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2= =A0 =C2=A0: entries 0, size 0 bytes, capacity 0 bytes, 0 hits, 0 requests, = NaN recent hit rate, 0 save period in seconds
Counter Cache =C2=A0 =C2= =A0 =C2=A0 =C2=A0 =C2=A0: entries 0, size 0 bytes, capacity 50 MiB, 0 hits,= 0 requests, NaN recent hit rate, 7200 save period in seconds
Chunk Cach= e =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0: entries 7680, size 480 MiB, ca= pacity 480 MiB, 8277584 misses, 147262566 requests, 0.944 recent hit rate, = 835.412 microseconds miss latency
Percent Repaired =C2=A0 =C2=A0 =C2=A0 = : 4.232661592656687%
Token =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2= =A0 =C2=A0 =C2=A0: (node is not joined to the cluster)

=
Any idea whats going wrong?

Same situation wa= s when I bootstrapped a node last time. In the end I just started the node = with auto_bootstrap=3Dfalse to get it up and running and I ran repair after= wards. I'd like to avoid that repair and all the inconsistencies this t= ime.
=

--
Benjamin Roth
Prokurist

Jaumo GmbH =C2=B7 www.jaumo.com
Wehrstra=C3=9Fe= 46 =C2=B7 73035 G=C3=B6ppingen =C2=B7 Germany
Phone +49 7161 30= 4880-6 =C2=B7 Fax +49 7161 304880-1
AG Ulm =C2=B7 HRB 73= 1058 =C2=B7 Managing Director: Jens Kammerer




--
=
Benjamin Roth
Prokurist

Jaumo GmbH = =C2=B7 www.jaumo.com=
Wehrstra=C3=9Fe 46 =C2=B7 73035 G=C3=B6ppingen =C2=B7 Germany
Phone = +49 7161 304880-6 =C2=B7 Fax +49 7161 304880-1
A= G Ulm =C2=B7 HRB 731058 =C2=B7 Managing Director: Jens Kammerer



--
=
Benjamin Roth
Prokurist

Jaumo GmbH =C2=B7 www.jaumo.com
Wehrstra=C3=9Fe 4= 6 =C2=B7 73035 G=C3=B6ppingen =C2=B7 Germany
Phone +49 7161 304880-6 =C2= =B7 Fax +49 7161 304880-1
AG Ulm =C2=B7 HRB 731058 =C2=B7 Managing Direc= tor: Jens Kammerer
--001a113cfa702f9b780542252d56--