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 29312200BAD for ; Tue, 11 Oct 2016 02:00:54 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 2753F160AEB; Tue, 11 Oct 2016 00:00:54 +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 0D388160AE1 for ; Tue, 11 Oct 2016 02:00:51 +0200 (CEST) Received: (qmail 88266 invoked by uid 500); 11 Oct 2016 00:00:50 -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 88256 invoked by uid 99); 11 Oct 2016 00:00:50 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 11 Oct 2016 00:00:50 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id C35F0C0BF8 for ; Tue, 11 Oct 2016 00:00:49 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 3 X-Spam-Level: *** X-Spam-Status: No, score=3 tagged_above=-999 required=6.31 tests=[HTML_MESSAGE=2, KAM_LAZY_DOMAIN_SECURITY=1] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id FdjXfk7dzG9o for ; Tue, 11 Oct 2016 00:00:41 +0000 (UTC) Received: from mx0b-00206401.pphosted.com (mx0b-00206401.pphosted.com [148.163.152.21]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTPS id 8661B5FAC8 for ; Tue, 11 Oct 2016 00:00:41 +0000 (UTC) Received: from pps.filterd (m0093025.ppops.net [127.0.0.1]) by mx0b-00206401.pphosted.com (8.16.0.17/8.16.0.17) with SMTP id u9ANws9e000596 for ; Mon, 10 Oct 2016 17:00:33 -0700 Received: from ee01.crowdstrike.sys (dragosx.crowdstrike.com [208.42.231.60]) by mx0b-00206401.pphosted.com with ESMTP id 25xypah3my-1 (version=TLSv1 cipher=ECDHE-RSA-AES256-SHA bits=256 verify=NOT) for ; Mon, 10 Oct 2016 17:00:33 -0700 Received: from casmbox02.crowdstrike.sys (10.3.0.20) by ee01.crowdstrike.sys (10.100.0.12) with Microsoft SMTP Server (TLS) id 15.0.847.32; Mon, 10 Oct 2016 17:00:24 -0700 Received: from Casmbox03.crowdstrike.sys (10.100.11.66) by casmbox02.crowdstrike.sys (10.3.0.20) with Microsoft SMTP Server (TLS) id 15.0.1210.3; Mon, 10 Oct 2016 17:00:18 -0700 Received: from Casmbox03.crowdstrike.sys ([fe80::f877:615e:4b03:23ef]) by Casmbox03.crowdstrike.sys ([fe80::f877:615e:4b03:23ef%25]) with mapi id 15.00.1210.000; Mon, 10 Oct 2016 17:00:18 -0700 From: Jeff Jirsa To: "user@cassandra.apache.org" Subject: Re: Bootstrapping data from Cassandra 2.2.5 datacenter to 3.0.8 datacenter fails because of streaming errors Thread-Topic: Bootstrapping data from Cassandra 2.2.5 datacenter to 3.0.8 datacenter fails because of streaming errors Thread-Index: AQHSI077V436A1phtk6Iqd7fHaWaf6CiXkeA Date: Tue, 11 Oct 2016 00:00:18 +0000 Message-ID: References: In-Reply-To: Accept-Language: en-US Content-Language: en-US X-MS-Has-Attach: yes X-MS-TNEF-Correlator: x-ms-exchange-messagesentrepresentingtype: 1 x-ms-exchange-transport-fromentityheader: Hosted x-originating-ip: [10.100.0.9] x-disclaimer: USA Content-Type: multipart/signed; protocol="application/pkcs7-signature"; micalg=sha256; boundary="B_3558963617_1968738120" MIME-Version: 1.0 X-Proofpoint-Virus-Version: vendor=fsecure engine=2.50.10432:,, definitions=2016-10-10_11:,, signatures=0 X-Proofpoint-Spam-Details: rule=russia_temp_notspam policy=russia_temp score=0 priorityscore=1501 malwarescore=0 suspectscore=0 phishscore=0 bulkscore=0 spamscore=0 clxscore=1015 lowpriorityscore=0 impostorscore=0 adultscore=0 classifier=spam adjust=0 reason=mlx scancount=1 engine=8.0.1-1609300000 definitions=main-1610100401 archived-at: Tue, 11 Oct 2016 00:00:54 -0000 --B_3558963617_1968738120 Content-type: multipart/alternative; boundary="B_3558963617_761215762" --B_3558963617_761215762 Content-transfer-encoding: quoted-printable Content-Type: text/plain; charset="utf-8" =20 No need to cc dev@, user@ is the right list for this question. =20 As Jon mentioned, you can=E2=80=99t stream (bootstrap/rebuild/repair) acros= s major versions, so don=E2=80=99t try to destroy the cluster =E2=80=93 jus= t upgrade in place. It IS a good idea to do one DC at a time, but an in-pla= ce upgrade is pretty straightforward =E2=80=93 flush, drain, stop Cassandra= , replace binaries, start Cassandra, run nodetool upgradesstables -a. =20 Note that you can run nodetool upgradesstables on more than one node at a t= ime if you can tolerate the hit to your read latencies. =20 It IS common, I imagine, for there to be schema mismatches temporarily whil= e you have a mixed version cluster =E2=80=93 this isn=E2=80=99t 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=E2=80=99s generally advisable. =20 =20 =20 =20 From: Abhishek Verma Reply-To: "user@cassandra.apache.org" Date: Monday, October 10, 2016 at 4:34 PM To: "user@cassandra.apache.org" , "dev@cassandra= .apache.org" Subject: Bootstrapping data from Cassandra 2.2.5 datacenter to 3.0.8 datace= nter fails because of streaming errors =20 Hi Cassandra users,=20 =20 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 or= der to preserve our data, we are trying to upgrade one datacenter at a time= .=20 =20 Initially both DCs (dc1 and dc2) are running 2.2.5. The idea is to tear dow= n dc1 completely (delete all the data in it), bring it up with 3.0.8, let d= ata replicate from dc2 to dc1, and then tear down dc2, bring it up with 3.0= .8 and replicate data from dc1. =20 I am able to reproduce the problem on bare metal clusters running on 3 node= s. I am using Oracle's server-jre-8u74-linux-x64 JRE. =20 Node A: Downloaded 2.2.5-bin.tar.gz, changed the seeds to include its own I= P address, changed listen_address and rpc_address to its own IP and changed= endpoint_snitch to GossipingPropertyFileSnitch. I changed conf/cassandra-r= ackdc.properties to dc=3Ddc2 rack=3Drack2 This node started up fine and is UN in nodetool status in dc2. =20 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 =20 CREATE KEYSPACE tmp WITH replication =3D {'class': 'NetworkTopologyStrategy= ', 'dc1': '1', 'dc2': '1'} AND durable_writes =3D true; =20 CREATE TABLE tmp.map ( key text PRIMARY KEY, value text )...; cqlsh> select * from tmp.map; =20 key | value -----+------- k1 | v1 k3 | v3 k2 | v2 =20 =20 Node B: Downloaded 3.0.8-bin.tar.gz, changed the seeds to include itself an= d node A, changed listen_address and rpc_address to its own IP, changed end= point_snitch to GossipingPropertyFileSnitch. I did not change conf/cassandr= a-rackdc.properties and its contents are dc=3Ddc1 rack=3Drack1 =20 In the logs, I see: INFO [main] 2016-10-10 22:42:42,850 MessagingService.java:557 - Starting M= essaging Service on /10.164.32.29:7000 (eth0) INFO [main] 2016-10-10 22:42:42,864 StorageService.java:784 - This node wi= ll not auto bootstrap because it is configured to be a seed node. =20 So I start a third node: Node C: Downloaded 3.0.8-bin.tar.gz, changed the seeds to include node A an= d node B, changed listen_address and rpc_address to its own IP, changed end= point_snitch to GossipingPropertyFileSnitch. I did not change conf/cassandr= a-rackdc.properties. Now, nodetool status shows: =20 verma@xxxxxxx:~/apache-cassandra-3.0.8$ bin/nodetool status Datacenter: dc1 =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D Status=3DUp/Down |/ State=3DNormal/Leaving/Joining/Moving -- Address Load Tokens Owns (effective) Host ID = Rack UJ 87.81 KB 256 ? 9064832d-ed5c-4= c42-ad5a-f754b52b670c rack1 UN 107.72 KB 256 100.0% 28b1043f-115b-46= a5-b6b6-8609829cde76 rack1 Datacenter: dc2 =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D Status=3DUp/Down |/ State=3DNormal/Leaving/Joining/Moving -- Address Load Tokens Owns (effective) Host ID = Rack UN 73.2 KB 256 100.0% 09cc542c-2299-= 45a5-a4d1-159c239ded37 rack2 =20 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: [, ] =20 9770e3c5-3135-32e2-b761-65a0f6d8824e: [] =20 Note that there are two schema versions and they don't match. =20 I see the following in the system.log:=20 =20 INFO [InternalResponseStage:1] 2016-10-10 22:48:36,055 ColumnFamilyStore.j= ava:390 - Initializing system_auth.roles INFO [main] 2016-10-10 22:48:36,316 StorageService.java:1149 - JOINING: wa= iting for schema information to complete INFO [main] 2016-10-10 22:48:36,316 StorageService.java:1149 - JOINING: sc= hema complete, ready to bootstrap INFO [main] 2016-10-10 22:48:36,316 StorageService.java:1149 - JOINING: wa= iting for pending range calculation INFO [main] 2016-10-10 22:48:36,317 StorageService.java:1149 - JOINING: ca= lculation complete, ready to bootstrap INFO [main] 2016-10-10 22:48:36,319 StorageService.java:1149 - JOINING: ge= tting bootstrap token INFO [main] 2016-10-10 22:48:36,357 StorageService.java:1149 - JOINING: sl= eeping 30000 ms for pending range setup INFO [main] 2016-10-10 22:49:06,358 StorageService.java:1149 - JOINING: St= arting 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 streami= ng to / INFO [StreamConnectionEstablisher:2] 2016-10-10 22:49:06,495 StreamSession= .java:242 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e] Starting streami= ng to / INFO [StreamConnectionEstablisher:2] 2016-10-10 22:49:06,500 StreamCoordin= ator.java:213 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e, ID#0] Beginn= ing stream session with / INFO [STREAM-IN-/] 2016-10-10 22:49:06,590 StreamResultFuture.j= ava:183 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e] Session with / is complete INFO [StreamConnectionEstablisher:1] 2016-10-10 22:49:06,635 StreamCoordin= ator.java:213 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e, ID#0] Beginn= ing stream session with / ERROR [STREAM-IN-/] 2016-10-10 22:49:06,639 StreamSession.java:5= 28 - [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_10= 2] 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:38= 5) ~[na:1.8.0_102] at org.apache.cassandra.streaming.messages.StreamMessage.deserialize(Stream= Message.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-/] 2016-10-10 22:49:06,639 StreamResultFuture.j= ava:183 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e] Session with / is complete WARN [STREAM-IN-/] 2016-10-10 22:49:06,640 StreamResultFuture.j= ava:210 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e] Stream failed WARN [STREAM-IN-/] 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.onFailu= re(StreamEventJMXNotifier.java:85) ~[apache-cassandra-3.0.8.jar:3.0.8] at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) [guav= a-18.0.jar:na] at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(M= oreExecutors.java:457) [guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.executeListener(Executio= nList.java:156) [guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.ja= va:145) [guava-18.0.jar:na] at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFu= ture.java:202) [guava-18.0.jar:na] at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamRe= sultFuture.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.Str= eamException: Stream failed at com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractF= uture.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.jav= a: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.ja= va:663) [apache-cassandra-3.0.8.jar:3.0.8] at org.apache.cassandra.service.StorageService.initServer(StorageService.ja= va: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.ja= va:557) [apache-cassandra-3.0.8.jar:3.0.8] at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:6= 85) [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.onFailu= re(StreamEventJMXNotifier.java:85) ~[apache-cassandra-3.0.8.jar:3.0.8] at com.google.common.util.concurrent.Futures$6.run(Futures.java:1310) ~[gua= va-18.0.jar:na] at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execute(M= oreExecutors.java:457) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.executeListener(Executio= nList.java:156) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.ja= va:145) ~[guava-18.0.jar:na] at com.google.common.util.concurrent.AbstractFuture.setException(AbstractFu= ture.java:202) ~[guava-18.0.jar:na] at org.apache.cassandra.streaming.StreamResultFuture.maybeComplete(StreamRe= sultFuture.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 st= reaming 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 b= acklog; 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=3Dnetty-buffer-4.0.23.Final.208198c, netty-codec=3Dnetty-cod= ec-4.0.23.Final.208198c, netty-codec-http=3Dnetty-codec-http-4.0.23.Final.2= 08198c, netty-codec-socks=3Dnetty-codec-socks-4.0.23.Final.208198c, netty-c= ommon=3Dnetty-common-4.0.23.Final.208198c, netty-handler=3Dnetty-handler-4.= 0.23.Final.208198c, netty-transport=3Dnetty-transport-4.0.23.Final.208198c,= netty-transport-rxtx=3Dnetty-transport-rxtx-4.0.23.Final.208198c, netty-tr= ansport-sctp=3Dnetty-transport-sctp-4.0.23.Final.208198c, netty-transport-u= dt=3Dnetty-transport-udt-4.0.23.Final.208198c] INFO [main] 2016-10-10 22:49:14,726 Server.java:160 - Starting listening f= or CQL clients on /:9042 (unencrypted)... INFO [main] 2016-10-10 22:49:14,748 CassandraDaemon.java:477 - Not startin= g RPC server as requested. Use JMX (StorageService->startRPCServer()) or no= detool (enablethrift) to start it =20 I tried resuming bootstrap but it fails with the same streaming errors: =20 verma@:~/apache-cassandra-3.0.8$ bin/nodetool bootstrap resume Resuming bootstrap [2016-10-10 23:15:11,816] session with / complete (progress: 0%) [2016-10-10 23:15:11,939] session with / complete (progress: 0%) [2016-10-10 23:15:11,940] Stream failed =20 and I see the same error in the system.log:=20 =20 StreamSession.java:528 - [Stream #64b73a20-8f3f-11e6-b69a-1b451159408e] Str= eaming error occurred java.io.IOException: Connection reset by peer ... =20 Does Cassandra support upgrading from 2.2.5 to 3.0.8 in this way? Am I miss= ing something?=20 =20 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 di= sclose, 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 emai= l and all attachments. --B_3558963617_761215762 Content-transfer-encoding: quoted-printable Content-Type: text/html; charset="UTF-8"

 

