SSTable files not getting deleted

2011-01-06 Thread Ching-Cheng Chen
My impression is that force GC should have deleted the SSTable files no
longer valid.

I performed a nodetool compact, all went good and finished.   All column
family now only have one big live SSTable file.

Then I use jconsole to force a GC, but those old SSTable files still not
getting deleted.   I thought this should trigger a deletion for those
SSTable files maked for delete.

I'm using 0.7-rc2.

When I perform the force GC, no exception in log file.

 INFO [ScheduledTasks:1] 2011-01-06 10:43:16,786 GCInspector.java (line 133)
GC for ConcurrentMarkSweep: 456 ms, 1529612464 reclaimed leaving 114919560
used; max is 4424663040
 INFO [ScheduledTasks:1] 2011-01-06 10:48:25,580 GCInspector.java (line 133)
GC for ConcurrentMarkSweep: 505 ms, 56558064 reclaimed leaving 121900224
used; max is 4424663040
 INFO [ScheduledTasks:1] 2011-01-06 10:50:47,760 GCInspector.java (line 133)
GC for ConcurrentMarkSweep: 455 ms, 28188544 reclaimed leaving 124425408
used; max is 4424663040
 INFO [ScheduledTasks:1] 2011-01-06 10:52:57,107 GCInspector.java (line 133)
GC for ConcurrentMarkSweep: 513 ms, 27953872 reclaimed leaving 126936960
used; max is 4424663040

By the way, those old SSTable files did get removed if I restart the node.

Regards,

Chen


Re: SSTable files not getting deleted

2011-01-06 Thread Ching-Cheng Chen
Yes, those SSTable files has "compacted" tag.

Those with compacted tag have size 0, so disk space is not an issue.

However, the matching Filter, Index, Statistics files were not removed,
either.
So I ended up with tons of file under data directory although they not using
much space.

I'm running cassandra 0.7-rc2.

Red Hat Linux 2.6.18-194.26.1.el5

SUN JDK
java version "1.6.0_22"
Java(TM) SE Runtime Environment (build 1.6.0_22-b04)
Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode)

Regards,

Chen

On Thu, Jan 6, 2011 at 12:18 PM, Robert Coli  wrote:

> On Thu, Jan 6, 2011 at 7:59 AM, Ching-Cheng Chen
>  wrote:
>
> > I performed a nodetool compact, all went good and finished.   All column
> > family now only have one big live SSTable file.
> > Then I use jconsole to force a GC, but those old SSTable files still not
> > getting deleted.   I thought this should trigger a deletion for those
> > SSTable files maked for delete.
>
> Which old SSTable files, specifically?
>
> Does their name include the tag "compacted"?
>
> And what version of cassandra are you running in what environment?
>
> =Rob
>


Re: UnserializableColumnFamilyException: Couldn't find cfId

2011-01-21 Thread Ching-Cheng Chen
We have similar exception before, and the root cause was like Aaron
mentioned.

You will encounter this exception If you have code create CF on the fly and
data was insert into the node which hasn't got schema synced yet.

You will have to call describe_schema_version() to ensure all nodes has same
schema before you start insert data into newly create CF.

Regards,

Chen

On Thu, Jan 20, 2011 at 5:34 PM, Aaron Morton wrote:

> Sounds like there are multiple versions of your schema around the cluster.
> What client API are you using? Does it support
> the describe_schema_versions() function? This will tell you how many
> versions there are.
>
> The easy solutions here is scrub the data and start a new 0.7 cluster using
> the release version.If possible you should not use data created in the non
> release versions once you get to production.
>
> Hope that helps.
> Aaron
>
>
> On 21 Jan, 2011,at 09:15 AM, Oleg Proudnikov 
> wrote:
>
> Hi All,
>
> Could you please help me understand the impact on my data?
>
> I am running a 6 node 0.7-rc4 Cassandra cluster with RF=2. Schema was
> defined
> when the cluster was created and did not change. I am doing batch load with
> CL=ONE. The cluster is under some stress in memory and I/O. Each node has
> 1G
> heap. CPU is around 10% but the latency is high.
>
> I saw this exception on 2 out of 6 nodes in a relatively short window of
> time.
> Hector clients received no exception and the nodes continued running. The
> exception has not happened since even though the load is continuing.
> I do get an occasional OOM and I am adjusting thresholds and other
> settings as I go. I also doubled RAM to 2G since the exception.
>
> Here is the exception - the same stack trace in all cases.
> org.apache.cassandra.db.UnserializableColumnFamilyException: C
> ouldn't find cfId=1004
> at org.apache.cassandra.dbColumnFamilySerializer.deserialize
>
> (ColumnFamilySerializer.java:117)
> at org.apache.cassandra.db.RowMutationSerializer.defreezeTheMaps
> (RowMutation.java:385)
> at org.apache.cassandra.db.RowMutationSerializer.deserialize
> (RowMutation.java:395)
> at org.apache.cassandra.db.RowMutationSerializer.deserialize
> (RowMutation.java:353)
> at org.apache.cassandra.db.RowMutationVerbHandler.doVerb
> (RowMutationVerbHandler.java:52)
> at org.apache.cassandra.net.MessageDeliveryTask.run
> (MessageDeliveryTask.java:63)
> at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown Source)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
> at java.lang.Thread.run(Unknown Source)
>
>
> It refers to two cfIds - cfId=1004 and cfId=1013. Mutation stages are
> always
> different even for the exceptions appearing within the same millisecond.
> As you can see below cfId=004 appears on both nodes several times but at
> different times while cfId=0013 appears only once on one node.
>
> It happened as a group within one second on one node and in 5 groups spread
> across 45 minutes on another node. I left the first log entry of each
> group.
>
> xxx.xxx.xxx.140 grep -i cfid -B 1 log/cassandra.log
> xxx.xxx.xxx.141 grep -i cfid -B 1 log/cassandra.log
> xxx.xxx.xxx.142 grep -i cfid -B 1 log/cassandra.log
> xxx.xxx.xxx.143 grep -i cfid -B 1 log/cassandra.log
>
>
> xxx.xxx.xxx.144 grep -i cfid -B 1 log/cassandra.log
> ERROR [MutationStage:11] 2011-01-14 15:02:03,911
> RowMutationVerbHandler.java
> (line 83) Error in row mutation
> org.apache.cassandra.db.UnserializableColumnFamilyException:
> Couldn't find cfId=1004
>
>
> xxx.xxx.xxx.145 grep -i cfid -B 1 log/cassandra.log
> ERROR [MutationStage:1] 2011-01-14 15:02:34,460 RowMutationVerbHandler.java
> (line 83) Error in row mutation
> org.apache.cassandra.db.UnserializableColumnFamilyException:
> Couldn't find cfId=1004
> --
> ERROR [MutationStage:13] 2011-01-14 15:03:28,637
> RowMutationVerbHandler.java
> (line 83) Error in row mutation
> org.apache.cassandra.db.UnserializableColumnFamilyException:
> Couldn't find cfId=1004
> --
> ERROR [MutationStage:27] 2011-01-14 15:05:02,513
> RowMutationVerbHandler.java
> (line 83) Error in row mutation
> org.apache.cassandra.db.UnserializableColumnFamilyException:
> Couldn't find cfId=1004
> --
> ERROR [MutationStage:4] 2011-01-14 15:12:30,731 RowMutationVerbHandler.java
> (line 83) Error in row mutation
> org.apache.cassandra.db.UnserializableColumnFamilyException:
> Couldn't find cfId=1004
> --
> ERROR [MutationStage:23] 2011-01-14 15:47:03,416
> RowMutationVerbHandler.java
> (line 83) Error in row mutation
> org.apache.cassandra.db.UnserializableColumnFamilyException:
> Couldn't find cfId=1013
>
>
>
> Q. What does this mean for the consistency? Am I still within my guarantee
> of
> CL=ONE?
>
>
>
> NOTE: I experienced similar exceptions in 0.7-rc2 but at that time cfIds
> looked
> corrupted. They were random/negative and these exceptions
> were followed by an OOM with an attempt to allocate a huge HeapByteBuffer.
>
> Thank you very much,
> Oleg
>
>
>
>


Files not deleted after compaction and GCed

2011-01-25 Thread Ching-Cheng Chen
Using cassandra 0.7.0

The class org.apache.cassandra.io.sstable.SSTableDeletingReference only
remove the -Data.db file, but leave the xxx-Compacted, xxx-Filter.db,
xxx-Index.db and xxx-Statistics.db intact.

And that's the behavior I saw.I ran manual compact then trigger a GC
from jconsole.   The Data.db file got removed but not the others.

Is this the expected behavior?

Regards,

Chen


Re: Files not deleted after compaction and GCed

2011-01-25 Thread Ching-Cheng Chen
Nope, no exception at all.

But if the same class
(org.apache.cassandra.io.sstable.SSTableDeletingReference) is responsible
for delete other files, then that's not right.
I checked the source code for SSTableDeletingReference, doesn't looks like
it will delete other files type.

Regards,

Chen

On Tue, Jan 25, 2011 at 4:05 PM, Jonathan Ellis  wrote:

