You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by gabriele renzi <rf...@gmail.com> on 2010/05/07 10:47:39 UTC

timeout while running simple hadoop job

Hi everyone,

I am trying to develop a mapreduce job that does a simple
selection+filter on the rows in our store.
Of course it is mostly based on the WordCount example :)


Sadly, while it seems the app runs fine on a test keyspace with little
data, when run on a larger test index (but still on a single node) I
reliably see this error in the logs

10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001
java.lang.RuntimeException: TimedOutException()
        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165)
        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215)
        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97)
        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91)
        at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423)
        at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67)
        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143)
        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583)
        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
        at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176)
Caused by: TimedOutException()
        at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015)
        at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623)
        at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597)
        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142)
        ... 11 more

and after that the job seems to finish "normally" but no results are produced.

FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if
it ain't broke don't fix it).

The single node has a data directory of about 127GB in two column
families, off which the one used in the mapred job is about 100GB.
The cassandra server is run with 6GB of heap on a box with 8GB
available and no swap enabled. read/write latency from cfstat are

        Read Latency: 0.8535837762577986 ms.
        Write Latency: 0.028849603764075547 ms.

row cache is not enabled, key cache percentage is default. Load on the
machine is basically zero when the job is not running.

As my code is 99% that from the wordcount contrib, I shall notice that
In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we
can supposedly change, but it's apparently not used anywhere, but as I
said, running on a single node this should not be an issue anyway.

Does anyone has suggestions or has seen this error before? On the
other hand, did people run this kind of jobs in similar conditions
flawlessly, so I can consider it just my problem?


Thanks in advance for any help.

Re: timeout while running simple hadoop job

Posted by gabriele renzi <rf...@gmail.com>.
On Wed, May 12, 2010 at 5:46 PM, Johan Oskarsson <jo...@oskarsson.nu> wrote:
> Looking over the code this is in fact an issue in 0.6.
> It's fixed in trunk/0.7. Connections will be reused and closed properly, see https://issues.apache.org/jira/browse/CASSANDRA-1017 for more details.
>
> We can either backport that patch or make at least close the connections properly in 0.6. Can you open an ticket for this bug?

done as https://issues.apache.org/jira/browse/CASSANDRA-1081

Re: timeout while running simple hadoop job

Posted by Johan Oskarsson <jo...@oskarsson.nu>.
Looking over the code this is in fact an issue in 0.6. 
It's fixed in trunk/0.7. Connections will be reused and closed properly, see https://issues.apache.org/jira/browse/CASSANDRA-1017 for more details.

We can either backport that patch or make at least close the connections properly in 0.6. Can you open an ticket for this bug?

/Johan

On 12 maj 2010, at 12.11, gabriele renzi wrote:

> a follow up for anyone that may end up on this conversation again:
> 
> I kept trying and neither changing the number of concurrent map tasks,
> nor the slice size helped.
> Finally, I found out a screw up in our logging system,  which had
> forbidden us from noticing a couple of recurring errors in the logs :
> 
> ERROR [ROW-READ-STAGE:1] 2010-05-11 16:43:32,328
> DebuggableThreadPoolExecutor.java (line 101) Error in
> ThreadPoolExecutor
> java.lang.RuntimeException: java.lang.RuntimeException: corrupt sstable
>        at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:53)
>        at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:40)
>        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.lang.RuntimeException: corrupt sstable
>        at org.apache.cassandra.io.SSTableScanner.seekTo(SSTableScanner.java:73)
>        at org.apache.cassandra.db.ColumnFamilyStore.getKeyRange(ColumnFamilyStore.java:907)
>        at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1000)
>        at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:41)
>        ... 4 more
> Caused by: java.io.FileNotFoundException:
> /path/to/data/Keyspace/CF-123-Index.db (Too many open files)
>        at java.io.RandomAccessFile.open(Native Method)
>        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
>        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:98)
>        at org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:143)
>        at org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:138)
>        at org.apache.cassandra.io.SSTableReader.getNearestPosition(SSTableReader.java:414)
>        at org.apache.cassandra.io.SSTableScanner.seekTo(SSTableScanner.java:62)
>        ... 7 more
> 
> and the related
> 
> WARN [main] 2010-05-11 16:43:38,076 TThreadPoolServer.java (line 190)
> Transport error occurred during acceptance of message.
> org.apache.thrift.transport.TTransportException:
> java.net.SocketException: Too many open files
>        at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:124)
>        at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:35)
>        at org.apache.thrift.transport.TServerTransport.accept(TServerTransport.java:31)
>        at org.apache.thrift.server.TThreadPoolServer.serve(TThreadPoolServer.java:184)
>        at org.apache.cassandra.thrift.CassandraDaemon.start(CassandraDaemon.java:149)
>        at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:190)
> Caused by: java.net.SocketException: Too many open files
>        at java.net.PlainSocketImpl.socketAccept(Native Method)
>        at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390)
>        at java.net.ServerSocket.implAccept(ServerSocket.java:453)
>        at java.net.ServerSocket.accept(ServerSocket.java:421)
>        at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:119)
>        ... 5 more
> 
> The client was reporting timeouts in this case.
> 
> 
> The max fd limit on the process was in fact not exceedingly high
> (1024) and raising it seems to have solved the problem.
> 
> Anyway It still seems that there may be two issues:
> 
> - since we had never seen this error before with normal client
> connections (as in: non hadoop), is it possible that the
> Cassandra/hadoop layer is not closing sockets properly between one
> connection and the other, or not reusing connections efficiently?
> E.g. TSocket seems to have a close() method but I don't see it used in
> ColumnFamilyInputFormat.(getSubSplits, getRangeMap) but it may well be
> inside CassandraClient.
> 
> Anyway, judging by lsof's output I can only see about a hundred TCP
> connections, but those from the hadoop jobs seem to always be below 60
> so this may just be my wrong impression.
> 
> - is it possible that such errors show up on the client side as
> timeoutErrors when they could be reported better? this would probably
> help other people in diagnosing/reporting internal errors in the
> future.
> 
> 
> Thanks again to everyone with this, I promise I'll put the discussion
> on the wiki for future reference :)


