Hi Donald,

The problem is that the return value of CellStoreV0::add is getting ignored
in AccessGroup::run_compaction.  The correct thing to do in this situation
would be for the code to throw an exception and let the system re-schedule
another compaction later.  It also appears that the return value is getting
ignored in the normal RangeServer::update code path as well.  I've filed issue
231 <http://code.google.com/p/hypertable/issues/detail?id=231> to take care
of both of these situations.

Not sure what to do about the HDFS exception.  If this is a common error
that is transient and there is a way to query HDFS to see if it is healthy
or unhealthy, then we could probably put the system into some sort of
readonly mode until HDFS became healthy again.  Unfortunately, I don't think
this API exists.  Have you tried this with KFS?

- Doug

On Sat, Feb 7, 2009 at 9:00 PM, donald <[email protected]> wrote:

>
> Hi All,
>
> Yesterday I get the following errors in my Hypertable.RangeServer.log
> after loading data into a 10-node Hypertable/Hadoop cluster for over
> 10 hours at about 30MB/s.
>
> 1234009104 ERROR Hypertable.RangeServer : (/home/work/src/hypertable/
> src/cc/Hypertable/RangeServer/CellStoreV0.cc:162) Problem writing to
> DFS file '/hypertable/tables/blip_session_raw/default/
> 48390E68F18E1750E11ECFA1/cs668' : Event: type=ERROR "COMM request
> timeout" from=127.0.0.1:38030
> 1234009104 ERROR Hypertable.RangeServer : (/home/work/src/hypertable/
> src/cc/Hypertable/RangeServer/CellStoreV0.cc:162) Problem writing to
> DFS file '/hypertable/tables/blip_session_raw/default/
> 48390E68F18E1750E11ECFA1/cs668' : Event: type=ERROR "COMM request
> timeout" from=127.0.0.1:38030
> 1234009104 ERROR Hypertable.RangeServer : (/home/work/src/hypertable/
> src/cc/Hypertable/RangeServer/CellStoreV0.cc:162) Problem writing to
> DFS file '/hypertable/tables/blip_session_raw/default/
> 48390E68F18E1750E11ECFA1/cs668' : Event: type=ERROR "COMM request
> timeout" from=127.0.0.1:38030
>
> And in DfsBroker.hadoop.log (I've modified HdfsBroker.java to make it
> more verbose):
> Feb 7, 2009 8:12:06 PM org.hypertable.DfsBroker.hadoop.HdfsBroker
> Write
> INFO: Write 69558 bytes from file /hypertable/tables/blip_session_raw/
> default/48390E68F18E1750E11ECFA1/cs668 handle 19514 cost 2682 ms.
> Feb 7, 2009 8:12:16 PM org.hypertable.DfsBroker.hadoop.HdfsBroker
> Write
> INFO: Write 70141 bytes from file /hypertable/tables/blip_session_raw/
> default/48390E68F18E1750E11ECFA1/cs668 handle 19514 cost 9668 ms.
> Feb 7, 2009 8:12:20 PM org.hypertable.DfsBroker.hadoop.HdfsBroker
> Write
> INFO: Write 69568 bytes from file /hypertable/tables/blip_session_raw/
> default/48390E68F18E1750E11ECFA1/cs668 handle 19514 cost 2881 ms.
> Feb 7, 2009 8:12:21 PM org.hypertable.DfsBroker.hadoop.HdfsBroker
> Write
> INFO: Write 60521 bytes from file /hypertable/tables/blip_session_raw/
> default/48390E68F18E1750E11ECFA1/cs668 handle 19514 cost 1042 ms.
> Feb 7, 2009 8:12:52 PM org.hypertable.DfsBroker.hadoop.HdfsBroker
> Write
> INFO: Write 68186 bytes from file /hypertable/tables/blip_session_raw/
> default/48390E68F18E1750E11ECFA1/cs668 handle 19514 cost 30879 ms.
> Feb 7, 2009 8:12:56 PM org.hypertable.DfsBroker.hadoop.HdfsBroker
> Write
> INFO: Write 68899 bytes from file /hypertable/tables/blip_session_raw/
> default/48390E68F18E1750E11ECFA1/cs668 handle 19514 cost 3764 ms.
> 09/02/07 20:14:02 WARN hdfs.DFSClient: DFSOutputStream
> ResponseProcessor exception  for block
> blk_1192910_586583java.net.SocketTimeoutException: 69000 millis
> timeout whil
> e waiting for channel to be ready for read. ch :
> java.nio.channels.SocketChannel[connected local=/10.65.25.151:52026
> remote=/10.73.4.160:50010]
>        at org.apache.hadoop.net.SocketIOWithTimeout.doIO
> (SocketIOWithTimeout.java:164)
>        at org.apache.hadoop.net.SocketInputStream.read
> (SocketInputStream.java:155)
>        at org.apache.hadoop.net.SocketInputStream.read
> (SocketInputStream.java:128)
>        at org.apache.hadoop.net.SocketInputStream.read
> (SocketInputStream.java:116)
>        at java.io.DataInputStream.readShort(DataInputStream.java:295)
>        at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream
> $ResponseProcessor.run(DFSClient.java:2350)
>
> Feb 7, 2009 8:15:24 PM org.hypertable.DfsBroker.hadoop.HdfsBroker
> Write
> INFO: Write 62994 bytes from file /hypertable/tables/blip_session_raw/
> default/48390E68F18E1750E11ECFA1/cs668 handle 19514 cost 147876 ms.
> 09/02/07 20:15:24 WARN hdfs.DFSClient: Error Recovery for block
> blk_1192910_586583 bad datanode[0] 10.73.4.160:50010
> 09/02/07 20:15:24 WARN hdfs.DFSClient: Error Recovery for block
> blk_1192910_586583 in pipeline 10.73.4.160:50010, 10.73.4.136:50010,
> 10.73.4.148:50010: bad datanode 10.73
> .4.160:50010
>
> After this error, the range server stopped doing compactions any more,
> pstack'ing the process shows that the maintenance thread was stucked
> in line 161 of CellStoreV0::add()
>
> 160     if (m_outstanding_appends >= MAX_APPENDS_OUTSTANDING) {
> 161       if (!m_sync_handler.wait_for_reply(event_ptr)) {
> 162         HT_ERRORF("Problem writing to DFS file '%s' : %s",
> m_filename.c_str(), Hypertable::Protocol::string_format_message
> (event_ptr).c_str());
> 163         return -1;
> 164       }
> 165       m_outstanding_appends--;
> 166     }
>
> I guess the problem is m_outstanding_appends is not decremented when
> there is an error. However subsequent calls to CellStoreV0::add()
> would still wait for the already failed appends which would never
> reply again. I couldn't think of a good fix for this problem yet. What
> do you think?
>
> Donald
>
>
> >
>

--~--~---------~--~----~------------~-------~--~----~
You received this message because you are subscribed to the Google Groups 
"Hypertable Development" group.
To post to this group, send email to [email protected]
To unsubscribe from this group, send email to 
[email protected]
For more options, visit this group at 
http://groups.google.com/group/hypertable-dev?hl=en
-~----------~----~----~----~------~----~------~--~---

Reply via email to