You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Bhaskar Singhal <bh...@yahoo.com> on 2014/07/08 06:30:03 UTC

TTransportException (java.net.SocketException: Broken pipe)

Hi,


I am using Cassandra 2.0.7 (with default settings and 16GB heap on quad core ubuntu server with 32gb ram) and trying to ingest 1MB values using cassandra-stress. It works fine for a 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 system.log I see the below mentioned errors.


ERROR [COMMIT-LOG-ALLOCATOR] 2014-07-07 22:39:23,617 CassandraDaemon.java (line 198) Exception in thread Thread[COMMIT-LOG-ALLOCATOR,5,main]
java.lang.NoClassDefFoundError: org/apache/cassandra/db/commitlog/CommitLog$4
        at org.apache.cassandra.db.commitlog.CommitLog.handleCommitError(CommitLog.java:374)
        at org.apache.cassandra.db.commitlog.CommitLogAllocator$1.runMayThrow(CommitLogAllocator.java:116)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at java.lang.Thread.run(Thread.java:744)
Caused by: java.lang.ClassNotFoundException: org.apache.cassandra.db.commitlog.CommitLog$4
        at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
        at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
        at java.security.AccessController.doPrivileged(Native Method)
        at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
        at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
        at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
        ... 4 more
Caused by: java.io.FileNotFoundException: /path/2.0.7/cassandra/build/classes/main/org/apache/cassandra/db/commitlog/CommitLog$4.class (Too many open files)
        at java.io.FileInputStream.open(Native Method)
        at java.io.FileInputStream.<init>(FileInputStream.java:146)
        at sun.misc.URLClassPath$FileLoader$1.getInputStream(URLClassPath.java:1086)
        at sun.misc.Resource.cachedInputStream(Resource.java:77)
        at sun.misc.Resource.getByteBuffer(Resource.java:160)
        at java.net.URLClassLoader.defineClass(URLClassLoader.java:436)
        at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
        at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
        ... 10 more
ERROR [FlushWriter:7] 2014-07-07 22:39:24,924 CassandraDaemon.java (line 198) Exception in thread Thread[FlushWriter:7,5,main]
FSWriteError in /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db
        at org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:475)
        at org.apache.cassandra.io.util.FileUtils.closeQuietly(FileUtils.java:212)
        at org.apache.cassandra.io.sstable.SSTableWriter.abort(SSTableWriter.java:301)
        at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:417)
        at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
        at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:744)
Caused by: java.io.FileNotFoundException: /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db (Too many open files)
        at java.io.FileOutputStream.open(Native Method)
        at java.io.FileOutputStream.<init>(FileOutputStream.java:221)
        at java.io.FileOutputStream.<init>(FileOutputStream.java:110)
        at org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:466)
        ... 9 more


There are around 9685 open files by the Cassandra server process (using lsof), 3938 commit log segments in /cassandra/commitlog and around 572 commit log segments deleted during the course of the test.

I am wondering what is causing Cassandra to open so many files, is the flushing slow? or something else?

I tried increasing the flush writers, but that didn't help. 



Regards,
Bhaskar


CREATE KEYSPACE "Keyspace1" WITH replication = {
  'class': 'SimpleStrategy',
  'replication_factor': '1'
};

CREATE TABLE "Standard1" (
  key blob,
  "C0" blob,
  PRIMARY KEY (key)
) WITH COMPACT STORAGE AND
  bloom_filter_fp_chance=0.010000 AND
  caching='KEYS_ONLY' AND
  comment='' AND
  dclocal_read_repair_chance=0.000000 AND
  gc_grace_seconds=864000 AND
  index_interval=128 AND
  read_repair_chance=0.100000 AND
  replicate_on_write='true' AND
  populate_io_cache_on_flush='false' AND
  default_time_to_live=0 AND
  speculative_retry='NONE' AND
  memtable_flush_period_in_ms=0 AND
  compaction={'class': 'SizeTieredCompactionStrategy'} AND
  compression={};

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Robert Coli <rc...@eventbrite.com>.
On Mon, Jul 21, 2014 at 8:07 AM, Bhaskar Singhal <bh...@yahoo.com>
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 insanely over-huge commitlog which indicates systemic failure in your
application's use of Cassandra. Congratulations on masking your actual
issue with your configuration change.