Re: timeout while running simple hadoop job

Posted by Héctor Izquierdo <iz...@strands.com>.
Have you checked your open file handler limit? You can do that by using 
"ulimit" in the shell. If it's too low, you will encounter the "too many 
open files" error. You can also see how many open handlers an 
application has with "lsof".

Héctor Izquierdo

On 12/05/10 17:00, gabriele renzi wrote:
> On Wed, May 12, 2010 at 4:43 PM, Jonathan Ellis<jb...@gmail.com>  wrote:
>    
>> On Wed, May 12, 2010 at 5:11 AM, gabriele renzi<rf...@gmail.com>  wrote:
>>      
>>> - is it possible that such errors show up on the client side as
>>> timeoutErrors when they could be reported better?
>>>        
>> No, if the node the client is talking to doesn't get a reply from the
>> data node, there is no way for it to magically find out what happened
>> since ipso facto it got no reply.
>>      
> Sorry I was not clear: I meant the first error (where we get a
> RuntimeException in reading the file, not in the socket.accept()).
> There we have a reasonable error message (either "too many open files"
> or "corrupt sstable") that does not appear client side.
>
>
>
>    


Re: timeout while running simple hadoop job

Posted by gabriele renzi <rf...@gmail.com>.
On Wed, May 12, 2010 at 4:43 PM, Jonathan Ellis <jb...@gmail.com> wrote:
> On Wed, May 12, 2010 at 5:11 AM, gabriele renzi <rf...@gmail.com> wrote:
>> - is it possible that such errors show up on the client side as
>> timeoutErrors when they could be reported better?
>
> No, if the node the client is talking to doesn't get a reply from the
> data node, there is no way for it to magically find out what happened
> since ipso facto it got no reply.

Sorry I was not clear: I meant the first error (where we get a
RuntimeException in reading the file, not in the socket.accept()).
There we have a reasonable error message (either "too many open files"
or "corrupt sstable") that does not appear client side.



-- 
blog en: http://www.riffraff.info
blog it: http://riffraff.blogsome.com

Re: timeout while running simple hadoop job

Posted by Jonathan Ellis <jb...@gmail.com>.
On Wed, May 12, 2010 at 5:11 AM, gabriele renzi <rf...@gmail.com> wrote:
> - is it possible that such errors show up on the client side as
> timeoutErrors when they could be reported better?

No, if the node the client is talking to doesn't get a reply from the
data node, there is no way for it to magically find out what happened
since ipso facto it got no reply.

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

Re: timeout while running simple hadoop job

Posted by gabriele renzi <rf...@gmail.com>.
a follow up for anyone that may end up on this conversation again:

I kept trying and neither changing the number of concurrent map tasks,
nor the slice size helped.
Finally, I found out a screw up in our logging system,  which had
forbidden us from noticing a couple of recurring errors in the logs :

