Re: Question about how sorted runs are picked for compaction with STCS

2021-11-23 Thread Jay Zhuang
Hi Mark, Non-adjacent files can be merged. For cassandra, it anyway needs to query all sorted runs as it's timestamps are not strictly ordered. Thanks, Jay On Tue, Nov 23, 2021 at 9:42 AM MARK CALLAGHAN wrote: > I am trying to understand how sorted runs are picked for compaction with > STCS an

Re: Difficulties after nodetool removenode

2019-07-04 Thread Jay Zhuang
Hi Morten, it might be a bug, which C* version are you using? To guarantee consistency, it's recommended to run repair on all nodes after removeNode (for NetworkTopologyStrategy, it could be all nodes in that specific datacenter). On Thu, Jul 4, 2019 at 8:30 AM Alain RODRIGUEZ wrote: > Hello, >

Re: Using CDC Feature to Stream C* to Kafka (Design Proposal)

2018-09-12 Thread Jay Zhuang
We have the similar use case: Streamific, the Ingestion Service for Hadoop Big Data at Uber Engineering . We had this data ingestion pipeline built on MySQL/schemaless before using Cassandra. For Cassandra, we used to do

Re: LEAK DETECTED while minor compaction

2018-02-24 Thread Jay Zhuang
Maybe it's because of https://issues.apache.org/jira/browse/CASSANDRA-12014 We had the similar issue in 3.0.14, when the SSTable has more than about 51 million keys: https://issues.apache.org/jira/browse/CASSANDRA-13785 If you upgrade to 2.2.11+, you should be able to see the real compaction exc

Re: CDC usability and future development

2018-02-01 Thread Jay Zhuang
We did a POC to improve CDC feature as an interface ( https://github.com/ngcc/ngcc2017/blob/master/CassandraDataIngestion.pdf), so the user doesn't have to read the commit log directly. We deployed the change to a test cluster and doing more tests for production traffics, will send out the design p

Is it recommended to enable debug log in production

2018-01-16 Thread Jay Zhuang
Hi, Do you guys enable debug log in production? Is it recommended? By default, the cassandra log level is set to debug: https://github.com/apache/cassandra/blob/trunk/conf/logback.xml#L100 We’re using 3.0.x, which generates lots of Gossip messages: FailureDetector.java:456 - Ignoring interval ti

Re: Do not use Cassandra 3.11.0+ or Cassandra 3.0.12+

2017-08-28 Thread Jay Zhuang
We're using 3.0.12+ for a few months and haven't seen the issue like that. Do we know what could trigger the problem? Or is 3.0.x really impacted? Thanks, Jay On 8/28/17 6:02 AM, Hannu Kröger wrote: > Hello, > > Current latest Cassandra version (3.11.0, possibly also 3.0.12+) has a race > condit

Re: Stress test

2017-07-27 Thread Jay Zhuang
The user and password should be in -mode section, for example: ./cassandra-stress user profile=table.yaml ops\(insert=1\) -mode native cql3 user=** password=** http://docs.datastax.com/en/cassandra/3.0/cassandra/tools/toolsCStress.html /Jay On 7/27/17 2:46 PM, Greg Lloyd wrote: > I am trying to

Re: commitlog_total_space_in_mb tuning

2017-07-05 Thread Jay Zhuang
blem is it's causing lots of read repair timeouts as the repair mutations are dropped. Which causes Cassandra JVM hang or sometimes crash. /Jay On 7/5/17 2:45 PM, Jeff Jirsa wrote: > > > On 2017-07-05 14:32 (-0700), Jay Zhuang wrote: >> Hi, >> >> commi

commitlog_total_space_in_mb tuning

2017-07-05 Thread Jay Zhuang
Hi, commitlog_total_space_in_mb is increased from 1G to 8G in CASSANDRA-7031. Sometimes we saw the number of dropped mutations spikes. Not sure if it's a sign that we should increase the commitlog_total_space_in_mb? For bean: org.apache.cassandra.metrics:name=WaitingOnSegmentAllocation,type=Commi

Ask for suggestions to de-duplicate data for Cassandra CDC

2017-06-20 Thread Jay Zhuang
Hi, For Cassandra CDC feature: http://cassandra.apache.org/doc/latest/operating/cdc.html The CDC data is duplicated RF number of times. Let's say replication factor is 3 in one DC, the same data will be sent out 3 times. One solution is adding another DC with RF=1, which will be only used for CDC

Re: Pluggable throttling of read and write queries

2017-02-22 Thread Jay Zhuang
Here is the Scheduler interface: https://github.com/apache/cassandra/blob/cassandra-3.11/conf/cassandra.yaml#L978 Seems like it could be used for this case. It is removed in 4.x with thrift, not sure why: https://github.com/apache/cassandra/commit/4881d9c308ccd6b5ca70925bf6ebedb70e7705fc Tha