You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Benjamin Roth <be...@jaumo.com> on 2016/11/25 17:59:12 UTC

Bootstrap fails on 3.10

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
========
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:440)
~[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$OnCompletionRunnable.run(StreamReceiveTask.java:205)
~[apache-cassandra-3.10.jar:3.10]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
[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(ThreadPoolExecutor.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-75828fa8d45c]
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-75828fa8d45c]
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-75828fa8d45c] 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-75828fa8d45c] 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 restarted.
java.util.concurrent.ExecutionException:
org.apache.cassandra.streaming.StreamException: 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(AbstractFuture.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(StorageService.java:948)
[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.service.StorageService.initServer(StorageService.java:667)
[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.service.StorageService.initServer(StorageService.java:598)
[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.service.CassandraDaemon.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.cassandra.streaming.StreamException: Stream failed
at
org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:215)
~[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
~[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:481)
~[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:571)
~[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:251)
~[apache-cassandra-3.10.jar:3.10]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
~[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(ThreadPoolExecutor.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.StreamEventJMXNotifier.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$DirectExecutor.execute(MoreExecutors.java:457)
[guava-18.0.jar:na]
at
com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionList.java:156)
[guava-18.0.jar:na]
at
com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
[guava-18.0.jar:na]
at
com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:202)
[guava-18.0.jar:na]
at
org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:215)
[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:191)
[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:481)
[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:571)
[apache-cassandra-3.10.jar:3.10]
at
org.apache.cassandra.streaming.StreamReceiveTask$OnCompletionRunnable.run(StreamReceiveTask.java:251)
[apache-cassandra-3.10.jar:3.10]
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
[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(ThreadPoolExecutor.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
==========
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/mc-97218-big-Data.db
29425392/29425392 bytes(100%) sent to idx:0/10.23.71.8
...

nodetool info on new node says
==========
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 save
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=false to get it up and running and I
ran repair afterwards. I'd like to avoid that repair and all the
inconsistencies this time.

-- 
Benjamin Roth
Prokurist

Jaumo GmbH · www.jaumo.com
Wehrstraße 46 · 73035 Göppingen · Germany
Phone +49 7161 304880-6 · Fax +49 7161 304880-1
AG Ulm · HRB 731058 · Managing Director: Jens Kammerer

Re: Bootstrap fails on 3.10

Posted by Benjamin Roth <be...@jaumo.com>.
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 <be...@jaumo.com>:

> 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 HintVerb
> 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 bootstrapping
> 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 <pa...@gmail.com>:
>
>> 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 the
>> failure? It may eventually succeed, since this is an MV lock contention
>> problem.
>>
>> 2016-11-25 15:59 GMT-02:00 Benjamin Roth <be...@jaumo.com>:
>>
>>> 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
>>> ========
>>> 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:440)
>>> ~[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:511)
>>> [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(ThreadPoolExecutor.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-75828fa8d45c]
>>> 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-75828fa8d45c]
>>> 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-75828fa8d45c]
>>> 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-75828fa8d45c]
>>> 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 restarted.
>>> java.util.concurrent.ExecutionException: org.apache.cassandra.streaming.StreamException:
>>> 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(AbstractFuture.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(StorageService.java:948)
>>> [apache-cassandra-3.10.jar:3.10]
>>> at org.apache.cassandra.service.StorageService.initServer(StorageService.java:667)
>>> [apache-cassandra-3.10.jar:3.10]
>>> at org.apache.cassandra.service.StorageService.initServer(StorageService.java:598)
>>> [apache-cassandra-3.10.jar:3.10]
>>> at org.apache.cassandra.service.CassandraDaemon.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.cassandra.streaming.StreamException: Stream failed
>>> 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(StreamSession.java:481)
>>> ~[apache-cassandra-3.10.jar:3.10]
>>> at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java: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:511)
>>> ~[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(ThreadPoolExecutor.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(ExecutionList.java:156)
>>> [guava-18.0.jar:na]
>>> at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
>>> [guava-18.0.jar:na]
>>> at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.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(StreamSession.java:481)
>>> [apache-cassandra-3.10.jar:3.10]
>>> at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java: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:511)
>>> [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(ThreadPoolExecutor.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
>>> ==========
>>> 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/mc-97218-big-Data.db
>>> 29425392/29425392 bytes(100%) sent to idx:0/10.23.71.8
>>> ...
>>>
>>> nodetool info on new node says
>>> ==========
>>> 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 save
>>> 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=false to get it up and running
>>> and I ran repair afterwards. I'd like to avoid that repair and all the
>>> inconsistencies this time.
>>>
>>> --
>>> Benjamin Roth
>>> Prokurist
>>>
>>> Jaumo GmbH · www.jaumo.com
>>> Wehrstraße 46 · 73035 Göppingen · Germany
>>> Phone +49 7161 304880-6 · Fax +49 7161 304880-1
>>> AG Ulm · HRB 731058 · Managing Director: Jens Kammerer
>>>
>>
>>
>
>
> --
> Benjamin Roth
> Prokurist
>
> Jaumo GmbH · www.jaumo.com
> Wehrstraße 46 · 73035 Göppingen · Germany
> Phone +49 7161 304880-6 · Fax +49 7161 304880-1
> AG Ulm · HRB 731058 · Managing Director: Jens Kammerer
>



-- 
Benjamin Roth
Prokurist

Jaumo GmbH · www.jaumo.com
Wehrstraße 46 · 73035 Göppingen · Germany
Phone +49 7161 304880-6 · Fax +49 7161 304880-1
AG Ulm · HRB 731058 · Managing Director: Jens Kammerer

Re: Bootstrap fails on 3.10

Posted by Benjamin Roth <be...@jaumo.com>.
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 HintVerb
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 bootstrapping
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 <pa...@gmail.com>:

> 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 the
> failure? It may eventually succeed, since this is an MV lock contention
> problem.
>
> 2016-11-25 15:59 GMT-02:00 Benjamin Roth <be...@jaumo.com>:
>
>> 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
>> ========
>> 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:440)
>> ~[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:511)
>> [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(ThreadPoolExecutor.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-75828fa8d45c]
>> 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-75828fa8d45c]
>> 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-75828fa8d45c]
>> 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-75828fa8d45c]
>> 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 restarted.
>> java.util.concurrent.ExecutionException: org.apache.cassandra.streaming.StreamException:
>> 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(AbstractFuture.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(StorageService.java:948)
>> [apache-cassandra-3.10.jar:3.10]
>> at org.apache.cassandra.service.StorageService.initServer(StorageService.java:667)
>> [apache-cassandra-3.10.jar:3.10]
>> at org.apache.cassandra.service.StorageService.initServer(StorageService.java:598)
>> [apache-cassandra-3.10.jar:3.10]
>> at org.apache.cassandra.service.CassandraDaemon.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.cassandra.streaming.StreamException: Stream failed
>> 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(StreamSession.java:481)
>> ~[apache-cassandra-3.10.jar:3.10]
>> at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java: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:511)
>> ~[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(ThreadPoolExecutor.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(ExecutionList.java:156)
>> [guava-18.0.jar:na]
>> at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:145)
>> [guava-18.0.jar:na]
>> at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.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.10]
>> at org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:481)
>> [apache-cassandra-3.10.jar:3.10]
>> at org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java: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:511)
>> [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(ThreadPoolExecutor.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
>> ==========
>> 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/mc-97218-big-Data.db
>> 29425392/29425392 bytes(100%) sent to idx:0/10.23.71.8
>> ...
>>
>> nodetool info on new node says
>> ==========
>> 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 save
>> 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=false to get it up and running
>> and I ran repair afterwards. I'd like to avoid that repair and all the
>> inconsistencies this time.
>>
>> --
>> Benjamin Roth
>> Prokurist
>>
>> Jaumo GmbH · www.jaumo.com
>> Wehrstraße 46 · 73035 Göppingen · Germany
>> Phone +49 7161 304880-6 · Fax +49 7161 304880-1
>> AG Ulm · HRB 731058 · Managing Director: Jens Kammerer
>>
>
>


-- 
Benjamin Roth
Prokurist

Jaumo GmbH · www.jaumo.com
Wehrstraße 46 · 73035 Göppingen · Germany
Phone +49 7161 304880-6 · Fax +49 7161 304880-1
AG Ulm · HRB 731058 · Managing Director: Jens Kammerer

Re: Bootstrap fails on 3.10

Posted by Paulo Motta <pa...@gmail.com>.
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 the
failure? It may eventually succeed, since this is an MV lock contention
problem.

2016-11-25 15:59 GMT-02:00 Benjamin Roth <be...@jaumo.com>:

> 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
> ========
> 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:440)
> ~[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$
> OnCompletionRunnable.run(StreamReceiveTask.java:205)
> ~[apache-cassandra-3.10.jar:3.10]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> [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(ThreadPoolExecutor.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-75828fa8d45c]
> 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-75828fa8d45c]
> 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-75828fa8d45c]
> 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-75828fa8d45c]
> 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 restarted.
> java.util.concurrent.ExecutionException: org.apache.cassandra.streaming.StreamException:
> 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(AbstractFuture.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(StorageService.java:948)
> [apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.service.StorageService.initServer(StorageService.java:667)
> [apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.service.StorageService.initServer(StorageService.java:598)
> [apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.service.CassandraDaemon.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.cassandra.streaming.StreamException: Stream failed
> at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:215)
> ~[apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamResultFuture.
> handleSessionComplete(StreamResultFuture.java:191)
> ~[apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamSession.
> closeSession(StreamSession.java:481) ~[apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamSession.
> onError(StreamSession.java:571) ~[apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamReceiveTask$
> OnCompletionRunnable.run(StreamReceiveTask.java:251)
> ~[apache-cassandra-3.10.jar:3.10]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> ~[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(ThreadPoolExecutor.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.StreamEventJMXNotifier.
> 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$DirectExecutor.execute(MoreExecutors.java:457)
> [guava-18.0.jar:na]
> at com.google.common.util.concurrent.ExecutionList.
> executeListener(ExecutionList.java:156) [guava-18.0.jar:na]
> at com.google.common.util.concurrent.ExecutionList.
> execute(ExecutionList.java:145) [guava-18.0.jar:na]
> at com.google.common.util.concurrent.AbstractFuture.
> setException(AbstractFuture.java:202) [guava-18.0.jar:na]
> at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:215)
> [apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamResultFuture.
> handleSessionComplete(StreamResultFuture.java:191)
> [apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamSession.
> closeSession(StreamSession.java:481) [apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamSession.
> onError(StreamSession.java:571) [apache-cassandra-3.10.jar:3.10]
> at org.apache.cassandra.streaming.StreamReceiveTask$
> OnCompletionRunnable.run(StreamReceiveTask.java:251)
> [apache-cassandra-3.10.jar:3.10]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> [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(ThreadPoolExecutor.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
> ==========
> 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/mc-97218-big-Data.db
> 29425392/29425392 bytes(100%) sent to idx:0/10.23.71.8
> ...
>
> nodetool info on new node says
> ==========
> 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 save
> 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=false to get it up and running and I
> ran repair afterwards. I'd like to avoid that repair and all the
> inconsistencies this time.
>
> --
> Benjamin Roth
> Prokurist
>
> Jaumo GmbH · www.jaumo.com
> Wehrstraße 46 · 73035 Göppingen · Germany
> Phone +49 7161 304880-6 · Fax +49 7161 304880-1
> AG Ulm · HRB 731058 · Managing Director: Jens Kammerer
>