ERROR [ROW-READ-STAGE:1] 2010-05-11 16:43:32,328
DebuggableThreadPoolExecutor.java (line 101) Error in
ThreadPoolExecutor
java.lang.RuntimeException: java.lang.RuntimeException: corrupt sstable
        at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:53)
        at org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:40)
        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.lang.RuntimeException: corrupt sstable
        at org.apache.cassandra.io.SSTableScanner.seekTo(SSTableScanner.java:73)
        at org.apache.cassandra.db.ColumnFamilyStore.getKeyRange(ColumnFamilyStore.java:907)
        at org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1000)
        at org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:41)
        ... 4 more
Caused by: java.io.FileNotFoundException:
/path/to/data/Keyspace/CF-123-Index.db (Too many open files)
        at java.io.RandomAccessFile.open(Native Method)
        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:212)
        at java.io.RandomAccessFile.<init>(RandomAccessFile.java:98)
        at org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:143)
        at org.apache.cassandra.io.util.BufferedRandomAccessFile.<init>(BufferedRandomAccessFile.java:138)
        at org.apache.cassandra.io.SSTableReader.getNearestPosition(SSTableReader.java:414)
        at org.apache.cassandra.io.SSTableScanner.seekTo(SSTableScanner.java:62)
        ... 7 more

and the related

 WARN [main] 2010-05-11 16:43:38,076 TThreadPoolServer.java (line 190)
Transport error occurred during acceptance of message.
org.apache.thrift.transport.TTransportException:
java.net.SocketException: Too many open files
        at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:124)
        at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:35)
        at org.apache.thrift.transport.TServerTransport.accept(TServerTransport.java:31)
        at org.apache.thrift.server.TThreadPoolServer.serve(TThreadPoolServer.java:184)
        at org.apache.cassandra.thrift.CassandraDaemon.start(CassandraDaemon.java:149)
        at org.apache.cassandra.thrift.CassandraDaemon.main(CassandraDaemon.java:190)
Caused by: java.net.SocketException: Too many open files
        at java.net.PlainSocketImpl.socketAccept(Native Method)
        at java.net.PlainSocketImpl.accept(PlainSocketImpl.java:390)
        at java.net.ServerSocket.implAccept(ServerSocket.java:453)
        at java.net.ServerSocket.accept(ServerSocket.java:421)
        at org.apache.thrift.transport.TServerSocket.acceptImpl(TServerSocket.java:119)
        ... 5 more

The client was reporting timeouts in this case.


The max fd limit on the process was in fact not exceedingly high
(1024) and raising it seems to have solved the problem.

Anyway It still seems that there may be two issues:

- since we had never seen this error before with normal client
connections (as in: non hadoop), is it possible that the
Cassandra/hadoop layer is not closing sockets properly between one
connection and the other, or not reusing connections efficiently?
E.g. TSocket seems to have a close() method but I don't see it used in
ColumnFamilyInputFormat.(getSubSplits, getRangeMap) but it may well be
inside CassandraClient.

Anyway, judging by lsof's output I can only see about a hundred TCP
connections, but those from the hadoop jobs seem to always be below 60
so this may just be my wrong impression.

- is it possible that such errors show up on the client side as
timeoutErrors when they could be reported better? this would probably
help other people in diagnosing/reporting internal errors in the
future.


Thanks again to everyone with this, I promise I'll put the discussion
on the wiki for future reference :)

Re: timeout while running simple hadoop job

Posted by Joost Ouwerkerk <jo...@openplaces.org>.
The number of map tasks for a job is a function of the InputFormat,
which in the case of ColumnInputFormat is a function of the global
number of keys in Cassandra.  The number of concurrent maps being
executed at any given time per TaskTracker (per node) is set by
mapred.tasktracker.reduce.tasks.maximum.
j

On Fri, May 7, 2010 at 9:57 AM, Joseph Stein <cr...@gmail.com> wrote:
> you can manage the number of map tasks by node
>
> mapred.tasktracker.map.tasks.maximum=1
>
>
> On Fri, May 7, 2010 at 9:53 AM, gabriele renzi <rf...@gmail.com> wrote:
>> On Fri, May 7, 2010 at 2:44 PM, Jonathan Ellis <jb...@gmail.com> wrote:
>>> Sounds like you need to configure Hadoop to not create a whole bunch
>>> of Map tasks at once
>>
>> interesting, from a  quick check it seems there are a dozen threads running.
>> Yet , setNumMapTasks seems to be deprecated (together with JobConf)
>> and while I guess
>>   -Dmapred.map.tasks=N
>> may still work, it seems that  so it seems the only way to manage the
>> number of map tasks is via a custom subclass of
>> ColumnFamilyInputFormat.
>>
>> But of course you have a point that in a single box this does not add anything.
>>
>
>
>
> --
> /*
> Joe Stein
> http://www.linkedin.com/in/charmalloc
> */
>

