Re: [EXTERNAL] Re: Nodetool refresh v/s sstableloader

2018-08-30 Thread Rajath Subramanyam
Thank you, everyone, for responding.


Rajath Subramanyam



On Thu, Aug 30, 2018 at 8:38 AM Carl Mueller
 wrote:

> - Range aware compaction strategy that subdivides data by the token range
> could help for this: you only bakcup data for the primary node and not the
> replica data
> - yes, if you want to use nodetool refresh as some sort of recovery
> solution, MAKE SURE YOU STORE THE TOKEN LIST with the
> sstables/snapshots/backups for the nodes.
>
> On Wed, Aug 29, 2018 at 8:57 AM Durity, Sean R <
> sean_r_dur...@homedepot.com> wrote:
>
>> Sstableloader, though, could require a lot more disk space – until
>> compaction can reduce. For example, if your RF=3, you will essentially be
>> loading 3 copies of the data. Then it will get replicated 3 more times as
>> it is being loaded. Thus, you could need up to 9x disk space.
>>
>>
>>
>>
>>
>> Sean Durity
>>
>> *From:* kurt greaves 
>> *Sent:* Wednesday, August 29, 2018 7:26 AM
>> *To:* User 
>> *Subject:* [EXTERNAL] Re: Nodetool refresh v/s sstableloader
>>
>>
>>
>> Removing dev...
>>
>> Nodetool refresh only picks up new SSTables that have been placed in the
>> tables directory. It doesn't account for actual ownership of the data like
>> SSTableloader does. Refresh will only work properly if the SSTables you are
>> copying in are completely covered by that nodes tokens. It doesn't work if
>> there's a change in topology, replication and token ownership will have to
>> be more or less the same.
>>
>>
>>
>> SSTableloader will break up the SSTables and send the relevant bits to
>> whichever node needs it, so no need for you to worry about tokens and
>> copying data to the right places, it will do that for you.
>>
>>
>>
>> On 28 August 2018 at 11:27, Rajath Subramanyam 
>> wrote:
>>
>> Hi Cassandra users, Cassandra dev,
>>
>>
>>
>> When recovering using SSTables from a snapshot, I want to know what are
>> the key differences between using:
>>
>> 1. Nodetool refresh and,
>>
>> 2. SSTableloader
>>
>>
>>
>> Does nodetool refresh have restrictions that need to be met?
>> Does nodetool refresh work even if there is a change in the topology
>> between the source cluster and the destination cluster? Does it work if the
>> token ranges don't match between the source cluster and the destination
>> cluster? Does it work when an old SSTable in the snapshot has a dropped
>> column that is not part of the current schema?
>>
>>
>>
>> I appreciate any help in advance.
>>
>>
>>
>> Thanks,
>>
>> Rajath
>>
>> 
>>
>> Rajath Subramanyam
>>
>>
>>
>>
>>
>> --
>>
>> The information in this Internet Email is confidential and may be legally
>> privileged. It is intended solely for the addressee. Access to this Email
>> by anyone else is unauthorized. If you are not the intended recipient, any
>> disclosure, copying, distribution or any action taken or omitted to be
>> taken in reliance on it, is prohibited and may be unlawful. When addressed
>> to our clients any opinions or advice contained in this Email are subject
>> to the terms and conditions expressed in any applicable governing The Home
>> Depot terms of business or client engagement letter. The Home Depot
>> disclaims all responsibility and liability for the accuracy and content of
>> this attachment and for any damages or losses arising from any
>> inaccuracies, errors, viruses, e.g., worms, trojan horses, etc., or other
>> items of a destructive nature, which may be contained in this attachment
>> and shall not be liable for direct, indirect, consequential or special
>> damages in connection with this e-mail message or its attachment.
>>
>


Nodetool refresh v/s sstableloader

2018-08-27 Thread Rajath Subramanyam
Hi Cassandra users, Cassandra dev,

When recovering using SSTables from a snapshot, I want to know what are the
key differences between using:
1. Nodetool refresh and,
2. SSTableloader

Does nodetool refresh have restrictions that need to be met?
Does nodetool refresh work even if there is a change in the topology
between the source cluster and the destination cluster? Does it work if the
token ranges don't match between the source cluster and the destination
cluster? Does it work when an old SSTable in the snapshot has a dropped
column that is not part of the current schema?

I appreciate any help in advance.

Thanks,
Rajath

Rajath Subramanyam


Re: SSTable Ancestors information in Cassandra 3.0.x

2017-03-25 Thread Rajath Subramanyam
Thanks Jeff.


Rajath Subramanyam


On Thu, Mar 23, 2017 at 4:50 PM, Jeff Jirsa <jji...@gmail.com> wrote:

> The ancestors were used primarily to clean up leftovers in the case that
> cassandra was killed right as compaction finished, where the
> source/origin/ancestors were still on the disk at the same time as the
> compaction result.
>
> It's not timestamp based, though - that compaction process has moved to
> using a transaction log, which tracks the source/results on a per
> compaction basis, and cassandra uses those logs/journals rather than
> inspecting the ancestors.
>
> - Jeff
>
>
>
> On Thu, Mar 23, 2017 at 4:35 PM, Rajath Subramanyam <rajat...@gmail.com>
> wrote:
>
> > Thanks, Jeff. Did all the internal tasks and the compaction tasks move
> to a
> > timestamp-based approach?
> >
> > Regards,
> > Rajath
> >
> > 
> > Rajath Subramanyam
> >
> >
> > On Thu, Mar 23, 2017 at 2:12 PM, Jeff Jirsa <jji...@gmail.com> wrote:
> >
> > > That information was removed, because it was really meant to be used
> for
> > a
> > > handful of internal tasks, most of which were no longer used. The
> > remaining
> > > use was cleaning up compaction leftovers, and the compaction leftover
> > code
> > > was rewritten in 3.0 / CASSANDRA-7066 (note, though, that it's somewhat
> > > incomplete in the upgrade case , so CASSANDRA-13313 may be interesting
> to
> > > people who are very very very very very very very sensitive to data
> > > consistency)
> > >
> > >
> > > On Thu, Mar 23, 2017 at 2:00 PM, Rajath Subramanyam <
> rajat...@gmail.com>
> > > wrote:
> > >
> > > > Hello Cassandra-Users and Cassandra-dev,
> > > >
> > > > One of the handy features in sstablemetadata that was part of
> Cassandra
> > > > 2.1.15 was that it displayed Ancestor information of an SSTable. Here
> > is
> > > a
> > > > sample output of the sstablemetadata tool with the ancestors
> > information
> > > in
> > > > C* 2.1.15:
> > > > [centos@chen-datos test1-b83746000fef11e7bdfc8bb2d6662df7]$
> > > > sstablemetadata
> > > > ks3-test1-ka-2-Statistics.db | grep "Ancestors"
> > > > Ancestors: [1]
> > > > [centos@chen-datos test1-b83746000fef11e7bdfc8bb2d6662df7]$
> > > >
> > > > However, the same tool in Cassandra 3.0.x no longer gives us that
> > > > information. Here is a sample output of the sstablemetadata grepping
> > for
> > > > Ancestors information in C* 3.0 (the output is empty since it is no
> > > longer
> > > > available):
> > > > [centos@rj-cassandra-1 elsevier1-ab7389f0fafb11e6ac23e7ccf62f494b]$
> > > > sstablemetadata mc-5-big-Statistics.db | grep "Ancestors"
> > > > [centos@rj-cassandra-1 elsevier1-ab7389f0fafb11e6ac23e7ccf62f494b]$
> > > >
> > > > My question, how can I get this information in C* 3.0.x ?
> > > >
> > > > Thank you !
> > > >
> > > > Regards,
> > > > Rajath
> > > >
> > > > 
> > > > Rajath Subramanyam
> > > >
> > >
> >
>


Re: SSTable Ancestors information in Cassandra 3.0.x

2017-03-23 Thread Rajath Subramanyam
Thanks, Jeff. Did all the internal tasks and the compaction tasks move to a
timestamp-based approach?

Regards,
Rajath


Rajath Subramanyam


On Thu, Mar 23, 2017 at 2:12 PM, Jeff Jirsa <jji...@gmail.com> wrote:

> That information was removed, because it was really meant to be used for a
> handful of internal tasks, most of which were no longer used. The remaining
> use was cleaning up compaction leftovers, and the compaction leftover code
> was rewritten in 3.0 / CASSANDRA-7066 (note, though, that it's somewhat
> incomplete in the upgrade case , so CASSANDRA-13313 may be interesting to
> people who are very very very very very very very sensitive to data
> consistency)
>
>
> On Thu, Mar 23, 2017 at 2:00 PM, Rajath Subramanyam <rajat...@gmail.com>
> wrote:
>
> > Hello Cassandra-Users and Cassandra-dev,
> >
> > One of the handy features in sstablemetadata that was part of Cassandra
> > 2.1.15 was that it displayed Ancestor information of an SSTable. Here is
> a
> > sample output of the sstablemetadata tool with the ancestors information
> in
> > C* 2.1.15:
> > [centos@chen-datos test1-b83746000fef11e7bdfc8bb2d6662df7]$
> > sstablemetadata
> > ks3-test1-ka-2-Statistics.db | grep "Ancestors"
> > Ancestors: [1]
> > [centos@chen-datos test1-b83746000fef11e7bdfc8bb2d6662df7]$
> >
> > However, the same tool in Cassandra 3.0.x no longer gives us that
> > information. Here is a sample output of the sstablemetadata grepping for
> > Ancestors information in C* 3.0 (the output is empty since it is no
> longer
> > available):
> > [centos@rj-cassandra-1 elsevier1-ab7389f0fafb11e6ac23e7ccf62f494b]$
> > sstablemetadata mc-5-big-Statistics.db | grep "Ancestors"
> > [centos@rj-cassandra-1 elsevier1-ab7389f0fafb11e6ac23e7ccf62f494b]$
> >
> > My question, how can I get this information in C* 3.0.x ?
> >
> > Thank you !
> >
> > Regards,
> > Rajath
> >
> > 
> > Rajath Subramanyam
> >
>