> No, that is not expected.  All the sstable components are removed in
> the same method; did you check the log for exceptions?
>
> On Tue, Jan 25, 2011 at 2:58 PM, Ching-Cheng Chen
>  wrote:
> > Using cassandra 0.7.0
> > The class org.apache.cassandra.io.sstable.SSTableDeletingReference only
> > remove the -Data.db file, but leave the xxx-Compacted, xxx-Filter.db,
> > xxx-Index.db and xxx-Statistics.db intact.
> > And that's the behavior I saw.I ran manual compact then trigger a GC
> > from jconsole.   The Data.db file got removed but not the others.
> > Is this the expected behavior?
> > Regards,
> > Chen
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of DataStax, the source for professional Cassandra support
> http://www.datastax.com
>


Re: Files not deleted after compaction and GCed

2011-01-26 Thread Ching-Cheng Chen
It's a bug.

In SSTableDeletingReference, it try this operation

components.remove(Component.DATA);

before

STable.delete(desc, components);

However, the components was reference to the components object which was
created inside SSTable by

this.components = Collections.unmodifiableSet(dataComponents);

As you can see, you can't try the remove operation on that componets object.

If I add a try block and output exception around the
components.remove(Component.DATA), I got this.

java.lang.UnsupportedOperationException
at java.util.Collections$UnmodifiableCollection.remove(Unknown
Source)
at
org.apache.cassandra.io.sstable.SSTableDeletingReference$CleanupTask.run(SSTableDeletingReference.java:103)
at java.util.concurrent.Executors$RunnableAdapter.call(Unknown
Source)
at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
at java.util.concurrent.FutureTask.run(Unknown Source)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(Unknown
Source)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown
Source)
at
org.apache.cassandra.concurrent.RetryingScheduledThreadPoolExecutor$LoggingScheduledFuture.run(RetryingScheduledThreadPoolExecutor.java:81)
at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown
Source)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown
Source)
at java.lang.Thread.run(Unknown Source)

Regards,

Chen

On Tue, Jan 25, 2011 at 4:21 PM, Jonathan Ellis  wrote:

> the other component types are deleted by this line:
>
>SSTable.delete(desc, components);
>
> On Tue, Jan 25, 2011 at 3:11 PM, Ching-Cheng Chen
>  wrote:
> > Nope, no exception at all.
> > But if the same class
> > (org.apache.cassandra.io.sstable.SSTableDeletingReference) is responsible
> > for delete other files, then that's not right.
> > I checked the source code for SSTableDeletingReference, doesn't looks
> like
> > it will delete other files type.
> > Regards,
> > Chen
> >
> > On Tue, Jan 25, 2011 at 4:05 PM, Jonathan Ellis 
> wrote:
> >>
> >> No, that is not expected.  All the sstable components are removed in
> >> the same method; did you check the log for exceptions?
> >>
> >> On Tue, Jan 25, 2011 at 2:58 PM, Ching-Cheng Chen
> >>  wrote:
> >> > Using cassandra 0.7.0
> >> > The class org.apache.cassandra.io.sstable.SSTableDeletingReference
> only
> >> > remove the -Data.db file, but leave the xxx-Compacted,
> >> > xxx-Filter.db,
> >> > xxx-Index.db and xxx-Statistics.db intact.
> >> > And that's the behavior I saw.I ran manual compact then trigger a
> GC
> >> > from jconsole.   The Data.db file got removed but not the others.
> >> > Is this the expected behavior?
> >> > Regards,
> >> > Chen
> >>
> >>
> >>
> >> --
> >> Jonathan Ellis
> >> Project Chair, Apache Cassandra
> >> co-founder of DataStax, the source for professional Cassandra support
> >> http://www.datastax.com
> >
> >
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of DataStax, the source for professional Cassandra support
> http://www.datastax.com
>


Re: Files not deleted after compaction and GCed

2011-01-26 Thread Ching-Cheng Chen
I think this might be what happening.

Since you are using ScheduledThreadPoolExecutor.schedule(), the exception
was swallowed by the FutureTask.

You will have to perform a get() method on the ScheduledFuture, and you will
get ExecutionException if there was any exception occured in run().

Regards,

Chen

On Wed, Jan 26, 2011 at 10:50 AM, Jonathan Ellis  wrote:

> Patch submitted.
>
> One thing I still don't understand is why
> RetryingScheduledThreadPoolExecutor isn't firing the
> DefaultUncaughtExceptionHandler, which should have logged that
> exception.
>
> On Wed, Jan 26, 2011 at 9:41 AM, Jonathan Ellis  wrote:
> > Thanks for tracking that down!  Created
> > https://issues.apache.org/jira/browse/CASSANDRA-2059 to fix.
> >
> > On Wed, Jan 26, 2011 at 8:17 AM, Ching-Cheng Chen
> >  wrote:
> >> It's a bug.
> >> In SSTableDeletingReference, it try this operation
> >> components.remove(Component.DATA);
> >> before
> >> STable.delete(desc, components);
> >> However, the components was reference to the components object which was
> >> created inside SSTable by
> >> this.components = Collections.unmodifiableSet(dataComponents);
> >> As you can see, you can't try the remove operation on that componets
> object.
> >> If I add a try block and output exception around the
> >> components.remove(Component.DATA), I got this.
> >> java.lang.UnsupportedOperationException
> >> at java.util.Collections$UnmodifiableCollection.remove(Unknown
> >> Source)
> >> at
> >>
> org.apache.cassandra.io.sstable.SSTableDeletingReference$CleanupTask.run(SSTableDeletingReference.java:103)
> >> at java.util.concurrent.Executors$RunnableAdapter.call(Unknown
> >> Source)
> >> at java.util.concurrent.FutureTask$Sync.innerRun(Unknown Source)
> >> at java.util.concurrent.FutureTask.run(Unknown Source)
> >> at
> >>
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(Unknown
> >> Source)
> >> at
> >>
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(Unknown
> >> Source)
> >> at
> >>
> org.apache.cassandra.concurrent.RetryingScheduledThreadPoolExecutor$LoggingScheduledFuture.run(RetryingScheduledThreadPoolExecutor.java:81)
> >> at
> java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown
> >> Source)
> >> at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown
> >> Source)
> >> at java.lang.Thread.run(Unknown Source)
> >> Regards,
> >> Chen
> >> On Tue, Jan 25, 2011 at 4:21 PM, Jonathan Ellis 
> wrote:
> >>>
> >>> the other component types are deleted by this line:
> >>>
> >>>SSTable.delete(desc, components);
> >>>
> >>> On Tue, Jan 25, 2011 at 3:11 PM, Ching-Cheng Chen
> >>>  wrote:
> >>> > Nope, no exception at all.
> >>> > But if the same class
> >>> > (org.apache.cassandra.io.sstable.SSTableDeletingReference) is
> >>> > responsible
> >>> > for delete other files, then that's not right.
> >>> > I checked the source code for SSTableDeletingReference, doesn't looks
> >>> > like
> >>> > it will delete other files type.
> >>> > Regards,
> >>> > Chen
> >>> >
> >>> > On Tue, Jan 25, 2011 at 4:05 PM, Jonathan Ellis 
> >>> > wrote:
> >>> >>
> >>> >> No, that is not expected.  All the sstable components are removed in
> >>> >> the same method; did you check the log for exceptions?
> >>> >>
> >>> >> On Tue, Jan 25, 2011 at 2:58 PM, Ching-Cheng Chen
> >>> >>  wrote:
> >>> >> > Using cassandra 0.7.0
> >>> >> > The class org.apache.cassandra.io.sstable.SSTableDeletingReference
> >>> >> > only
> >>> >> > remove the -Data.db file, but leave the xxx-Compacted,
> >>> >> > xxx-Filter.db,
> >>> >> > xxx-Index.db and xxx-Statistics.db intact.
> >>> >> > And that's the behavior I saw.I ran manual compact then
> trigger a
> >>> >> > GC
> >>> >> > from jconsole.   The Data.db file got removed but not the others.
> >>> >> > Is this the expected behavior?
> >>> >> > Regards,
> >>> >> > Chen
> >>> >>
> >>> >>
> >>> >>
> >>> >> --
> >>> >> Jonathan Ellis
> >>> >> Project Chair, Apache Cassandra
> >>> >> co-founder of DataStax, the source for professional Cassandra
> support
> >>> >> http://www.datastax.com
> >>> >
> >>> >
> >>>
> >>>
> >>>
> >>> --
> >>> Jonathan Ellis
> >>> Project Chair, Apache Cassandra
> >>> co-founder of DataStax, the source for professional Cassandra support
> >>> http://www.datastax.com
> >>
> >>
> >
> >
> >
> > --
> > Jonathan Ellis
> > Project Chair, Apache Cassandra
> > co-founder of DataStax, the source for professional Cassandra support
> > http://www.datastax.com
> >
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of DataStax, the source for professional Cassandra support
> http://www.datastax.com
>


memtable_flush_after_mins setting not working

2011-02-17 Thread Ching-Cheng Chen
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


Re: memtable_flush_after_mins setting not working

2011-02-17 Thread Ching-Cheng Chen
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  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
>  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


Re: memtable_flush_after_mins setting not working

2011-02-17 Thread Ching-Cheng Chen
https://issues.apache.org/jira/browse/CASSANDRA-2183

<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 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
>>  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
>


Re: [RELEASE] 0.7.2