Re: timeout while running simple hadoop job

Posted by Joseph Stein <cr...@gmail.com>.
you can manage the number of map tasks by node

mapred.tasktracker.map.tasks.maximum=1


On Fri, May 7, 2010 at 9:53 AM, gabriele renzi <rf...@gmail.com> wrote:
> On Fri, May 7, 2010 at 2:44 PM, Jonathan Ellis <jb...@gmail.com> wrote:
>> Sounds like you need to configure Hadoop to not create a whole bunch
>> of Map tasks at once
>
> interesting, from a  quick check it seems there are a dozen threads running.
> Yet , setNumMapTasks seems to be deprecated (together with JobConf)
> and while I guess
>   -Dmapred.map.tasks=N
> may still work, it seems that  so it seems the only way to manage the
> number of map tasks is via a custom subclass of
> ColumnFamilyInputFormat.
>
> But of course you have a point that in a single box this does not add anything.
>



-- 
/*
Joe Stein
http://www.linkedin.com/in/charmalloc
*/

Re: timeout while running simple hadoop job

Posted by gabriele renzi <rf...@gmail.com>.
On Fri, May 7, 2010 at 2:44 PM, Jonathan Ellis <jb...@gmail.com> wrote:
> Sounds like you need to configure Hadoop to not create a whole bunch
> of Map tasks at once

interesting, from a  quick check it seems there are a dozen threads running.
Yet , setNumMapTasks seems to be deprecated (together with JobConf)
and while I guess
   -Dmapred.map.tasks=N
may still work, it seems that  so it seems the only way to manage the
number of map tasks is via a custom subclass of
ColumnFamilyInputFormat.

But of course you have a point that in a single box this does not add anything.

Re: timeout while running simple hadoop job

Posted by Jonathan Ellis <jb...@gmail.com>.
The whole point is to parallelize to use the available capacity across
multiple machines.  If you go past that point (fairly easy when you
have a single machine) then you're just contending for resources, not
making things faster.

On Fri, May 7, 2010 at 7:48 AM, Joost Ouwerkerk <jo...@openplaces.org> wrote:
> Huh? Isn't that the whole point of using Map/Reduce?
>
> On Fri, May 7, 2010 at 8:44 AM, Jonathan Ellis <jb...@gmail.com> wrote:
>> Sounds like you need to configure Hadoop to not create a whole bunch
>> of Map tasks at once
>>
>> On Fri, May 7, 2010 at 3:47 AM, gabriele renzi <rf...@gmail.com> wrote:
>>> Hi everyone,
>>>
>>> I am trying to develop a mapreduce job that does a simple
>>> selection+filter on the rows in our store.
>>> Of course it is mostly based on the WordCount example :)
>>>
>>>
>>> Sadly, while it seems the app runs fine on a test keyspace with little
>>> data, when run on a larger test index (but still on a single node) I
>>> reliably see this error in the logs
>>>
>>> 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001
>>> java.lang.RuntimeException: TimedOutException()
>>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165)
>>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215)
>>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97)
>>>        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>>>        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91)
>>>        at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423)
>>>        at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67)
>>>        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143)
>>>        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583)
>>>        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
>>>        at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176)
>>> Caused by: TimedOutException()
>>>        at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015)
>>>        at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623)
>>>        at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597)
>>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142)
>>>        ... 11 more
>>>
>>> and after that the job seems to finish "normally" but no results are produced.
>>>
>>> FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if
>>> it ain't broke don't fix it).
>>>
>>> The single node has a data directory of about 127GB in two column
>>> families, off which the one used in the mapred job is about 100GB.
>>> The cassandra server is run with 6GB of heap on a box with 8GB
>>> available and no swap enabled. read/write latency from cfstat are
>>>
>>>        Read Latency: 0.8535837762577986 ms.
>>>        Write Latency: 0.028849603764075547 ms.
>>>
>>> row cache is not enabled, key cache percentage is default. Load on the
>>> machine is basically zero when the job is not running.
>>>
>>> As my code is 99% that from the wordcount contrib, I shall notice that
>>> In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we
>>> can supposedly change, but it's apparently not used anywhere, but as I
>>> said, running on a single node this should not be an issue anyway.
>>>
>>> Does anyone has suggestions or has seen this error before? On the
>>> other hand, did people run this kind of jobs in similar conditions
>>> flawlessly, so I can consider it just my problem?
>>>
>>>
>>> Thanks in advance for any help.
>>>
>>
>>
>>
>> --
>> Jonathan Ellis
>> Project Chair, Apache Cassandra
>> co-founder of Riptano, the source for professional Cassandra support
>> http://riptano.com
>>
>



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

