On Mon, Jul 21, 2014 at 8:07 AM, Bhaskar Singhal
wrote:
> I have not seen the issue after changing the commit log segment size to
> 1024MB.
>
Yes... your insanely over-huge commitlog will be contained in fewer files
if you increase the size of segments that will not make it any less of
an in
I have not seen the issue after changing the commit log segment size to 1024MB.
tpstats output:
Pool Name Active Pending Completed Blocked All
time blocked
ReadStage 0 0 0 0
0
RequestResponseStage
On Mon, Jul 7, 2014 at 9:30 PM, Bhaskar Singhal
wrote:
> I am using Cassandra 2.0.7 (with default settings and 16GB heap on quad
> core ubuntu server with 32gb ram)
>
16GB of heap will lead to significant GC pauses, and probably will not
improve total performance versus 8gb heap.
I continue to
Well with 4k maximum open files that still looks to be your culprit :)
I suggest you increase the size of your CL segments; the default is 32Mb,
and this is probably too small for the size of record you are writing. I
suspect that a 'too many open files' exception is crashing a flush which
then ca
Yes, I am.
lsof lists around 9000 open file handles.. and there were around 3000 commitlog
segments.
On Thursday, 17 July 2014 1:24 PM, Benedict Elliott Smith
wrote:
Are you still seeing the same exceptions about too many open files?
On Thu, Jul 17, 2014 at 6:28 AM, Bhaskar Singhal
Are you still seeing the same exceptions about too many open files?
On Thu, Jul 17, 2014 at 6:28 AM, Bhaskar Singhal
wrote:
> Even after changing ulimits and moving to the recommended production
> settings, we are still seeing the same issue.
>
> root@lnx148-76:~# cat /proc/17663/limits
> Lim
Even after changing ulimits and moving to the recommended production settings,
we are still seeing the same issue.
root@lnx148-76:~# cat /proc/17663/limits
Limit Soft Limit Hard Limit Units
Max cpu time unlimited unlimited
On Tue, Jul 8, 2014 at 10:17 AM, Bhaskar Singhal
wrote:
> But I am wondering why does Cassandra need to keep 3000+ commit log
> segment files open?
>
Because you are writing faster than you can flush to disk.
=Rob
00secs) but after ingesting
> around 120GB data, I start getting the following error:
> Operation [70668] retried 10 times - error inserting key 0070668
> ((TTransportException): java.net.SocketException: Broken pipe)
>
> The cassandra server is still running but in the system.log
ter ingesting
>around 120GB data, I start getting the following error:
>Operation [70668] retried 10 times - error inserting key 0070668
>((TTransportException): java.net.SocketException: Broken pipe)
>
>
>
>The cassandra server is still running but in the system.log I see
while(1600secs) but after ingesting
> around 120GB data, I start getting the following error:
> Operation [70668] retried 10 times - error inserting key 0070668
> ((TTransportException): java.net.SocketException: Broken pipe)
>
> The cassandra server is still running but in the syste
[70668] retried 10 times - error inserting key 0070668
((TTransportException): java.net.SocketException: Broken pipe)
The cassandra server is still running but in the system.log I see the below
mentioned errors.
ERROR [COMMIT-LOG-ALLOCATOR] 2014-07-07 22:39:23,617 CassandraDaemon.java (line
198
12 matches
Mail list logo