Yes. On Fri, Feb 25, 2011 at 4:29 PM, Jeffrey Wang <jw...@palantir.com> wrote: > I just noticed this thread. Does this mean that (assuming the same setup of > an empty keyspace and CFs added later) if I have a CF that I write to for > some time, but not enough to hit the flush limits, it will never get flushed > until the server is restarted? I believe this is causing commit logs to not > be deleted, which is taking up a ton of disk space (in addition to a bunch > of small memtables being stuck in memory). > > > > -Jeffrey > > > > From: Ching-Cheng Chen [mailto:cc...@evidentsoftware.com] > Sent: Thursday, February 17, 2011 8:52 AM > To: user@cassandra.apache.org > Cc: Jonathan Ellis > Subject: Re: memtable_flush_after_mins setting not working > > > > https://issues.apache.org/jira/browse/CASSANDRA-2183 > > > > Regards, > > > > Chen > > > > www.evidentsoftware.com > > On Thu, Feb 17, 2011 at 11:47 AM, Ching-Cheng Chen > <cc...@evidentsoftware.com> wrote: > > Certainly, I'll open a ticket to track this issue. > > > > Regards, > > > > Chen > > > > www.evidentsoftware.com > > > > On Thu, Feb 17, 2011 at 11:42 AM, Jonathan Ellis <jbel...@gmail.com> wrote: > > Your analysis sounds correct to me. Can you open a ticket on > https://issues.apache.org/jira/browse/CASSANDRA ? > > On Thu, Feb 17, 2011 at 10:17 AM, Ching-Cheng Chen > <cc...@evidentsoftware.com> wrote: >> We have observed the behavior that memtable_flush_after_mins setting not >> working occasionally. After some testing and code digging, we finally >> figured out what going on. >> The memtable_flush_after_mins won't work on certain condition with current >> implementation in Cassandra. >> >> In org.apache.cassandra.db.Table, the scheduled flush task is setup by >> the >> following code during construction. >> >> int minCheckMs = Integer.MAX_VALUE; >> >> for (ColumnFamilyStore cfs : columnFamilyStores.values()) >> { >> minCheckMs = Math.min(minCheckMs, cfs.getMemtableFlushAfterMins() * 60 >> * >> 1000); >> } >> Runnable runnable = new Runnable() >> { >> public void run() >> { >> for (ColumnFamilyStore cfs : columnFamilyStores.values()) >> { >> cfs.forceFlushIfExpired(); >> } >> } >> }; >> flushTask = StorageService.scheduledTasks.scheduleWithFixedDelay(runnable, >> minCheckMs, minCheckMs, TimeUnit.MILLISECONDS); >> >> Now for our application, we will create a keyspacewithout any columnfamily >> first. And only add needed columnfamily later depends on request. >> However, when keyspacegot created (without any columnfamily ), the above >> code will actually schedule a fixed delay flush check task with >> Integer.MAX_VALUE ms >> since there is no columnfamily yet. >> Later when you add columnfamily to this empty keyspace, the initCf() >> method >> in Table.java doesn't check whether the scheduled flush check task >> interval >> need >> to be updated or not. To fix this, we'd need to restart the Cassandra >> after columnfamily added into the keyspace. >> I would suggest that add additional logic in initCf() method to recreate a >> scheduled flush check task if needed. >> Regards, >> Chen >> www.evidentsoftware.com > > > -- > Jonathan Ellis > Project Chair, Apache Cassandra > co-founder of Riptano, the source for professional Cassandra support > http://riptano.com > > -- > www.evidentsoftware.com > >
-- Jonathan Ellis Project Chair, Apache Cassandra co-founder of DataStax, the source for professional Cassandra support http://www.datastax.com