Re: timeout while running simple hadoop job

Posted by Joost Ouwerkerk <jo...@openplaces.org>.
Huh? Isn't that the whole point of using Map/Reduce?

On Fri, May 7, 2010 at 8:44 AM, Jonathan Ellis <jb...@gmail.com> wrote:
> Sounds like you need to configure Hadoop to not create a whole bunch
> of Map tasks at once
>
> On Fri, May 7, 2010 at 3:47 AM, gabriele renzi <rf...@gmail.com> wrote:
>> Hi everyone,
>>
>> I am trying to develop a mapreduce job that does a simple
>> selection+filter on the rows in our store.
>> Of course it is mostly based on the WordCount example :)
>>
>>
>> Sadly, while it seems the app runs fine on a test keyspace with little
>> data, when run on a larger test index (but still on a single node) I
>> reliably see this error in the logs
>>
>> 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001
>> java.lang.RuntimeException: TimedOutException()
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97)
>>        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>>        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91)
>>        at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423)
>>        at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67)
>>        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143)
>>        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583)
>>        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
>>        at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176)
>> Caused by: TimedOutException()
>>        at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015)
>>        at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623)
>>        at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142)
>>        ... 11 more
>>
>> and after that the job seems to finish "normally" but no results are produced.
>>
>> FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if
>> it ain't broke don't fix it).
>>
>> The single node has a data directory of about 127GB in two column
>> families, off which the one used in the mapred job is about 100GB.
>> The cassandra server is run with 6GB of heap on a box with 8GB
>> available and no swap enabled. read/write latency from cfstat are
>>
>>        Read Latency: 0.8535837762577986 ms.
>>        Write Latency: 0.028849603764075547 ms.
>>
>> row cache is not enabled, key cache percentage is default. Load on the
>> machine is basically zero when the job is not running.
>>
>> As my code is 99% that from the wordcount contrib, I shall notice that
>> In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we
>> can supposedly change, but it's apparently not used anywhere, but as I
>> said, running on a single node this should not be an issue anyway.
>>
>> Does anyone has suggestions or has seen this error before? On the
>> other hand, did people run this kind of jobs in similar conditions
>> flawlessly, so I can consider it just my problem?
>>
>>
>> Thanks in advance for any help.
>>
>
>
>
> --
> Jonathan Ellis
> Project Chair, Apache Cassandra
> co-founder of Riptano, the source for professional Cassandra support
> http://riptano.com
>

Re: timeout while running simple hadoop job

Posted by Jonathan Ellis <jb...@gmail.com>.
Sounds like you need to configure Hadoop to not create a whole bunch
of Map tasks at once

On Fri, May 7, 2010 at 3:47 AM, gabriele renzi <rf...@gmail.com> wrote:
> Hi everyone,
>
> I am trying to develop a mapreduce job that does a simple
> selection+filter on the rows in our store.
> Of course it is mostly based on the WordCount example :)
>
>
> Sadly, while it seems the app runs fine on a test keyspace with little
> data, when run on a larger test index (but still on a single node) I
> reliably see this error in the logs
>
> 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001
> java.lang.RuntimeException: TimedOutException()
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97)
>        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91)
>        at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423)
>        at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67)
>        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143)
>        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583)
>        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
>        at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176)
> Caused by: TimedOutException()
>        at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015)
>        at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623)
>        at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142)
>        ... 11 more
>
> and after that the job seems to finish "normally" but no results are produced.
>
> FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if
> it ain't broke don't fix it).
>
> The single node has a data directory of about 127GB in two column
> families, off which the one used in the mapred job is about 100GB.
> The cassandra server is run with 6GB of heap on a box with 8GB
> available and no swap enabled. read/write latency from cfstat are
>
>        Read Latency: 0.8535837762577986 ms.
>        Write Latency: 0.028849603764075547 ms.
>
> row cache is not enabled, key cache percentage is default. Load on the
> machine is basically zero when the job is not running.
>
> As my code is 99% that from the wordcount contrib, I shall notice that
> In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we
> can supposedly change, but it's apparently not used anywhere, but as I
> said, running on a single node this should not be an issue anyway.
>
> Does anyone has suggestions or has seen this error before? On the
> other hand, did people run this kind of jobs in similar conditions
> flawlessly, so I can consider it just my problem?
>
>
> Thanks in advance for any help.
>



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

