RE: incremental repairs with -pr flag?
Thanks. Sean From: Alexander Dejanovski [a...@thelastpickle.com] Sent: Monday, October 24, 2016 10:39 AM To: user@cassandra.apache.org Subject: Re: incremental repairs with -pr flag? Hi Sean, In order to mitigate its impact, anticompaction is not fully executed when incremental repair is run with -pr. What you'll observe is that running repair on all nodes with -pr will leave sstables marked as unrepaired on all of them. Then, if you think about it you realize it's no big deal as -pr is useless with incremental repair : data is repaired only once with incremental repair, which is what -pr intended to fix on full repair, by repairing all token ranges only once instead of times the replication factor. Cheers, Le lun. 24 oct. 2016 18:05, Sean Bridges <sean.brid...@globalrelay.net<mailto:sean.brid...@globalrelay.net>> a écrit : Hey, In the datastax documentation on repair [1], it says, "The partitioner range option is recommended for routine maintenance. Do not use it to repair a downed node. Do not use with incremental repair (default for Cassandra 3.0 and later)." Why is it not recommended to use -pr with incremental repairs? Thanks, Sean [1] https://docs.datastax.com/en/cassandra/3.x/cassandra/operations/opsRepairNodesManualRepair.html -- Sean Bridges senior systems architect Global Relay sean.brid...@globalrelay.net <mailto:sean.brid...@globalrelay.net> 866.484.6630 New York | Chicago | Vancouver | London (+44.0800.032.9829) | Singapore (+65.3158.1301) Global Relay Archive supports email, instant messaging, BlackBerry, Bloomberg, Thomson Reuters, Pivot, YellowJacket, LinkedIn, Twitter, Facebook and more. Ask about Global Relay Message<http://www.globalrelay.com/services/message> - The Future of Collaboration in the Financial Services World All email sent to or from this address will be retained by Global Relay's email archiving system. This message is intended only for the use of the individual or entity to which it is addressed, and may contain information that is privileged, confidential, and exempt from disclosure under applicable law. Global Relay will not be liable for any compliance or technical information provided herein. All trademarks are the property of their respective owners. -- - Alexander Dejanovski France @alexanderdeja Consultant Apache Cassandra Consulting http://www.thelastpickle.com<http://www.thelastpickle.com/>
incremental repairs with -pr flag?
Hey, In the datastax documentation on repair [1], it says, "The partitioner range option is recommended for routine maintenance. Do not use it to repair a downed node. Do not use with incremental repair (default for Cassandra 3.0 and later)." Why is it not recommended to use -pr with incremental repairs? Thanks, Sean [1] https://docs.datastax.com/en/cassandra/3.x/cassandra/operations/opsRepairNodesManualRepair.html -- Sean Bridges senior systems architect Global Relay _sean.bridges@globalrelay.net_ <mailto:sean.brid...@globalrelay.net> *866.484.6630 * New York | Chicago | Vancouver | London (+44.0800.032.9829) | Singapore (+65.3158.1301) Global Relay Archive supports email, instant messaging, BlackBerry, Bloomberg, Thomson Reuters, Pivot, YellowJacket, LinkedIn, Twitter, Facebook and more. Ask about *_Global Relay Message_* <http://www.globalrelay.com/services/message> - The Future of Collaboration in the Financial Services World All email sent to or from this address will be retained by Global Relay's email archiving system. This message is intended only for the use of the individual or entity to which it is addressed, and may contain information that is privileged, confidential, and exempt from disclosure under applicable law. Global Relay will not be liable for any compliance or technical information provided herein. All trademarks are the property of their respective owners.
Re: non incremental repairs with cassandra 2.2+
Thanks, we will try that. Sean On 16-10-19 09:34 AM, Alexander Dejanovski wrote: Hi Sean, you should be able to do that by running subrange repairs, which is the only type of repair that wouldn't trigger anticompaction AFAIK. Beware that now you will have sstables marked as repaired and others marked as unrepaired, which will never be compacted together. You might want to flag all sstables as unrepaired before moving on, if you do not intend to switch to incremental repair for now. Cheers, On Wed, Oct 19, 2016 at 6:31 PM Sean Bridges <sean.brid...@globalrelay.net <mailto:sean.brid...@globalrelay.net>> wrote: Hey, We are upgrading from cassandra 2.1 to cassandra 2.2. With cassandra 2.1 we would periodically repair all nodes, using the -pr flag. With cassandra 2.2, the same repair takes a very long time, as cassandra does an anti compaction after the repair. This anti compaction causes most (all?) the sstables to be rewritten. Is there a way to do full repairs without continually anti compacting? If we do a full repair on each node with the -pr flag, will subsequent full repairs also force anti compacting most (all?) sstables? Thanks, Sean -- - Alexander Dejanovski France @alexanderdeja Consultant Apache Cassandra Consulting http://www.thelastpickle.com <http://www.thelastpickle.com/> -- Sean Bridges senior systems architect Global Relay _sean.bridges@globalrelay.net_ <mailto:sean.brid...@globalrelay.net> *866.484.6630 * New York | Chicago | Vancouver | London (+44.0800.032.9829) | Singapore (+65.3158.1301) Global Relay Archive supports email, instant messaging, BlackBerry, Bloomberg, Thomson Reuters, Pivot, YellowJacket, LinkedIn, Twitter, Facebook and more. Ask about *_Global Relay Message_* <http://www.globalrelay.com/services/message> - The Future of Collaboration in the Financial Services World All email sent to or from this address will be retained by Global Relay's email archiving system. This message is intended only for the use of the individual or entity to which it is addressed, and may contain information that is privileged, confidential, and exempt from disclosure under applicable law. Global Relay will not be liable for any compliance or technical information provided herein. All trademarks are the property of their respective owners.
non incremental repairs with cassandra 2.2+
Hey, We are upgrading from cassandra 2.1 to cassandra 2.2. With cassandra 2.1 we would periodically repair all nodes, using the -pr flag. With cassandra 2.2, the same repair takes a very long time, as cassandra does an anti compaction after the repair. This anti compaction causes most (all?) the sstables to be rewritten. Is there a way to do full repairs without continually anti compacting? If we do a full repair on each node with the -pr flag, will subsequent full repairs also force anti compacting most (all?) sstables? Thanks, Sean
does DC_LOCAL require manually truncating system.paxos on failover?
We are using lightweight transactions, two datacenters and DC_LOCAL consistency level. There is a comment in CASSANDRA-5797, This would require manually truncating system.paxos when failing over. Is that required? I don't see it documented anywhere else. Thanks, Sean https://issues.apache.org/jira/browse/CASSANDRA-5797
Re: are repairs in 2.0 more expensive than in 1.2
Janne, I filed CASSANDRA-8177 [1] for this. Maybe comment on the jira that you are having the same problem. Sean [1] https://issues.apache.org/jira/browse/CASSANDRA-8177 On Thu, Oct 23, 2014 at 2:04 PM, Janne Jalkanen janne.jalka...@ecyrd.com wrote: On 23 Oct 2014, at 21:29 , Robert Coli rc...@eventbrite.com wrote: On Thu, Oct 23, 2014 at 9:33 AM, Sean Bridges sean.brid...@gmail.com wrote: The change from parallel to sequential is very dramatic. For a small cluster with 3 nodes, using cassandra 2.0.10, a parallel repair takes 2 hours, and io throughput peaks at 6 mb/s. Sequential repair takes 40 hours, with average io around 27 mb/s. Should I file a jira? As you are an actual user actually encountering the problem I had only conjectured about, you are the person best suited to file such a ticket on the reasonableness of the -par default. :D Hm? I’ve been banging my head against the exact same problem (cluster size five nodes, RF=3, ~40GB/node) - paraller repair takes about 6 hrs whereas serial takes some 48 hours or so. In addition, the compaction impact is roughly the same - that is, there’s the same number of compactions triggered per minute, but serial runs eight times more of them. There does not seem to be a difference between the node response latency during parallel or serial repair. NB: We do increase our compaction throughput during calmer times, and lower it through busy times, and the serial compaction takes enough time to hit the busy period - that might also have an impact to the overall performance. If I had known that this had so far been a theoretical problem, I would’ve spoken up earlier. Perhaps serial repair is not the best default. /Janne
Re: are repairs in 2.0 more expensive than in 1.2
We switched to to parallel repairs, and now our repairs in 2.0 are behaving like the repairs in 1.2. The change from parallel to sequential is very dramatic. For a small cluster with 3 nodes, using cassandra 2.0.10, a parallel repair takes 2 hours, and io throughput peaks at 6 mb/s. Sequential repair takes 40 hours, with average io around 27 mb/s. Should I file a jira? Sean On Wed, Oct 15, 2014 at 9:23 PM, Sean Bridges sean.brid...@gmail.com wrote: Thanks Robert. Does the switch to sequential from parallel explain why IO increases, we see significantly higher IO with 2.10. The nodetool docs [1] hint at the reason for defaulting to sequential, This allows the dynamic snitch to maintain performance for your application via the other replicas, because at least one replica in the snapshot is not undergoing repair. Sean [1] http://www.datastax.com/documentation/cassandra/2.0/cassandra/tools/toolsRepair.html On Wed, Oct 15, 2014 at 5:36 PM, Robert Coli rc...@eventbrite.com wrote: On Wed, Oct 15, 2014 at 4:54 PM, Sean Bridges sean.brid...@gmail.com wrote: We upgraded a cassandra cluster from 1.2.18 to 2.0.10, and it looks like repair is significantly more expensive now. Is this expected? It depends on what you mean by expected. Operators usually don't expect defaults with such dramatic impacts to change without them understanding why, but there is a reason for it. In 2.0 the default for repair was changed to be non-parallel. To get the old behavior, you need to supply -par as an argument. The context-free note you missed the significance of in NEWS.txt for version 2.0.2 says : - Nodetool defaults to Sequential mode for repair operations What this doesn't say is how almost certainly unreasonable this is as a default, because this means that repair is predictably slower in direct relationship to your replication factor, and the default for gc_grace_seconds (the time box in which one must complete a repair) did not change at the same time. The ticket where the change happens [1] does not specify a rationale, so your guess is as good as mine as to the reasoning which not only felt the change was necessary but reasonable. Leaving aside the problem you've encountered (upgraders notice that their repairs (which already took forever) are suddenly WAY SLOWER) this default is also quite pathological for anyone operating with a RF over 3, which are valid, if very uncommon, configurations. In summary, if, as an operator, you disagree that making repair slower by default as a factor of replication factor is reasonable, I suggest filing a JIRA and letting the project know. At least in that case there is a chance they might explain the rationale for so blithely making a change that has inevitable impact on operators... ? =Rob [1] https://issues.apache.org/jira/browse/CASSANDRA-5950 http://twitter.com/rcolidba
are all files for an sstable immutable?
Hello, I thought an sstable was immutable once written to disk. Before upgrading from 1.2.18 to 2.0.10 we took a snapshot of our sstables. Now when I compare the files in the snaphot dir and the original files, the Summary.db files have a newer modified date, and the file sizes have changed. Thanks, Sean
Re: are all files for an sstable immutable?
Thanks Tyler, that explains it. Sean On Thu, Oct 16, 2014 at 10:40 AM, Tyler Hobbs ty...@datastax.com wrote: The summary files are immutable, but can be replaced periodically. See https://issues.apache.org/jira/browse/CASSANDRA-5519 for more details. The summary files aren't particularly important, they're primarily an optimization for startup time. On Thu, Oct 16, 2014 at 12:20 PM, Sean Bridges sean.brid...@gmail.com wrote: Hello, I thought an sstable was immutable once written to disk. Before upgrading from 1.2.18 to 2.0.10 we took a snapshot of our sstables. Now when I compare the files in the snaphot dir and the original files, the Summary.db files have a newer modified date, and the file sizes have changed. Thanks, Sean -- Tyler Hobbs DataStax http://datastax.com/
are repairs in 2.0 more expensive than in 1.2
Hello, We upgraded a cassandra cluster from 1.2.18 to 2.0.10, and it looks like repair is significantly more expensive now. Is this expected? We schedule rolling repairs through the cluster. With 1.2.18 a repair would take 3 hours or so. The first repair after the upgrade has been going on for over a day now, and still hasn't finished. The repair is doing a lot more io as well. We have 24 nodes in our cluster, about 400 gigs of sstables per node, and are using vnodes. All machines are using ssds. Thanks, Sean
Re: are repairs in 2.0 more expensive than in 1.2
Thanks Robert. Does the switch to sequential from parallel explain why IO increases, we see significantly higher IO with 2.10. The nodetool docs [1] hint at the reason for defaulting to sequential, This allows the dynamic snitch to maintain performance for your application via the other replicas, because at least one replica in the snapshot is not undergoing repair. Sean [1] http://www.datastax.com/documentation/cassandra/2.0/cassandra/tools/toolsRepair.html On Wed, Oct 15, 2014 at 5:36 PM, Robert Coli rc...@eventbrite.com wrote: On Wed, Oct 15, 2014 at 4:54 PM, Sean Bridges sean.brid...@gmail.com wrote: We upgraded a cassandra cluster from 1.2.18 to 2.0.10, and it looks like repair is significantly more expensive now. Is this expected? It depends on what you mean by expected. Operators usually don't expect defaults with such dramatic impacts to change without them understanding why, but there is a reason for it. In 2.0 the default for repair was changed to be non-parallel. To get the old behavior, you need to supply -par as an argument. The context-free note you missed the significance of in NEWS.txt for version 2.0.2 says : - Nodetool defaults to Sequential mode for repair operations What this doesn't say is how almost certainly unreasonable this is as a default, because this means that repair is predictably slower in direct relationship to your replication factor, and the default for gc_grace_seconds (the time box in which one must complete a repair) did not change at the same time. The ticket where the change happens [1] does not specify a rationale, so your guess is as good as mine as to the reasoning which not only felt the change was necessary but reasonable. Leaving aside the problem you've encountered (upgraders notice that their repairs (which already took forever) are suddenly WAY SLOWER) this default is also quite pathological for anyone operating with a RF over 3, which are valid, if very uncommon, configurations. In summary, if, as an operator, you disagree that making repair slower by default as a factor of replication factor is reasonable, I suggest filing a JIRA and letting the project know. At least in that case there is a chance they might explain the rationale for so blithely making a change that has inevitable impact on operators... ? =Rob [1] https://issues.apache.org/jira/browse/CASSANDRA-5950 http://twitter.com/rcolidba
Re: Consistency model
If you are reading and writing at quorum, then what you are seeing shouldn't happen. You shouldn't be able to read N+1 until N+1 has been committed to a quorum of servers. At this point you should not be able to read N anymore, since there is no quorum that contains N. Dan - I think you are right, except that quorum reads should be consistent even during a quorum write. You are not guaranteed to read N+1 until *after* a successful quorum write of N+1, but once you see N+1, you should never see N again, even if the write failed. Sean On Fri, Apr 15, 2011 at 1:29 PM, Dan Hendry dan.hendry.j...@gmail.com wrote: So Cassandra does not use an atomic commit protocol at the cluster level. Strong consistency on a quorum read is only guaranteed *after* a successful quorum write. The behaviour you are seeing is possible if you are reading in the middle of a write or the write failed (which should be reported to your code via an exception). Dan -Original Message- From: James Cipar [mailto:jci...@cmu.edu] Sent: April-15-11 14:15 To: user@cassandra.apache.org Subject: Consistency model I've been experimenting with the consistency model of Cassandra, and I found something that seems a bit unexpected. In my experiment, I have 2 processes, a reader and a writer, each accessing a Cassandra cluster with a replication factor greater than 1. In addition, sometimes I generate background traffic to simulate a busy cluster by uploading a large data file to another table. The writer executes a loop where it writes a single row that contains just an sequentially increasing sequence number and a timestamp. In python this looks something like: while time.time() start_time + duration: target_server = random.sample(servers, 1)[0] target_server = '%s:9160'%target_server row = {'seqnum':str(seqnum), 'timestamp':str(time.time())} seqnum += 1 # print 'uploading to server %s, %s'%(target_server, row) pool = pycassa.connect('Keyspace1', [target_server]) cf = pycassa.ColumnFamily(pool, 'Standard1') cf.insert('foo', row, write_consistency_level=consistency_level) pool.dispose() if sleeptime 0.0: time.sleep(sleeptime) The reader simply executes a loop reading this row and reporting whenever a sequence number is *less* than the previous sequence number. As expected, with consistency_level=ConsistencyLevel.ONE there are many inconsistencies, especially with a high replication factor. What is unexpected is that I still detect inconsistencies when it is set at ConsistencyLevel.QUORUM. This is unexpected because the documentation seems to imply that QUORUM will give consistent results. With background traffic the average difference in timestamps was 0.6s, and the maximum was 3.5s. This means that a client sees a version of the row, and can subsequently see another version of the row that is 3.5s older than the previous. What I imagine is happening is this, but I'd like someone who knows that they're talking about to tell me if it's actually the case: I think Cassandra is not using an atomic commit protocol to commit to the quorum of servers chosen when the write is made. This means that at some point in the middle of the write, some subset of the quorum have seen the write, while others have not. At this time, there is a quorum of servers that have not seen the update, so depending on which quorum the client reads from, it may or may not see the update. Of course, I understand that the client is not *choosing* a bad quorum to read from, it is just the first `q` servers to respond, but in this case it is effectively random and sometimes an bad quorum is chosen. Does anyone have any other insight into what is going on here?= No virus found in this incoming message. Checked by AVG - www.avg.com Version: 9.0.894 / Virus Database: 271.1.1/3574 - Release Date: 04/15/11 02:34:00
Re: Consistency model
Tyler, your answer seems to contradict this email by Jonathan Ellis [1]. In it Jonathan says, The important guarantee this gives you is that once one quorum read sees the new value, all others will too. You can't see the newest version, then see an older version on a subsequent write [sic, I assume he meant read], which is the characteristic of non-strong consistency Jonathan also says, {X, Y} and {X, Z} are equivalent: one node with the write, and one without. The read will recognize that X's version needs to be sent to Z, and the write will be complete. This read and all subsequent ones will see the write. (Z [sic, I assume he meant Y] will be replicated to asynchronously via read repair.) To me, the statement this read and all subsequent ones will see the write implies that the new value must be committed to Y or Z before the read can return. If not, the statement must be false. Sean [1] : http://mail-archives.apache.org/mod_mbox/cassandra-user/201102.mbox/%3caanlktimegp8h87mgs_bxzknck-a59whxf-xx58hca...@mail.gmail.com%3E Sean On Sat, Apr 16, 2011 at 7:44 PM, Tyler Hobbs ty...@datastax.com wrote: Here's what's probably happening: I'm assuming RF=3 and QUORUM writes/reads here. I'll call the replicas A, B, and C. 1. Writer process writes sequence number 1 and everything works fine. A, B, and C all have sequence number 1. 2. Writer process writes sequence number 2. Replica A writes successfully, B and C fail to respond in time, and a TimedOutException is returned. pycassa waits to retry the operation. 3. Reader process reads, gets a response from A and B. When the row from A and B is merged, sequence number 2 is the newest and is returned. A read repair is pushed to B and C, but they don't yet update their data. 4. Reader process reads again, gets a response from B and C (before they've repaired). These both report sequence number 1, so that's returned to the client. This is were you get a decreasing sequence number. 5. pycassa eventually retries the write; B and C eventually repair their data. Either way, both B and C shortly have sequence number 2. I've left out some of the details of read repair, and this scenario could happen in several slightly different ways, but it should give you an idea of what's happening. On Sat, Apr 16, 2011 at 8:35 PM, James Cipar jci...@cmu.edu wrote: Here it is. There is some setup code and global variable definitions that I left out of the previous code, but they are pretty similar to the setup code here. import pycassa import random import time consistency_level = pycassa.cassandra.ttypes.ConsistencyLevel.QUORUM duration = 600 sleeptime = 0.0 hostlist = 'worker-hostlist' def read_servers(fn): f = open(fn) servers = [] for line in f: servers.append(line.strip()) f.close() return servers servers = read_servers(hostlist) start_time = time.time() seqnum = -1 timestamp = 0 while time.time() start_time + duration: target_server = random.sample(servers, 1)[0] target_server = '%s:9160'%target_server try: pool = pycassa.connect('Keyspace1', [target_server]) cf = pycassa.ColumnFamily(pool, 'Standard1') row = cf.get('foo', read_consistency_level=consistency_level) pool.dispose() except: time.sleep(sleeptime) continue sq = int(row['seqnum']) ts = float(row['timestamp']) if sq seqnum: print 'Row changed: %i %f - %i %f'%(seqnum, timestamp, sq, ts) seqnum = sq timestamp = ts if sleeptime 0.0: time.sleep(sleeptime) On Apr 16, 2011, at 5:20 PM, Tyler Hobbs wrote: James, Would you mind sharing your reader process code as well? On Fri, Apr 15, 2011 at 1:14 PM, James Cipar jci...@cmu.edu wrote: I've been experimenting with the consistency model of Cassandra, and I found something that seems a bit unexpected. In my experiment, I have 2 processes, a reader and a writer, each accessing a Cassandra cluster with a replication factor greater than 1. In addition, sometimes I generate background traffic to simulate a busy cluster by uploading a large data file to another table. The writer executes a loop where it writes a single row that contains just an sequentially increasing sequence number and a timestamp. In python this looks something like: while time.time() start_time + duration: target_server = random.sample(servers, 1)[0] target_server = '%s:9160'%target_server row = {'seqnum':str(seqnum), 'timestamp':str(time.time())} seqnum += 1 # print 'uploading to server %s, %s'%(target_server, row) pool = pycassa.connect('Keyspace1', [target_server]) cf = pycassa.ColumnFamily(pool, 'Standard1') cf.insert('foo', row,
Re: 10 minute cassandra pause
I see about 3000 lines of, INFO [COMMIT-LOG-WRITER] 2010-06-23 16:40:29,107 CommitLog.java (line 412) Discarding obsolete commit log:CommitLogSegment(/data1/cass/commitlog/CommitLog-1277302220723.log) Then, http://pastebin.com/YQA0mpRG It's around 16:50 that cassandra writes stop timing out. Some writes are getting through during this 10 minutes, but they shouldn't be enough to cause the index memtables to flush. Thanks, Sean On Wed, Jun 23, 2010 at 3:30 PM, Benjamin Black b...@b3k.us wrote: Are you seeing any sort of log messages from Cassandra at all? On Wed, Jun 23, 2010 at 2:26 PM, Sean Bridges sean.brid...@gmail.com wrote: We were running a load test against a single 0.6.2 cassandra node. 24 hours into the test, Cassandra appeared to be nearly frozen for 10 minutes. Our write rate went to almost 0, and we had a large number of write timeouts. We weren't swapping or gc'ing at the time. It looks like the problems were caused by our memtables flushing after 24 hours (we have MemtableFlushAfterMinutes=1440). Some of our column families are written to infrequently so that they don't hit the flush thresholds in MemtableOperationsInMillions and MemtableThroughputInMB. After 24 hours we had ~3000 commit log files. Is this flushing causing Cassandra to become unresponsive? I would have thought Cassandra could flush in the background without blocking new writes. Thanks, Sean
using more than 50% of disk space
We're investigating Cassandra, and we are looking for a way to get Cassandra use more than 50% of it's data disks. Is this possible? For major compactions, it looks like we can use more than 50% of the disk if we use multiple similarly sized column families. If we had 10 column families of the same size, we could use 90% of the disk, since a major compaction would only need as much free space as the largest column family (in reality we would use less). Is that right? For bootstrapping new nodes, it looks like adding a new node will require that an existing node does anti-compaction. This anti-compaction could take nearly 50% of the disk. Is there a way around this? Is there anything else that would prevent us from using more than 50% of the data disk. Thanks, Sean
Re: using more than 50% of disk space
So after CASSANDRA-579, anti compaction won't be done on the source node, and we can use more than 50% of the disk space if we use multiple column families? Thanks, Sean On Wed, May 26, 2010 at 10:01 AM, Stu Hood stu.h...@rackspace.com wrote: See https://issues.apache.org/jira/browse/CASSANDRA-579 for some background here: I was just about to start working on this one, but it won't make it in until 0.7. -Original Message- From: Sean Bridges sean.brid...@gmail.com Sent: Wednesday, May 26, 2010 11:50am To: user@cassandra.apache.org Subject: using more than 50% of disk space We're investigating Cassandra, and we are looking for a way to get Cassandra use more than 50% of it's data disks. Is this possible? For major compactions, it looks like we can use more than 50% of the disk if we use multiple similarly sized column families. If we had 10 column families of the same size, we could use 90% of the disk, since a major compaction would only need as much free space as the largest column family (in reality we would use less). Is that right? For bootstrapping new nodes, it looks like adding a new node will require that an existing node does anti-compaction. This anti-compaction could take nearly 50% of the disk. Is there a way around this? Is there anything else that would prevent us from using more than 50% of the data disk. Thanks, Sean