You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Aaron Morton <aa...@thelastpickle.com> on 2010/09/29 06:25:53 UTC

Truncate + Snapshot + Cannot Allocate Memory == Timeout

Running on the current 0.7.0 beta2 #3. 

On my dev workstation (unubtu 10.4) I accidentally let cassandra have 4GB and free memory got down to <100 MB . I was running some tests that added a few (< 100) rows to the CF's in Keyspace1, then Truncated the CF's for the next tests. 

 I got the following error in the cassandra system.log 

DEBUG [pool-1-thread-44] 2010-09-29 16:44:39,166 CassandraServer.java (line 954) truncating Standard1 in Keyspace1
DEBUG [pool-1-thread-44] 2010-09-29 16:44:39,166 StorageProxy.java (line 759) Starting a blocking truncate operation on keyspace Keyspace1, CF 
DEBUG [pool-1-thread-44] 2010-09-29 16:44:39,166 StorageProxy.java (line 774) Starting to send truncate messages to hosts [localhost/127.0.0.1]
DEBUG [MUTATION_STAGE:25] 2010-09-29 16:44:39,166 TruncateVerbHandler.java (line 50) Applying Truncation(keyspace='Keyspace1', cf='Standard1')
DEBUG [MUTATION_STAGE:25] 2010-09-29 16:44:39,166 Table.java (line 630) Truncating...
 INFO [MUTATION_STAGE:25] 2010-09-29 16:44:39,166 ColumnFamilyStore.java (line 459) switching in a fresh Memtable for Standard1 at CommitLogContext(file='/local1/junkbox/cassandra/commitlog/CommitLog-1285728080403.log', position=10022050)
 INFO [MUTATION_STAGE:25] 2010-09-29 16:44:39,166 ColumnFamilyStore.java (line 771) Enqueuing flush of Memtable-Standard1@1701009673(40104 bytes, 1152 operations)
DEBUG [pool-1-thread-44] 2010-09-29 16:44:39,167 StorageProxy.java (line 780) Sent all truncate messages, now waiting for 1 responses
 INFO [FLUSH-WRITER-POOL:1] 2010-09-29 16:44:39,167 Memtable.java (line 150) Writing Memtable-Standard1@1701009673(40104 bytes, 1152 operations)
 INFO [FLUSH-WRITER-POOL:1] 2010-09-29 16:44:39,268 Memtable.java (line 157) Completed flushing /local1/junkbox/cassandra/data/Keyspace1/Standard1-e-10-Data.db
DEBUG [CompactionExecutor:1] 2010-09-29 16:44:39,268 CompactionManager.java (line 95) Checking to see if compaction of Standard1 would be useful
DEBUG [COMMIT-LOG-WRITER] 2010-09-29 16:44:39,268 CommitLog.java (line 422) discard completed log segments for CommitLogContext(file='/local1/junkbox/cassandra/commitlog/CommitLog-1285728080403.log', position=10022050), column family 1000.
DEBUG [COMMIT-LOG-WRITER] 2010-09-29 16:44:39,268 CommitLog.java (line 448) Marking replay position 10022050 on commit log CommitLogSegment(/local1/junkbox/cassandra/commitlog/CommitLog-1285728080403.log)
ERROR [MUTATION_STAGE:25] 2010-09-29 16:44:39,341 AbstractCassandraDaemon.java (line 88) Fatal exception in thread Thread[MUTATION_STAGE:25,5,main]
java.io.IOError: java.io.IOException: Cannot run program "ln": java.io.IOException: error=12, Cannot allocate memory
        at org.apache.cassandra.db.ColumnFamilyStore.snapshot(ColumnFamilyStore.java:1368)
        at org.apache.cassandradb.ColumnFamilyStore.truncate(ColumnFamilyStore.java:1511)
        at org.apache.cassandra.db.Table.truncate(Table.java:633)
        at org.apache.cassandra.db.TruncateVerbHandler.doVerb(TruncateVerbHandler.java:54)
        at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:50)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
        at java.lang.Thread.run(Thread.java:619)
Caused by: java.io.IOException: Cannot run program "ln": java.io.IOException: error=12, Cannot allocate memory
        at java.lang.ProcessBuilder.start(ProcessBuilder.java:460)
        at org.apache.cassandra.io.util.FileUtils.createHardLinkWithExec(FileUtils.java:263)
        at org.apache.cassandra.io.util.FileUtils.createHardLink(FileUtils.java:229)
        at org.apache.cassandra.db.ColumnFamilyStore.snapshot(ColumnFamilyStore.java:1360)
        ... 7 more