Re: timeout while running simple hadoop job

Posted by gabriele renzi <rf...@gmail.com>.
On Fri, May 7, 2010 at 3:02 PM, Joost Ouwerkerk <jo...@openplaces.org> wrote:
> Joseph, the stacktrace suggests that it's Thrift that's timing out,
> not the Task.
>
> Gabriele, I believe that your problem is caused by too much load on
> Cassandra.  Get_range_slices is presently an expensive operation. I
> had some success in reducing (although, it turns out, not eliminating)
> this problem by requesting smaller batches from get_range_slices.  See
> ConfigHelper.setRangeBatchSize()

interesting, I will try and report.

Re: timeout while running simple hadoop job

Posted by Joost Ouwerkerk <jo...@openplaces.org>.
Joseph, the stacktrace suggests that it's Thrift that's timing out,
not the Task.

Gabriele, I believe that your problem is caused by too much load on
Cassandra.  Get_range_slices is presently an expensive operation. I
had some success in reducing (although, it turns out, not eliminating)
this problem by requesting smaller batches from get_range_slices.  See
ConfigHelper.setRangeBatchSize()

joost

On Fri, May 7, 2010 at 8:49 AM, Joseph Stein <cr...@gmail.com> wrote:
> The problem could be that you are crunching more data than will be
> completed within the interval expire setting.
>
> In Hadoop you need to kind of tell the task tracker that you are still
> doing stuff which is done by setting status or incrementing counter on
> the Reporter object.
>
> http://allthingshadoop.com/2010/04/28/map-reduce-tips-tricks-your-first-real-cluster/
>
> "In your Java code there is a little trick to help the job be “aware”
> within the cluster of tasks that are not dead but just working hard.
> During execution of a task there is no built in reporting that the job
> is running as expected if it is not writing out.  So this means that
> if your tasks are taking up a lot of time doing work it is possible
> the cluster will see that task as failed (based on the
> mapred.task.tracker.expiry.interval setting).
>
> Have no fear there is a way to tell cluster that your task is doing
> just fine.  You have 2 ways todo this you can either report the status
> or increment a counter.  Both of these will cause the task tracker to
> properly know the task is ok and this will get seen by the jobtracker
> in turn.  Both of these options are explained in the JavaDoc
> http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/Reporter.html"
>
> Hope this helps
>
> On Fri, May 7, 2010 at 4:47 AM, gabriele renzi <rf...@gmail.com> wrote:
>> Hi everyone,
>>
>> I am trying to develop a mapreduce job that does a simple
>> selection+filter on the rows in our store.
>> Of course it is mostly based on the WordCount example :)
>>
>>
>> Sadly, while it seems the app runs fine on a test keyspace with little
>> data, when run on a larger test index (but still on a single node) I
>> reliably see this error in the logs
>>
>> 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001
>> java.lang.RuntimeException: TimedOutException()
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97)
>>        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>>        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91)
>>        at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423)
>>        at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67)
>>        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143)
>>        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583)
>>        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
>>        at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176)
>> Caused by: TimedOutException()
>>        at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015)
>>        at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623)
>>        at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142)
>>        ... 11 more
>>
>> and after that the job seems to finish "normally" but no results are produced.
>>
>> FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if
>> it ain't broke don't fix it).
>>
>> The single node has a data directory of about 127GB in two column
>> families, off which the one used in the mapred job is about 100GB.
>> The cassandra server is run with 6GB of heap on a box with 8GB
>> available and no swap enabled. read/write latency from cfstat are
>>
>>        Read Latency: 0.8535837762577986 ms.
>>        Write Latency: 0.028849603764075547 ms.
>>
>> row cache is not enabled, key cache percentage is default. Load on the
>> machine is basically zero when the job is not running.
>>
>> As my code is 99% that from the wordcount contrib, I shall notice that
>> In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we
>> can supposedly change, but it's apparently not used anywhere, but as I
>> said, running on a single node this should not be an issue anyway.
>>
>> Does anyone has suggestions or has seen this error before? On the
>> other hand, did people run this kind of jobs in similar conditions
>> flawlessly, so I can consider it just my problem?
>>
>>
>> Thanks in advance for any help.
>>
>
>
>
> --
> /*
> Joe Stein
> http://www.linkedin.com/in/charmalloc
> */
>