SSTable Ancestors information in Cassandra 3.0.x

2017-03-23 Thread Rajath Subramanyam
Hello Cassandra-Users and Cassandra-dev,

One of the handy features in sstablemetadata that was part of Cassandra
2.1.15 was that it displayed Ancestor information of an SSTable. Here is a
sample output of the sstablemetadata tool with the ancestors information in
C* 2.1.15:
[centos@chen-datos test1-b83746000fef11e7bdfc8bb2d6662df7]$ sstablemetadata
ks3-test1-ka-2-Statistics.db | grep "Ancestors"
Ancestors: [1]
[centos@chen-datos test1-b83746000fef11e7bdfc8bb2d6662df7]$

However, the same tool in Cassandra 3.0.x no longer gives us that
information. Here is a sample output of the sstablemetadata grepping for
Ancestors information in C* 3.0 (the output is empty since it is no longer
available):
[centos@rj-cassandra-1 elsevier1-ab7389f0fafb11e6ac23e7ccf62f494b]$
sstablemetadata mc-5-big-Statistics.db | grep "Ancestors"
[centos@rj-cassandra-1 elsevier1-ab7389f0fafb11e6ac23e7ccf62f494b]$

My question, how can I get this information in C* 3.0.x ?

Thank you !

Regards,
Rajath

--------
Rajath Subramanyam


Re: Single cluster node restore

2016-12-05 Thread Rajath Subramanyam
Hi Petr,

>From the question, it looks like you are trying to find a shortcut to avoid
longer restore times without having data loss. The TLDR answer to your
question is that you cannot workaround by doing node level restore since
the corrupted data will eventually get propagated. But there are some other
ways you can make the restore quicker and ensure that you don't have any
data loss.

1) Fine-grained Recovery (to save you time)
If you know the column family(ies) or at least the keyspace(s) where the
data corruption happened, you can save time by doing a fine-grained restore
of only those column(ies) or keyspace(s).

2) Point-in-time Recovery (to prevent data loss)
Assuming, you had a last correct snapshot, you can restore that version.
However, all the newer data that came in after the snapshot and before the
data corruption was introduced will still be lost. In order to avoid this,
you will need to devise a point-in-time recovery strategy.

3) Repair-free Recovery (to save you time)
By going back to last correct snapshot, you might undo the corruption, but
you will also undo all the good work done by Cassandra via repairs since
the snapshot was taken and before the corruption was introduced. For this,
you will have to devise a repair-free recovery strategy.

Hope this helps.

Thank you !

Regards,
Rajath

----
Rajath Subramanyam


On Fri, Dec 2, 2016 at 10:19 AM, Anuj Wadehra <anujw_2...@yahoo.co.in>
wrote:

> Hi Petr,
>
> If data corruption means accidental data deletions via Cassandra commands,
> you have to restore entire cluster with latest snapshots. This may lead to
> data loss as there may be valid updates after the snapshot was taken but
> before the data deletion. Restoring single node with snapshot wont help as
> Cassandra replicated the accidental deletes to all nodes.
>
> If data corruption means accidental deletion of some sstable files from
> file system of a node, repair would fix it.
>
> If data corruption means unreadable data due to hardware issues etc, you
> will have two options after replacing the disk: bootstrap or restore
> snapshot on the single affected node. If you have huge data per node e.g.
> 300Gb , you may want to restore from Snapshot followed by repair. Restoring
> snapshot on single node is faster than streaming all data via bootstrap.
> If the node is not recoverable and must be replaced, you should be able to
> do auto-boostrap or restore from snapshot with auto-bootstrap set to false.
> I havent replaced a dead node with snapshot but there should not be any
> issues as token ranges dont change when you replace a node.
>
>
>
> Thanks
> Anuj
>
> On Tue, 29 Nov, 2016 at 11:08 PM, Petr Malik
> <pma...@tesora.com> wrote:
>
>
> Hi.
>
> I have a question about Cassandra backup-restore strategies.
>
> As far as I understand Cassandra has been designed to survive hardware
> failures by relying on data replication.
>
>
> It seems like people still want backup/restore for case when somebody
> accidentally deletes data or the data gets otherwise corrupted.
>
> In that case restoring all keyspace/table snapshots on all nodes should
> bring it back.
>
>
> I am asking because I often read directions on restoring a single node in
> a cluster. I am just wondering under what circumstances could this be done
> safely.
>
>
> Please correct me if i am wrong but restoring just a single node does not
> really roll back the data as the newer (corrupt) data will be served by
> other replicas and eventually propagated to the restored node. Right?
>
> In fact by doing so one may end up reintroducing deleted data back...
>
>
> Also since Cassandra distributes the data throughout the cluster it is not
> clear on which mode any particular (corrupt) data resides and hence which
> to restore.
>
>
> I guess this is a long way of asking whether there is an advantage of
> trying to restore just a single node in a Cassandra cluster as opposed to
> say replacing the dead node and letting Cassandra handle the replication.
>
>
> Thanks.
>
>