2011-02-17 Thread Ching-Cheng Chen
Looks like it's

https://issues.apache.org/jira/browse/CASSANDRA-2172

Regards,

Chen

www.evidentsoftware.com

On Thu, Feb 17, 2011 at 1:06 PM, Damick, Jeffrey  wrote:

>  So after upgrade to 0.7.2, I see this on startup – should I just blow
> away these cache files?
>
>
>  WARN [main] 2011-02-17 18:03:24,161 ColumnFamilyStore.java (line 281)
> error reading saved cache /var/lib/cassandra/saved_caches/xx-KeyCache
> java.io.EOFException
> at java.io.ObjectInputStream$PeekInputStream.readFully(Unknown Source)
> at java.io.ObjectInputStream$BlockDataInputStream.readShort(Unknown
> Source)
> at java.io.ObjectInputStream.readStreamHeader(Unknown Source)
> at java.io.ObjectInputStream.(Unknown Source)
> at
> org.apache.cassandra.db.ColumnFamilyStore.readSavedCache(ColumnFamilyStore.java:255)
> at
> org.apache.cassandra.db.ColumnFamilyStore.(ColumnFamilyStore.java:198)
> at
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:451)
> at
> org.apache.cassandra.db.ColumnFamilyStore.createColumnFamilyStore(ColumnFamilyStore.java:432)
> at org.apache.cassandra.db.Table.initCf(Table.java:360)
> at org.apache.cassandra.db.Table.(Table.java:290)
> at org.apache.cassandra.db.Table.open(Table.java:107)
> at
> org.apache.cassandra.service.AbstractCassandraDaemon.setup(AbstractCassandraDaemon.java:162)
> at
> org.apache.cassandra.service.AbstractCassandraDaemon.activate(AbstractCassandraDaemon.java:316)
> at
> org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:79)
>
>
> thanks
>
>
> On 2/16/11 7:56 PM, "Eric Evans"  wrote:
>
>
>
> CASSANDRA-2165[1] became evident almost as soon as 0.7.1 released, and
> it's ugly enough that we didn't want to wait.
>
> Be sure you've read the changelog[2] and release notes[3], and let us
> know[4] if you encounter any problems.
>
> Thanks!
>
>
> [1]: https://issues.apache.org/jira/browse/CASSANDRA-2165
> [2]: http://goo.gl/iI7U2 (CHANGES.txt)
> [3]: http://goo.gl/b2dCq (NEWS.txt)
> [4]: https://issues.apache.org/jira/browse/CASSANDRA
>
> --
> Eric Evans
> eev...@rackspace.com
>
>
>


Re: Error when bringing up 3rd node

2011-02-18 Thread Ching-Cheng Chen
41
82
123

These certainly not correct.  Can't just use 2 ^ 127, will overflow

You can't use Java's primitive type to do this calculation.   long only use
64 bit.

You'd need to use BigInteger class to do this calculation.

Regards,

Chen

www.evidentsoftware.com

On Fri, Feb 18, 2011 at 4:04 PM, mcasandra  wrote:

>
> Thanks! This is what I got. Is this right?
>
> public class TokenCalc{
>  public static void main(String ...args){
>   int nodes=3;
>   for(int i = 1 ; i <= nodes; i++) {
> System.out.println( (2 ^ 127) / nodes * i);
>   }
>  }
> }
>
> 41
> 82
> 123
> --
> View this message in context:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Error-when-bringing-up-3rd-node-tp6041409p6041471.html
> Sent from the cassandra-u...@incubator.apache.org mailing list archive at
> Nabble.com.
>


Re: Error when bringing up 3rd node

2011-02-18 Thread Ching-Cheng Chen
try this

BigInteger bi = new BigInteger("2");
BigInteger or = new BigInteger("2");
 for (int i=1;i<127;i++) {
or = or.multiply(bi);
}
or = or.divide(new BigInteger("3"));
 for (int i=0;i<3;i++) {
System.out.println(or.multiply(new BigInteger(""+i)));
}

which generate

0
56713727820156410577229101238628035242
113427455640312821154458202477256070484

Regards,

Chen

www.evidentsoftware.com

On Fri, Feb 18, 2011 at 4:24 PM, Eric Gilmore  wrote:

> I'm not sure I can say exactly why, but I'm sure those numbers can't be
> correct.  One node should be zero and the other values should be very long
> numbers like 85070591730234615865843651857942052863.
>
> We need another Java expert's opinion here, but it looks like your snippet
> may have "integer 
> overflow"
> or "integer overload" going on.
>
>
> On Fri, Feb 18, 2011 at 1:04 PM, mcasandra  wrote:
>
>>
>> Thanks! This is what I got. Is this right?
>>
>> public class TokenCalc{
>>  public static void main(String ...args){
>>   int nodes=3;
>>   for(int i = 1 ; i <= nodes; i++) {
>> System.out.println( (2 ^ 127) / nodes * i);
>>   }
>>  }
>> }
>>
>> 41
>> 82
>> 123
>> --
>> View this message in context:
>> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Error-when-bringing-up-3rd-node-tp6041409p6041471.html
>> Sent from the cassandra-u...@incubator.apache.org mailing list archive at
>> Nabble.com.
>>
>
>