Re: timeout while running simple hadoop job

Posted by Matt Revelle <mr...@gmail.com>.

On May 7, 2010, at 9:40, gabriele renzi <rf...@gmail.com> wrote:

> On Fri, May 7, 2010 at 2:53 PM, Matt Revelle <mr...@gmail.com>  
> wrote:
> re: not reporting, I thought this was not needed with the new  mapred
> api (Mapper class vs Mapper interface), plus I can see that the
> mappers do work, report percentage and happily terminate (I believe
> Mapper.Context.write does a call to progress() behind the scenes).
>
> I will check if it makes any difference though.

Joost is write, the exception indicates that Thrift is timing out, not  
the Hadoop task.

Re: timeout while running simple hadoop job

Posted by gabriele renzi <rf...@gmail.com>.
On Fri, May 7, 2010 at 2:53 PM, Matt Revelle <mr...@gmail.com> wrote:
> There's also the mapred.task.timeout property that can be tweaked.  But reporting is the correct way to fix timeouts during execution.


re: not reporting, I thought this was not needed with the new  mapred
api (Mapper class vs Mapper interface), plus I can see that the
mappers do work, report percentage and happily terminate (I believe
Mapper.Context.write does a call to progress() behind the scenes).

I will check if it makes any difference though.

Re: timeout while running simple hadoop job

Posted by Matt Revelle <mr...@gmail.com>.
There's also the mapred.task.timeout property that can be tweaked.  But reporting is the correct way to fix timeouts during execution.

On May 7, 2010, at 8:49 AM, Joseph Stein wrote:

> The problem could be that you are crunching more data than will be
> completed within the interval expire setting.
> 
> In Hadoop you need to kind of tell the task tracker that you are still
> doing stuff which is done by setting status or incrementing counter on
> the Reporter object.
> 
> http://allthingshadoop.com/2010/04/28/map-reduce-tips-tricks-your-first-real-cluster/
> 
> "In your Java code there is a little trick to help the job be “aware”
> within the cluster of tasks that are not dead but just working hard.
> During execution of a task there is no built in reporting that the job
> is running as expected if it is not writing out.  So this means that
> if your tasks are taking up a lot of time doing work it is possible
> the cluster will see that task as failed (based on the
> mapred.task.tracker.expiry.interval setting).
> 
> Have no fear there is a way to tell cluster that your task is doing
> just fine.  You have 2 ways todo this you can either report the status
> or increment a counter.  Both of these will cause the task tracker to
> properly know the task is ok and this will get seen by the jobtracker
> in turn.  Both of these options are explained in the JavaDoc
> http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/Reporter.html"
> 
> Hope this helps
> 
> On Fri, May 7, 2010 at 4:47 AM, gabriele renzi <rf...@gmail.com> wrote:
>> Hi everyone,
>> 
>> I am trying to develop a mapreduce job that does a simple
>> selection+filter on the rows in our store.
>> Of course it is mostly based on the WordCount example :)
>> 
>> 
>> Sadly, while it seems the app runs fine on a test keyspace with little
>> data, when run on a larger test index (but still on a single node) I
>> reliably see this error in the logs
>> 
>> 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001
>> java.lang.RuntimeException: TimedOutException()
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97)
>>        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>>        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91)
>>        at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423)
>>        at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67)
>>        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143)
>>        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583)
>>        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
>>        at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176)
>> Caused by: TimedOutException()
>>        at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015)
>>        at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623)
>>        at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597)
>>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142)
>>        ... 11 more
>> 
>> and after that the job seems to finish "normally" but no results are produced.
>> 
>> FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if
>> it ain't broke don't fix it).
>> 
>> The single node has a data directory of about 127GB in two column
>> families, off which the one used in the mapred job is about 100GB.
>> The cassandra server is run with 6GB of heap on a box with 8GB
>> available and no swap enabled. read/write latency from cfstat are
>> 
>>        Read Latency: 0.8535837762577986 ms.
>>        Write Latency: 0.028849603764075547 ms.
>> 
>> row cache is not enabled, key cache percentage is default. Load on the
>> machine is basically zero when the job is not running.
>> 
>> As my code is 99% that from the wordcount contrib, I shall notice that
>> In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we
>> can supposedly change, but it's apparently not used anywhere, but as I
>> said, running on a single node this should not be an issue anyway.
>> 
>> Does anyone has suggestions or has seen this error before? On the
>> other hand, did people run this kind of jobs in similar conditions
>> flawlessly, so I can consider it just my problem?
>> 
>> 
>> Thanks in advance for any help.
>> 
> 
> 
> 
> -- 
> /*
> Joe Stein
> http://www.linkedin.com/in/charmalloc
> */