"
Pool Name                    Active   Pending      Completed   Blocked  All
time blocked
FlushWriter                       6        10          12442
1              2940
"

1/4 attempts to flush blocked waiting for resources, and you have 6 actives
flushes and 10 pending, because YOU'RE WRITING TOO FAST.

As a meta aside, I am unlikely to respond to further questions of yours
which do not engage with what I have now told you three or four times, that
YOU'RE WRITING TOO FAST.

=Rob

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Bhaskar Singhal <bh...@yahoo.com>.
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              0         0              0         0                 0
MutationStage                    32        40        2526143         0                 0
ReadRepairStage                   0         0              0         0                 0
ReplicateOnWriteStage             0         0              0         0                 0
GossipStage                       0         0              0         0                 0
AntiEntropyStage                  0         0              0         0                 0
MigrationStage                    0         0              3         0                 0
MemoryMeter                       0         0          24752         0                 0
MemtablePostFlusher               1        19          12939         0                 0
FlushWriter                       6        10          12442         1              2940
MiscStage                         0         0              0         0                 0
PendingRangeCalculator            0         0              1         0                 0
commitlog_archiver                0         0              0         0                 0
InternalResponseStage             0         0              0         0                 0
HintedHandoff                     0         0              0         0                 0

Message type           Dropped
RANGE_SLICE                  0
READ_REPAIR                  0
PAGED_RANGE                  0
BINARY                       0
READ                         0
MUTATION                     0
_TRACE                       0
REQUEST_RESPONSE             0
COUNTER_MUTATION             0




On Saturday, 19 July 2014 1:32 AM, Robert Coli <rc...@eventbrite.com> wrote:
 


On Mon, Jul 7, 2014 at 9:30 PM, Bhaskar Singhal <bh...@yahoo.com> 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 maintain that your "problem" is that you are writing faster than you can flush.

Paste the output of "nodetool tpstats"?

=Rob

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Robert Coli <rc...@eventbrite.com>.
On Mon, Jul 7, 2014 at 9:30 PM, Bhaskar Singhal <bh...@yahoo.com>
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 maintain that your "problem" is that you are writing faster
than you can flush.

Paste the output of "nodetool tpstats"?

=Rob

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Ruchir Jha <ru...@gmail.com>.
We have these precise settings but are still seeing the broken pipe exception in our gc logs. Any clues?

Sent from my iPhone

