Another thing to notice is :

system_auth WITH replication = {'class': 'SimpleStrategy',
'replication_factor': '1'}

system_auth has a replication factor of 1 and even if one node is down it
may impact the system because of the replication factor.



On Wed, 12 Sep 2018 at 09:46, Steinmaurer, Thomas <
thomas.steinmau...@dynatrace.com> wrote:

> Hi,
>
>
>
> I remember something that a client using the native protocol gets notified
> too early by Cassandra being ready due to the following issue:
>
> https://issues.apache.org/jira/browse/CASSANDRA-8236
>
>
>
> which looks similar, but above was marked as fixed in 2.2.
>
>
>
> Thomas
>
>
>
> *From:* Riccardo Ferrari <ferra...@gmail.com>
> *Sent:* Mittwoch, 12. September 2018 18:25
> *To:* user@cassandra.apache.org
> *Subject:* Re: Read timeouts when performing rolling restart
>
>
>
> Hi Alain,
>
>
>
> Thank you for chiming in!
>
>
>
> I was thinking to perform the 'start_native_transport=false' test as well
> and indeed the issue is not showing up. Starting the/a node with native
> transport disabled and letting it cool down lead to no timeout exceptions
> no dropped messages, simply a crystal clean startup. Agreed it is a
> workaround
>
>
>
> # About upgrading:
>
> Yes, I desperately want to upgrade despite is a long and slow task. Just
> reviewing all the changes from 3.0.6 to 3.0.17
> is going to be a huge pain, top of your head, any breaking change I should
> absolutely take care of reviewing ?
>
>
>
> # describecluster output: YES they agree on the same schema version
>
>
>
> # keyspaces:
>
> system WITH replication = {'class': 'LocalStrategy'}
>
> system_schema WITH replication = {'class': 'LocalStrategy'}
>
> system_auth WITH replication = {'class': 'SimpleStrategy',
> 'replication_factor': '1'}
>
> system_distributed WITH replication = {'class': 'SimpleStrategy',
> 'replication_factor': '3'}
>
> system_traces WITH replication = {'class': 'SimpleStrategy',
> 'replication_factor': '2'}
>
>
>
> <custom1> WITH replication = {'class': 'SimpleStrategy',
> 'replication_factor': '3'}
>
> <custom2>  WITH replication = {'class': 'SimpleStrategy',
> 'replication_factor': '3'}
>
>
>
> # Snitch
>
> Ec2Snitch
>
>
>
> ## About Snitch and replication:
>
> - We have the default DC and all nodes are in the same RACK
>
> - We are planning to move to GossipingPropertyFileSnitch configuring the
> cassandra-rackdc accortingly.
>
> -- This should be a transparent change, correct?
>
>
>
> - Once switched to GPFS, we plan to move to 'NetworkTopologyStrategy' with
> 'us-xxxx' DC and replica counts as before
>
> - Then adding a new DC inside the VPC, but this is another story...
>
>
>
> Any concerns here ?
>
>
>
> # nodetool status <ks>
>
> --  Address         Load       Tokens       Owns (effective)  Host
> ID                               Rack
> UN  10.x.x.a  177 GB     256          50.3%
> d8bfe4ad-8138-41fe-89a4-ee9a043095b5  rr
> UN  10.x.x.b    152.46 GB  256          51.8%
> 7888c077-346b-4e09-96b0-9f6376b8594f  rr
> UN  10.x.x.c   159.59 GB  256          49.0%
> 329b288e-c5b5-4b55-b75e-fbe9243e75fa  rr
> UN  10.x.x.d  162.44 GB  256          49.3%
> 07038c11-d200-46a0-9f6a-6e2465580fb1  rr
> UN  10.x.x.e    174.9 GB   256          50.5%
> c35b5d51-2d14-4334-9ffc-726f9dd8a214  rr
> UN  10.x.x.f  194.71 GB  256          49.2%
> f20f7a87-d5d2-4f38-a963-21e24167b8ac  rr
>
>
>
> # gossipinfo
>
> /10.x.x.a
>   STATUS:827:NORMAL,-1350078789194251746
>   LOAD:289986:1.90078037902E11
>   SCHEMA:281088:af4461c3-d269-39bc-9d03-3566031c1e0a
>   DC:6:<some-ec2-dc>
>   RACK:8:rr
>   RELEASE_VERSION:4:3.0.6
>   SEVERITY:290040:0.5934718251228333
>   NET_VERSION:1:10
>   HOST_ID:2:d8bfe4ad-8138-41fe-89a4-ee9a043095b5
>   RPC_READY:868:true
>   TOKENS:826:<hidden>
> /10.x.x.b
>   STATUS:16:NORMAL,-1023229528754013265
>   LOAD:7113:1.63730480619E11
>   SCHEMA:10:af4461c3-d269-39bc-9d03-3566031c1e0a
>   DC:6:<some-ec2-dc>
>   RACK:8:rr
>   RELEASE_VERSION:4:3.0.6
>   SEVERITY:7274:0.5988024473190308
>   NET_VERSION:1:10
>   HOST_ID:2:7888c077-346b-4e09-96b0-9f6376b8594f
>   TOKENS:15:<hidden>
> /10.x.x.c
>   STATUS:732:NORMAL,-1117172759238888547
>   LOAD:245839:1.71409806942E11
>   SCHEMA:237168:af4461c3-d269-39bc-9d03-3566031c1e0a
>   DC:6:<some-ec2-dc>
>   RACK:8:rr
>   RELEASE_VERSION:4:3.0.6
>   SEVERITY:245989:0.0
>   NET_VERSION:1:10
>   HOST_ID:2:329b288e-c5b5-4b55-b75e-fbe9243e75fa
>   RPC_READY:763:true
>   TOKENS:731:<hidden>
> /10.x.x.d
>   STATUS:14:NORMAL,-1004942496246544417
>   LOAD:313125:1.74447964917E11
>   SCHEMA:304268:af4461c3-d269-39bc-9d03-3566031c1e0a
>   DC:6:<some-ec2-dc>
>   RACK:8:rr
>   RELEASE_VERSION:4:3.0.6
>   SEVERITY:313215:0.25641027092933655
>   NET_VERSION:1:10
>   HOST_ID:2:07038c11-d200-46a0-9f6a-6e2465580fb1
>   RPC_READY:56:true
>   TOKENS:13:<hidden>
> /10.x.x.e
>   STATUS:520:NORMAL,-1058809960483771749
>   LOAD:276118:1.87831573032E11
>   SCHEMA:267327:af4461c3-d269-39bc-9d03-3566031c1e0a
>   DC:6:<some-ec2-dc>
>   RACK:8:rr
>   RELEASE_VERSION:4:3.0.6
>   SEVERITY:276217:0.32786884903907776
>   NET_VERSION:1:10
>   HOST_ID:2:c35b5d51-2d14-4334-9ffc-726f9dd8a214
>   RPC_READY:550:true
>   TOKENS:519:<hidden>
> /10.x.x.f
>   STATUS:1081:NORMAL,-1039671799603495012
>   LOAD:239114:2.09082017545E11
>   SCHEMA:230229:af4461c3-d269-39bc-9d03-3566031c1e0a
>   DC:6:<some-ec2-dc>
>   RACK:8:rr
>   RELEASE_VERSION:4:3.0.6
>   SEVERITY:239180:0.5665722489356995
>   NET_VERSION:1:10
>   HOST_ID:2:f20f7a87-d5d2-4f38-a963-21e24167b8ac
>   RPC_READY:1118:true
>   TOKENS:1080:<hidden>
>
>
>
> ## About load and tokens:
>
> - While load is pretty even this does not apply to tokens, I guess we have
> some table with uneven distribution. This should not be the case for high
> load tabels as partition keys are are build with some 'id + <some time
> format>'
>
> - I was not able to find some documentation about the numbers printed next
> to LOAD, SCHEMA, SEVERITY, RPC_READY ... Is there any doc around ?
>
>
>
> # Tombstones
>
> No ERRORS, only WARN about a very specific table that we are aware of. It
> is an append only table read by spark from a batch job. (I guess it is a
> read_repair chance or DTCS misconfig)
>
>
>
> ## Closing note!
>
> We are on very only m1.xlarge 4 vcpu and raid0 (stripe) on the 4 spinning
> drives, some changes to the cassandra.yml:
>
>
>
> - dynamic_snitch: false
>
> - concurrent_reads: 48
>
> - concurrent_compactors: 1 (was 2)
>
> - disk_optimization_strategy: spinning
>
>
>
> I have some concerns about the number of concurrent_compactors, what do
> you think?
>
>
>
> Thanks!
>
>
> The contents of this e-mail are intended for the named addressee only. It
> contains information that may be confidential. Unless you are the named
> addressee or an authorized designee, you may not copy or use it, or
> disclose it to anyone else. If you received it in error please notify us
> immediately and then destroy it. Dynatrace Austria GmbH (registration
> number FN 91482h) is a company registered in Linz whose registered office
> is at 4040 Linz, Austria, Freistädterstraße 313
>

Reply via email to