No need to cc dev@= , user@ is the right list for this question.

&nbs= p;

As Jon mentioned, you can’t stream (bootstrap/rebui= ld/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, b= ut an in-place upgrade is pretty straightforward – flush, drain, stop= Cassandra, replace binaries, start Cassandra, run nodetool upgradesstables= -a.

 

<= span style=3D'font-size:11.0pt;font-family:Calibri'>Note that you can run n= odetool 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 necessaril= y a huge problem, but do try to get to 3.0.8 as quickly as possible once yo= u start, and if you can avoid administrative tasks (such as those that will= change the schema) during the process, that’s generally advisable.

 

 

<= p class=3DMsoNormal> 

 

From: Abhishek Verma <verma@u= ber.com>
Reply-To: "user@cassandra.apache.org" <u= ser@cassandra.apache.org>
Date: Monday, October 10, 2016 at 4:= 34 PM
To: "user@cassandra.apache.org" <user@cassandr= a.apache.org>, "dev@cassandra.apache.org" <dev@cassandra.ap= ache.org>
Subject: Bootstrapping data from Cassandra 2.2.5 dat= acenter to 3.0.8 datacenter fails because of streaming errors

 

Hi Cassandra users,

 

We are t= rying to upgrade our Cassandra version from 2.2.5 to 3.0.8 (running on Meso= s, but that's besides the point). We have two datacenters, so in order to p= reserve our data, we are trying to upgrade one datacenter at a time. <= o:p>

 

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 d= c2, 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 in= clude its own IP address, changed listen_address and rpc_address to its own= IP and changed endpoint_snitch to GossipingPropertyFileSnitch. I changed&n= bsp;conf/cassandra-rackdc.properties to

dc=3Ddc2

rack=3Drack2

=

This node started up fine and is UN in nodetool status= in dc2.

 

I used CQL shell to create a table and ins= ert 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 h= elp.

cqlsh> desc tmp

 

CREATE KEYSPACE tmp WITH replicatio= n =3D {'class': 'NetworkTopologyStrategy', 'dc1': '1', 'dc2': '1'}  AN= D durable_writes =3D true;

 

CREATE TABLE tmp.map (

    = key text PRIMARY KEY,

=     value text

)...;

cqlsh> sel= ect * from tmp.map;

 

 key | value

<= p class=3DMsoNormal>-----+-= ------

  k1 |    v1

  k3 |    v3

  k2= |    v2

 

 

Node B: Downloaded 3.0.8-bin.tar.g= z, changed the seeds to include itself and node A, changed listen_address a= nd rpc_address to its own IP, changed endpoint_snitch to GossipingPropertyF= ileSnitch. I did not change conf/cassandra-rackdc.properties and its conten= ts are

dc=3Ddc1

rack=3Dra= ck1

 <= /o:p>

In the logs, I see:

=

INFO  [main] 2016-10-10 22:42:42,850 MessagingService.java:5= 57 - Starting Messaging Service on /10.164.32.29:7000= (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.g= z, changed the seeds to include node A and node B, changed listen_address a= nd rpc_address to its own IP, changed endpoint_snitch to GossipingPropertyF= ileSnitch. I did not change conf/cassandra-rackdc.properties.

Now, nodetool status shows:

 

verma= @xxxxxxx:~/apache-cassandra-3.0.8$ bin/nodetool status

Datacenter: dc1

= =3D=3D=3D=3D=3D=3D=3D=3D=3D= =3D=3D=3D=3D=3D=3D

Status=3DUp/Down

|/ State=3DNormal/Leaving/Joining/Moving

=

--  Address       Load       Tokens=       Owns (effective)  Host ID       &= nbsp;                    =   Rack

UJ  <Node C IP>   87= .81 KB   256          ?       =           9064832d-ed5c-4c42-ad5a-f754b52b670c &nb= sp;rack1

UN  <Node B IP>  107.7= 2 KB  256          100.0%      = ;      28b1043f-115b-46a5-b6b6-8609829cde76  rack1

Datacenter: dc2

=3D=3D=3D= =3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D=3D

Status=3DUp/= Down

|/ State=3DNormal/Leaving/Joining/Moving

--  Address       Load   &n= bsp;   Tokens       Owns (effective)  Host ID &nbs= p;                     &n= bsp;       Rack

UN  <Node = A IP>    73.2 KB    256         =  100.0%            09cc542c-2299-45a5-a4= d1-159c239ded37  rack2

 

Nodetool = describe cluster shows:

= verma@xxxxxxx:~/apache-cass= andra-3.0.8$ bin/nodetool describecluster

<= p class=3DMsoNormal>Cluster= Information:

Name: Test Cluster

Snitch: org.apache.cassandra.locator.DynamicEndpointSnitch=

Partitioner: org.apache.cassandra.dht.Murmur3Partitione= r

Schema versions:

c2a2= bb4f-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:&nbs= p;

 

INFO  [InternalResponseStage:1] 2016-10-10 22:48:36,055 ColumnFam= ilyStore.java:390 - Initializing system_auth.roles

INFO  [main] 2016-10-10 22:48:36,316 StorageService.java:1149 - JOIN= ING: waiting for schema information to complete

=

I= NFO  [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] 2= 016-10-10 22:48:36,317 StorageService.java:1149 - JOINING: calculation comp= lete, ready to bootstrap

INFO  [main] 2016-1= 0-10 22:48:36,319 StorageService.java:1149 - JOINING: getting bootstrap tok= en

INFO  [main] 2016-10-10 22:48:36,357 Stor= ageService.java:1149 - JOINING: sleeping 30000 ms for pending range setup

INFO  [main] 2016-10-10 22:49:06,358 StorageS= ervice.java:1149 - JOINING: Starting to bootstrap...

<= /div>

INFO  [main] 2016-10-10 22:49:06,494 StreamResultFuture.java:87 - = [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e] Executing streaming plan for= Bootstrap

INFO  [StreamConnectionEstablishe= r:1] 2016-10-10 22:49:06,495 StreamSession.java:242 - [Stream #bfb5e470-8f3= b-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-1b45115= 9408e] Starting streaming to /<Node B IP>

=

I= NFO  [StreamConnectionEstablisher:2] 2016-10-10 22:49:06,500 StreamCoo= rdinator.java:213 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e, ID#0] Be= ginning stream session with /<Node B IP>

<= div>

IN= FO  [STREAM-IN-/<Node B IP>] 2016-10-10 22:49:06,590 StreamResul= tFuture.java:183 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e] Session w= ith /<Node B IP> is complete

INFO  [St= reamConnectionEstablisher:1] 2016-10-10 22:49:06,635 StreamCoordinator.java= :213 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e, ID#0] Beginning strea= m session with /<Node A IP>

ERROR [STREAM-= IN-/<Node A IP>] 2016-10-10 22:49:06,639 StreamSession.java:528 - [St= ream #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(S= ocketDispatcher.java:39) ~[na:1.8.0_102]

at sun.n= io.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(SocketChan= nelImpl.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(ChannelIn= putStream.java:103) ~[na:1.8.0_102]

at java.nio.c= hannels.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.j= ar:3.0.8]

at org.apache.cassandra.streaming.Conn= ectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:287) ~[apac= he-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.ja= va:183 - [Stream #bfb5e470-8f3b-11e6-b69a-1b451159408e] Session with /<N= ode A IP> is complete

WARN  [STREAM-IN-/&= lt;Node A IP>] 2016-10-10 22:49:06,640 StreamResultFuture.java:210 - [St= ream #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.StreamEventJMXN= otifier.onFailure(StreamEventJMXNotifier.java:85) ~[apache-cassandra-3.0.8.= jar:3.0.8]

at com.google.common.util.concurrent.F= utures$6.run(Futures.java:1310) [guava-18.0.jar:na]

at com.google.common.util.concurrent.MoreExecutors$DirectExecutor.execut= e(MoreExecutors.java:457) [guava-18.0.jar:na]

at = com.google.common.util.concurrent.ExecutionList.executeListener(ExecutionLi= st.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.concurren= t.AbstractFuture.setException(AbstractFuture.java:202) [guava-18.0.jar:na]<= /span>

at org.apache.cassandra.streaming.StreamResultFut= ure.maybeComplete(StreamResultFuture.java:211) [apache-cassandra-3.0.8.jar:= 3.0.8]

at org.apache.cassandra.streaming.StreamRe= sultFuture.handleSessionComplete(StreamResultFuture.java:187) [apache-cassa= ndra-3.0.8.jar:3.0.8]

= at org.apache.cassandra.str= eaming.StreamSession.closeSession(StreamSession.java:429) [apache-cassandra= -3.0.8.jar:3.0.8]

at org.apache.cassandra.streami= ng.StreamSession.onError(StreamSession.java:534) [apache-cassandra-3.0.8.ja= r:3.0.8]

at org.apache.cassandra.streaming.Conn= ectionHandler$IncomingMessageHandler.run(ConnectionHandler.java:305) [apach= e-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 b= ootstrap to complete. Bootstrap will have to be restarted.

java.util.concurrent.ExecutionException: org.apache.cassandra.str= eaming.StreamException: Stream failed

at com.goog= le.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:= 299) ~[guava-18.0.jar:na]

at com.google.common.ut= il.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:286) ~[guava-18.0= .jar:na]

at com.google.common.util.concurrent.A= bstractFuture.get(AbstractFuture.java:116) ~[guava-18.0.jar:na]=

at org.apache.cassandra.service.StorageService.bootstrap(Sto= rageService.java:1213) [apache-cassandra-3.0.8.jar:3.0.8]=

at org.apache.cassandra.service.StorageService.joinTokenRing(Stora= geService.java:889) [apache-cassandra-3.0.8.jar:3.0.8]

at org.apache.cassandra.service.StorageService.initServer(StorageServ= ice.java:663) [apache-cassandra-3.0.8.jar:3.0.8]

= at org.apache.cassandra.service.StorageService.initServer(StorageService.ja= va: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) [apach= e-cassandra-3.0.8.jar:3.0.8]

at org.apache.cassan= dra.service.CassandraDaemon.main(CassandraDaemon.java:685) [apache-cassandr= a-3.0.8.jar:3.0.8]

Caused by: org.apache.cassandr= a.streaming.StreamException: Stream failed

=

at org= .apache.cassandra.streaming.management.StreamEventJMXNotifier.onFailure(Str= eamEventJMXNotifier.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.com= mon.util.concurrent.MoreExecutors$DirectExecutor.execute(MoreExecutors.java= :457) ~[guava-18.0.jar:na]

at com.google.common.u= til.concurrent.ExecutionList.executeListener(ExecutionList.java:156) ~[guav= a-18.0.jar:na]

at com.google.common.util.concurre= nt.ExecutionList.execute(ExecutionList.java:145) ~[guava-18.0.jar:na]

at com.google.common.util.concurrent.AbstractFuture.se= tException(AbstractFuture.java:202) ~[guava-18.0.jar:na]<= /p>

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.handle= SessionComplete(StreamResultFuture.java:187) ~[apache-cassandra-3.0.8.jar:3= .0.8]

at org.apache.cassandra.streaming.StreamSes= sion.closeSession(StreamSession.java:429) ~[apache-cassandra-3.0.8.jar:3.0.= 8]

at org.apache.cassandra.streaming.StreamSessio= n.onError(StreamSession.java:534) ~[apache-cassandra-3.0.8.jar:3.0.8]

at org.apache.cassandra.streaming.ConnectionHandler$In= comingMessageHandler.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-1= 0 22:49:06,646 StorageService.java:944 - Some data streaming failed. Use no= detool to check bootstrap state and resume. For more, see `nodetool help bo= otstrap`. IN_PROGRESS

= INFO  [main] 2016-10-1= 0 22:49:06,647 CassandraDaemon.java:644 - Waiting for gossip to settle befo= re accepting client requests...

INFO  [main]= 2016-10-10 22:49:14,648 CassandraDaemon.java:675 - No gossip backlog; proc= eeding

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:1= 59 - Using Netty Version: [netty-buffer=3Dnetty-buffer-4.0.23.Final.208198c= , netty-codec=3Dnetty-codec-4.0.23.Final.208198c, netty-codec-http=3Dnetty-= codec-http-4.0.23.Final.208198c, netty-codec-socks=3Dnetty-codec-socks-4.0.= 23.Final.208198c, netty-common=3Dnetty-common-4.0.23.Final.208198c, netty-h= andler=3Dnetty-handler-4.0.23.Final.208198c, netty-transport=3Dnetty-transp= ort-4.0.23.Final.208198c, netty-transport-rxtx=3Dnetty-transport-rxtx-4.0.2= 3.Final.208198c, netty-transport-sctp=3Dnetty-transport-sctp-4.0.23.Final.2= 08198c, netty-transport-udt=3Dnetty-transport-udt-4.0.23.Final.208198c]

INFO  [main] 2016-10-10 22:49:14,726 Server.jav= a:160 - Starting listening for CQL clients on /<Node C IP>:9042 (unen= crypted)...

INFO  [main] 2016-10-10 22:49:14= ,748 CassandraDaemon.java:477 - Not starting RPC server as requested. Use J= MX (StorageService->startRPCServer()) or nodetool (enablethrift) to star= t it

 =

I tried resuming bootstrap but it fails with the same streaming errors:

 

verma@<Node C>:~/apache-cassandra-3.0.8$ bin/nodetool bootstra= p 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 w= ith /<Node A IP> complete (progress: 0%)

<= div>

[2= 016-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 di= sclose, 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 emai= l and all attachments.
--B_3558963617_761215762-- --B_3558963617_1968738120 Content-Type: application/pkcs7-signature; name="smime.p7s" Content-Transfer-Encoding: base64 Content-Disposition: attachment; filename="smime.p7s" MIIRKwYJKoZIhvcNAQcCoIIRHDCCERgCAQExDzANBglghkgBZQMEAgEFADALBgkqhkiG9w0B BwGggg6dMIIFTDCCBDSgAwIBAgIRAIeX7oQRaz3bAAAAAEw1XuAwDQYJKoZIhvcNAQELBQAw gaUxCzAJBgNVBAYTAlVTMRYwFAYDVQQKEw1FbnRydXN0LCBJbmMuMTkwNwYDVQQLEzB3d3cu ZW50cnVzdC5uZXQvQ1BTIGlzIGluY29ycG9yYXRlZCBieSByZWZlcmVuY2UxHzAdBgNVBAsT FihjKSAyMDEwIEVudHJ1c3QsIEluYy4xIjAgBgNVBAMTGUVudHJ1c3QgQ2xhc3MgMiBDbGll bnQgQ0EwHhcNMTYwOTIwMjIxMTIwWhcNMTkwOTMwMjI0MTE3WjCBjTELMAkGA1UEBhMCVVMx EzARBgNVBAgTCkNhbGlmb3JuaWExDzANBgNVBAcTBklydmluZTEaMBgGA1UEChMRQ3Jvd2RT dHJpa2UsIEluYy4xPDARBgNVBAMTCkplZmYgSmlyc2EwJwYJKoZIhvcNAQkBFhpqZWZmLmpp cnNhQGNyb3dkc3RyaWtlLmNvbTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAM8/ kM42VwiKXDU7EgjPU7wyr7KRidCCUqlqfSJ9pcvlNzqluaTAYfoAALsc8vYhIxw7h9qJPVC9 xXdgQGXJcHeVfHwslf0jUWezmnk4jXOuhhiGKF8hCDR2OK1vwl495dCVl8ui+Xly59MMxIvc uAVieWJ8+E5JLa0/IQVPHg3OHB4vWfipOnp9ZXyXWvwtbU6px4vV5tG80PXBeMPUO3vT7XTe rQuua+nZTiqh3VnVuOxdxr1ttkxu3Gn5SqBLwbuPlMrBYtJVa5nAMPo+fVgUmV+aSCCjG/x+ Vy6dFutaIyLXyB2jiQx3t9mX0Iu2Nnc2rtpezj+g0FP6dB703nsCAwEAAaOCAYswggGHMA4G A1UdDwEB/wQEAwIFoDAdBgNVHSUEFjAUBggrBgEFBQcDAgYIKwYBBQUHAwQwQgYDVR0gBDsw OTA3BgtghkgBhvpsCgEEAjAoMCYGCCsGAQUFBwIBFhpodHRwOi8vd3d3LmVudHJ1c3QubmV0 L3JwYTBqBggrBgEFBQcBAQReMFwwIwYIKwYBBQUHMAGGF2h0dHA6Ly9vY3NwLmVudHJ1c3Qu bmV0MDUGCCsGAQUFBzAChilodHRwOi8vYWlhLmVudHJ1c3QubmV0LzIwNDhjbGFzczJzaGEy LmNlcjA0BgNVHR8ELTArMCmgJ6AlhiNodHRwOi8vY3JsLmVudHJ1c3QubmV0L2NsYXNzMmNh LmNybDAlBgNVHREEHjAcgRpqZWZmLmppcnNhQGNyb3dkc3RyaWtlLmNvbTAfBgNVHSMEGDAW gBQJkaW66fIuKnXfzX7+d8ry3mubJDAdBgNVHQ4EFgQUSw+neOez3ZJWkkEF36O0c2skDkAw CQYDVR0TBAIwADANBgkqhkiG9w0BAQsFAAOCAQEAoQiIaUSkRZecrnLGP6/as+GANvfMnFNL i5wawcZljyeJg8e7p6+ZcXUSI0GOPs/Wl9paitiIIhGuvD2iD3+cvJQlrC+8LT2PFkRUu81B riyF3QzWygI1hCdFQcRY+9Fox1zKT0+5SwfOPstSBLHuYAUfRQrc9WtoqF70xbngPUCfGZVJ +8l9kJgCnXqwmfTu8s2d1Q5MCdz68g8geVU3nYnJ7ONPvvgsdlgywW0sNLLhn4iqGY6y5xSh uR2GYgSwcYrvKfU56sHYc2JLyyUzUm3r3BWE+CedpBg+B4Al6XsgqJPu2t2hgSrcDoHrpEsV +hTUoTgWxZlqHh7bcQdRhjCCBOkwggPRoAMCAQICBEwOjDgwDQYJKoZIhvcNAQEFBQAwgbQx FDASBgNVBAoTC0VudHJ1c3QubmV0MUAwPgYDVQQLFDd3d3cuZW50cnVzdC5uZXQvQ1BTXzIw NDggaW5jb3JwLiBieSByZWYuIChsaW1pdHMgbGlhYi4pMSUwIwYDVQQLExwoYykgMTk5OSBF bnRydXN0Lm5ldCBMaW1pdGVkMTMwMQYDVQQDEypFbnRydXN0Lm5ldCBDZXJ0aWZpY2F0aW9u IEF1dGhvcml0eSAoMjA0OCkwHhcNMTExMTExMTUzODM0WhcNMjExMTEyMDAxNzM0WjCBpTEL MAkGA1UEBhMCVVMxFjAUBgNVBAoTDUVudHJ1c3QsIEluYy4xOTA3BgNVBAsTMHd3dy5lbnRy dXN0Lm5ldC9DUFMgaXMgaW5jb3Jwb3JhdGVkIGJ5IHJlZmVyZW5jZTEfMB0GA1UECxMWKGMp IDIwMTAgRW50cnVzdCwgSW5jLjEiMCAGA1UEAxMZRW50cnVzdCBDbGFzcyAyIENsaWVudCBD QTCCASIwDQYJKoZIhvcNAQEBBQADggEPADCCAQoCggEBAMQyjULQnhmdW5BaEEy1EAAhuQdI 3q5ugNb/FFAG6HWva0aO56VPrcOMsPp74BmR/fBjrXFJ86gcH6s0GSBOS1TpAJO+cAgx3olT rFe8JO8qj0LU9+qVJV0UdtLNpxL6G7K0XGFAvV/dV5tEVdjFiRk8ZT256NSlLcIs0+qDMaII PF5ZrhIuKgqMXvOzMa4KrX7ssEkJ/KcuIh5oZDSdFuOmPQMxQBb3lPZLGTTJl+YinEjeZKCD C1gFmMQiRokF/aO+9klMYQMWpPgKmRziwMZ+aQIyV5ADrwCUobnczq/v9HwYzjALyof41V8f WVHYiwu5OMZYwlN82ibU2/K9kM0CAwEAAaOCAQ4wggEKMA4GA1UdDwEB/wQEAwIBBjASBgNV HRMBAf8ECDAGAQH/AgEAMDMGCCsGAQUFBwEBBCcwJTAjBggrBgEFBQcwAYYXaHR0cDovL29j c3AuZW50cnVzdC5uZXQwMgYDVR0fBCswKTAnoCWgI4YhaHR0cDovL2NybC5lbnRydXN0Lm5l dC8yMDQ4Y2EuY3JsMDsGA1UdIAQ0MDIwMAYEVR0gADAoMCYGCCsGAQUFBwIBFhpodHRwOi8v d3d3LmVudHJ1c3QubmV0L3JwYTAdBgNVHQ4EFgQUCZGluunyLip1381+/nfK8t5rmyQwHwYD VR0jBBgwFoAUVeSB0RGAvtiJuQijMfmhJAkWuXAwDQYJKoZIhvcNAQEFBQADggEBAAqJtbEz ORCxLAl57vMbbah2SrTDeOPn/ydhNMxK7NiC7h9jSuF9RXpERqpWxoBM38h1CZxhIdk+Tcug GvSRiiWlem0buWcZPyUz1EEfYT8YIpPIPvfD6Q/nWPSeH07jn+HV3ze6/LHtgDZmZoUmV2K1 4m6wgmrQrCMT0RcVRglZds6ncKeIHnEnPh3e2eqdCIp/K5byi5sUf8pFck8KLVu/zrl76IyI TI/XXgmQoOfI+YA+rcEyskbD/c0MDOXC/U8Jt4IgkrzTZJ8HMU32zzVpN6TvRz8lK3sO35s7 snE9J86ULnsmrUifBH+fG4fMeh2xIJAVCK4CEdPDAD2o60cwggRcMIIDRKADAgECAgQ4Y7lm MA0GCSqGSIb3DQEBBQUAMIG0MRQwEgYDVQQKEwtFbnRydXN0Lm5ldDFAMD4GA1UECxQ3d3d3 LmVudHJ1c3QubmV0L0NQU18yMDQ4IGluY29ycC4gYnkgcmVmLiAobGltaXRzIGxpYWIuKTEl MCMGA1UECxMcKGMpIDE5OTkgRW50cnVzdC5uZXQgTGltaXRlZDEzMDEGA1UEAxMqRW50cnVz dC5uZXQgQ2VydGlmaWNhdGlvbiBBdXRob3JpdHkgKDIwNDgpMB4XDTk5MTIyNDE3NTA1MVoX DTE5MTIyNDE4MjA1MVowgbQxFDASBgNVBAoTC0VudHJ1c3QubmV0MUAwPgYDVQQLFDd3d3cu ZW50cnVzdC5uZXQvQ1BTXzIwNDggaW5jb3JwLiBieSByZWYuIChsaW1pdHMgbGlhYi4pMSUw IwYDVQQLExwoYykgMTk5OSBFbnRydXN0Lm5ldCBMaW1pdGVkMTMwMQYDVQQDEypFbnRydXN0 Lm5ldCBDZXJ0aWZpY2F0aW9uIEF1dGhvcml0eSAoMjA0OCkwggEiMA0GCSqGSIb3DQEBAQUA A4IBDwAwggEKAoIBAQCtTUupEoay6qMgBxUWZCorS9G/C0pNju2AdqVnt3hAwHNCyGjA21Mr 3V64dpg1k4sanXwTOg4fW7cez+UkFB6xgamNfbjMa0sD8QIM3KulQCQAf3SUoZ0IKbOIC/WH d51VzeTDftdqZKuFFIaVW5cyUG89yLpmDOP8vbhJwXaJSRn9wKi9iaNnL8afvHEZYLgt6SzJ kHZme5Tir3jWZVNdPNacss8pA/kvpFCy1EjOBTJViv2yZEwO5JgHddt/37kIVWCFMCn5e0ik aYbjNT8ehl16ehW97wCOFSJUFwCQJpO8Dklokb/4R9OdlULBDk3fbybPwxghYmZDcNbVwAfh AgMBAAGjdDByMBEGCWCGSAGG+EIBAQQEAwIABzAfBgNVHSMEGDAWgBRV5IHREYC+2Im5CKMx +aEkCRa5cDAdBgNVHQ4EFgQUVeSB0RGAvtiJuQijMfmhJAkWuXAwHQYJKoZIhvZ9B0EABBAw DhsIVjUuMDo0LjADAgSQMA0GCSqGSIb3DQEBBQUAA4IBAQBZR6whhIoXyZyJUx66gIUaxjxO PrGctnzGkl0YZALj0wYIEWF8Y+MrnTEDcHbSoyig9LuaY3PtbeUq2+0UqSvGNhHQK+sHi6Xa nlwZnVYS9VQpyAXtshIqjfQDG//nkhCHsDq1w50FNxKjx/QVudWkORabUzojkfGogqJqiGjB eQIivKqm1q7fsBRfuIfQ3Xx/e/+vHM/m2wetXtuFndArDTPbBNHmSUATK3b7PumciQ8Vzhiw hXghT2tPDvo2Z80H8v8I0OLe2b8qr7iHhiE8BMq3lGh/zzzpmNc4/+zA2VDwLktYrkZv0C7D YNpyVXK9TEWeYbq/hIGSA9HSaXzFMYICUjCCAk4CAQEwgbswgaUxCzAJBgNVBAYTAlVTMRYw FAYDVQQKEw1FbnRydXN0LCBJbmMuMTkwNwYDVQQLEzB3d3cuZW50cnVzdC5uZXQvQ1BTIGlz IGluY29ycG9yYXRlZCBieSByZWZlcmVuY2UxHzAdBgNVBAsTFihjKSAyMDEwIEVudHJ1c3Qs IEluYy4xIjAgBgNVBAMTGUVudHJ1c3QgQ2xhc3MgMiBDbGllbnQgQ0ECEQCHl+6EEWs92wAA AABMNV7gMA0GCWCGSAFlAwQCAQUAoGkwLwYJKoZIhvcNAQkEMSIEIFWl/DMEvk/1ByXV4JZw uEBpEEM/lcryMVJEd108390GMBgGCSqGSIb3DQEJAzELBgkqhkiG9w0BBwEwHAYJKoZIhvcN AQkFMQ8XDTE2MTAxMTAwMDAxN1owDQYJKoZIhvcNAQEBBQAEggEAHPLkw58y88nDuXNBtOWT pUtN6XpNlwhBnltjCaQWK5xlsxC6EekZijkoLHQvy3N1huAzSm+3uYm9LrZ/WvrvarUhs/ZF H7Zbyfn3A3z3N7QAxMTTCei2JPvECZSjygeAV9GbDjo1PLHiBDHfR/ZEZ7WHI1ZlNOyOAJVg jEsoLhY052X/JfNlUiohaojXFlFGFUUz4uH72V57UtSPrOaGg+PBQxMeTMAetYoOEK4E1bDN 6u9QTwpXd6ILnNWudABTWaAaJh8AfRYGuGHipB6dnRlnfrPG7Lnj5ZavmiNNoqaYLVunIGkP isBt7s1mxysJnu4dNyU+dRSreg/hSJ7Zyg== --B_3558963617_1968738120--