Re: Error when bringing up 3rd node

2011-02-18 Thread Ching-Cheng Chen
If you know you will have 3 nodes, you should set the initial token inside
the cassandra.yaml for each node.

Then you won't need to run nodetool move.

Regards,

Chen

www.evidentsoftware.com

On Fri, Feb 18, 2011 at 5:24 PM, mcasandra  wrote:

>
> Thanks! I feel so horrible after realizing what mistaked I made :)
>
> After I bring up the new node I just need to run the following on old
> nodes?
>
> 1) New node set the initial token to 56713727820156410577229101238628035242
> 2) start new node
> 3) On second node run nodetool move 113427455640312821154458202477256070484
>
> --
> View this message in context:
> http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Error-when-bringing-up-3rd-node-tp6041409p6041649.html
> Sent from the cassandra-u...@incubator.apache.org mailing list archive at
> Nabble.com.
>


Re: Is it possible to get list of row keys?

2011-02-23 Thread Ching-Cheng Chen
You can use the setRowCount() method to specify how many keys to return per
call.
By default is 100.

Beware don't set it too high since it might cause OOM.

And underline code will pre-allocate an array list with size you speify in
setRowCount().   So you might get a OOM if
you used something like Interger.MAX.

Regards,

Chen

www.evidentsoftware.com

On Wed, Feb 23, 2011 at 2:24 AM, Roshan Dawrani wrote:

> Does it help:
> https://github.com/rantav/hector/blob/master/core/src/test/java/me/prettyprint/cassandra/model/RangeSlicesQueryTest.java
>
>
> It
> uses setReturnKeysOnly()...
>
> Same for index queries in:
> https://github.com/rantav/hector/blob/master/core/src/test/java/me/prettyprint/cassandra/model/IndexedSlicesQueryTest.java
>
> I think it will not return all keys from the ColumnFamily at one shot (as
> with rows)
>
> So, if you want all, you will need to keep paging forward and collecting
> the keys.
>
> On Wed, Feb 23, 2011 at 12:41 PM, Joshua Partogi wrote:
>
>> Hi,
>>
>> Assuming the application does not know the list of keys that is stored
>> inside cassandra, how would it be possible to get list of row keys?
>> This list of row keys is going to be used to get a range of slices.
>>
>> Thank you for your help.
>>
>> --
>> http://twitter.com/jpartogi
>>
>
>
>
> --
> Roshan
> Blog: http://roshandawrani.wordpress.com/
> Twitter: @roshandawrani 
> Skype: roshandawrani
>
>


Re: Is it possible to get list of row keys?

2011-02-23 Thread Ching-Cheng Chen
Actually, if you want to get ALL keys, I believe you can still use
RangeSliceQuery with RP.

Just use setKeys("","") as first batch call.

Then use the last key from previous batch as startKey for next batch.
Beware that since startKey is inclusive, so you'd need to ignore first key
from now on.

Keep going until you finish all batches.  You will know you'd need to stop
when setKeys(key_xyz,"") return you only one key.

This should get you all keys even with RP.

Regards,

Chen

www.evidentsoftware.com

On Wed, Feb 23, 2011 at 8:23 AM, Norman Maurer  wrote:

> query per ranges is only possible with OPP or BPP.
>
> Bye,
> Norman
>
>
> 2011/2/23 Sasha Dolgy :
> > What if i want 20 rows and the next 20 rows in a subsequent query?  can
> this
> > only be achieved with OPP?
> >
> > --
> > Sasha Dolgy
> > sasha.do...@gmail.com
> >
> > On 23 Feb 2011 13:54, "Ching-Cheng Chen" 
> wrote:
> >
>


Re: Is it possible to get list of row keys?

2011-03-02 Thread Ching-Cheng Chen
Stable order mean the keys will return in some kind of ordering (not
alphanumeric, not numeric,), just in certain order.

For example, if you use setKeys("","") as first batch call and you got
following result.

keyabc
key456
keydsg
key8jkg
keyag87
key45s
...

Then if you call with setKeys("keydsg","") again, you will get following
result.

keydsg
key8jkg
keyag87
key45s
...

Regards,

Chen

www.evidentsoftware.com