> On Jul 8, 2014, at 1:17 PM, Bhaskar Singhal <bh...@yahoo.com> wrote:
> 
> Thanks Mark. Yes the 1024 is the limit. I haven't changed it as per the recommended production settings.
> 
> But I am wondering why does Cassandra need to keep 3000+ commit log segment files open?
> 
> Regards,
> Bhaskar
> 
> 
> On Tuesday, 8 July 2014 1:50 PM, Mark Reddy <ma...@boxever.com> wrote:
> 
> 
> Hi Bhaskar,
> 
> Can you check your limits using 'ulimit -a'? The default is 1024, which needs to be increased if you have not done so already.
> 
> Here you will find a list of recommended production settings: http://www.datastax.com/documentation/cassandra/2.0/cassandra/install/installRecommendSettings.html
> 
> 
> Mark
> 
> On Tue, Jul 8, 2014 at 5:30 AM, Bhaskar Singhal <bh...@yahoo.com> wrote:
> Hi,
> 
> I am using Cassandra 2.0.7 (with default settings and 16GB heap on quad core ubuntu server with 32gb ram) and trying to ingest 1MB values using cassandra-stress. It works fine for a 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 system.log I see the below mentioned errors.
> 
> ERROR [COMMIT-LOG-ALLOCATOR] 2014-07-07 22:39:23,617 CassandraDaemon.java (line 198) Exception in thread Thread[COMMIT-LOG-ALLOCATOR,5,main]
> java.lang.NoClassDefFoundError: org/apache/cassandra/db/commitlog/CommitLog$4
>         at org.apache.cassandra.db.commitlog.CommitLog.handleCommitError(CommitLog.java:374)
>         at org.apache.cassandra.db.commitlog.CommitLogAllocator$1.runMayThrow(CommitLogAllocator.java:116)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>         at java.lang.Thread.run(Thread.java:744)
> Caused by: java.lang.ClassNotFoundException: org.apache.cassandra.db.commitlog.CommitLog$4
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>         at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>         ... 4 more
> Caused by: java.io.FileNotFoundException: /path/2.0.7/cassandra/build/classes/main/org/apache/cassandra/db/commitlog/CommitLog$4.class (Too many open files)
>         at java.io.FileInputStream.open(Native Method)
>         at java.io.FileInputStream.<init>(FileInputStream.java:146)
>         at sun.misc.URLClassPath$FileLoader$1.getInputStream(URLClassPath.java:1086)
>         at sun.misc.Resource.cachedInputStream(Resource.java:77)
>         at sun.misc.Resource.getByteBuffer(Resource.java:160)
>         at java.net.URLClassLoader.defineClass(URLClassLoader.java:436)
>         at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
>         ... 10 more
> ERROR [FlushWriter:7] 2014-07-07 22:39:24,924 CassandraDaemon.java (line 198) Exception in thread Thread[FlushWriter:7,5,main]
> FSWriteError in /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db
>         at org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:475)
>         at org.apache.cassandra.io.util.FileUtils.closeQuietly(FileUtils.java:212)
>         at org.apache.cassandra.io.sstable.SSTableWriter.abort(SSTableWriter.java:301)
>         at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:417)
>         at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
>         at org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:744)
> Caused by: java.io.FileNotFoundException: /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db (Too many open files)
>         at java.io.FileOutputStream.open(Native Method)
>         at java.io.FileOutputStream.<init>(FileOutputStream.java:221)
>         at java.io.FileOutputStream.<init>(FileOutputStream.java:110)
>         at org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:466)
>         ... 9 more
> 
> There are around 9685 open files by the Cassandra server process (using lsof), 3938 commit log segments in /cassandra/commitlog and around 572 commit log segments deleted during the course of the test.
> 
> I am wondering what is causing Cassandra to open so many files, is the flushing slow? or something else?
> 
> I tried increasing the flush writers, but that didn't help. 
> 
> 
> Regards,
> Bhaskar
> 
> 
> CREATE KEYSPACE "Keyspace1" WITH replication = {
>   'class': 'SimpleStrategy',
>   'replication_factor': '1'
> };
> 
> CREATE TABLE "Standard1" (
>   key blob,
>   "C0" blob,
>   PRIMARY KEY (key)
> ) WITH COMPACT STORAGE AND
>   bloom_filter_fp_chance=0.010000 AND
>   caching='KEYS_ONLY' AND
>   comment='' AND
>   dclocal_read_repair_chance=0.000000 AND
>   gc_grace_seconds=864000 AND
>   index_interval=128 AND
>   read_repair_chance=0.100000 AND
>   replicate_on_write='true' AND
>   populate_io_cache_on_flush='false' AND
>   default_time_to_live=0 AND
>   speculative_retry='NONE' AND
>   memtable_flush_period_in_ms=0 AND
>   compaction={'class': 'SizeTieredCompactionStrategy'} AND
>   compression={};
> 
> 
> 
> 

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Benedict Elliott Smith <be...@datastax.com>.
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 causes the application to get into an inconsistent state wrt commit
log reclaim.


On Thu, Jul 17, 2014 at 10:42 AM, Bhaskar Singhal <bh...@yahoo.com>
wrote:

> 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 <
> belliottsmith@datastax.com> wrote:
>
>
> Are you still seeing the same exceptions about too many open files?
>
>
>
>
> On Thu, Jul 17, 2014 at 6:28 AM, Bhaskar Singhal <bhaskarsinghal@yahoo.com
> > 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
> Limit                     Soft Limit           Hard Limit           Units
> Max cpu time              unlimited            unlimited            seconds
> Max file size             unlimited            unlimited            bytes
> Max data size             unlimited            unlimited            bytes
> Max stack size            8388608              unlimited            bytes
> Max core file size        0                    unlimited            bytes
> Max resident set          unlimited            unlimited            bytes
> Max processes             256502               256502
> processes
> Max open files            4096                 4096                 files
> Max locked memory         65536                65536                bytes
> Max address space         unlimited            unlimited            bytes
> Max file locks            unlimited            unlimited            locks
> Max pending signals       256502               256502               signals
> Max msgqueue size         819200               819200               bytes
> Max nice priority         0                    0
> Max realtime priority     0                    0
> Max realtime timeout      unlimited            unlimited            us
>
>
> Regards,
> Bhaskar
>
>
>   On Thursday, 10 July 2014 12:09 AM, Robert Coli <rc...@eventbrite.com>
> wrote:
>
>
>  On Tue, Jul 8, 2014 at 10:17 AM, Bhaskar Singhal <
> bhaskarsinghal@yahoo.com> 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
>
>
>
>
>
>
>

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Bhaskar Singhal <bh...@yahoo.com>.
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 <be...@datastax.com> wrote:
 


Are you still seeing the same exceptions about too many open files?





On Thu, Jul 17, 2014 at 6:28 AM, Bhaskar Singhal <bh...@yahoo.com> 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
>Limit                     Soft Limit           Hard Limit           Units
>Max cpu time              unlimited            unlimited            seconds
>Max file size             unlimited            unlimited            bytes
>Max data size            
 unlimited            unlimited            bytes
>Max stack size            8388608              unlimited            bytes
>Max core file size        0                    unlimited            bytes
>Max resident set          unlimited            unlimited            bytes
>Max processes             256502               256502               processes
>Max open files            4096                 4096                 files
>Max locked memory         65536                65536                bytes
>Max address
 space         unlimited            unlimited            bytes
>Max file locks            unlimited            unlimited            locks
>Max pending signals       256502               256502               signals
>Max msgqueue size         819200              
 819200               bytes
>Max nice priority         0                    0
>Max realtime priority     0                    0
>Max realtime timeout      unlimited            unlimited            us
>
>
>
>
>Regards,
>Bhaskar
>
>
>
>
>On Thursday, 10 July 2014 12:09 AM, Robert Coli <rc...@eventbrite.com> wrote:
> 
>
>
>On Tue, Jul 8, 2014 at 10:17 AM, Bhaskar Singhal <bh...@yahoo.com> 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
> 
>
>

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Benedict Elliott Smith <be...@datastax.com>.
Are you still seeing the same exceptions about too many open files?