Re: Backup restore with a different name

2016-11-03 Thread Rajath Subramanyam
Hi Jens,

Looks like what you need is an "any point in time" recovery solution. I
suggest that you go back to the snapshot that you issued that was closest
to "20161102" and restore that snapshot using the bulk loader to a new
table called "users_20161102". If you need to recover precisely to a
particular timestamp, you might have to parse every row in the SSTable and
filter out some rows.

Btw, we at Datos IO are working on exactly this solution. We have built a
data protection software for scale-out databases called RecoverX. We also
support Cassandra. One of the features that RecoverX supports is a repair
free recovery/restore that allows you to go back any point in time.

If you need more information, visit our website datos.io or drop us a note
at i...@datos.io, joe.schwa...@datos.io.

Hope this helps.

Full disclaimer: I am an engineer at Datos.io.

Regards,
Rajath

--------
Rajath Subramanyam


On Wed, Nov 2, 2016 at 3:10 PM, Jens Rantil <jens.ran...@tink.se> wrote:

> Bryan,
>
> On Wed, Nov 2, 2016 at 11:38 AM, Bryan Cheng <br...@blockcypher.com>
> wrote:
>
>> do you mean restoring the cluster to that state, or just exposing that
>> state for reference while keeping the (corrupt) current state in the live
>> cluster?
>
>
> I mean "exposing that state for reference while keeping the (corrupt)
> current state in the live cluster".
>
> Cheers,
> Jens
>
> --
> Jens Rantil
> Backend engineer
> Tink AB
>
> Email: jens.ran...@tink.se
> Phone: +46 708 84 18 32
> Web: www.tink.se
>
> Facebook <https://www.facebook.com/#!/tink.se> Linkedin
> <http://www.linkedin.com/company/2735919?trk=vsrp_companies_res_photo=VSRPsearchId%3A1057023381369207406670%2CVSRPtargetId%3A2735919%2CVSRPcmpt%3Aprimary>
>  Twitter <https://twitter.com/tink>
>


Cassandra snapshot JMX commands

2016-10-25 Thread Rajath Subramanyam
Hello Cassandra-users,

I have a question about issuing snapshot using JMX commands.


   - Issuing snapshot on a single column family:

Nodetool command:
nodetool snapshot -cf  -t ** 

The equivalent JMX command is:
run -d org.apache.cassandra.db -b
org.apache.cassandra.db:type=StorageService takeColumnFamilySnapshot <
ks_name>  


   - Issuing snapshot to multiple column families spanning keyspaces:

Nodetool command:
nodetool snapshot -kc .,.,.,. -t


Can anybody help me with JMX equivalent of the above command ?

Thanks in advance.

Regards,
Rajath
----
Rajath Subramanyam


Re: Anticompaction Question

2016-10-25 Thread Rajath Subramanyam
Hi Anubhav,

According to the Datastax documentation here
<http://www.datastax.com/dev/blog/more-efficient-repairs>, after the
anti-compaction process splits the ranges to repaired and un-repaired
SSTables, they are compacted in their own separate pools.

Regards,
Rajath



----
Rajath Subramanyam


On Tue, Oct 25, 2016 at 4:56 PM, Anubhav Kale <anubhav.k...@microsoft.com>
wrote:

> Hello,
>
>
>
> If incremental repairs are enabled, there is logic in every compaction
> strategy to make sure not to mix repaired and unrepaired SS Tables.
>
>
>
> Does this mean if some SS Table files are repaired and some aren’t and
> incremental repairs don’t work reliably, the unrepaired tables will
> *never* get compacted into repaired ones ?
>
>
>
> Thanks !
>


Set of SSTables have the same set of ancestors

2016-10-11 Thread Rajath Subramanyam
Hello Cassandra-users,

I logged into my test Cassandra cluster and saw this today. I find it very
unusual that a set of SSTables have the same set of ancestors:

$ sstablemetadata b800ks3-colla1-ka-*-Statistics.db | grep "Ancestors\|SSTable:"
SSTable: ./b800ks3-colla1-ka-1876
Ancestors: [1840, 1745, 1827, 1782, 1862, 1863, 1864, 1865, 1849, 1407]
SSTable: ./b800ks3-colla1-ka-2419
Ancestors: [2417, 2418, 2373, 2374, 2375, 2376, 2377, 2378]
SSTable: ./b800ks3-colla1-ka-2420
Ancestors: [2417, 2418, 2373, 2374, 2375, 2376, 2377, 2378]
SSTable: ./b800ks3-colla1-ka-2421
Ancestors: [2417, 2418, 2373, 2374, 2375, 2376, 2377, 2378]
SSTable: ./b800ks3-colla1-ka-2422
Ancestors: [2417, 2418, 2373, 2374, 2375, 2376, 2377, 2378]
SSTable: ./b800ks3-colla1-ka-2423
Ancestors: [2417, 2418, 2373, 2374, 2375, 2376, 2377, 2378]
SSTable: ./b800ks3-colla1-ka-2424
Ancestors: [2417, 2418, 2373, 2374, 2375, 2376, 2377, 2378]
SSTable: ./b800ks3-colla1-ka-2425
Ancestors: [2417, 2418, 2373, 2374, 2375, 2376, 2377, 2378]
SSTable: ./b800ks3-colla1-ka-2430
Ancestors: [2428, 2429]


What could have potentially caused this ? Nobody has run a sstablesplit on
any of these SSTables and I do not see any "anticompaction" words in the
log file.

Thank you !

Regards,
Rajath
--------
Rajath Subramanyam


Re: [Marketing Mail] Re: sstableloader question

2016-10-11 Thread Rajath Subramanyam
How many sstables are you trying to load ? Running sstableloaders in
parallel will help. Did you try setting the "-t" parameter and see if you
are getting the expected throughput ?

- Rajath

--------
Rajath Subramanyam


On Mon, Oct 10, 2016 at 2:02 PM, Osman YOZGATLIOGLU <
osman.yozgatlio...@krontech.com> wrote:

> Hello,
>
> Thank you Adam and Rajath.
>
> I'll split input sstables and run parallel jobs for each.
> I tested this approach and run 3 parallel sstableloader job without -t
> parameter.
> I raised stream_throughput_outbound_megabits_per_sec parameter from 200
> to 600 Mbit/sec at all of target nodes.
> But each job runs about 10MB/sec only and generates about 100Mbit'sec
> network traffic.
> At total this can be much more. Source and target servers has plenty of
> unused cpu, io and network resource.
> Do you have any idea how can I increase speed of sstableloader job?
>
> Regards,
> Osman
>
> On 10-10-2016 22:05, Rajath Subramanyam wrote:
> Hi Osman,
>
> You cannot restart the streaming only to the failed nodes specifically.
> You can restart the sstableloader job itself. Compaction will eventually
> take care of the redundant rows.
>
> - Rajath
>
> 
> Rajath Subramanyam
>
>
> On Sun, Oct 9, 2016 at 7:38 PM, Adam Hutson <a...@datascale.io<mailto:adam
> @datascale.io>> wrote:
> It'll start over from the beginning.
>
>
> On Sunday, October 9, 2016, Osman YOZGATLIOGLU <
> osman.yozgatlio...@krontech.com<mailto:osman.yozgatlio...@krontech.com>>
> wrote:
> Hello,
>
> I have running a sstableloader job.
> Unfortunately some of nodes restarted since beginnig streaming.
> I see streaming stop for those nodes.
> Can I restart those streaming somehow?
> Or if I restart sstableloader job, will it start from beginning?
>
> Regards,
> Osman
>
>
> This e-mail message, including any attachments, is for the sole use of the
> person to whom it has been sent, and may contain information that is
> confidential or legally protected. If you are not the intended recipient or
> have received this message in error, you are not authorized to copy,
> distribute, or otherwise use this message or its attachments. Please notify
> the sender immediately by return e-mail and permanently delete this message
> and any attachments. KRON makes no warranty that this e-mail is error or
> virus free.
>
>
> --
>
> Adam Hutson
> Data Architect | DataScale
> +1 (417) 224-5212<tel:%2B1%20%28417%29%20224-5212>
> a...@datascale.io<mailto:a...@datascale.io>
>
>
>
>
> This e-mail message, including any attachments, is for the sole use of the
> person to whom it has been sent, and may contain information that is
> confidential or legally protected. If you are not the intended recipient or
> have received this message in error, you are not authorized to copy,
> distribute, or otherwise use this message or its attachments. Please notify
> the sender immediately by return e-mail and permanently delete this message
> and any attachments. KRON makes no warranty that this e-mail is error or
> virus free.
>