Caused by: java.io.IOException: java.io.IOException: error=12, Cannot allocate memory
        at java.langUNIXProcess.<init>(UNIXProcess.java:148)
        at java.langProcessImpl.start(ProcessImpl.java:65)
        at java.lang.ProcessBuilder.start(ProcessBuilder.java:453)
        ... 10 more
DEBUG [GC inspection] 2010-09-29 16:44:39,539 GCInspector.java (line 131) GC for ParNew: 43 ms, 29106728 reclaimed leaving 3567200784 used; max is 5498404864
DEBUG [pool-1-thread-44] 2010-09-29 16:44:49,189 ClientState.java (line 107) logged out: #<User allow_all groups=[]>

Fair enough, though it appeared as a timeout on the client (with a 10 second timeout) 

  File "/tech/home//git_home/trojan/trojan/cassandra/Cassandra.py", line 846, in truncate
    self.recv_truncate()
  File "/tech/home//git_home/trojan/trojan/cassandra/Cassandra.py", line 857, in recv_truncate
    (fname, mtype, rseqid) = self._iprot.readMessageBegin()
  File "/tech/home//git_home/trojan/trojan/thrift/protocol/TBinaryProtocol.py", line 126, in readMessageBegin
    sz = self.readI32()
<snip>
    chunk = self.read(sz-have)
  File "/tech/home//git_home/trojan/trojan/thrift/transport/TSocket.py", line 92, in read
    buff = self.handle.recv(sz)
timeout: timed out

Here's the output of ulimit -a 

core file size          (blocks, -c) 0
data seg size           (kbytes, -d) unlimited
scheduling priority             (-e) 20
file size               (blocks, -f) unlimited
pending signals                 (-i) 16382
max locked memory       (kbytes, -l) 64
max memory size         (kbytes, -m) unlimited
open files                      (-n) 65536
pipe size            (512 bytes, -p) 8
POSIX message queues     (bytes, -q) 819200
real-time priority              (-r) 0
stack size              (kbytes, -s) 8192
cpu time               (seconds, -t) unlimited
max user processes              (-u) unlimited
virtual memory          (kbytes, -v) unlimited
file locks                      (-x) unlimited


Two questions:

1) Is the memory error just a result of me letting my machine run stupidly low on memory?

2) Should it have returned an ApplicationError or some such in this case? The code in ColumnFamilyStore:1368 is catching the IOException from the call to FileUtils.createHardLink and wrapping it in an IOError. However the code in TruncateVerbHandler:56 is looking for the IOException. 

Cheers
Aaron


Re: Truncate + Snapshot + Cannot Allocate Memory == Timeout

Posted by aaron morton <aa...@thelastpickle.com>.
Created CASSANDRA-1557 for the error masking. 

Aaron

On 29 Sep 2010, at 18:19, Jonathan Ellis wrote:

> On Tue, Sep 28, 2010 at 11:25 PM, Aaron Morton <aa...@thelastpickle.com> wrote:
>> 1) Is the memory error just a result of me letting my machine run stupidly
>> low on memory?
> 
> No, it's the JVM forking to run ln.  Enable overcommit, or get JNA so
> it does the link w/ native code instead.
> 
>> 2) Should it have returned an ApplicationError or some such in this case?
>> The code in ColumnFamilyStore:1368 is catching the IOException from the call
>> to FileUtils.createHardLink and wrapping it in an IOError. However the code
>> in TruncateVerbHandler:56 is looking for the IOException.
> 
> That does sound like a bug.
> 
> -- 
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of Riptano, the source for professional Cassandra support
> http://riptano.com


Re: Truncate + Snapshot + Cannot Allocate Memory == Timeout

Posted by Jonathan Ellis <jb...@gmail.com>.
On Tue, Sep 28, 2010 at 11:25 PM, Aaron Morton <aa...@thelastpickle.com> wrote:
> 1) Is the memory error just a result of me letting my machine run stupidly
> low on memory?

No, it's the JVM forking to run ln.  Enable overcommit, or get JNA so
it does the link w/ native code instead.

> 2) Should it have returned an ApplicationError or some such in this case?
> The code in ColumnFamilyStore:1368 is catching the IOException from the call
> to FileUtils.createHardLink and wrapping it in an IOError. However the code
> in TruncateVerbHandler:56 is looking for the IOException.

That does sound like a bug.

-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of Riptano, the source for professional Cassandra support
http://riptano.com