nge_repair options parsing.
>
> On 2.1 it was added support to simultaneous -pr and -local options on
> CASSANDRA-7450, so if you need that you can either upgade to 2.1 or
> backport that to 2.0.
>
>
> 2016-08-10 5:20 GMT-03:00 Anishek Agarwal <anis...@gmail.com>:
>
Hello,
We have 2.0.17 cassandra cluster(*DC1*) with a cross dc setup with a
smaller cluster(*DC2*). After reading various blogs about
scheduling/running repairs looks like its good to run it with the following
-pr for primary range only
-st -et for sub ranges
-par for parallel
-dc to make sure
Looks like some problem with our monitoring framework. Thanks for you help !
On Mon, Apr 18, 2016 at 2:46 PM, Anishek Agarwal <anis...@gmail.com> wrote:
> OS used : Cent OS 6 on all nodes except *10*.125.138.59 ( which runs Cent
> OS 7)
> All of them are running Cassandra 2.
; "socketAddress"-> host.getSocketAddress.toString,
> "cassandraVersion" -> host.getCassandraVersion.toString,
> "isUp" -> host.isUp,
> "hostDistance" -> lbp.distance(host).toString
>
he output of nodetool status would really help answer some questions. I
> take it the 8 hosts in your graph are in the same DC. Are the four serving
> writes in the same logical or physical rack (as Cassandra sees it), while
> the others are not?
>
> On Tue, Apr 12, 2016 at 10:48
e in another?
>
> On Tue, Apr 12, 2016, 1:01 AM Anishek Agarwal <anis...@gmail.com> wrote:
>
>> hello,
>>
>> we have 8 nodes in one cluster and attached is the traffic patterns
>> across the nodes.
>>
>> its very surprising that only 4 nodes show transmitt
At that point you're running a data warehouse and lose some
> of the advantages of seemless cluster membership.
>
> On Wed, Mar 30, 2016 at 5:43 AM, Anishek Agarwal <anis...@gmail.com>
> wrote:
>
>> Hey Guys,
>>
>> We did the necessary changes and were trying to get th
tc), I actually don't mind it it takes more than a few hours to
> do a full repair. But I am not sure about 4 days... I guess it depends on
> the size of the cluster and data...
>
> On Tue, Mar 29, 2016 at 6:04 AM, Anishek Agarwal <anis...@gmail.com>
> wrote:
>
>> I would
this can't be done, so we have to setup two
different analytics cluster ? can't we just get data from CLUSTER_1/2 to
same cluster CLUSTER_3 ?
thanks
anishek
On Mon, Mar 21, 2016 at 3:31 PM, Anishek Agarwal <anis...@gmail.com> wrote:
> Hey Clint,
>
> we have two separate rings w
I would really like to know the answer for above because on some nodes
repair takes almost 4 days for us :(.
On Tue, Mar 29, 2016 at 8:34 AM, Jack Krupansky
wrote:
> Someone recently asked me for advice when their repair time was 2-3 days.
> I thought that was
ically a hard link to the
> involved SSTables, so it's not considered as data load from Cassandra but
> it is effectively using disk space.
>
> Hope this helps.
>
> Carlos Alonso | Software Engineer | @calonso <https://twitter.com/calonso>
>
> On 22 March 2016 at 07:57, Anish
>
>> Or do you mean that you have two keyspaces in one cluster?
>>
>> Or?
>>
>> Clint
>> On Mar 14, 2016 2:11 AM, "Anishek Agarwal" <anis...@gmail.com> wrote:
>>
>>> Hello,
>>>
>>> We are using cassandra 2.0.1
Hello,
We are using cassandra 2.0.17 and have two logical DC having different
Keyspaces but both having same logical name DC1.
we want to setup another cassandra cluster for analytics which should get
data from both the above DC.
if we setup the new DC with name DC2 and follow the steps
Hello,
we used to run repair on each node using
https://github.com/BrianGallew/cassandra_range_repair.git. most of the time
repairs finished in under 12 hrs per node, we had then 4 nodes. gradually
the repair time kept increasing as traffic increased, we also added more
nodes meanwhile, we have 7
parnew,
>> increasing new gen instead of decreasing it should help drop (faster)
>> rather than promoting to sv/oldgen (slower) ?
>>
>>
>>
>> From: Anishek Agarwal
>> Reply-To: "user@cassandra.apache.org"
>> Date: Thursday, March 3, 201
her 5 nodes it varies between 1-3.
>>
>>
>>
>> - Is Memory, CPU or disk a bottleneck? Is one of those running at the
>> limits?
>>
>>
>>
>> concurrent_compactors: 48
>>
>>
>>
>> Reducing this to 8 would free some space for transac
>
> Is there some iowait ? Could point to a bottleneck or bad hardware.
> iostats -mx 5 100
>
> ...
>
> Hope one of those will point you to an issue, but there are many more
> thing you could check.
>
> Let us know how it goes,
>
> C*heers,
> -----
also MAX_HEAP_SIZE=6G and HEAP_NEWSIZE=4G.
On Wed, Mar 2, 2016 at 1:40 PM, Anishek Agarwal <anis...@gmail.com> wrote:
> Hey Jeff,
>
> one of the nodes with high GC has 1400 SST tables, all other nodes have
> about 500-900 SST tables. the other node with high GC
SANDRA-9754 for work in progress
> to help mitigate that type of pain).
>
> - Jeff
>
> From: Anishek Agarwal
> Reply-To: "user@cassandra.apache.org"
> Date: Tuesday, March 1, 2016 at 11:12 PM
> To: "user@cassandra.apache.org"
> Subject: Lot of GC on t
Hello,
we have a cassandra cluster of 7 nodes, all of them have the same JVM GC
configurations, all our writes / reads use the TokenAware Policy wrapping
a DCAware policy. All nodes are part of same Datacenter.
We are seeing that two nodes are having high GC collection times. Then
mostly seem
her reason? Also, why do you
>>>> have --delete-before when you're copying data to a temp (assumed empty)
>>>> directory?
>>>>
>>>> On Thu, Feb 18, 2016 at 4:12 AM, Alain RODRIGUEZ <arodr...@gmail.com>
>>>> wrote:
>>>>
>
Looks like that sstablemetadata is available in 2.2 , we are on 2.0.x do
you know anything that will work on 2.0.x
On Tue, Feb 23, 2016 at 1:48 PM, Anishek Agarwal <anis...@gmail.com> wrote:
> Thanks Jeff, Awesome will look at the tools and JMX endpoint.
>
> our settings are b
likely that some sstables stick around longer than you expect.
>>
>> 2) max_sstable_age_days tells cassandra when to stop compacting that
>> file, not when to delete it.
>>
>> 3) You can change the window size using both the base_time_seconds
>> parameter and max
you’re
> not simply hitting GC pauses that cause your queries to run longer than you
> expect? Do you have graphs of GC time (first derivative of total gc time is
> common for tools like graphite), or do you see ‘gcinspector’ in your logs
> indicating pauses > 200ms?
>
> From: An
thanks
anishek
On Mon, Feb 22, 2016 at 10:23 AM, Anishek Agarwal <anis...@gmail.com> wrote:
> We are using DTCS have a 30 day window for them before they are cleaned
> up. I don't think with DTCS we can do anything about table sizing. Please
> do let me know if there are oth
uggestion about reducing this unless you partition your
> data.
>
>
> Bloom filter space used, bytes: 493777336 (400MB is huge)
>
> If number of keys are reduced then this will automatically reduce bloom
> filter size I believe.
>
>
>
> Jaydeep
>
> On Thu, Feb 18,
ld be
>>> inaccurate: https://issues.apache.org/jira/browse/CASSANDRA-8525
>>>
>>> There are also a couple of recent improvements to bloom filters:
>>> * https://issues.apache.org/jira/browse/CASSANDRA-8413
>>> * https://issues.apache.org/jira/browse/CASSANDR
Hello,
We have a table with composite partition key with humungous cardinality,
its a combination of (long,long). On the table we have
bloom_filter_fp_chance=0.01.
On doing "nodetool cfstats" on the 5 nodes we have in the cluster we are
seeing "Bloom filter false ratio:" in the range of 0.7
gt; On Wed, Feb 17, 2016 at 12:45 PM, Robert Coli <rc...@eventbrite.com>
> wrote:
>
>>
>>
>> On Tue, Feb 16, 2016 at 11:29 PM, Anishek Agarwal <anis...@gmail.com>
>> wrote:
>>>
>>> To accomplish this can I just copy the data from d
Additional note we are using cassandra 2.0.15 have 5 nodes in cluster ,
going to expand to 8 nodes.
On Wed, Feb 17, 2016 at 12:59 PM, Anishek Agarwal <anis...@gmail.com> wrote:
> Hello,
>
> We started with two 800GB SSD on each cassandra node based on our initial
> estimations
Hello,
We started with two 800GB SSD on each cassandra node based on our initial
estimations of read/write rate. As we started on boarding additional
traffic we find that CPU is becoming a bottleneck and we are not able to
run the NICE jobs like compaction very well. We have started expanding the
, Feb 15, 2016 at 4:21 PM, Anishek Agarwal <anis...@gmail.com> wrote:
> Hello,
>
> I have installed Ops center 5.2.3 along with agents on three cassandra
> nodes in my test cluster version 2.0.15. This has two tables in one
> keyspace. I have a program that is reading
Hello,
I have installed Ops center 5.2.3 along with agents on three cassandra
nodes in my test cluster version 2.0.15. This has two tables in one
keyspace. I have a program that is reading values only from one of the
tables(table1) with in a keyspace.
I am looking at two graphs
- Read
Bulkloader_t.html
>
>
>
> Sent from Yahoo Mail on Android
> <https://overview.mail.yahoo.com/mobile/?.src=Android>
> --
>
> *From*:"Anishek Agarwal" <anis...@gmail.com>
> *Date*:Wed, Nov 18, 2015 at 11:24
> *Subject*:Strategy too
Hello
We have 5 node prod cluster and 3 node test cluster. Is there a way i can
take snapshot of a table in prod and load it test cluster. The cassandra
versions are same.
Even if there is a tool that can help with this it will be great.
If not, how do people handle scenarios where data in prod
@Rob interesting something i will try next time, for step 3 you mentioned
-- I just remove the -Dcassandra.join_ring=false option and restart the
cassandra service?
@Anuj, gc_grace_seconds dictates how long hinted handoff are stored right.
These might be good where we explicitly delete values
nope its not
On Mon, Nov 16, 2015 at 5:48 PM, sai krishnam raju potturi <
pskraj...@gmail.com> wrote:
> Is that a seed node?
>
> On Mon, Nov 16, 2015, 05:21 Anishek Agarwal <anis...@gmail.com> wrote:
>
>> Hello,
>>
>> We are having a 3 node
r I
> get when I forget to set the replace_address on Cassandra-env.
>
>
>
> JVM_OPTS="$JVM_OPTS -Dcassandra.replace_address=address_of_dead_node
>
>
>
>
>
> *From:* Anishek Agarwal [mailto:anis...@gmail.com]
> *Sent:* Monday, November 16, 2015 9:25 AM
> *To:
address? That is usually the error I
> get when I forget to set the replace_address on Cassandra-env.
>
> JVM_OPTS="$JVM_OPTS -Dcassandra.replace_address=address_of_dead_node
>
>
> *From:* Anishek Agarwal [mailto:anis...@gmail.com]
> *Sent:* Monday, November 16, 2015 9:25 AM
> *To:*
Hello,
We are having a 3 node cluster and one of the node went down due to a
hardware memory failure looks like. We followed the steps below after the
node was down for more than the default value of *max_hint_window_in_ms*
I tried to restart cassandra by following the steps @
1.
if its some sort of timeseries DTCS might turn out to be better for
compaction. also some disk monitoring might help to understand if disk is
the bottleneck.
On Sun, Oct 25, 2015 at 3:47 PM, 曹志富 wrote:
> I will try to trace a read that take > 20msec
> .
>
> just HDD.no
Anyone has seen similar behavior with LCS, please do let me know, It will
be good to know this can happen.
On Fri, Oct 9, 2015 at 5:19 PM, Anishek Agarwal <anis...@gmail.com> wrote:
> Looks like some of the nodes have higher sstables on L0 and compaction is
> running there, so on
, Anishek Agarwal <anis...@gmail.com> wrote:
> hello,
>
> on doing cfstats for the column family i see
>
> SSTables in each level: [1, 10, 109/100, 1, 0, 0, 0, 0, 0]
>
> i thought compaction would trigger since the 3rd level tables are move
> than expected number,
&g
hello,
on doing cfstats for the column family i see
SSTables in each level: [1, 10, 109/100, 1, 0, 0, 0, 0, 0]
i thought compaction would trigger since the 3rd level tables are move than
expected number,
but on doing compactionstats its shows "n/a" -- any reason why its not
triggering, should
Hey all,
We are using DTCS and we have a ttl of 30 days for all inserts, there are
no deletes/updates we do.
When the SST tables is dropped by DTCS what kind of logging do we see in C*
logs.
any help would be useful. The reason is my db size is not hovering around a
size it is increasing, there
Hello all,
We are running c* version 2.0.15. We have 5 nodes with RF=3. We are using
DTCS and on all inserts we have a TTL of 30 days. We have no deletes.We
just have one CF. When i run nodetool repair on a node i notice a lot of
extra sst tables created, this I think is due to the fact that its
may be just increase the read and write timeouts at cassandra currently at
5 sec i think. i think the datastax java client driver provides ability to
say how many max requests per connection are to be sent, you can try and
lower that to limit excessive requests along with limiting the number of
Hello,
I am seeing that even though the bloom filter fp ratio being set to 0.1 the
actual is at about .55 and on looking at the histograms of the table i see
that there are reads going to 3+ SSTtables even though the way i am
querying for read it should look at the most recent row only since i
Hello,
I have a text partition key for one of the CF. The cfstats on that table
seems to show that the bloom filter false positive ratio is always 1. Also
the bloom filter is using very less space.
Do bloom filters not work well with text partition keys ? I can assume this
as it can no way
Hello everyone,
i have a 3 node cluster with Cassandra 2.0.14 on centos in the same Data
center with RF=3 and i am using CL=Local_Quorum by default for the read and
write operations. I have given about 5 GB of heap space to cassandra.
I have 40 core machines with 3 separate SATA disks with
how many sst tables were there? what compaction are you using ? These
properties define how many possible disk reads cassandra has to do to get
all the data you need depending on which SST Tables have data for your
partition key.
On Fri, May 8, 2015 at 6:25 PM, Alprema alpr...@alprema.com
the memory cassandra is trying to allocate is pretty small. you sure there
is no hardware failure on the machine. what is the free ram on the box ?
On Mon, May 11, 2015 at 3:28 PM, Rahul Bhardwaj
rahul.bhard...@indiamart.com wrote:
Hi All,
We have cluster of 3 nodes with 64GB RAM each. My
(-x) unlimited
Also attaching complete error file
On Mon, May 11, 2015 at 3:35 PM, Anishek Agarwal anis...@gmail.com
wrote:
the memory cassandra is trying to allocate is pretty small. you sure
there is no hardware failure on the machine. what is the free ram on the
box
did u setup CQLSH_HOST variable to the ip so cqlsh uses that ?
On Tue, May 5, 2015 at 8:50 PM, Björn Hachmann bjoern.hachm...@metrigo.de
wrote:
Hello,
I am unable to connect to the nodes of our second datacenter, not even
from localhost.
The error message I receive is:
Connection error:
I think these will help speed up
- removing compression
- you have lot of independent columns mentioned. If you are always going to
query all of them together one other thing that will help is have a full
json(or some custom obj representation) of the value data and change the
model to just have
also might want to go through a thread here in with subject High latencies
for simple queries
On Wed, Apr 22, 2015 at 1:55 PM, Anishek Agarwal anis...@gmail.com wrote:
I think these will help speed up
- removing compression
- you have lot of independent columns mentioned. If you are always
sorry i take that back we will modify different keys across threads not the
same key, our storm topology is going to use field grouping to get updates
for same keys to same set of bolts.
On Tue, Apr 21, 2015 at 6:17 PM, Anishek Agarwal anis...@gmail.com wrote:
@Bruice : I dont think so as i am
, but you can correlate I guess, tools/bin/sstablemetadata gives you
sstable level information
and, it is also likely that since you get so many L0 sstables, you will
be doing size tiered compaction in L0 for a while.
On Tue, Apr 21, 2015 at 1:40 PM, Anishek Agarwal anis...@gmail.com
wrote
at 2:48 PM, Anishek Agarwal anis...@gmail.com
wrote:
sorry i take that back we will modify different keys across threads
not the same key, our storm topology is going to use field grouping to
get
updates for same keys to same set of bolts.
On Tue, Apr 21, 2015 at 6:17 PM, Anishek Agarwal
L0
- L1 compaction going on that blocks other compactions from starting
On Tue, Apr 21, 2015 at 1:06 PM, Anishek Agarwal anis...@gmail.com
wrote:
the some_bits column has about 14-15 bytes of data per key.
On Tue, Apr 21, 2015 at 4:34 PM, Anishek Agarwal anis...@gmail.com
wrote:
Hello
Hello,
We are using cassandra 2.0.14 and have a cluster of 3 nodes. I have a
writer test (written in java) that runs 50 threads to populate data to a
single table in a single keyspace.
when i look at the iftop I see that the amount of network transfer
happening on two nodes is same but on one
the some_bits column has about 14-15 bytes of data per key.
On Tue, Apr 21, 2015 at 4:34 PM, Anishek Agarwal anis...@gmail.com wrote:
Hello,
I am inserting about 100 million entries via datastax-java driver to a
cassandra cluster of 3 nodes.
Table structure is as
create keyspace test
Hello,
I am inserting about 100 million entries via datastax-java driver to a
cassandra cluster of 3 nodes.
Table structure is as
create keyspace test with replication = {'class':
'NetworkTopologyStrategy', 'DC' : 3};
CREATE TABLE test_bits(id bigint primary key , some_bits text) with
| Linkedin: *linkedin.com/in/carlosjuzarterolo
http://linkedin.com/in/carlosjuzarterolo*
Mobile: +31 6 159 61 814 | Tel: +1 613 565 8696 x1649
www.pythian.com
On Tue, Apr 21, 2015 at 1:06 PM, Anishek Agarwal anis...@gmail.com
wrote:
the some_bits column has about 14-15 bytes of data per key
Hello,
I was trying to find what protocol versions are supported in Cassandara
2.0.14 and after reading multiple links i am very very confused.
Please correct me if my understanding is correct:
- Binary Protocol version and CQL Spec version are different ?
- Cassandra 2.0.x supports CQL 3
Thanks Tyler for the validations,
I have a follow up question.
One SSTable doesn't have precedence over another. Instead, when the same
cell exists in both sstables, the one with the higher write timestamp wins.
if my table has 5(non partition key columns) and i update only 1 of them
then the
ready PHP driver?
On Fri, Apr 10, 2015 at 5:47 AM, Anishek Agarwal anis...@gmail.com
wrote:
Hello,
As part of using this for our project one of our teams need PHP driver
for cassandra. the datastax page says its in ALPHA, is there some release
candidate that people have used or any way to get
, 2015 at 8:47 AM, Anishek Agarwal anis...@gmail.com
wrote:
Hello,
As part of using this for our project one of our teams need PHP driver
for cassandra. the datastax page says its in ALPHA, is there some release
candidate that people have used or any way to get this working with PHP ?
Thanks
to increase MaxTenuringThreshold,
to keep CMS off of moving data to old gen.
If you sure that young gen is filled not so fast, that you can increase
*CMSWaitDuration* to avoid useless calls of CMS.
On 04/10/2015 03:42 PM, Anishek Agarwal wrote:
Sorry i forgot to update but i am not using
Hello,
We have only on CF as
CREATE TABLE t1(id bigint, ts timestamp, definition text, primary key (id,
ts))
with clustering order by (ts desc) and gc_grace_seconds=0
and compaction = {'class': 'DateTieredCompactionStrategy',
'timestamp_resolution':'SECONDS', 'base_time_seconds':'20',
Hey Peter,
This is from the perspective of 2.0.13 but there should be something
similar in your version. Can you enable debug log for cassandra and see if
the log files have additional info. Depending on how soon/later in you test
you get the error, you might also want to modify the
Colin,
When you said larger number of tokens has Query performance hit, is it read
or write performance. Also if you have any links you could share to shed
some light on this it would be great.
Thanks
Anishek
On Sun, Mar 29, 2015 at 2:20 AM, Colin Clark co...@clark.ws wrote:
I typically use a
Are you frequently updating same rows ? What is the memtable flush size ?
can you post the table create query here in please.
On Thu, Mar 26, 2015 at 1:21 PM, Dave Galbraith david92galbra...@gmail.com
wrote:
Hey! So I'm running Cassandra 2.1.2 and using the
SizeTieredCompactionStrategy. I'm
Hello,
If i have a custom type EventDefinition and i create a table like
create table TestTable {
user_id long,
ts timestamp,
definition 'com.anishek.EventDefinition',
Primary Key (user_id, ts))
with clustering order by (ts desc) and compression={'sstable_compression' :
'SnappyCompressor'}
and
Forgot to mention I am using Cassandra 2.0.13
On Mon, Mar 23, 2015 at 5:59 PM, Anishek Agarwal anis...@gmail.com wrote:
Hello,
I am using a single node server class machine with 16 CPUs with 32GB RAM
with a single drive attached to it.
my table structure is as below
CREATE TABLE t1(id
Hello,
I am using a single node server class machine with 16 CPUs with 32GB RAM
with a single drive attached to it.
my table structure is as below
CREATE TABLE t1(id bigint, ts timestamp, cat1 settext, cat2
settext, lat float, lon float, a bigint, primary key (id, ts));
I am trying to insert
76 matches
Mail list logo