On Wed, Mar 2, 2011 at 4:53 AM, Eric Charles wrote:

> Hi,
>
> I'm also facing the need to retrieve all row keys.
>
> What do you mean with "stable" order?
> From this thread, I understand paging method with RandomPartitioner will
> return all keys (shuffled, but missing key, no double key).
> This seems to have already told, but I prefer to double-check...
>
> Tks,
> - Eric
>
>
>
> On 23/02/2011 15:24, Daniel Lundin wrote:
>
>> They are, however, in *stable* order, which is important.
>>
>> On Wed, Feb 23, 2011 at 3:20 PM, Norman Maurer  wrote:
>>
>>> yes but be aware that the keys will not in the "right order".
>>>
>>> Bye,
>>> Norman
>>>
>>> 2011/2/23 Roshan Dawrani:
>>>
>>>> On Wed, Feb 23, 2011 at 7:17 PM, Ching-Cheng Chen
>>>>   wrote:
>>>>
>>>>> Actually, if you want to get ALL keys, I believe you can still use
>>>>> RangeSliceQuery with RP.
>>>>> Just use setKeys("","") as first batch call.
>>>>> Then use the last key from previous batch as startKey for next batch.
>>>>> Beware that since startKey is inclusive, so you'd need to ignore first
>>>>> key
>>>>> from now on.
>>>>> Keep going until you finish all batches.  You will know you'd need to
>>>>> stop
>>>>> when setKeys(key_xyz,"") return you only one key.
>>>>>
>>>> This is what I meant to suggest when I earlier said "So, if you want
>>>> all,
>>>> you will need to keep paging forward and collecting the keys." :-)
>>>>
>>>
>


Re: Cassandra 0.7.2 - Enable/Disable HH via JMX (Jconsole)

2011-03-03 Thread Ching-Cheng Chen
The HintedHandoffEnabled attribute is read/write.

You can always set it to true/false to enable/disable HH.

Regards,

Chen

www.evidentosftware.com

On Thu, Mar 3, 2011 at 1:47 PM, Narendra Sharma
wrote:

> I am unable to enable/disable HH via JMX (JConsole).
>
> Even though the load is on and read/writes happening, I don't see
> "operations" component on Jconsole. To clarify further, I see only
> Jconsole->MBeans->org.apache.cassandra.db.StorageProxy.Attributes. I don't
> see Jconsole->MBeans->org.apache.cassandra.db.StorageProxy.Operations. As a
> result I cannot operation like enable/disable HH.
>
> Is this is a bug or I am missing something?
>
> Thanks,
>
> Naren
>


Re: Cassandra 0.6.3 ring not balance in terms of data size

2011-03-17 Thread Ching-Cheng Chen
>From OrderPreservingPartition.java

