Re: Low compactionthroughput blocks reads?
rformance of > the whole cluster. > > We have multiple C* 2.2 (2.2.5) clusters, basically running the same > application but with different amount of load. This symptom only appears on > one of our clusters, which has significantly more usage than most others. > This is also our largest cluster, having about 3 times as many machines as > most other ones (and RF 5 instead of RF 3). > > We did not see this before 2.0 and also only started to see this on that > particular cluster. We didn't see this on any other cluster after upgrading > form 2.0 (but then again, they're not as loaded). > > At first we suspected the incremental repair, because we did have issues > with that as well with heap pressure with 4GB of heap. Went to 6GB and no > more heap pressure but the problem persists. The suspect does not change as > this bad behaviour coincides with repair and specifically with > anticompaction going on. > > We see a clear timely correlation between open file handles rising and at > the same time active validations skyrocketing and the beginning of such an > "event". Same goes for the end of that event, which is clearly timely > correlated to the validations being completed and the number of file > handles dropping dramatically. However, these "beginning markers" seem to > be both present. Active validations skyrocketing without open file handlers > skyrocketing at the same time does not produce these symptoms. > > The only thing we have found so far that made a difference in these > situations is compaction throughput. When we decreased the compaction > throughput in these events, pending reads piled up even more and even more > quickly. Beyond that we're still pretty much in the dark. Anyway something > is locking up Cassandra internally. > > We suspect that there may be a "synchronized" somewhere it shouldn't be > (or should be solved differently) but that's just a guess. We'll try to > produce some jstacks but the events are pretty elusive because they happen > suddenly and don't last very long (except when we're not watching closely > -_-). > > Anyway, here's some graphs to illustrate what I've tried to describe: > > (1) CPU Usage of Cassandra (green) and open file descriptors (yellow, > second y-axis) > > > (2) Threads: new threads per second (orange line, second y-axis) > > > (3) See below; active and pending tasks on second y-axis > > > (4) Compactions and Validations: Active Tasks (Bars, second y-axis) and > Completed Tasks / s (lines) > > > You can see around 08:03 an event started with a sudden rise in active > validations and multiple sudden increases in open file descriptors. The > event lasts until 08:46 with a sudden drop in open file descriptors and a > huge peak in new threads per second. > > During the event you can see Cassandra's CPU usage drops significantly. > Same goes for GC activity (graph not included here, because STW GC only > happens about once every 50 minutes and then takes only a fraction of a > second). > > As you can see there's another such event later on but much smaller and > shorter and between the events the pattern with the validations continues > the same way without problems - only difference: No significant change in > open file descriptor count. > > I have system graphs as well but not included because they show no > problems: CPU usage goes down during that event, no I/O wait on the CPU and > disk OP/s as well as throughput actually go down as well. > > During the depicted time frame there was a repair (incremental, parallel, > partitioner range) running on a different machine within the cluster. We've > switched back to -pr because when running it without -pr these event happen > more often and more articulated but I think that it's just affected by the > same underlying problem. > > Interestingly we had a similar issue in another cluster last night, which > runs C* 2.1.13 and does NOT yet use incremental repair (just full repair > with -pr). > > Any chance something in the read path is affected by the set compaction > throughput and/or running compactions? It definitely seems that Cassandra > is severly restricting itself here. > > Best regards, > Dominik > > > Am 26.02.2016 um 17:42 schrieb horschi: > > Hi, > > I just had a weird behaviour on one of our Cassandra nodes, which I would > like to share: > > Short version: > My pending reads went up from ~0 to the hundreds when I reduced the > compactionthroughput from 16 to 2. > > > Long version: > > One of our more powerful nodes had a few pending reads, while the other > ones didn't. So far nothing
Re: Low compactionthroughput blocks reads?
Hi, I just found another one. Its basically the same, but I'll post it anyway: Thread 84311: (state = BLOCKED) - java.lang.Thread.sleep(long) @bci=0 (Compiled frame; information may be imprecise) - java.lang.Thread.sleep(long, int) @bci=57, line=340 (Compiled frame) - java.util.concurrent.TimeUnit.sleep(long) @bci=23, line=386 (Compiled frame) - com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly(long, java.util.concurrent.TimeUnit) @bci=22, line=273 (Compiled frame) - com.google.common.util.concurrent.RateLimiter$SleepingTicker$1.sleepMicrosUninterruptibly(long) @bci=10, line=701 (Compiled frame) - com.google.common.util.concurrent.RateLimiter.acquire(int) @bci=42, line=405 (Compiled frame) - org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer() @bci=11, line=43 (Compiled frame) - org.apache.cassandra.io.util.RandomAccessReader.seek(long) @bci=147, line=287 (Compiled frame) - org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(long) @bci=22, line=65 (Compiled frame) - org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(long) @bci=5, line=1751 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.SimpleSliceReader.(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.RowIndexEntry, org.apache.cassandra.io.util.FileDataInput, org.apache.cassandra.db.composites.Composite) @bci=36, line=57 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.SSTableSliceIterator.createReader(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.RowIndexEntry, org.apache.cassandra.io.util.FileDataInput, org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=38, line=66 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.SSTableSliceIterator.(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.DecoratedKey, org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=36, line=43 (Compiled frame) - org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(org.apache.cassandra.db.DecoratedKey, org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=8, line=75 (Compiled frame) - org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(org.apache.cassandra.io.sstable.format.SSTableReader, org.apache.cassandra.db.DecoratedKey) @bci=10, line=246 (Compiled frame) - org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(org.apache.cassandra.io.sstable.format.SSTableReader) @bci=9, line=62 (Compiled frame) - org.apache.cassandra.db.CollationController.collectAllData(boolean) @bci=350, line=270 (Compiled frame) - org.apache.cassandra.db.CollationController.getTopLevelColumns(boolean) @bci=39, line=64 (Compiled frame) - org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(org.apache.cassandra.db.filter.QueryFilter, int) @bci=40, line=2011 (Compiled frame) - org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(org.apache.cassandra.db.filter.QueryFilter) @bci=141, line=1815 (Compiled frame) - org.apache.cassandra.db.Keyspace.getRow(org.apache.cassandra.db.filter.QueryFilter) @bci=11, line=360 (Compiled frame) - org.apache.cassandra.db.SliceFromReadCommand.getRow(org.apache.cassandra.db.Keyspace) @bci=222, line=85 (Compiled frame) - org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow() @bci=16, line=1587 (Compiled frame) - org.apache.cassandra.service.StorageProxy$DroppableRunnable.run() @bci=37, line=2232 (Compiled frame) - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 (Compiled frame) - org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run() @bci=5, line=164 (Compiled frame) - org.apache.cassandra.concurrent.SEPWorker.run() @bci=87, line=105 (Compiled frame) - java.lang.Thread.run() @bci=11, line=745 (Compiled frame) kind regards, Christian On Thu, Mar 3, 2016 at 2:31 PM, horschi wrote: > Hi, > > we just had such an incident again, and this time we were able to grab a > jstack-threaddump. > > One thread from this stackdump actually shows a ReadVerbHandler calling > the RateLimiter. From my understanding this should not be: > > Thread 30346: (state = BLOCKED) > - java.lang.Thread.sleep(long) @bci=0 (Compiled frame; information may be > imprecise) > - java.lang.Thread.sleep(long, int) @bci=57, line=340 (Compiled frame) > - java.util.concurrent.TimeUnit.sleep(long) @bci=23, line=386 (Compiled > frame) > - > com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly(long, > java.util.concurrent.TimeUnit) @bci=22, line=273 (Compiled frame) > - > com.google.common.util.concurrent.RateLimiter$SleepingTicker$1.sleepMicrosUninterruptibly(long) > @bci=10, line=701 (Compiled frame) > - com.google.common.util.concurrent.RateLimiter.acquire(int) @bci=42, > line=405 (Compiled frame) > - org.apache.cassandra.io.compre
Re: Low compactionthroughput blocks reads?
Thanks. That's good to know. On Thu, Mar 3, 2016 at 8:48 PM, Benedict Elliott Smith wrote: > Yep, definitely a bug. Introduced by CASSANDRA-9240 (me; mea culpa). > > I've filed a JIRA for you: CASSANDRA-11301 > > On 3 March 2016 at 14:10, horschi wrote: > > > Hi, > > > > I just found another one. Its basically the same, but I'll post it > anyway: > > > > Thread 84311: (state = BLOCKED) > > - java.lang.Thread.sleep(long) @bci=0 (Compiled frame; information may > be > > imprecise) > > - java.lang.Thread.sleep(long, int) @bci=57, line=340 (Compiled frame) > > - java.util.concurrent.TimeUnit.sleep(long) @bci=23, line=386 (Compiled > > frame) > > - > > > > > com.google.common.util.concurrent.Uninterruptibles.sleepUninterruptibly(long, > > java.util.concurrent.TimeUnit) @bci=22, line=273 (Compiled frame) > > - > > > > > com.google.common.util.concurrent.RateLimiter$SleepingTicker$1.sleepMicrosUninterruptibly(long) > > @bci=10, line=701 (Compiled frame) > > - com.google.common.util.concurrent.RateLimiter.acquire(int) @bci=42, > > line=405 (Compiled frame) > > - org.apache.cassandra.io.compress.CompressedThrottledReader.reBuffer() > > @bci=11, line=43 (Compiled frame) > > - org.apache.cassandra.io.util.RandomAccessReader.seek(long) @bci=147, > > line=287 (Compiled frame) > > - org.apache.cassandra.io.util.PoolingSegmentedFile.getSegment(long) > > @bci=22, line=65 (Compiled frame) > > - > > > org.apache.cassandra.io.sstable.format.SSTableReader.getFileDataInput(long) > > @bci=5, line=1751 (Compiled frame) > > - > > > > > org.apache.cassandra.io.sstable.format.big.SimpleSliceReader.(org.apache.cassandra.io.sstable.format.SSTableReader, > > org.apache.cassandra.db.RowIndexEntry, > > org.apache.cassandra.io.util.FileDataInput, > > org.apache.cassandra.db.composites.Composite) @bci=36, line=57 (Compiled > > frame) > > - > > > > > org.apache.cassandra.io.sstable.format.big.SSTableSliceIterator.createReader(org.apache.cassandra.io.sstable.format.SSTableReader, > > org.apache.cassandra.db.RowIndexEntry, > > org.apache.cassandra.io.util.FileDataInput, > > org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=38, line=66 > > (Compiled frame) > > - > > > > > org.apache.cassandra.io.sstable.format.big.SSTableSliceIterator.(org.apache.cassandra.io.sstable.format.SSTableReader, > > org.apache.cassandra.db.DecoratedKey, > > org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=36, line=43 > > (Compiled frame) > > - > > > > > org.apache.cassandra.io.sstable.format.big.BigTableReader.iterator(org.apache.cassandra.db.DecoratedKey, > > org.apache.cassandra.db.filter.ColumnSlice[], boolean) @bci=8, line=75 > > (Compiled frame) > > - > > > > > org.apache.cassandra.db.filter.SliceQueryFilter.getSSTableColumnIterator(org.apache.cassandra.io.sstable.format.SSTableReader, > > org.apache.cassandra.db.DecoratedKey) @bci=10, line=246 (Compiled frame) > > - > > > > > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(org.apache.cassandra.io.sstable.format.SSTableReader) > > @bci=9, line=62 (Compiled frame) > > - org.apache.cassandra.db.CollationController.collectAllData(boolean) > > @bci=350, line=270 (Compiled frame) > > - > org.apache.cassandra.db.CollationController.getTopLevelColumns(boolean) > > @bci=39, line=64 (Compiled frame) > > - > > > > > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(org.apache.cassandra.db.filter.QueryFilter, > > int) @bci=40, line=2011 (Compiled frame) > > - > > > > > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(org.apache.cassandra.db.filter.QueryFilter) > > @bci=141, line=1815 (Compiled frame) > > - > > > > > org.apache.cassandra.db.Keyspace.getRow(org.apache.cassandra.db.filter.QueryFilter) > > @bci=11, line=360 (Compiled frame) > > - > > > > > org.apache.cassandra.db.SliceFromReadCommand.getRow(org.apache.cassandra.db.Keyspace) > > @bci=222, line=85 (Compiled frame) > > - > > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow() > > @bci=16, line=1587 (Compiled frame) > > - org.apache.cassandra.service.StorageProxy$DroppableRunnable.run() > > @bci=37, line=2232 (Compiled frame) > > - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 > > (Compiled frame) > > - > > > > > org.apache.cassandra.concurrent.AbstractLocalAwareExecutorService$FutureTask.run() > &g
Re: URGENT: CASSANDRA-14092 causes Data Loss
The assertion was working fine until yesterday 03:14 UTC. The long term solution would be to work with a long instead of a int. The serialized seems to be a variable-int already, so that should be fine already. If you change the assertion to 15 years, then applications might fail, as they might be setting a 15+ year ttl. regards, Christian On Thu, Jan 25, 2018 at 9:19 PM, Paulo Motta wrote: > Thanks for raising this. Agreed this is bad, when I filed > CASSANDRA-14092 I thought a write would fail when localDeletionTime > overflows (as it is with 2.1), but that doesn't seem to be the case on > 3.0+ > > I propose adding the assertion back so writes will fail, and reduce > the max TTL to something like 15 years for the time being while we > figure a long term solution. > > 2018-01-25 18:05 GMT-02:00 Jeremiah D Jordan : > > If you aren’t getting an error, then I agree, that is very bad. Looking > at the 3.0 code it looks like the assertion checking for overflow was > dropped somewhere along the way, I had only been looking into 2.1 where you > get an assertion error that fails the query. > > > > -Jeremiah > > > >> On Jan 25, 2018, at 2:21 PM, Anuj Wadehra > wrote: > >> > >> > >> Hi Jeremiah, > >> Validation is on TTL value not on (system_time+ TTL). You can test it > with below example. Insert is successful, overflow happens silently and > data is lost: > >> create table test(name text primary key,age int); > >> insert into test(name,age) values('test_20yrs',30) USING TTL 63072; > >> select * from test where name='test_20yrs'; > >> > >> name | age > >> --+- > >> > >> (0 rows) > >> > >> insert into test(name,age) values('test_20yr_plus_1',30) USING TTL > 630720001;InvalidRequest: Error from server: code=2200 [Invalid query] > message="ttl is too large. requested (630720001) maximum (63072)" > >> ThanksAnuj > >>On Friday 26 January 2018, 12:11:03 AM IST, J. D. Jordan < > jeremiah.jor...@gmail.com> wrote: > >> > >> Where is the dataloss? Does the INSERT operation return successfully > to the client in this case? From reading the linked issues it sounds like > you get an error client side. > >> > >> -Jeremiah > >> > >>> On Jan 25, 2018, at 1:24 PM, Anuj Wadehra > wrote: > >>> > >>> Hi, > >>> > >>> For all those people who use MAX TTL=20 years for inserting/updating > data in production, https://issues.apache.org/jira/browse/CASSANDRA-14092 > can silently cause irrecoverable Data Loss. This seems like a certain TOP > MOST BLOCKER to me. I think the category of the JIRA must be raised to > BLOCKER from Major. Unfortunately, the JIRA is still "Unassigned" and no > one seems to be actively working on it. Just like any other critical > vulnerability, this vulnerability demands immediate attention from some > very experienced folks to bring out an Urgent Fast Track Patch for all > currently Supported Cassandra versions 2.1,2.2 and 3.x. As per my > understanding of the JIRA comments, the changes may not be that trivial for > older releases. So, community support on the patch is very much appreciated. > >>> > >>> Thanks > >>> Anuj > >> > >> - > >> To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org > >> For additional commands, e-mail: dev-h...@cassandra.apache.org > > > > > > - > > To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org > > For additional commands, e-mail: dev-h...@cassandra.apache.org > > > > - > To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org > For additional commands, e-mail: dev-h...@cassandra.apache.org > >
Re: URGENT: CASSANDRA-14092 causes Data Loss
Paulo: Is readUnsignedVInt() limited to 32 bits? I would expect it to be of variable size. That would mean that the format would be fine. Correct me if I'm wong! Brandon: Some applications might set the TTL dynamically. Of course the TTL could be capped and or removed in the application. But it might not be so obvious as you make it sound. On Thu, Jan 25, 2018 at 9:49 PM, Paulo Motta wrote: > > The long term solution would be to work with a long instead of a int. The > serialized seems to be a variable-int already, so that should be fine > already. > > Agreed but apparently it needs a new sstable format as well as > mentioned on CASSANDRA-14092. > > > If you change the assertion to 15 years, then applications might fail, as > they might be setting a 15+ year ttl. > > This is an emergency measure while we provide a longer term fix. Any > application using TTL ~= 20 years will need to be lower the TTL anyway > to prevent data loss. > > 2018-01-25 18:40 GMT-02:00 Brandon Williams : > > My guess is they don't know how to NOT set a TTL (perhaps with a default > in > > the schema), so they chose max value. Someone else's problem by then. > > > > On Thu, Jan 25, 2018 at 2:38 PM, Michael Kjellman > > wrote: > > > >> why are people inserting data with a 15+ year TTL? sorta curious about > the > >> actual use case for that. > >> > >> > On Jan 25, 2018, at 12:36 PM, horschi wrote: > >> > > >> > The assertion was working fine until yesterday 03:14 UTC. > >> > > >> > The long term solution would be to work with a long instead of a int. > The > >> > serialized seems to be a variable-int already, so that should be fine > >> > already. > >> > > >> > If you change the assertion to 15 years, then applications might > fail, as > >> > they might be setting a 15+ year ttl. > >> > > >> > regards, > >> > Christian > >> > > >> > On Thu, Jan 25, 2018 at 9:19 PM, Paulo Motta < > pauloricard...@gmail.com> > >> > wrote: > >> > > >> >> Thanks for raising this. Agreed this is bad, when I filed > >> >> CASSANDRA-14092 I thought a write would fail when localDeletionTime > >> >> overflows (as it is with 2.1), but that doesn't seem to be the case > on > >> >> 3.0+ > >> >> > >> >> I propose adding the assertion back so writes will fail, and reduce > >> >> the max TTL to something like 15 years for the time being while we > >> >> figure a long term solution. > >> >> > >> >> 2018-01-25 18:05 GMT-02:00 Jeremiah D Jordan < > jeremiah.jor...@gmail.com > >> >: > >> >>> If you aren’t getting an error, then I agree, that is very bad. > >> Looking > >> >> at the 3.0 code it looks like the assertion checking for overflow was > >> >> dropped somewhere along the way, I had only been looking into 2.1 > where > >> you > >> >> get an assertion error that fails the query. > >> >>> > >> >>> -Jeremiah > >> >>> > >> >>>> On Jan 25, 2018, at 2:21 PM, Anuj Wadehra >> INVALID> > >> >> wrote: > >> >>>> > >> >>>> > >> >>>> Hi Jeremiah, > >> >>>> Validation is on TTL value not on (system_time+ TTL). You can test > it > >> >> with below example. Insert is successful, overflow happens silently > and > >> >> data is lost: > >> >>>> create table test(name text primary key,age int); > >> >>>> insert into test(name,age) values('test_20yrs',30) USING TTL > >> 63072; > >> >>>> select * from test where name='test_20yrs'; > >> >>>> > >> >>>> name | age > >> >>>> --+- > >> >>>> > >> >>>> (0 rows) > >> >>>> > >> >>>> insert into test(name,age) values('test_20yr_plus_1',30) USING TTL > >> >> 630720001;InvalidRequest: Error from server: code=2200 [Invalid > query] > >> >> message="ttl is too large. requested (630720001) maximum (63072)" > >> >>>> ThanksAnuj > >> >>>> On Friday 26 January 2018, 12:11:03 AM IST, J. D. Jordan < > >> >> jeremiah.jor...@gmail.com> wrote: > >> >
Re: URGENT: CASSANDRA-14092 causes Data Loss
If its a 32 bit timestamp, can't we just save/read localDeletionTime as unsinged int? That would give it another 68 years. I think everyone involved here could live with that limitation :-) On Fri, Jan 26, 2018 at 8:16 AM, Anuj Wadehra < anujw_2...@yahoo.co.in.invalid> wrote: > Hi Jeff, > > Thanks for the prompt action! I agree that patching an application MAY > have a shorter life cycle than patching Cassandra in production. But, in > the interest of the larger Cassandra user community, we should put our best > effort to avoid breaking all the affected applications in production. We > should also consider that updating business logic as per the new 15 year > TTL constraint may have business implications for many users. I have a > limited understanding about the complexity of the code patch, but it may be > more feasible to extend the 20 year limit in Cassandra in 2.1/2.2 rather > than asking all impacted users to do an immediate business logic > adaptation. Moreover, now that we officially support Cassandra 2.1 & 2.2 > until 4.0 release and provide critical fixes for 2.1, it becomes even more > reasonable to provide this extremely critical patch for 2.1 & 2.2 (unless > its absolutely impossible). Still, many users use Cassandra 2.1 and 2.2 in > their most critical production systems. > > Thanks > Anuj > > On Friday 26 January 2018, 11:06:30 AM IST, Jeff Jirsa < > jji...@gmail.com> wrote: > > We’ll get patches out. They almost certainly aren’t going to change the > sstable format for old versions (unless whoever writes the patch makes a > great argument for it), so there’s probably not going to be post-2038 ttl > support for 2.1/2.2. For those old versions, we can definitely make it not > lose data, but we almost certainly aren’t going to make the ttl go past > 2038 in old versions. > > More importantly, any company trying to do 20 year ttl’s that’s waiting > for a patched version should start by patching their app to not write > invalid ttls - your app release cycle is almost certainly faster than db > patch / review / test / release / validation, and you can avoid the data > loss application side by calculating the ttl explicitly. It’s not the best > solution, but it beats doing nothing, and we’re not rushing out a release > in less than a day (we haven’t even started a vote, and voting window is 72 > hours for members to review and approve or reject the candidate). > > > > -- > Jeff Jirsa > > > > On Jan 25, 2018, at 9:07 PM, Jeff Jirsa wrote: > > > > Patches welcome. > > > > -- > > Jeff Jirsa > > > > > >> On Jan 25, 2018, at 8:15 PM, Anuj Wadehra > wrote: > >> > >> Hi Paulo, > >> > >> Thanks for looking into the issue on priority. I have serious concerns > regarding reducing the TTL to 15 yrs.The patch will immediately break all > existing applications in Production which are using 15+ yrs TTL. And then > they would be stuck again until all the logic in Production software is > modified and the software is upgraded immediately. This may take days. Such > heavy downtime is generally not acceptable for any business. Yes, they will > not have silent data loss but they would not be able to do any business > either. I think the permanent fix must be prioritized and put on extremely > fast track. This is a certain Blocker and the impact could be > enormous--with and without the 15 year short-term patch. > >> > >> And believe me --there are plenty such business use cases where you use > very long TTLs such as 20 yrs for compliance and other reasons. > >> > >> Thanks > >> Anuj > >> > >> On Friday 26 January 2018, 4:57:13 AM IST, Michael Kjellman < > kjell...@apple.com> wrote: > >> > >> why are people inserting data with a 15+ year TTL? sorta curious about > the actual use case for that. > >> > >>> On Jan 25, 2018, at 12:36 PM, horschi wrote: > >>> > >>> The assertion was working fine until yesterday 03:14 UTC. > >>> > >>> The long term solution would be to work with a long instead of a int. > The > >>> serialized seems to be a variable-int already, so that should be fine > >>> already. > >>> > >>> If you change the assertion to 15 years, then applications might fail, > as > >>> they might be setting a 15+ year ttl. > >>> > >>> regards, > >>> Christian > >>> > >>> On Thu, Jan 25, 2018 at 9:19 PM, Paulo Motta > > >>> wrote: > >>> > >>>> Thanks for raising this. Agreed this is bad, whe
Re: [VOTE CLOSED] Release Apache Cassandra 2.0.10
Would it be possible to have CASSANDRA-7511 reviewed also? On Mon, Aug 11, 2014 at 4:34 PM, Sylvain Lebresne wrote: > Ok, ok, closing this vote for now. We'll re-roll as soon as the pig stuff > are fixed. > > > On Fri, Aug 8, 2014 at 10:07 PM, Jeremiah D Jordan > wrote: > > > I'm -1 on this until we get CqlRecordReader fixed (which will also fix > the > > newly added in 2.0.10 Pig CqlNativeStoarge): > > https://issues.apache.org/jira/browse/CASSANDRA-7725 > > https://issues.apache.org/jira/browse/CASSANDRA-7726 > > > > Without those two things anyone using CqlStorage previously (which > removed > > with the removal of CPRR) who updates to using CqlNativeStoarge will have > > broken scripts unless they are very very careful. > > > > > > -Jeremiah > > > > On Aug 8, 2014, at 5:03 AM, Sylvain Lebresne > wrote: > > > > > I propose the following artifacts for release as 2.0.10. > > > > > > sha1: cd37d07baf5394d9bac6763de4556249e9837bb0 > > > Git: > > > > > > http://git-wip-us.apache.org/repos/asf?p=cassandra.git;a=shortlog;h=refs/tags/2.0.10-tentative > > > Artifacts: > > > > > > https://repository.apache.org/content/repositories/orgapachecassandra-1023/org/apache/cassandra/apache-cassandra/2.0.10/ > > > Staging repository: > > > > > > https://repository.apache.org/content/repositories/orgapachecassandra-1023/ > > > > > > The artifacts as well as the debian package are also available here: > > > http://people.apache.org/~slebresne/ > > > > > > The vote will be open for 72 hours (longer if needed). > > > > > > [1]: http://goo.gl/xzb9ky (CHANGES.txt) > > > [2]: http://goo.gl/nBI37B (NEWS.txt) > > > > >
Re: [VOTE CLOSED] Release Apache Cassandra 2.0.10
Saw it. Thanks! On Mon, Aug 11, 2014 at 11:53 PM, Michael Shuler wrote: > On 08/11/2014 04:21 PM, Michael Shuler wrote: > >> On 08/11/2014 09:50 AM, horschi wrote: >> >>> Would it be possible to have CASSANDRA-7511 reviewed also? >>> >> >> That was committed to the c*-2.0 branch prior to the vote and was in the >> tentative-2.0.10 CHANGES.txt: >> >> * Fix truncate to always flush (CASSANDRA-7511) >> > > I didn't pair up your comment on 7511 and this email - see: > > https://issues.apache.org/jira/browse/CASSANDRA-7750 > > -- > Michael > >