As I learned the hard way (and has already been implied), design your
tables to support your queries.
We have, for example, 9 tables storing the same data, because users wish to
query in different ways. Could be several more tables (if one was being a
purist), but indexes get us the rest of the wa
>This means that from the client driver perspective when I define the
contact points I can specify any node in the cluster as contact point and
not necessary a seed node?
Correct.
On Wed, Feb 12, 2020 at 11:48 AM Sergio wrote:
> So if
> 1) I stop the a Cassandra node that doesn't have in the
Been thinking about it, and I can't really see how with 4 nodes and RF=3,
any 2 nodes would *not* have all the data; but am more than willing to
learn.
On the other thing: that's an attractive option, but in our case, the
target cluster will likely come into use before the source-cluster data is
a
loading (sstableloader) only 2 of
the nodes rather than all 4?
Apologies for the spate of hypotheticals lately, this project is making
life interesting.
Thanks,
Voytek Jarnot
Odd. Have you seen this behavior? I ran a test last week, loaded snapshots
from 4 nodes to 4 nodes (RF 3 on both ends) and did not notice a spike.
That's not to say that it didn't happen, but I think I'd have noticed as I
was loading approx 250GB x 4 (although sequentially rather than 4x
sstableloa
de the case for num_tokens=4. His
>>>> Accelerate 2019 talk is available at:
>>>>
>>>>
>>>>
>>>> https://www.youtube.com/watch?v=swL7bCnolkU
>>>>
>>>>
>>>>
>>>> You might want to check that
t to check that out. Also I think the amount of effort you
>> put into evening out the token distribution increases as vnode count
>> shrinks. The caveats are explored at:
>>
>>
>>
>>
>> https://thelastpickle.com/blog/2019/02/21/set-up-a-cluster-with-ev
Running 3.11.x, 4 nodes RF=3, default 256 tokens; moving to a different 4
node RF=3 cluster.
I've read that 256 is not an optimal default num_tokens value, and that 32
is likely a better option.
We have the "opportunity" to switch, as we're migrating environments and
will likely be using sstablel
Needing to know when Cassandra is finished initializing and is up & running.
Had some scripts which were looking through system.log for "No gossip
backlog; proceeding", but that turns out not to be 100% reliable.
Is looking for "Starting listening for CQL clients" considered definitive?
I.E., alw
Apologies for the bump, but I'm wondering if anyone has any thoughts on the
question below - specifically about running nodetool rebuild on a
destination that has data that does not exist in the source
Thanks.
On Wed, Sep 11, 2019 at 2:41 PM Voytek Jarnot
wrote:
> Pardon the co
enter. Would the rebuild end up
dropping our post 2019-09-01 data?
Thanks,
Voytek Jarnot
cluster if you wished in the future.
>
> I would also be interested if anyone has the definitive answer one this.
>
> Thanks
>
> Paul
> www.redshots.com
>
> On 29 Jul 2019, at 17:06, Voytek Jarnot wrote:
>
> Just a quick bump - hoping someone can shed some light
ultiRegionSnitch? This setup would be temporary, existing
> until the new DC nodes have rebuilt and the old DC is decommissioned.
>
> Thanks,
> Voytek Jarnot
>
ew DC nodes have rebuilt and the old DC is decommissioned.
Thanks,
Voytek Jarnot
turned off, both clusters will TRY to talk to
> each other, but the different cluster names will prevent it.
> - At that point, you can nodetool removenode / nodetool assassinate the
> 'old' IPs in 'new' and the 'new' IPs in 'old'
> - Fina
nodes in each current DC to "forget" about the
other DC.
On Thu, Jul 11, 2019 at 10:03 AM Voytek Jarnot
wrote:
> My google-fu is failing me this morning. I'm looking for any tips on
> splitting a 2 DC cluster into two separate clusters. I see a lot of docs
> about decomiss
My google-fu is failing me this morning. I'm looking for any tips on
splitting a 2 DC cluster into two separate clusters. I see a lot of docs
about decomissioning a datacenter, but not much in the way of disconnecting
datacenters into individual clusters, but keeping each one as-is data-wise
(aside
NODE?_ELASTIC_IP to destination of NODE?_PRIVATE_IP solves
the issue, but that seems like a hack.
(For Example: iptables -t nat -A OUTPUT -p tcp -d ELASTIC_IP -j DNAT
--to-destination PRIVATE_IP)
Not sure if I'm missing a config item, or something in AWS is blocking me,
or if 3.11.2 has an issue.
Thanks,
Voytek Jarnot
o minimize changes to the existing DC, if that means Amazon
charges me for transferring data via public IPs, so be it.
On Wed, Jun 26, 2019, 10:19 AM Voytek Jarnot
wrote:
> I started an higher-level thread years ago about moving a cluster by
> expanding from 1 to 2 datacenters, replic
C living in AWS, whereas the new DC will
be in a different AZ. Do I need to get into switching
from GossipingPropertyFileSnitch to Ec2MultiRegionSnitch? If so, could
someone shed a bit of light on that process, and the associated changes
needed for listen_address and broadcast_address?
Thanks for getting this far,
Voytek Jarnot
Have you ruled out EBS snapshot initialization issues (
https://docs.aws.amazon.com/AWSEC2/latest/UserGuide/ebs-initialize.html)?
On Tue, Mar 27, 2018 at 2:24 PM, Mike Torra wrote:
> Hi There -
>
> I have noticed an issue where I consistently see high p999 read latency on
> a node for a few hour
Have a scenario where it's necessary to migrate a cluster to a different
set of hardware with minimal downtime. Setup is:
Current cluster: 4 nodes, RF 3
New cluster: 6 nodes, RF 3
My initial inclination is to follow this writeup on setting up the 6 new
nodes as a new DC:
https://docs.datastax.com
Multiple versions of python can coexist, the cqlsh shell script will
attempt to execute via a python2.7 executable if it finds one.
On Tue, Apr 4, 2017 at 9:49 AM, Jacob Shadix wrote:
> I've recently upgraded to 3.0.12 and unable to run CQLSH.
> No appropriate python interpreter found.
>
> The c
y thoughts,
Thanks.
On Mon, Apr 3, 2017 at 9:56 AM, Voytek Jarnot
wrote:
> Further info - tablehistograms reports zeros for all percentiles for Read
> Latency; tablestats also reports really low numbers for Bloom filter usage
> (3-4 KiB, depending on node, whereas I'd expect orders of magn
most
written-to and read-from table in the keyspace, seems to keep up with
tracking of writes, but not reads.
Full repair on this table is the only thing I can think of; but that's a
guess and doesn't get me any closer to understanding what has happened.
On Fri, Mar 31, 2017 at 11:11 PM
Cassandra 3.9
Have a keyspace with 5 tables, one of which is exhibiting rather poor read
performance. In starting an attempt to get to the bottom of the issues, I
noticed that, when running nodetool tablestats against the keyspace, that
particular table reports "Local read count: 0" on all nodes -
Was about to optimize some queries, given tracing out, but then saw
CASSANDRA-13120 (https://issues.apache.org/jira/browse/CASSANDRA-13120) and
am now wondering if there's nothing to be gained.
We have a table with a (quite simplified/sanitized) structure as such:
created_week_year int,
created_w
Apologies for the stream-of-consciousness replies, but are the dropped
message stats output by tpstats an accumulation since the node came up, or
are there processes which clear and/or time-out the info?
On Mon, Mar 20, 2017 at 3:18 PM, Voytek Jarnot
wrote:
> No dropped messages in tpstats
No dropped messages in tpstats on any of the nodes.
On Mon, Mar 20, 2017 at 3:11 PM, Voytek Jarnot
wrote:
> Appreciate the reply, Kurt.
>
> I sanitized it out of the traces, but all trace outputs listed the same
> node for all three queries (1 working, 2 not working). Read repair
Appreciate the reply, Kurt.
I sanitized it out of the traces, but all trace outputs listed the same
node for all three queries (1 working, 2 not working). Read repair chance
set to 0.0 as recommended when using TWCS.
I'll check tpstats - in this environment, load is not an issue, but network
issu
A wrinkle further confounds the issue: running a repair on the node which
was servicing the queries has cleared things up and all the queries now
work.
That doesn't make a whole lot of sense to me - my assumption was that a
repair shouldn't have fixed it.
On Fri, Mar 17, 2017 at 12:03
Cassandra 3.9, 4 nodes, rf=3
Hi folks, we're see 0 results returned from queries that (a) should return
results, and (b) will return results with minor tweaks.
I've attached the sanitized trace outputs for the following 3 queries (pk1
and pk2 are partition keys, ck1 is clustering key, val1 is SAS
We use QueryLogger which is baked in to the datastax java driver; gives you
basic query execution times (and bind params) in your logs, can be tweaked
using log levels.
On Thu, Jan 12, 2017 at 12:31 PM, Jonathan Haddad wrote:
> You're likely to benefit a lot more if you log query times from your
Opened https://issues.apache.org/jira/browse/CASSANDRA-13105 because it
does seem like this should work.
On Wed, Jan 4, 2017 at 5:53 PM, Voytek Jarnot
wrote:
> Seeing queries return 0 rows incorrectly, running 3.9
>
> Setup:
>
> create table test1(id1 text PRIMARY KEY, val1
Seeing queries return 0 rows incorrectly, running 3.9
Setup:
create table test1(id1 text PRIMARY KEY, val1 text, val2 text);
create custom index test1_idx_val1 on test1(val1) using
'org.apache.cassandra.index.sasi.SASIIndex';
create custom index test1_idx_val2 on test1(val2) using
'org.apache.ca
onkar wrote:
>
> In the first case, the partitioning is based on key1,key2,key3.
>
> In the second case, partitioning is based on key1 , key2. Additionally you
> have a clustered key key3. This means within a partition you can do range
> queries on key3 efficiently. That is the di
concerned about
> conflicting writes getting applied in the correct order.
>
> Probably worth a footnote in the documentation indicating that if you're
> doing both USING TTL and WITH TIMESTAMP that those don't relate to each
> other. At rest TTL'd records get written with an
It appears as though, when inserting with "using ttl [foo] and timestamp
[bar]" that the TTL does not take the provided timestamp into account.
In other words, the TTL starts at insert time, not at the time specified by
the timestamp.
Similarly, if inserting with just "using timestamp [bar]" and
what partition.
>
> If you don't ever have to find all key2 for a given key1 I don't see any
> reason to do case 1
>
>
> > On 27 Dec 2016, at 16:42, Voytek Jarnot wrote:
> >
> > Wondering if there's a difference when querying by primary key betw
Wondering if there's a difference when querying by primary key between the
two definitions below:
primary key ((key1, key2, key3))
primary key ((key1, key2), key3)
In terms of read speed/efficiency... I don't have much of a reason
otherwise to prefer one setup over the other, so would prefer the
cassandra.yaml has various timeouts such as read_request_timeout,
range_request_timeout, write_request_timeout, etc. The driver does as well
(via Cluster -> Configuration -> SocketOptions -> setReadTimeoutMillis).
Not sure if you can (or would want to) set them to "forever", but it's a
starting p
Reading that article the only conclusion I can reach (unless I'm
misreading) is that all the stuff that was never free is still not free -
the change is that Oracle may actually be interested in the fact that some
are using non-free products for free.
Pretty much a non-story, it seems like.
On Tu
l and
tombstone_threshold? Or am I was off - is there more to it?
On Sat, Dec 17, 2016 at 11:08 AM, Voytek Jarnot
wrote:
> Thanks again.
>
> I swear I'd look this up instead, but my google-fu is failing me
> completely ... That said, I presume that they're enab
rsa
wrote:
> With the caveat that tombstone compactions are disabled by default in TWCS
> (and DTCS)
>
> --
> Jeff Jirsa
>
>
> On Dec 16, 2016, at 8:34 PM, Voytek Jarnot
> wrote:
>
> Gotcha. "never compacted" has an implicit asterisk referencing
> tombs
do, if there’s an sstable
> over 24 hours old that’s at least 80% tombstones, it’ll compact it in a
> single sstable compaction).
>
>
>
> - Jeff
>
>
>
> *From: *Voytek Jarnot
> *Reply-To: *"user@cassandra.apache.org"
> *Date: *Friday, Decem
gt; anyway (memTable flush to sstable isn’t going to happen exactly when the
> window expires, so it’ll bleed a bit anyway), so I bet no meaningful impact.
>
>
>
> - Jeff
>
>
>
> *From: *Voytek Jarnot
> *Reply-To: *"user@cassandra.apache.org&q
t reading from all of them, and with a partition-per-week the
> bloom filter is going to make things nice and easy for you.
>
>
>
> - Jeff
>
>
>
>
>
> *From: *Voytek Jarnot
> *Reply-To: *"user@cassandra.apache.org"
> *Date: *Friday, December 16,
Scenario:
Converting an Oracle table to Cassandra, one Oracle table to 4 Cassandra
tables, basically time-series - think log or auditing. Retention is 10
years, but greater than 95% of reads will occur on data written within the
last year. 7 day TTL used on a small percentage of the records, major
e "Writing large partition" WARNing (exposes the partition
key). There are probably others. The large partition warning would
probably be mostly useless without logging the partition key, but - in any
case - there are usage scenarios where data in logs is prohibited.
Thanks,
Voytek Jarnot
rformance if you never clear out old failed transactions. The
> tables we have that use this generally use TTLs so we don't really care as
> long as irrecoverable transaction failures are very rare.
>
> Thanks,
> Cody
>
> On Wed, Dec 7, 2016 at 1:56 PM, Voytek Jarnot
> wr
ll, with some deletes via TTL, but anyway), so that's
somewhat tempting, but I'd rather not prematurely optimize. Unless, of
course, anyone's got experience such that "batches over XXkb are definitely
going to be a problem".
Appreciate everyone's time.
--Voytek Jarn
>
> Am 07.12.2016 17:27 schrieb "Voytek Jarnot" :
>
>> Sure, about which part?
>>
>> default batch size warning is 5kb
>> I've increased it to 30kb, and will need to increase to 40kb (8x default
>> setting) to avoid WARN log messages about
>
> Am 07.12.2016 17:10 schrieb "Voytek Jarnot" :
>
>> Should've mentioned - running 3.9. Also - please do not recommend MVs: I
>> tried, they're broken, we punted.
>>
>> On Wed, Dec 7, 2016 at 10:06 AM, Voytek Jarnot
>> wrote:
>>
>
The low default value for batch_size_warn_threshold_in_kb is making me
wonder if I'm perhaps approaching the problem of atomicity in a non-ideal
fashion.
With one data set duplicated/denormalized into 5 tables to support queries,
we use batches to ensure inserts make it to all or 0 tables. This w
Should've mentioned - running 3.9. Also - please do not recommend MVs: I
tried, they're broken, we punted.
On Wed, Dec 7, 2016 at 10:06 AM, Voytek Jarnot
wrote:
> The low default value for batch_size_warn_threshold_in_kb is making me
> wonder if I'm perhaps approaching the
situation
>
> On Sat, Nov 5, 2016 at 5:15 AM, Jonathan Haddad wrote:
>
>> Can you file a Jira for this? Would be good to make sure 3.10 doesn't get
>> released with this bug.
>> On Fri, Nov 4, 2016 at 6:11 PM Voytek Jarnot
>> wrote:
>>
>>> Tho
Yep, already done: https://issues.apache.org/jira/browse/CASSANDRA-12877
On Fri, Nov 4, 2016 at 11:15 PM, Jonathan Haddad wrote:
> Can you file a Jira for this? Would be good to make sure 3.10 doesn't get
> released with this bug.
> On Fri, Nov 4, 2016 at 6:11 PM Voytek J
1-2','6-3','asdf');
insert into tester(id1,id2,id3, val1) values ('1-1','1-2','7-3','asdf');
insert into tester(id1,id2,id3, val1) values ('1-1','1-2','8-3','asdf');
insert into tester(id1,id2,
Wondering if anyone has encountered the same...
Full story and stacktraces below, short version is that creating a SASI
index fails for me when running a 3.10-SNAPSHOT build. One caveat: creating
the index on an empty table doesn't fail; however, soon after I start
pumping data into the table simi
y
>
> On Sat, Oct 29, 2016 at 3:25 AM, Voytek Jarnot
> wrote:
>
>> Scenario (running 3.9, by the way):
>>
>> CREATE TABLE atc_test1.idxtest (
>> pk text PRIMARY KEY,
>> col1 text,
>> col2 text);
>> CREATE CUSTOM INDEX idx2 ON atc_
Scenario (running 3.9, by the way):
CREATE TABLE atc_test1.idxtest (
pk text PRIMARY KEY,
col1 text,
col2 text);
CREATE CUSTOM INDEX idx2 ON atc_test1.idxtest (col2) USING
'org.apache.cassandra.index.sasi.SASIIndex';
CREATE INDEX idx1 ON atc_test1.idxtest (col1);
Queries:
Works: sele
61 matches
Mail list logo