During the upgrade you'll want to avoid the following operations that
result in data streaming:

1. Bootstrapping nodes
2. Decomissioning nodes
3. Repair


On Mon, Oct 10, 2016 at 5:00 PM Jeff Jirsa <jeff.ji...@crowdstrike.com>
wrote:

>
>
> No need to cc dev@, user@ is the right list for this question.
>
>
>
> As Jon mentioned, you can’t stream (bootstrap/rebuild/repair) across major
> versions, so don’t try to destroy the cluster – just upgrade in place. It
> IS a good idea to do one DC at a time, but an in-place upgrade is pretty
> straightforward – flush, drain, stop Cassandra, replace binaries, start
> Cassandra, run nodetool upgradesstables -a.
>
>
>
> Note that you can run nodetool upgradesstables on more than one node at a
> time if you can tolerate the hit to your read latencies.
>
>
>
> It IS common, I imagine, for there to be schema mismatches temporarily
> while you have a mixed version cluster – this isn’t necessarily a huge
> problem, but do try to get to 3.0.8 as quickly as possible once you start,
> and if you can avoid administrative tasks (such as those that will change
> the schema) during the process, that’s generally advisable.
>
>
>
>
>
>
>
>
>
> *From: *Abhishek Verma <ve...@uber.com>
> *Reply-To: *"user@cassandra.apache.org" <user@cassandra.apache.org>
> *Date: *Monday, October 10, 2016 at 4:34 PM
> *To: *"user@cassandra.apache.org" <user@cassandra.apache.org>, "
> d...@cassandra.apache.org" <d...@cassandra.apache.org>
> *Subject: *Bootstrapping data from Cassandra 2.2.5 datacenter to 3.0.8
> datacenter fails because of streaming errors
>
>
>
> Hi Cassandra users,
>
>
>
> We are trying to upgrade our Cassandra version from 2.2.5 to 3.0.8
> (running on Mesos, but that's besides the point). We have two datacenters,
> so in order to preserve our data, we are trying to upgrade one datacenter
> at a time.
>
>
>
> Initially both DCs (dc1 and dc2) are running 2.2.5. The idea is to tear
> down dc1 completely (delete all the data in it), bring it up with 3.0.8,
> let data replicate from dc2 to dc1, and then tear down dc2, bring it up
> with 3.0.8 and replicate data from dc1.
>
>
>
> I am able to reproduce the problem on bare metal clusters running on 3
> nodes. I am using Oracle's server-jre-8u74-linux-x64 JRE.
>
>
>
> *Node A*: Downloaded 2.2.5-bin.tar.gz, changed the seeds to include its
> own IP address, changed listen_address and rpc_address to its own IP and
> changed endpoint_snitch to GossipingPropertyFileSnitch. I
> changed conf/cassandra-rackdc.properties to
>
> dc=dc2
>
> rack=rack2
>
> This node started up fine and is UN in nodetool status in dc2.
>
>
>
> I used CQL shell to create a table and insert 3 rows:
>
> verma@xxxxx:~/apache-cassandra-2.2.5$ bin/cqlsh $HOSTNAME
>
> Connected to Test Cluster at xxxxx:9042.
>
> [cqlsh 5.0.1 | Cassandra 2.2.5 | CQL spec 3.3.1 | Native protocol v4]
>
> Use HELP for help.
>
> cqlsh> desc tmp
>
>
>
> CREATE KEYSPACE tmp WITH replication = {'class':
> 'NetworkTopologyStrategy', 'dc1': '1', 'dc2': '1'}  AND durable_writes =
> true;
>
>
>
> CREATE TABLE tmp.map (
>
>     key text PRIMARY KEY,
>
>     value text
>
> )...;
>
> cqlsh> select * from tmp.map;
>
>
>
>  key | value
>
> -----+-------
>
>   k1 |    v1
>
>   k3 |    v3
>
>   k2 |    v2
>
>
>
>
>
> *Node B:* Downloaded 3.0.8-bin.tar.gz, changed the seeds to include
> itself and node A, changed listen_address and rpc_address to its own IP,
> changed endpoint_snitch to GossipingPropertyFileSnitch. I did not change
> conf/cassandra-rackdc.properties and its contents are
>
> dc=dc1
>
> rack=rack1
>
>
>
> In the logs, I see:
>
> INFO  [main] 2016-10-10 22:42:42,850 MessagingService.java:557 - Starting
> Messaging Service on /10.164.32.29:7000
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.164.32.29-3A7000&d=DQMFaQ&c=08AGY6txKsvMOP6lYkHQpPMRA1U6kqhAwGa8-0QCg3M&r=yfYEBHVkX6l0zImlOIBID0gmhluYPD5Jje-3CtaT3ow&m=aDxgr_M4K2TNU2MqOBLrREEaFoUbWmFpnP4l67CROag&s=t4xNW-nnA0EVZWnt-PSAoq4vx6Lep2ObgkG3ZuUUg7c&e=>
> (eth0)
>
> INFO  [main] 2016-10-10 22:42:42,864 StorageService.java:784 - This node
> will not auto bootstrap because it is configured to be a seed node.
>
>
>
> So I start a third node:
>
> *Node C:* Downloaded 3.0.8-bin.tar.gz, changed the seeds to include node
> A and node B, changed listen_address and rpc_address to its own IP, changed
> endpoint_snitch to GossipingPropertyFileSnitch. I did not change
> conf/cassandra-rackdc.properties.
>
> Now, nodetool status shows:
>
>
>
> verma@xxxxxxx:~/apache-cassandra-3.0.8$ bin/nodetool status
>
> Datacenter: dc1
>
> ===============
>
> Status=Up/Down
>
> |/ State=Normal/Leaving/Joining/Moving
>
> --  Address       Load       Tokens       Owns (effective)  Host ID
>                         Rack
>
> UJ  <Node C IP>   87.81 KB   256          ?
> 9064832d-ed5c-4c42-ad5a-f754b52b670c  rack1
>
> UN  <Node B IP>  107.72 KB  256          100.0%
>  28b1043f-115b-46a5-b6b6-8609829cde76  rack1
>
> Datacenter: dc2
>
> ===============
>
> Status=Up/Down
>
> |/ State=Normal/Leaving/Joining/Moving
>
> --  Address       Load       Tokens       Owns (effective)  Host ID
>                         Rack
>
> UN  <Node A IP>    73.2 KB    256          100.0%
>  09cc542c-2299-45a5-a4d1-159c239ded37  rack2
>
>
>
> Nodetool describe cluster shows:
>
> verma@xxxxxxx:~/apache-cassandra-3.0.8$ bin/nodetool describecluster
>
> Cluster Information:
>
> Name: Test Cluster
>
> Snitch: org.apache.cassandra.locator.DynamicEndpointSnitch
>
> Partitioner: org.apache.cassandra.dht.Murmur3Partitioner
>
> Schema versions:
>
> c2a2bb4f-7d31-3fb8-a216-00b41a643650: [<Node B IP>, <Node C IP>]
>
>
>
> 9770e3c5-3135-32e2-b761-65a0f6d8824e: [<Node A IP>]
>
>
>
> Note that there are two schema versions and they don't match.
>
>
>
> I see the following in the system.log:
>
>
>
> INFO  [InternalResponseStage:1] 2016-10-10 22:48:36,055
> ColumnFamilyStore.java:390 - Initializing system_auth.roles
>
> INFO  [main] 2016-10-10 22:48:36,316 StorageService.java:1149 - JOINING:
> waiting for schema information to complete
>
> INFO  [main] 2016-10-10 22:48:36,316 StorageService.java:1149 - JOINING:
> schema complete, ready to bootstrap
>
> INFO  [main] 2016-10-10 22:48:36,316 StorageService.java:1149 - JOINING:
> waiting for pending range calculation
>
> INFO  [main] 2016-10-10 22:48:36,317 StorageService.java:1149 - JOINING:
> calculation complete, ready to bootstrap
>
> INFO  [main] 2016-10-10 22:48:36,319 StorageService.java:1149 - JOINING:
> getting bootstrap token
>
> INFO  [main] 2016-10-10 22:48:36,357 StorageService.java:1149 - JOINING:
> sleeping 30000 ms for pending range setup
>
> INFO  [main] 2016-10-10 22:49:06,358 StorageService.java:1149 - JOINING:
> Starting to bootstrap...
>
> INFO  [main] 2016-10-10 22:49:06,494 StreamResultFuture.java:87 - [Stream
> #bfb5e470-8f3b-11e6-b69a-1b451159408e] Executing streaming plan for
> Bootstrap
>
> INFO  [StreamConnectionEstablisher:1] 2016-10-10 22:49:06,495
> StreamSession.java:242 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e]
> Starting streaming to /<Node A IP>
>
> INFO  [StreamConnectionEstablisher:2] 2016-10-10 22:49:06,495
> StreamSession.java:242 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e]
> Starting streaming to /<Node B IP>
>
> INFO  [StreamConnectionEstablisher:2] 2016-10-10 22:49:06,500
> StreamCoordinator.java:213 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e,
> ID#0] Beginning stream session with /<Node B IP>
>
> INFO  [STREAM-IN-/<Node B IP>] 2016-10-10 22:49:06,590
> StreamResultFuture.java:183 - [Stream
> #bfb5e470-8f3b-11e6-b69a-1b451159408e] Session with /<Node B IP> is complete
>
> INFO  [StreamConnectionEstablisher:1] 2016-10-10 22:49:06,635
> StreamCoordinator.java:213 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e,
> ID#0] Beginning stream session with /<Node A IP>
>
> ERROR [STREAM-IN-/<Node A IP>] 2016-10-10 22:49:06,639
> StreamSession.java:528 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e]
> Streaming error occurred
>
> java.io.IOException: Connection reset by peer
>
> at sun.nio.ch.FileDispatcherImpl.read0(Native Method) ~[na:1.8.0_102]
>
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
> ~[na:1.8.0_102]
>
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223) ~[na:1.8.0_102]
>
> at sun.nio.ch.IOUtil.read(IOUtil.java:197) ~[na:1.8.0_102]
>
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> ~[na:1.8.0_102]
>
> at sun.nio.ch.SocketAdaptor$SocketInputStream.read(SocketAdaptor.java:206)
> ~[na:1.8.0_102]
>
> at sun.nio.ch.ChannelInputStream.read(ChannelInputStream.java:103)
> ~[na:1.8.0_102]
>
> at
> java.nio.channels.Channels$ReadableByteChannelImpl.read(Channels.java:385)
> ~[na:1.8.0_102]
>
> at
> org.apache.cassandra.streaming.messages.StreamMessage.deserialize(StreamMessage.java:54)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:287)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]
>
> INFO  [STREAM-IN-/<Node A IP>] 2016-10-10 22:49:06,639
> StreamResultFuture.java:183 - [Stream
> #bfb5e470-8f3b-11e6-b69a-1b451159408e] Session with /<Node A IP> is complete
>
> WARN  [STREAM-IN-/<Node A IP>] 2016-10-10 22:49:06,640
> StreamResultFuture.java:210 - [Stream
> #bfb5e470-8f3b-11e6-b69a-1b451159408e] Stream failed
>
> WARN  [STREAM-IN-/<Node A IP>] 2016-10-10 22:49:06,640
> StorageService.java:1208 - Error during bootstrap.
>
> org.apache.cassandra.streaming.StreamException: Stream failed
>
> at
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> 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:211)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:187)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:429)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:534)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:305)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at java.lang.Thread.run(Thread.java:745) [na:1.8.0_102]
>
> ERROR [main] 2016-10-10 22:49:06,641 StorageService.java:1218 - 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:1213)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.service.StorageService.joinTokenRing(StorageService.java:889)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:663)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.service.StorageService.initServer(StorageService.java:528)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:339)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:557)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:685)
> [apache-cassandra-3.0.8.jar:3.0.8]
>
> Caused by: org.apache.cassandra.streaming.StreamException: Stream failed
>
> at
> org.apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(StreamEventJMXNotifier.java:85)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> 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:211)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.StreamResultFuture.handleSessionComplete(StreamResultFuture.java:187)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.StreamSession.closeSession(StreamSession.java:429)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.StreamSession.onError(StreamSession.java:534)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> at
> org.apache.cassandra.streaming.ConnectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:305)
> ~[apache-cassandra-3.0.8.jar:3.0.8]
>
> at java.lang.Thread.run(Thread.java:745) ~[na:1.8.0_102]
>
> WARN  [main] 2016-10-10 22:49:06,646 StorageService.java:944 - Some data
> streaming failed. Use nodetool to check bootstrap state and resume. For
> more, see `nodetool help bootstrap`. IN_PROGRESS
>
> INFO  [main] 2016-10-10 22:49:06,647 CassandraDaemon.java:644 - Waiting
> for gossip to settle before accepting client requests...
>
> INFO  [main] 2016-10-10 22:49:14,648 CassandraDaemon.java:675 - No gossip
> backlog; proceeding
>
> INFO  [main] 2016-10-10 22:49:14,694 NativeTransportService.java:70 -
> Netty using native Epoll event loop
>
> INFO  [main] 2016-10-10 22:49:14,726 Server.java:159 - Using Netty
> Version: [netty-buffer=netty-buffer-4.0.23.Final.208198c,
> netty-codec=netty-codec-4.0.23.Final.208198c,
> netty-codec-http=netty-codec-http-4.0.23.Final.208198c,
> netty-codec-socks=netty-codec-socks-4.0.23.Final.208198c,
> netty-common=netty-common-4.0.23.Final.208198c,
> netty-handler=netty-handler-4.0.23.Final.208198c,
> netty-transport=netty-transport-4.0.23.Final.208198c,
> netty-transport-rxtx=netty-transport-rxtx-4.0.23.Final.208198c,
> netty-transport-sctp=netty-transport-sctp-4.0.23.Final.208198c,
> netty-transport-udt=netty-transport-udt-4.0.23.Final.208198c]
>
> INFO  [main] 2016-10-10 22:49:14,726 Server.java:160 - Starting listening
> for CQL clients on /<Node C IP>:9042 (unencrypted)...
>
> INFO  [main] 2016-10-10 22:49:14,748 CassandraDaemon.java:477 - Not
> starting RPC server as requested. Use JMX
> (StorageService->startRPCServer()) or nodetool (enablethrift) to start it
>
>
>
> I tried resuming bootstrap but it fails with the same streaming errors:
>
>
>
> verma@<Node C>:~/apache-cassandra-3.0.8$ bin/nodetool bootstrap resume
>
> Resuming bootstrap
>
> [2016-10-10 23:15:11,816] session with /<Node B IP> complete (progress: 0%)
>
> [2016-10-10 23:15:11,939] session with /<Node A IP> complete (progress: 0%)
>
> [2016-10-10 23:15:11,940] Stream failed
>
>
>
> and I see the same error in the system.log:
>
>
>
> StreamSession.java:528 - [Stream #64b73a20-8f3f-11e6-b69a-1b451159408e]
> Streaming error occurred
>
> java.io.IOException: Connection reset by peer
>
> ...
>
>
>
> Does Cassandra support upgrading from 2.2.5 to 3.0.8 in this way? Am I
> missing something?
>
>
>
> Thanks for your time.
>
> -Abhishek.
> ____________________________________________________________________
> CONFIDENTIALITY NOTE: This e-mail and any attachments are confidential and
> may be legally privileged. If you are not the intended recipient, do not
> disclose, copy, distribute, or use this email or any attachments. If you
> have received this in error please let the sender know and then delete the
> email and all attachments.
>

Reply via email to