Re: sstableloader question

2016-10-10 Thread Rajath Subramanyam
Hi Osman,

You cannot restart the streaming only to the failed nodes specifically. You
can restart the sstableloader job itself. Compaction will eventually take
care of the redundant rows.

- Rajath


Rajath Subramanyam


On Sun, Oct 9, 2016 at 7:38 PM, Adam Hutson <a...@datascale.io> wrote:

> It'll start over from the beginning.
>
>
> On Sunday, October 9, 2016, Osman YOZGATLIOGLU <
> osman.yozgatlio...@krontech.com> wrote:
>
>> Hello,
>>
>> I have running a sstableloader job.
>> Unfortunately some of nodes restarted since beginnig streaming.
>> I see streaming stop for those nodes.
>> Can I restart those streaming somehow?
>> Or if I restart sstableloader job, will it start from beginning?
>>
>> Regards,
>> Osman
>>
>>
>> This e-mail message, including any attachments, is for the sole use of
>> the person to whom it has been sent, and may contain information that is
>> confidential or legally protected. If you are not the intended recipient or
>> have received this message in error, you are not authorized to copy,
>> distribute, or otherwise use this message or its attachments. Please notify
>> the sender immediately by return e-mail and permanently delete this message
>> and any attachments. KRON makes no warranty that this e-mail is error or
>> virus free.
>>
>
>
> --
>
> Adam Hutson
> Data Architect | DataScale
> +1 (417) 224-5212
> a...@datascale.io
>


Replacing a dead node in a live Cassandra Cluster

2016-09-30 Thread Rajath Subramanyam
Hello Cassandra-users,

I was running some tests today. My end goal was to learn more about
replacing a dead node in a live Cassandra cluster with minimal disruption
to the existing cluster and figure out a better and faster way of doing the
same.

I am running a package installation of the following version of Cassandra.

[centos@rj-cassandra-1 testcf-97896450869d11e6a84c4381bf5c5035]$ nodetool
version
ReleaseVersion: 2.1.12

I setup a 4 node Cassandra in the lab. I got one non-seed node (lets say
node1) down by issuing a 'sudo service cassandra stop'. Then following
following instructions from this link
<https://docs.datastax.com/en/cassandra/2.1/cassandra/operations/opsReplaceNode.html>,
I tried to replace node1 with the JMX
option -Dcassandra.replace_address=. However, when I
do this the bootstrap fails with the following error in the log:

ERROR [main] 2016-09-30 23:54:17,104 CassandraDaemon.java:579 - Exception
encountered during startup
java.lang.RuntimeException: Unable to gossip with any seeds
at org.apache.cassandra.gms.Gossiper.doShadowRound(Gossiper.java:1337)
~[apache-cassandra-2.1.12.jar:2.1.12]
at
org.apache.cassandra.service.StorageService.prepareReplacementInfo(StorageService.java:512)
~[apache-cassandra-2.1.12.jar:2.1.12]
at
org.apache.cassandra.service.StorageService.prepareToJoin(StorageService.java:783)
~[apache-cassandra-2.1.12.jar:2.1.12]
at
org.apache.cassandra.service.StorageService.initServer(StorageService.java:721)
~[apache-cassandra-2.1.12.jar:2.1.12]
at
org.apache.cassandra.service.StorageService.initServer(StorageService.java:612)
~[apache-cassandra-2.1.12.jar:2.1.12]
at
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:387)
[apache-cassandra-2.1.12.jar:2.1.12]
at
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:562)
[apache-cassandra-2.1.12.jar:2.1.12]
at
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:651)
[apache-cassandra-2.1.12.jar:2.1.12]
WARN  [StorageServiceShutdownHook] 2016-09-30 23:54:17,109
Gossiper.java:1454 - No local state or state is in silent shutdown, not
announcing shutdown
INFO  [StorageServiceShutdownHook] 2016-09-30 23:54:17,109
MessagingService.java:734 - Waiting for messaging service to quiesce
INFO  [ACCEPT-/10.7.0.232] 2016-09-30 23:54:17,110
MessagingService.java:1018 - MessagingService has terminated the accept()
thread

How do I recover from this error message ?

----
Rajath Subramanyam


Re: Verify cassandra backup and restore in C * 2.1

2016-08-08 Thread Rajath Subramanyam
Hi Indranil,

One approach is to do a row count on the original source table and the
table that is restored from backup. How are you backing up data ? I am
assuming you are issuing snapshot commands (either incremental or
otherwise).

I hope this helps.

- Rajath


Rajath Subramanyam


On Mon, Aug 8, 2016 at 4:29 AM, INDRANIL BASU <indranil...@yahoo.com> wrote:

> Hello,
>
>How can we verify that cassandra data is backed up and restored
> correctly?
>
> *-- IB*
>
>
>
>


Re: What is the merit of incremental backup

2016-07-15 Thread Rajath Subramanyam
Hi Satoshi,

Incremental Backup if set to True, copies SSTables to the backup folder as
soon as a SSTable is flushed to disk. Hence these backed up SSTables miss
out on the opportunity to go through compaction. Does that explain the
longer time ?

- Rajath


Rajath Subramanyam


On Fri, Jul 15, 2016 at 12:20 AM, Satoshi Hikida <sahik...@gmail.com> wrote:

> Hi Prasenjit
>
> Thank you for your reply.
>
> However, I doubt that incremental backup can reduce RTO. I think the
> demerit of incremental backup is to take longer repair time rather than
> without incremental backup.
>
> Because I've compared the repair time of two cases like below.
>
> (a) snapshot(10GB, full repaired) + incremental backup(1GB)
> (b) snapshot(10GB, full repaired)
>
> Each case consists of 3 node cluster, replication factor is 3 and total
> data size is 12GB/node. And we assume one node got failure then we restore
> the node. The result showed that case (b) is faster than case (a). The
> repair process of the token ranges included in incremental backup was very
> slow. However, the just transferring replicated data from existing nodes to
> repairing node is faster than repair.
>
> So far, I think Pros and Cons of incremental back is as following:
>
> - Pros (There are already agreed by you)
> - It allows storing backups offsite without transferring entire snapshots
> - With incremental backups and snapshots, it can provide more recent RPO
> (Recovery Point Objective)
> - Cons
> - It takes much longer repair time rather than without incremental backup
> (longer RTO)
>
>
> Is it correct understand? I would appreciate you can give me any advice or
> ideas if I was misunderstanding.
>
>
> Regards,
> Satoshi
>
>
> On Fri, Jul 15, 2016 at 1:46 AM, Prasenjit Sarkar <
> prasenjit.sar...@datos.io> wrote:
>
>> Hi Satoshi
>>
>> You are correct that incremental backups offer you the opportunity to
>> reduce the amount of data you need to transfer offsite. On the recovery
>> path, you need to piece together the full backup and subsequent incremental
>> backups.
>>
>> However, where incremental backups help is with respect to the RTO due to
>> the data reduction effect you mentioned. The RPO can be reduced only if you
>> take more frequent incremental backups than full backups.
>>
>> Hope this helps,
>> Prasenjit
>>
>> On Wed, Jul 13, 2016 at 11:54 PM, Satoshi Hikida <sahik...@gmail.com>
>> wrote:
>>
>>> Hi,
>>>
>>> I want to know the actual advantage of using incremental backup.
>>>
>>> I've read through the DataStax document and it says the merit of using
>>> incremental backup is as follows:
>>>
>>> - It allows storing backups offsite without transferring entire snapshots
>>> - With incremental backups and snapshots, it can provide more recent RPO
>>> (Recovery Point Objective)
>>>
>>> Is my understanding correct? I would appreciate if someone gives me some
>>> advice or correct me.
>>>
>>> References:
>>> - DataStax, "Enabling incremental backups",
>>> http://docs.datastax.com/en/cassandra/2.2/cassandra/operations/opsBackupIncremental.html
>>>
>>> Regards,
>>> Satoshi
>>>
>>
>>
>


Cassandra SSTables' metadata and corruption

2016-06-06 Thread Rajath Subramanyam
Hello C*-users community,

I have a question about Cassandra data files corruption. In one of our test
clusters (which runs *C* 2.1*), we observed that when we run the tool
*'sstablemetada'* on one of the sstables, it returns an *invalid value for
minimum timestamp*:

[datos_db_user@sgp-src-1 pre-scrub-1464981676212]$ java -jar
sstablemetadata.jar vt1-table_1-ka-12-Data.db | grep "timestamp"
Minimum timestamp: -9223372036854775808
Maximum timestamp: 146438090618

>From the sstablemetadata tool source code, I know that it parses only the
Statistics.db component to print the metadata. This seems to indicate three
possibilities:

   1. There is a corruption in the Statistics.db component only and the
   Data in the Data.db is fine
   2. There is a corruption in the data in the Data.db which is causing the
   invalid timestamp to show up in the Statistics.db component
   3. There is a bug in the sstablemetadata tool

However, when I run sstablescrub on the sstable, the problem is solved and
I get the right metadata. This rules out 3 above. Hence it is either 1 or
2.

[datos_db_user@sgp-src-1 table_1-99d6ca40244611e697fb176f47d4f7f7]$ java
-jar sstablemetadata.jar vt1-table_1-ka-16-Data.db | grep
"timestamp\|Ancestors"
Minimum timestamp: 1464380729962999
Maximum timestamp: 146438090618
Ancestors: [12]
[datos_db_user@sgp-src-1 table_1-99d6ca40244611e697fb176f47d4f7f7]$

