Well, that certainly looks like you've hit potentially multiple bugs there.
You could set auto_bootstrap to false, but you'll probably still want to
run a repair afterwards (note this is not good advice for a production
cluster).

You should probably raise a JIRA with all the relevant info you've posted
here and the full logs if you can.

On Thu, May 13, 2021 at 5:18 AM Joe Obernberger <
joseph.obernber...@gmail.com> wrote:

> HI All - I tried adding another node with RC1, it streams for a while, but
> then I get:
>
> INFO  [NonPeriodicTasks:1] 2021-05-12 13:29:22,382 LogTransaction.java:240
> - Unfinished transaction log, deleting
> /data/8/cassandra/data/doc/doc-839e5850ad9f11ebade9fdc1d34336d3/na-982-big-Data.db
> INFO  [NonPeriodicTasks:1] 2021-05-12 13:29:22,382 SSTable.java:111 -
> Deleting sstable:
> /data/8/cassandra/data/doc/doc-839e5850ad9f11ebade9fdc1d34336d3/na-982-big
> INFO  [NonPeriodicTasks:1] 2021-05-12 13:29:22,382 FileUtils.java:545 -
> Deleting file during startup:
> /data/8/cassandra/data/doc/doc-839e5850ad9f11ebade9fdc1d34336d3/na-982-big-Summary.db
> INFO  [NonPeriodicTasks:1] 2021-05-12 13:29:22,387 LogTransaction.java:240
> - Unfinished transaction log, deleting
> /data/8/cassandra/data/doc/doc-839e5850ad9f11ebade9fdc1d34336d3/na-969-big-Data.db
> INFO  [NonPeriodicTasks:1] 2021-05-12 13:29:22,397 SSTable.java:111 -
> Deleting sstable:
> /data/8/cassandra/data/doc/doc-839e5850ad9f11ebade9fdc1d34336d3/na-969-big
> INFO  [NonPeriodicTasks:1] 2021-05-12 13:29:22,398 FileUtils.java:545 -
> Deleting file during startup:
> /data/8/cassandra/data/doc/doc-839e5850ad9f11ebade9fdc1d34336d3/na-969-big-Summary.db
> INFO  [Stream-Deserializer-/172.16.100.39:7000-d9722b1c] 2021-05-12
> 13:29:22,605 StreamResultFuture.java:192 - [Stream
> #e4f33df0-b33f-11eb-85f3-15cea6735fa9] Session with /172.16.100.39:7000
> is complete
> WARN  [Stream-Deserializer-/172.16.100.39:7000-d9722b1c] 2021-05-12
> 13:29:22,619 StreamResultFuture.java:219 - [Stream
> #e4f33df0-b33f-11eb-85f3-15cea6735fa9] Stream failed
> ERROR [main] 2021-05-12 13:29:22,621 StorageService.java:1773 - 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.getDoneValue(AbstractFuture.java:552)
>         at
> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
>         at
> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1766)
>         at
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:1054)
>         at
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:1015)
>         at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:799)
>         at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:729)
>         at
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:420)
>         at
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:763)
>         at
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:887)
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
>         at
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
>         at
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
>         at
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:506)
>         at
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:837)
>         at
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:596)
>         at
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
>         at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>         at java.base/java.lang.Thread.run(Thread.java:829)
> WARN  [main] 2021-05-12 13:29:22,678 StorageService.java:1090 - Some data
> streaming failed. Use nodetool to check bootstrap state and resume. For
> more, see `nodetool help bootstrap`. IN_PROGRESS
> INFO  [main] 2021-05-12 13:29:22,678 Gossiper.java:2128 - Waiting for
> gossip to settle...
>
> Status:
>
> Datacenter: datacenter1
> =======================
> Status=Up/Down
> |/ State=Normal/Leaving/Joining/Moving
> --  Address         Load        Tokens  Owns (effective)  Host
> ID                               Rack
> UN  172.16.100.208  15.18 GiB   30      9.3%
> 2529b6ed-cdb2-43c2-bdd7-171cfe308bd3  rack1
> UJ  172.16.100.252  82.54 GiB   200     ?
> e83aa851-69b4-478f-88f6-60e657ea6539  rack1
> UN  172.16.100.249  112.64 GiB  200     62.9%
> 49e4f571-7d1c-4e1e-aca7-5bbe076596f7  rack1
> UN  172.16.100.36   111.08 GiB  200     62.9%
> d9702f96-256e-45ae-8e12-69a42712be50  rack1
> UN  172.16.100.39   113.87 GiB  200     63.0%
> 93f9cb0f-ea71-4e3d-b62a-f0ea0e888c47  rack1
> UN  172.16.100.253  1.97 GiB    4       1.3%
> a1a16910-9167-4174-b34b-eb859d36347e  rack1
> UN  172.16.100.248  112.25 GiB  200     62.9%
> 4bbbe57c-6219-41e5-bbac-de92a9594d53  rack1
> UN  172.16.100.37   66.09 GiB   120     37.7%
> 08a19658-40be-4e55-8709-812b3d4ac750  rack1
>
> I then executed:
> nodetool bootstrap resume
>
> It ran for about 20min:
>
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2706%)
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2707%)
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2707%)
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2707%)
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2708%)
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2708%)
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2709%)
> [2021-05-12 14:16:35,034] received file doc/indexorganize-24 (progress:
> 2709%)
> [2021-05-12 14:16:35,049] received file
> system_distributed/repair_history-0 (progress: 2710%)
> [2021-05-12 14:16:35,065] received file
> system_distributed/repair_history-1 (progress: 2710%)
> [2021-05-12 14:16:35,082] received file
> system_distributed/repair_history-2 (progress: 2710%)
> [2021-05-12 14:16:35,094] received file
> system_distributed/repair_history-3 (progress: 2711%)
> [2021-05-12 14:16:35,102] received file
> system_distributed/repair_history-4 (progress: 2711%)
> [2021-05-12 14:16:35,111] received file
> system_distributed/repair_history-5 (progress: 2712%)
> [2021-05-12 14:16:35,160] received file
> system_distributed/repair_history-6 (progress: 2712%)
> [2021-05-12 14:16:35,200] received file
> system_distributed/repair_history-7 (progress: 2713%)
> [2021-05-12 14:16:35,232] received file
> system_distributed/repair_history-8 (progress: 2713%)
> [2021-05-12 14:16:36,113] session with /172.16.100.37:7000 complete
> (progress: 2713%)
> [2021-05-12 14:16:36,113] Stream failed
> [2021-05-12 14:16:36,113] Error during bootstrap: Stream failed
> [2021-05-12 14:16:36,113] Resume bootstrap complete
>
> The node is still in the unjoined state.  I tried it again and got:
>
> Resuming bootstrap
> [2021-05-12 15:13:54,071] prepare with /172.16.100.249:7000 complete
> (progress: 0%)
> [2021-05-12 15:13:54,469] session with /172.16.100.249:7000 complete
> (progress: 0%)
> [2021-05-12 15:13:56,478] prepare with /172.16.100.253:7000 complete
> (progress: 0%)
> [2021-05-12 15:13:57,280] session with /172.16.100.253:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:00,082] prepare with /172.16.100.36:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:00,487] session with /172.16.100.36:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:04,890] prepare with /172.16.100.37:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:05,291] session with /172.16.100.37:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:07,285] prepare with /172.16.100.39:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:07,686] session with /172.16.100.39:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:16,219] prepare with /172.16.100.208:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:16,620] session with /172.16.100.208:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:41,424] prepare with /172.16.100.248:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:41,825] session with /172.16.100.248:7000 complete
> (progress: 0%)
> [2021-05-12 15:14:41,827] Stream failed
> [2021-05-12 15:14:41,827] Error during bootstrap: Stream failed
> [2021-05-12 15:14:41,827] Resume bootstrap complete
>
> The debug log has:
>
> ERROR [Stream-Deserializer-/172.16.100.248:7000-4810ecd6] 2021-05-12
> 15:14:41,824 StorageService.java:1874 - Error during bootstrap: Stream
> failed
> org.apache.cassandra.streaming.StreamException: Stream failed
>         at
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
>         at
> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
>         at
> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
>         at
> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
>         at
> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
>         at
> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
>         at
> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
>         at
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
>         at
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:506)
>         at
> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:837)
>         at
> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:596)
>         at
> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
>         at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>         at java.base/java.lang.Thread.run(Thread.java:829)
> DEBUG [Stream-Deserializer-/172.16.100.248:7000-4810ecd6] 2021-05-12
> 15:14:41,824 StreamSession.java:638 - [Stream
> #0fb4f950-b356-11eb-85f3-15cea6735fa9] Socket closed after session
> completed with state COMPLETE
>
> At this point, I plan on setting auto_bootstrap to false?
>
> -Joe
> On 5/10/2021 8:17 PM, Kane Wilson wrote:
>
> Well, that sounds like a dangerous sequence of events, but should have
> worked in the end regardless. Probably next time give it a bit more time
> and keep an eye on netstats and compactionstats.
>
>
> raft.so - Cassandra consulting, support, and managed services
>
>
> On Mon, May 10, 2021 at 10:23 PM Joe Obernberger <
> joseph.obernber...@gmail.com> wrote:
>
>> Hi - I waited 3 hours.  It was syncing up data; I could see network
>> traffic, but then it stopped.  I didn't check netstats, but I did check
>> compactionstats and there were no pending tasks.  I then set auto_bootstrap
>> to false on both new machines and they joined.  Then ran a repair.
>>
>> -Joe
>> On 5/9/2021 7:12 PM, Kane Wilson wrote:
>>
>> How long are you waiting for the node to join? Have you checked nodetool
>> netstats and compactionstats to see if all streams/compactions are complete?
>>
>> raft.so - Cassandra consulting, support, and managed services
>>
>>
>> On Sat, May 8, 2021 at 11:23 AM Joe Obernberger <
>> joseph.obernber...@gmail.com> wrote:
>>
>>> Whoops - had it in the wrong datacenter.  Same issue - new node is
>>> stuck in UJ, but I can start/stop OK with systemctl.
>>>
>>> Datacenter: datacenter1
>>> =======================
>>> Status=Up/Down
>>> |/ State=Normal/Leaving/Joining/Moving
>>> --  Address                    Load
>>> Tokens  Owns (effective)  Host
>>> ID                               Rack
>>> UN  helene.querymasters.com    423.92 MiB  30    Â
>>> 18.6%             2529b6ed-cdb2-43c2-bdd7-171cfe308bd3  rack1
>>> UJ  fortuna.querymasters.com   1.75 GiB    200   Â
>>> ?                 49e4f571-7d1c-4e1e-aca7-5bbe076596f7
>>> rack1
>>> UN  charon.querymasters.com    2.22 GiB    200   Â
>>> 98.5%             d9702f96-256e-45ae-8e12-69a42712be50  rack1
>>> UN  eros.querymasters.com      2.21 GiB    200   Â
>>> 98.5%             93f9cb0f-ea71-4e3d-b62a-f0ea0e888c47  rack1
>>> UN  hercules.querymasters.com  58.65 MiB   4     Â
>>> 2.6%              a1a16910-9167-4174-b34b-eb859d36347e  rack1
>>> UN  chaos.querymasters.com     1.82 GiB    120   Â
>>> 81.8%             08a19658-40be-4e55-8709-812b3d4ac750  rack1
>>>
>>> I am able to restart the server (fortuna - after about 3 hours), but I
>>> then get this:
>>>
>>> ERROR [Stream-Deserializer-/172.16.100.253:7000-493728e3] 2021-05-07
>>> 21:17:35,805 StreamingInboundHandler.java:205 - [Stream channel:
>>> 493728e3] stream operation from /172.16.100.253:7000 failed
>>> java.lang.IllegalStateException: unknown stream session:
>>> 27c00760-af9b-11eb-b7ee-5d6a136b5405 - 0
>>>         at
>>>
>>> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:45)
>>>         at
>>>
>>> org.apache.cassandra.streaming.messages.IncomingStreamMessage$1.deserialize(IncomingStreamMessage.java:38)
>>>         at
>>>
>>> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:53)
>>>         at
>>>
>>> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:172)
>>>         at
>>>
>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>>>         at java.base/java.lang.Thread.run(Thread.java:829)
>>> ERROR [Stream-Deserializer-/172.16.100.253:7000-e313e37d] 2021-05-07
>>> 21:17:36,208 StreamSession.java:882 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405] Remote peer /172.16.100.253:7000
>>> failed stream session.
>>> INFO  [Stream-Deserializer-/172.16.100.253:7000-e313e37d] 2021-05-07
>>> 21:17:36,209 StreamResultFuture.java:192 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405] Session with /172.16.100.253:7000
>>> is complete
>>> INFO  [Stream-Deserializer-/172.16.100.253:7000-e313e37d] 2021-05-07
>>> 21:17:36,209 StreamSession.java:359 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405] Starting streaming to
>>> /172.16.100.37:7000
>>> INFO  [Stream-Deserializer-/172.16.100.253:7000-e313e37d] 2021-05-07
>>> 21:17:36,214 StreamCoordinator.java:263 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405, ID#0] Beginning stream session
>>> with /172.16.100.37:7000
>>> INFO  [Stream-Deserializer-/172.16.100.36:7000-9d343b7e] 2021-05-07
>>> 21:17:37,808 StreamResultFuture.java:178 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405 ID#0] Prepare completed. Receiving
>>> 0 files(0.000KiB), sending 0 files(0.000KiB)
>>> INFO  [Stream-Deserializer-/172.16.100.39:7000-1c5eddba] 2021-05-07
>>> 21:17:37,809 StreamResultFuture.java:178 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405 ID#0] Prepare completed. Receiving
>>> 0 files(0.000KiB), sending 0 files(0.000KiB)
>>> INFO  [Stream-Deserializer-/172.16.100.36:7000-9d343b7e] 2021-05-07
>>> 21:17:38,209 StreamResultFuture.java:192 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405] Session with /172.16.100.36:7000
>>> is complete
>>> INFO  [Stream-Deserializer-/172.16.100.39:7000-1c5eddba] 2021-05-07
>>> 21:17:38,210 StreamResultFuture.java:192 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405] Session with /172.16.100.39:7000
>>> is complete
>>> INFO  [Stream-Deserializer-/172.16.100.37:7000-d2676988] 2021-05-07
>>> 21:17:41,416 StreamResultFuture.java:178 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405 ID#0] Prepare completed. Receiving
>>> 0 files(0.000KiB), sending 0 files(0.000KiB)
>>> INFO  [Stream-Deserializer-/172.16.100.37:7000-d2676988] 2021-05-07
>>> 21:17:41,818 StreamResultFuture.java:192 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405] Session with /172.16.100.37:7000
>>> is complete
>>> WARN  [Stream-Deserializer-/172.16.100.37:7000-d2676988] 2021-05-07
>>> 21:17:41,822 StreamResultFuture.java:219 - [Stream
>>> #27c00760-af9b-11eb-b7ee-5d6a136b5405] Stream failed
>>> ERROR [main] 2021-05-07 21:17:41,823 StorageService.java:1773 - 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.getDoneValue(AbstractFuture.java:552)
>>>         at
>>>
>>> com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:533)
>>>         at
>>>
>>> org.apache.cassandra.service.StorageService.bootstrap(StorageService.java:1766)
>>>         at
>>>
>>> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:1054)
>>>         at
>>>
>>> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:1015)
>>>         at
>>>
>>> org.apache.cassandra.service.StorageService.initServer(StorageService.java:799)
>>>         at
>>>
>>> org.apache.cassandra.service.StorageService.initServer(StorageService.java:729)
>>>         at
>>>
>>> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:420)
>>>         at
>>>
>>> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:763)
>>>         at
>>>
>>> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:887)
>>> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
>>>         at
>>>
>>> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:88)
>>>         at
>>>
>>> com.google.common.util.concurrent.Futures$CallbackListener.run(Futures.java:1056)
>>>         at
>>>
>>> com.google.common.util.concurrent.DirectExecutor.execute(DirectExecutor.java:30)
>>>         at
>>>
>>> com.google.common.util.concurrent.AbstractFuture.executeListener(AbstractFuture.java:1138)
>>>         at
>>>
>>> com.google.common.util.concurrent.AbstractFuture.complete(AbstractFuture.java:958)
>>>         at
>>>
>>> com.google.common.util.concurrent.AbstractFuture.setException(AbstractFuture.java:748)
>>>         at
>>>
>>> org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamResultFuture.java:220)
>>>         at
>>>
>>> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:196)
>>>         at
>>>
>>> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:506)
>>>         at
>>>
>>> org.apache.cassandra.streaming.StreamSession.complete(StreamSession.java:837)
>>>         at
>>>
>>> org.apache.cassandra.streaming.StreamSession.messageReceived(StreamSession.java:596)
>>>         at
>>>
>>> org.apache.cassandra.streaming.async.StreamingInboundHandler$StreamDeserializingTask.run(StreamingInboundHandler.java:189)
>>>         at
>>>
>>> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>>>         at java.base/java.lang.Thread.run(Thread.java:829)
>>> WARN  [main] 2021-05-07 21:17:41,843 StorageService.java:1090 - Some
>>> data streaming failed. Use nodetool to check bootstrap state and resume.
>>> For more, see `nodetool help bootstrap`. IN_PROGRESS
>>>
>>> -Joe
>>>
>>> On 5/7/2021 5:37 PM, Joe Obernberger wrote:
>>> > When I try to halt the joining node with systemctl stop cassandra, it
>>> > hangs.  I don't see it doing any network, disk, or CPU activity using
>>> > tools like iotop, atop, and top.
>>> >
>>> > I ended up kill -9'ing the process.  I tried the same join on a
>>> > different machine, and the same issue occurs.  It hangs in UJ.  I
>>> > deleted all data on the new node (not much there cuz it's new!), and
>>> > tried again.  Same issue.
>>> >
>>> > In other news, java 11 is working.  :)
>>> >
>>> > -Joe
>>> >
>>> >
>>> > On 5/7/2021 5:07 PM, Joe Obernberger wrote:
>>> >> Have an existing 5 node RC1 cluster and trying to join two more nodes
>>> >> to it.
>>> >> The new node is stuck in the UJ status:
>>> >>
>>> >> Datacenter: datacenter1
>>> >> =======================
>>> >> Status=Up/Down
>>> >> |/ State=Normal/Leaving/Joining/Moving
>>> >> --  Address         Load        Tokens  Owns
>>> >> (effective)  Host
>>> >> ID                               Rack
>>> >> UN  172.16.100.208  410.12 MiB  30
>>> >> 18.6%           � 2529b6ed-cdb2-43c2-bdd7-171cfe308bd3
>>> >> rack1
>>> >> UN  172.16.100.36   2.15 GiB    200
>>> >> 98.5%           � d9702f96-256e-45ae-8e12-69a42712be50
>>> >> rack1
>>> >> UN  172.16.100.39   2.14 GiB    200
>>> >> 98.5%           � 93f9cb0f-ea71-4e3d-b62a-f0ea0e888c47
>>> >> rack1
>>> >> UN  172.16.100.253  56.97 MiB   4
>>> >> 2.6%            �
>>> >> a1a16910-9167-4174-b34b-eb859d36347e  rack1
>>> >> UN  172.16.100.37   1.77 GiB    120
>>> >> 81.8%           � 08a19658-40be-4e55-8709-812b3d4ac750
>>> >> rack1
>>> >>
>>> >> Datacenter: dc1
>>> >> ===============
>>> >> Status=Up/Down
>>> >> |/ State=Normal/Leaving/Joining/Moving
>>> >> --  Address         Load        Tokens  Owns
>>> >> (effective)  Host
>>> >> ID                               Rack
>>> >> UJ  172.16.100.248  1.31 MiB    200
>>> >> ?               �
>>> >> 054109ad-3a5e-4680-b4ad-f9c08089238c  rack1
>>> >>
>>> >> What can I check?
>>> >>
>>> >> -Joe
>>> >>
>>>
>>
>>
>> <http://www.avg.com/email-signature?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=emailclient>
>>  Virus-free.
>> www.avg.com
>> <http://www.avg.com/email-signature?utm_medium=email&utm_source=link&utm_campaign=sig-email&utm_content=emailclient>
>> <#m_7266124636560615226_m_8128678561155722873_DAB4FAD8-2DD7-40BB-A1B8-4E2AA1F9FDF2>
>>
>>

-- 
raft.so - Cassandra consulting, support, and managed services

Reply via email to