public StringToken getToken(ByteBuffer key)
{
String skey;
try
{
skey = ByteBufferUtil.string(key, Charsets.UTF_8);
}
catch (CharacterCodingException e)
{
throw new RuntimeException("The provided key was not UTF8
encoded.", e);
}
return new StringToken(skey);
}

Regards,

Chen

Senior Developer, EvidentSoftware(Leaders in Monitoring of NoSQL & JAVA )

http://www.evidentsoftware.com



On Thu, Mar 17, 2011 at 2:06 PM, Ali Ahsan wrote:

>  Please can any one give their comment on this
>
> On 03/17/2011 07:02 PM, Ali Ahsan wrote:
>
> Dear Aaron,
>
> We are little confused about OPP token.How to calculate OPP Token? Few of
> our column families have UUID as key and other's  have  integer as key.
>
>
>
>


Re: Meaning of TotalReadLatencyMicros and TotalWriteLatencyMicrosStatistics

2011-03-22 Thread Ching-Cheng Chen
Just as what it named, it's the total microseconds spent on read operations
so far.

Not average.

Regards,

Chen

Senior Developer, EvidentSoftware(Leaders in Monitoring of NoSQL & JAVA )

http://www.evidentsoftware.com

On Tue, Mar 22, 2011 at 11:11 AM, Jonathan Colby
wrote:

> Hi -
>
> On our recently live cassandra cluster of 5 nodes, we've noticed that the
> latency readings, especially Reads have gone up drastically.
>
> TotalReadLatencyMicros  5413483
> TotalWriteLatencyMicros 1811824
>
>
> I understand these are in microseconds, but what meaning do they have for
> the performance of the cluster?   In other words what do these numbers
> actually measure.
>
> In our case, it looks like we have  a read latency of 5.4 seconds, which is
> very troubling if I interpret this correctly.
>
> Are reads really taking an average of 5 seconds to complete??
>
>
>
>


Re: Problem about freeing space after a major compaction

2011-03-28 Thread Ching-Cheng Chen
tombstones removal also depends on your gc grace period setting.

If you are pretty sure that you have proper gc grace period set and still on
0.7.0, then probably related to this bug.

https://issues.apache.org/jira/browse/CASSANDRA-2059

Regards,



Chen

Senior Developer, EvidentSoftware(Leaders in Monitoring of NoSQL & JAVA )

http://www.evidentsoftware.com

On Mon, Mar 28, 2011 at 10:40 AM, Roberto Bentivoglio <
roberto.bentivog...@gmail.com> wrote:

> Hi all,
> we're working on a Cassandra 0.7.0 production enviroment with a store of
> data near to 500 GB.
> We need to periodically remove the tombstones from deleted/expired data
> performing a major compaction operation through nodetool.
> After invoking the compaction on a single column family we can see from
> JConsole that the LiveSSTableCount is going from 15 to 3 while the
> LiveDiskSpaceUsed is going from 90GB to 50GB.
> The problem now is that the space on the file system is been taken from
> Cassandra (I assumed from the old SSTable) and it isn't freed. We have tried
> to perform a full GC from the JConsole as described in
> http://wiki.apache.org/cassandra/MemtableSSTable without any success. The
> space is freed only after a database restart.
>
> How can we free this disk space without restart the db?
>
> Thanks you very much,
> Roberto Bentivoglio
>


Re: Problem about freeing space after a major compaction

2011-03-28 Thread Ching-Cheng Chen
AFAIK, setting gc_grace_period to 0 shouldn't cause this issue.   In fact,
that what I'm using now in a single node environment like yours.

However, I'm using 0.7.2 with some patches.   If you are still using 0.7.0,
most likely you got hit with this bug.
You might want to patch it or upgrade to latest release.

https://issues.apache.org/jira/browse/CASSANDRA-2059

Regards,

<https://issues.apache.org/jira/browse/CASSANDRA-2059>

Chen

Senior Developer, EvidentSoftware(Leaders in Monitoring of NoSQL & JAVA )

http://www.evidentsoftware.com

On Mon, Mar 28, 2011 at 11:04 AM, Roberto Bentivoglio <
roberto.bentivog...@gmail.com> wrote:

> Hi Chen,
> we've set the gc grace period of the column families to 0 as suggest in a
> single node enviroment.
> Can this setting cause the problem? I don't think so...
>
> Thanks,
> Roberto
>
> On 28 March 2011 16:54, Ching-Cheng Chen wrote:
>
>> tombstones removal also depends on your gc grace period setting.
>>
>> If you are pretty sure that you have proper gc grace period set and still
>> on 0.7.0, then probably related to this bug.
>>
>> https://issues.apache.org/jira/browse/CASSANDRA-2059
>>
>> Regards,
>>
>> <https://issues.apache.org/jira/browse/CASSANDRA-2059>
>>
>> Chen
>>
>> Senior Developer, EvidentSoftware(Leaders in Monitoring of NoSQL & JAVA )
>>
>> http://www.evidentsoftware.com
>>
>> On Mon, Mar 28, 2011 at 10:40 AM, Roberto Bentivoglio <
>> roberto.bentivog...@gmail.com> wrote:
>>
>>> Hi all,
>>> we're working on a Cassandra 0.7.0 production enviroment with a store of
>>> data near to 500 GB.
>>> We need to periodically remove the tombstones from deleted/expired data
>>> performing a major compaction operation through nodetool.
>>> After invoking the compaction on a single column family we can see from
>>> JConsole that the LiveSSTableCount is going from 15 to 3 while the
>>> LiveDiskSpaceUsed is going from 90GB to 50GB.
>>> The problem now is that the space on the file system is been taken from
>>> Cassandra (I assumed from the old SSTable) and it isn't freed. We have tried
>>> to perform a full GC from the JConsole as described in
>>> http://wiki.apache.org/cassandra/MemtableSSTable without any success.
>>> The space is freed only after a database restart.
>>>
>>> How can we free this disk space without restart the db?
>>>
>>> Thanks you very much,
>>> Roberto Bentivoglio
>>>
>>
>>
>


Secondary index query return row should not return?

2010-11-11 Thread Ching-Cheng Chen
Not sure if this the intended behavior of the indexed query.

I created a column family and added index on column A,B,C.

Now I insert three rows.

row1 : A=123, B=456, C=789
row2 : A=123, C=789
row3 : A=123, B=789, C=789

Now if I perform an indexed query for A=123 and B=456, both row1 and row2
are returned.

Is this the expected behavior?  Since row2 doesn't have column B, I would
expect B=456 expression should return false.

I'm using cassandra-0.7.0-beta3 and hector-0.7.0-19_11042010

Regards,

Chen