On Thu, Jul 17, 2014 at 6:28 AM, Bhaskar Singhal <bh...@yahoo.com>
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
> Limit                     Soft Limit           Hard Limit           Units
> Max cpu time              unlimited            unlimited            seconds
> Max file size             unlimited            unlimited            bytes
> Max data size             unlimited            unlimited            bytes
> Max stack size            8388608              unlimited            bytes
> Max core file size        0                    unlimited            bytes
> Max resident set          unlimited            unlimited            bytes
> Max processes             256502               256502
> processes
> Max open files            4096                 4096                 files
> Max locked memory         65536                65536                bytes
> Max address space         unlimited            unlimited            bytes
> Max file locks            unlimited            unlimited            locks
> Max pending signals       256502               256502               signals
> Max msgqueue size         819200               819200               bytes
> Max nice priority         0                    0
> Max realtime priority     0                    0
> Max realtime timeout      unlimited            unlimited            us
>
>
> Regards,
> Bhaskar
>
>
>   On Thursday, 10 July 2014 12:09 AM, Robert Coli <rc...@eventbrite.com>
> wrote:
>
>
> On Tue, Jul 8, 2014 at 10:17 AM, Bhaskar Singhal <bhaskarsinghal@yahoo.com
> > 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
>
>
>
>

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Bhaskar Singhal <bh...@yahoo.com>.
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            seconds
Max file size             unlimited            unlimited            bytes
Max data size             unlimited            unlimited            bytes
Max stack size            8388608              unlimited            bytes
Max core file size        0                    unlimited            bytes
Max resident set          unlimited            unlimited            bytes
Max processes             256502               256502               processes
Max open files            4096                 4096                 files
Max locked memory         65536                65536                bytes
Max address space         unlimited            unlimited            bytes
Max file locks            unlimited            unlimited            locks
Max pending signals       256502               256502               signals
Max msgqueue size         819200               819200               bytes
Max nice priority         0                    0
Max realtime priority     0                    0
Max realtime timeout      unlimited            unlimited            us



Regards,
Bhaskar



On Thursday, 10 July 2014 12:09 AM, Robert Coli <rc...@eventbrite.com> wrote:
 


On Tue, Jul 8, 2014 at 10:17 AM, Bhaskar Singhal <bh...@yahoo.com> 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

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Robert Coli <rc...@eventbrite.com>.
On Tue, Jul 8, 2014 at 10:17 AM, Bhaskar Singhal <bh...@yahoo.com>
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

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Bhaskar Singhal <bh...@yahoo.com>.
Thanks Mark. Yes the 1024 is the limit. I haven't changed it as per the recommended production settings.

But I am wondering why does Cassandra need to keep 3000+ commit log segment files open?

Regards,
Bhaskar



On Tuesday, 8 July 2014 1:50 PM, Mark Reddy <ma...@boxever.com> wrote:
 


Hi Bhaskar,

Can you check your limits using 'ulimit -a'? The default is 1024, which needs to be increased if you have not done so already.

Here you will find a list of recommended production settings: http://www.datastax.com/documentation/cassandra/2.0/cassandra/install/installRecommendSettings.html


Mark


On Tue, Jul 8, 2014 at 5:30 AM, Bhaskar Singhal <bh...@yahoo.com> wrote:

Hi,
>
>
>
>I am using Cassandra 2.0.7 (with default settings and 16GB heap on quad core ubuntu server with 32gb ram) and trying to ingest 1MB values using cassandra-stress. It works fine for a 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 system.log I see the below mentioned errors.
>
>
>
>ERROR [COMMIT-LOG-ALLOCATOR] 2014-07-07 22:39:23,617 CassandraDaemon.java (line 198) Exception in thread Thread[COMMIT-LOG-ALLOCATOR,5,main]
>java.lang.NoClassDefFoundError: org/apache/cassandra/db/commitlog/CommitLog$4
>        at org.apache.cassandra.db.commitlog.CommitLog.handleCommitError(CommitLog.java:374)
>        at org.apache.cassandra.db.commitlog.CommitLogAllocator$1.runMayThrow(CommitLogAllocator.java:116)
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>        at java.lang.Thread.run(Thread.java:744)
>Caused by: java.lang.ClassNotFoundException: org.apache.cassandra.db.commitlog.CommitLog$4
>        at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
>        at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>        at java.security.AccessController.doPrivileged(Native Method)
>        at
 java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>        at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>        at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