Re: timeout while running simple hadoop job

Posted by Joseph Stein <cr...@gmail.com>.
The problem could be that you are crunching more data than will be
completed within the interval expire setting.

In Hadoop you need to kind of tell the task tracker that you are still
doing stuff which is done by setting status or incrementing counter on
the Reporter object.

http://allthingshadoop.com/2010/04/28/map-reduce-tips-tricks-your-first-real-cluster/

"In your Java code there is a little trick to help the job be “aware”
within the cluster of tasks that are not dead but just working hard.
During execution of a task there is no built in reporting that the job
is running as expected if it is not writing out.  So this means that
if your tasks are taking up a lot of time doing work it is possible
the cluster will see that task as failed (based on the
mapred.task.tracker.expiry.interval setting).

Have no fear there is a way to tell cluster that your task is doing
just fine.  You have 2 ways todo this you can either report the status
or increment a counter.  Both of these will cause the task tracker to
properly know the task is ok and this will get seen by the jobtracker
in turn.  Both of these options are explained in the JavaDoc
http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/Reporter.html"

Hope this helps

On Fri, May 7, 2010 at 4:47 AM, gabriele renzi <rf...@gmail.com> wrote:
> Hi everyone,
>
> I am trying to develop a mapreduce job that does a simple
> selection+filter on the rows in our store.
> Of course it is mostly based on the WordCount example :)
>
>
> Sadly, while it seems the app runs fine on a test keyspace with little
> data, when run on a larger test index (but still on a single node) I
> reliably see this error in the logs
>
> 10/05/06 16:37:58 WARN mapred.LocalJobRunner: job_local_0001
> java.lang.RuntimeException: TimedOutException()
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:165)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:215)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.computeNext(ColumnFamilyRecordReader.java:97)
>        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:135)
>        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:130)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader.nextKeyValue(ColumnFamilyRecordReader.java:91)
>        at org.apache.hadoop.mapred.MapTask$NewTrackingRecordReader.nextKeyValue(MapTask.java:423)
>        at org.apache.hadoop.mapreduce.MapContext.nextKeyValue(MapContext.java:67)
>        at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:143)
>        at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:583)
>        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305)
>        at org.apache.hadoop.mapred.LocalJobRunner$Job.run(LocalJobRunner.java:176)
> Caused by: TimedOutException()
>        at org.apache.cassandra.thrift.Cassandra$get_range_slices_result.read(Cassandra.java:11015)
>        at org.apache.cassandra.thrift.Cassandra$Client.recv_get_range_slices(Cassandra.java:623)
>        at org.apache.cassandra.thrift.Cassandra$Client.get_range_slices(Cassandra.java:597)
>        at org.apache.cassandra.hadoop.ColumnFamilyRecordReader$RowIterator.maybeInit(ColumnFamilyRecordReader.java:142)
>        ... 11 more
>
> and after that the job seems to finish "normally" but no results are produced.
>
> FWIW this is on 0.6.0 (we didn't move to 0.6.1 yet because, well, if
> it ain't broke don't fix it).
>
> The single node has a data directory of about 127GB in two column
> families, off which the one used in the mapred job is about 100GB.
> The cassandra server is run with 6GB of heap on a box with 8GB
> available and no swap enabled. read/write latency from cfstat are
>
>        Read Latency: 0.8535837762577986 ms.
>        Write Latency: 0.028849603764075547 ms.
>
> row cache is not enabled, key cache percentage is default. Load on the
> machine is basically zero when the job is not running.
>
> As my code is 99% that from the wordcount contrib, I shall notice that
> In 0.6.1's contrib (and trunk) there is a RING_DELAY constant that we
> can supposedly change, but it's apparently not used anywhere, but as I
> said, running on a single node this should not be an issue anyway.
>
> Does anyone has suggestions or has seen this error before? On the
> other hand, did people run this kind of jobs in similar conditions
> flawlessly, so I can consider it just my problem?
>
>
> Thanks in advance for any help.
>



-- 
/*
Joe Stein
http://www.linkedin.com/in/charmalloc
*/