Hmm - we read/write with Local Quorum always - I'd recommend that as that is your 'consistency' defense.
We use python, so I am not familiar with the java driver - but 'file not found' indicates something is inconsistent. On Tue, Apr 21, 2015 at 12:22 PM, Walsh, Stephen <stephen.wa...@aspect.com> wrote: > Thanks for all your help Michael, > > > > Our data will change through the day, so data with a TTL will eventually > get dropped, and new data will appear. > > I’d imagine the entire table maybe expire and start over 7-10 times a day. > > > > > > > > But on the GC topic, now java Driver now gives this error on the query > > I also get “Request did not complete within rpc_timeout.” In cqlsh. > > > > ################################# > > com.datastax.driver.core.exceptions.ReadTimeoutException: Cassandra > timeout during read query at consistency ONE (1 responses were required but > only 0 replica responded) > > at > com.datastax.driver.core.exceptions.ReadTimeoutException.copy(ReadTimeoutException.java:69) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.driver.core.Responses$Error.asException(Responses.java:100) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.driver.core.DefaultResultSetFuture.onSet(DefaultResultSetFuture.java:140) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.driver.core.RequestHandler.setFinalResult(RequestHandler.java:249) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.driver.core.RequestHandler.onSet(RequestHandler.java:433) > ~[cassandra-driver-core-2.1.4.jar:na] > > Caused by: com.datastax.driver.core.exceptions.ReadTimeoutException: > Cassandra timeout during read query at consistency ONE (1 responses were > required but only 0 replica responded) > > at > com.datastax.driver.core.Responses$Error$1.decode(Responses.java:61) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.driver.core.Responses$Error$1.decode(Responses.java:38) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.driver.core.Message$ProtocolDecoder.decode(Message.java:168) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.shaded.netty.handler.codec.oneone.OneToOneDecoder.handleUpstream(OneToOneDecoder.java:66) > ~[cassandra-driver-core-2.1.4.jar:na] > > at > com.datastax.shaded.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:564) > ~[cassandra-driver-core-2.1.4.jar:na] > > ################################# > > > > > > These queries where taking about 1 second to run when the gc was at 10 > seconds (same duration as the TTL). > > > > Also seeing a lot of this this stuff in the log file > > > > ################################# > > ERROR [ReadStage:71] 2015-04-21 17:11:07,597 CassandraDaemon.java (line > 199) Exception in thread Thread[ReadStage:71,5,main] > > java.lang.RuntimeException: java.lang.RuntimeException: > java.io.FileNotFoundException: > /var/lib/cassandra/data/keyspace/table/keyspace-table-jb-5-Data.db (No such > file or directory) > > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2008) > > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > > at java.lang.Thread.run(Thread.java:745) > > Caused by: java.lang.RuntimeException: java.io.FileNotFoundException: > /var/lib/cassandra/data/keyspace/table/keyspace-table-jb-5-Data.db > > ################################ > > > > > > Maybe this is a 1 step back 2 steps forward approach? > > Any ideas? > > > > > > > > > > *From:* Laing, Michael [mailto:michael.la...@nytimes.com] > *Sent:* 21 April 2015 17:09 > > *To:* user@cassandra.apache.org > *Subject:* Re: Cassandra tombstones being created by updating rows with > TTL's > > > > Discussions previously on the list show why this is not a problem in much > more detail. > > > > If something changes in your cluster: node down, new node, etc - you run > repair for sure. > > > > We also run periodic repairs prophylactically. > > > > But if you never delete and always ttl by the same amount, you do not have > to worry about zombie data being resurrected - the main reason for running > repair within gc_grace_seconds. > > > > > > > > On Tue, Apr 21, 2015 at 11:49 AM, Walsh, Stephen <stephen.wa...@aspect.com> > wrote: > > Maybe thanks Michael, > > I will give these setting a go, > > How do you do you periodic node-tool repairs in the situation, for what I > read we need to start doing this also. > > > > https://wiki.apache.org/cassandra/Operations#Frequency_of_nodetool_repair > > > > > > *From:* Laing, Michael [mailto:michael.la...@nytimes.com] > *Sent:* 21 April 2015 16:26 > *To:* user@cassandra.apache.org > *Subject:* Re: Cassandra tombstones being created by updating rows with > TTL's > > > > If you never delete except by ttl, and always write with the same ttl (or > monotonically increasing), you can set gc_grace_seconds to 0. > > > > That's what we do. There have been discussions on the list over the last > few years re this topic. > > > > ml > > > > On Tue, Apr 21, 2015 at 11:14 AM, Walsh, Stephen <stephen.wa...@aspect.com> > wrote: > > We were chatting to Jon Haddena about a week ago about our tombstone > issue using Cassandra 2.0.14 > > To Summarize > > > > We have a 3 node cluster with replication-factor=3 and compaction = > SizeTiered > > We use 1 keyspace with 1 table > > Each row have about 40 columns > > Each row has a TTL of 10 seconds > > > > We insert about 500 rows per second in a prepared batch** (about 3mb in > network overhead) > > We query the entire table once per second > > > > **This is too enable consistent data, E.G batch in transactional, so we > get all queried data from one insert and not a mix of 2 or more. > > > > > > Seems every second we insert, the rows are never deleted by the TTL, or so > we thought. > > After some time we got this message on the query side > > > > > > ####################################### > > ERROR [ReadStage:91] 2015-04-21 12:27:03,902 SliceQueryFilter.java (line > 206) Scanned over 100000 tombstones in keyspace.table; query aborted (see > tombstone_failure_threshold) > > ERROR [ReadStage:91] 2015-04-21 12:27:03,931 CassandraDaemon.java (line > 199) Exception in thread Thread[ReadStage:91,5,main] > > java.lang.RuntimeException: > org.apache.cassandra.db.filter.TombstoneOverwhelmingException > > at > org.apache.cassandra.service.StorageProxy$DroppableRunnable.run(StorageProxy.java:2008) > > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) > > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) > > at java.lang.Thread.run(Thread.java:745) > > Caused by: org.apache.cassandra.db.filter.TombstoneOverwhelmingException > > ####################################### > > > > > > So we know tombstones are infact being created. > > Solution was to change the table schema and set gc_grace_seconds to run > every 60 seconds. > > This worked for 20 seconds, then we saw this > > > > > > ####################################### > > Read 500 live and 30000 tombstoned cells in keyspace.table (see > tombstone_warn_threshold). 10000 columns was requested, slices=[-], > delInfo={deletedAt=-9223372036854775808, localDeletion=2147483647} > > ####################################### > > > > So every 20 seconds (500 inserts x 20 seconds = 10,000 tombstones) > > So now we have the gc_grace_seconds set to 10 seoncds. > > But its feels very wrong to have it at a low number, especially if we move > to a larger cluster. This just wont fly. > > What are we doing wrong? > > > > We shouldn’t increase the tombstone threshold as that is extremely > dangerous. > > > > > > Best Regards > > Stephen Walsh > > > > > > > > > > > > > > This email (including any attachments) is proprietary to Aspect Software, > Inc. and may contain information that is confidential. If you have received > this message in error, please do not read, copy or forward this message. > Please notify the sender immediately, delete it from your system and > destroy any copies. You may not further disclose or distribute this email > or its attachments. > > > > This email (including any attachments) is proprietary to Aspect Software, > Inc. and may contain information that is confidential. If you have received > this message in error, please do not read, copy or forward this message. > Please notify the sender immediately, delete it from your system and > destroy any copies. You may not further disclose or distribute this email > or its attachments. > > > This email (including any attachments) is proprietary to Aspect > Software, Inc. and may contain information that is confidential. If you > have received this message in error, please do not read, copy or forward > this message. Please notify the sender immediately, delete it from your > system and destroy any copies. You may not further disclose or distribute > this email or its attachments. >