So my question is, for those of you who have been running Cassandra for a
long time in production, how often does the Statistics.db component get
corrupted resulting in sstablemetadata showing invalid metadata ? Does
Cassandra guarantee that the metadata shown by sstablemetadata is reliable
? If the metadata being shown by sstablemetadata is invalid, does it point
to a bigger data corruption in the Data.db component ?

Thanks in advance for your thoughts about this.

Thank you !

Regards,
Rajath

--------
Rajath Subramanyam


Autobootstrap in Cassandra

2016-05-23 Thread Rajath Subramanyam
Hi Cassandra users,

Is there a way to find if auto_bootstrap is set to false on a Cassandra
node if we didn't know the location of the cassandra.yaml or the cassandra
installation directory (for e.g., through means like JMX, etc) ?

Thank you !

Regards,
Rajath

Rajath Subramanyam


Re: Regarding cassandra-stress results

2016-03-14 Thread Rajath Subramanyam
I opened CASSANDRA-11352
<https://issues.apache.org/jira/browse/CASSANDRA-11352> to add this minor
improvement to the cassandra-stress tool where the units are part of the
output.

- Rajath

----
Rajath Subramanyam


On Mon, Mar 14, 2016 at 3:43 PM, John Wong <gokoproj...@gmail.com> wrote:

> On Mon, Mar 14, 2016 at 6:13 PM, Robert Coli <rc...@eventbrite.com> wrote:
>
>> On Mon, Mar 14, 2016 at 11:38 AM, Rajath Subramanyam <rajat...@gmail.com>
>> wrote:
>>
>>> When cassandra-stress tool dumps the output at the end of the
>>> benchmarking run, what is the unit of latency statistics ?
>>>
>>
>> This is becoming a FAQ. Perhaps the docs for the tool (and/or the tool
>> itself) should be modified to specify units.
>>
>> I have bcced docs AT datastax regarding the docs.
>>
>> =Rob
>>
>>
>
> Probably also worth adding to the actual output. I am not sure if there's
> a good reason to not include in the output.
>
> John
>
>


Regarding cassandra-stress results

2016-03-14 Thread Rajath Subramanyam
Hello Cassandra Community,

When cassandra-stress tool dumps the output at the end of the benchmarking
run, what is the unit of latency statistics ?

latency mean  : 0.7 [READ:0.7, WRITE:0.7]
latency median: 0.6 [READ:0.6, WRITE:0.6]
latency 95th percentile   : 0.8 [READ:0.8, WRITE:0.8]
latency 99th percentile   : 1.2 [READ:1.2, WRITE:1.2]
latency 99.9th percentile : 8.8 [READ:8.9, WRITE:9.0]
latency max   : 448.7 [READ:162.3, WRITE:448.7]

Thanks in advance.

- Rajath

Rajath Subramanyam


Practical use of counters in the industry

2014-12-18 Thread Rajath Subramanyam
Hi Folks,

Have any of you come across blogs that describe how companies in the
industry are using Cassandra counters practically.

Thanks in advance.

Regards,
Rajath

Rajath Subramanyam


Re: Practical use of counters in the industry

2014-12-18 Thread Rajath Subramanyam
Thanks Ken. Any other use cases where counters are used apart from Rainbird
?


Rajath Subramanyam


On Thu, Dec 18, 2014 at 5:12 PM, Ken Hancock ken.hanc...@schange.com
wrote:

 Here's one from Twitter...


 http://www.slideshare.net/kevinweil/rainbird-realtime-analytics-at-twitter-strata-2011


 On Thu, Dec 18, 2014 at 6:08 PM, Rajath Subramanyam rajat...@gmail.com
 wrote:

 Hi Folks,

 Have any of you come across blogs that describe how companies in the
 industry are using Cassandra counters practically.

 Thanks in advance.

 Regards,
 Rajath
 
 Rajath Subramanyam



 --
 *Ken Hancock *| System Architect, Advanced Advertising
 SeaChange International
 50 Nagog Park
 Acton, Massachusetts 01720
 ken.hanc...@schange.com | www.schange.com | NASDAQ:SEAC
 http://www.schange.com/en-US/Company/InvestorRelations.aspx
 Office: +1 (978) 889-3329 | [image: Google Talk:] ken.hanc...@schange.com
  | [image: Skype:]hancockks | [image: Yahoo IM:]hancockks[image: LinkedIn]
 http://www.linkedin.com/in/kenhancock

 [image: SeaChange International]
 http://www.schange.com/This e-mail and any attachments may contain
 information which is SeaChange International confidential. The information
 enclosed is intended only for the addressees herein and may not be copied
 or forwarded without permission from SeaChange International.