>        at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>        ... 4 more
>Caused by: java.io.FileNotFoundException: /path/2.0.7/cassandra/build/classes/main/org/apache/cassandra/db/commitlog/CommitLog$4.class (Too many open files)
>        at java.io.FileInputStream.open(Native Method)
>        at java.io.FileInputStream.<init>(FileInputStream.java:146)
>        at sun.misc.URLClassPath$FileLoader$1.getInputStream(URLClassPath.java:1086)
>        at sun.misc.Resource.cachedInputStream(Resource.java:77)
>        at sun.misc.Resource.getByteBuffer(Resource.java:160)
>        at java.net.URLClassLoader.defineClass(URLClassLoader.java:436)
>        at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
>        at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
>        ... 10 more
>ERROR [FlushWriter:7] 2014-07-07 22:39:24,924 CassandraDaemon.java (line 198) Exception in thread
 Thread[FlushWriter:7,5,main]
>FSWriteError in /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db
>        at org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:475)
>        at org.apache.cassandra.io.util.FileUtils.closeQuietly(FileUtils.java:212)
>        at org.apache.cassandra.io.sstable.SSTableWriter.abort(SSTableWriter.java:301)
>        at org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:417)
>        at org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
>        at
 org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>        at java.lang.Thread.run(Thread.java:744)
>Caused by: java.io.FileNotFoundException: /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db (Too many open files)
>        at java.io.FileOutputStream.open(Native Method)
>        at
 java.io.FileOutputStream.<init>(FileOutputStream.java:221)
>        at java.io.FileOutputStream.<init>(FileOutputStream.java:110)
>        at org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:466)
>        ... 9 more
>
>
>
>There are around 9685 open files by the Cassandra server process (using lsof), 3938 commit log segments in /cassandra/commitlog and around 572 commit log segments deleted during the course of the test.
>
>
>I am wondering what is causing Cassandra to open so many files, is the flushing slow? or something else?
>
>
>I tried increasing the flush writers, but that didn't help. 
>
>
>
>
>
>Regards,
>Bhaskar
>
>
>
>CREATE KEYSPACE "Keyspace1" WITH replication = {
>  'class': 'SimpleStrategy',
>  'replication_factor': '1'
>};
>
>CREATE TABLE "Standard1" (
>  key blob,
>  "C0" blob,
>  PRIMARY KEY (key)
>) WITH COMPACT STORAGE AND
>  bloom_filter_fp_chance=0.010000 AND
>  caching='KEYS_ONLY' AND
>  comment='' AND
>  dclocal_read_repair_chance=0.000000 AND
>  gc_grace_seconds=864000 AND
>  index_interval=128 AND
>  read_repair_chance=0.100000 AND
>  replicate_on_write='true' AND
>  populate_io_cache_on_flush='false' AND
>  default_time_to_live=0 AND
> 
 speculative_retry='NONE' AND
>  memtable_flush_period_in_ms=0 AND
>  compaction={'class': 'SizeTieredCompactionStrategy'} AND
>  compression={};
>
>

Re: TTransportException (java.net.SocketException: Broken pipe)

Posted by Mark Reddy <ma...@boxever.com>.
Hi Bhaskar,

Can you check your limits using 'ulimit -a'? The default is 1024, which
needs to be increased if you have not done so already.

Here you will find a list of recommended production settings:
http://www.datastax.com/documentation/cassandra/2.0/cassandra/install/installRecommendSettings.html


Mark

On Tue, Jul 8, 2014 at 5:30 AM, Bhaskar Singhal <bh...@yahoo.com>
wrote:

> Hi,
>
> I am using Cassandra 2.0.7 (with default settings and 16GB heap on quad
> core ubuntu server with 32gb ram) and trying to ingest 1MB values using
> cassandra-stress. It works fine for a 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 system.log I see the
> below mentioned errors.
>
> ERROR [COMMIT-LOG-ALLOCATOR] 2014-07-07 22:39:23,617 CassandraDaemon.java
> (line 198) Exception in thread Thread[COMMIT-LOG-ALLOCATOR,5,main]
> java.lang.NoClassDefFoundError:
> org/apache/cassandra/db/commitlog/CommitLog$4
>         at
> org.apache.cassandra.db.commitlog.CommitLog.handleCommitError(CommitLog.java:374)
>         at
> org.apache.cassandra.db.commitlog.CommitLogAllocator$1.runMayThrow(CommitLogAllocator.java:116)
>         at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>         at java.lang.Thread.run(Thread.java:744)
> Caused by: java.lang.ClassNotFoundException:
> org.apache.cassandra.db.commitlog.CommitLog$4
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:363)
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
>         at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
>         at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
>         ... 4 more
> Caused by: java.io.FileNotFoundException:
> /path/2.0.7/cassandra/build/classes/main/org/apache/cassandra/db/commitlog/CommitLog$4.class
> (Too many open files)
>         at java.io.FileInputStream.open(Native Method)
>         at java.io.FileInputStream.<init>(FileInputStream.java:146)
>         at
> sun.misc.URLClassPath$FileLoader$1.getInputStream(URLClassPath.java:1086)
>         at sun.misc.Resource.cachedInputStream(Resource.java:77)
>         at sun.misc.Resource.getByteBuffer(Resource.java:160)
>         at java.net.URLClassLoader.defineClass(URLClassLoader.java:436)
>         at java.net.URLClassLoader.access$100(URLClassLoader.java:71)
>         at java.net.URLClassLoader$1.run(URLClassLoader.java:361)
>         ... 10 more
> ERROR [FlushWriter:7] 2014-07-07 22:39:24,924 CassandraDaemon.java (line
> 198) Exception in thread Thread[FlushWriter:7,5,main]
> FSWriteError in
> /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db
>         at
> org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:475)
>         at
> org.apache.cassandra.io.util.FileUtils.closeQuietly(FileUtils.java:212)
>         at
> org.apache.cassandra.io.sstable.SSTableWriter.abort(SSTableWriter.java:301)
>         at
> org.apache.cassandra.db.Memtable$FlushRunnable.writeSortedContents(Memtable.java:417)
>         at
> org.apache.cassandra.db.Memtable$FlushRunnable.runWith(Memtable.java:350)
>         at
> org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48)
>         at
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28)
>         at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:744)
> Caused by: java.io.FileNotFoundException:
> /cassandra/data4/Keyspace1/Standard1/Keyspace1-Standard1-tmp-jb-593-Filter.db
> (Too many open files)
>         at java.io.FileOutputStream.open(Native Method)
>         at java.io.FileOutputStream.<init>(FileOutputStream.java:221)
>         at java.io.FileOutputStream.<init>(FileOutputStream.java:110)
>         at
> org.apache.cassandra.io.sstable.SSTableWriter$IndexWriter.close(SSTableWriter.java:466)
>         ... 9 more
>
> There are around 9685 open files by the Cassandra server process (using
> lsof), 3938 commit log segments in /cassandra/commitlog and around 572
> commit log segments deleted during the course of the test.
>
> I am wondering what is causing Cassandra to open so many files, is the
> flushing slow? or something else?
>
> I tried increasing the flush writers, but that didn't help.
>
>
> Regards,
> Bhaskar
>
>
> CREATE KEYSPACE "Keyspace1" WITH replication = {
>   'class': 'SimpleStrategy',
>   'replication_factor': '1'
> };
>
> CREATE TABLE "Standard1" (
>   key blob,
>   "C0" blob,
>   PRIMARY KEY (key)
> ) WITH COMPACT STORAGE AND
>   bloom_filter_fp_chance=0.010000 AND
>   caching='KEYS_ONLY' AND
>   comment='' AND
>   dclocal_read_repair_chance=0.000000 AND
>   gc_grace_seconds=864000 AND
>   index_interval=128 AND
>   read_repair_chance=0.100000 AND
>   replicate_on_write='true' AND
>   populate_io_cache_on_flush='false' AND
>   default_time_to_live=0 AND
>   speculative_retry='NONE' AND
>   memtable_flush_period_in_ms=0 AND
>   compaction={'class': 'SizeTieredCompactionStrategy'} AND
>   compression={};
>
>