You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-dev@hadoop.apache.org by Colin McCabe <cm...@alumni.cmu.edu> on 2013/11/07 18:09:09 UTC

Re: HDFS single datanode cluster issues

First of all, HDFS isn't really the right choice for single-node
environments.  I would recommend using LocalFileSystem in this case.
If you're evaluating HDFS and only have one computer, it will really
be better to run several VMs to see how it works, rather than running
just one Datanode.

You are correct that there are some issues with the pipeline recovery
code on small clusters.  In a lot of cases, pipeline recovery can make
the whole output stream fail, when it is unable to find enough nodes.
We filed HDFS-5131 to address those issues.

In the meantime, you can set
dfs.client.block.write.replace-datanode-on-failure.enable to false,
and dfs.client.block.write.replace-datanode-on-failure.policy to
NEVER.  Based on your comment, it seems like you are already trying to
do this.  Make sure you are setting this configuration on the client
side as well as on the server side-- then you should not see error
messages about pipeline recovery.

best,
Colin

On Wed, Oct 30, 2013 at 8:06 AM, David Mankellow
<dm...@swiftserve.com> wrote:
> We are mapping a 1:1 replication.
>
> We have tried setting
> dfs.client.block.write.replace-datanode-on-failure.enable to NEVER but it
> seems to be ignored.
>
> We have tried the following:
> ===
>   <property>
> <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>
>     <value>false</value>
>   </property>
>   <property>
> <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
>     <value>NEVER</value>
>   </property>
> ===
>   <property>
> <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>
>     <value>true</value>
>   </property>
>   <property>
> <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
>     <value>NEVER</value>
>   </property>
> ===
>   <property>
> <name>dfs.client.block.write.replace-datanode-on-failure.enable</name>
>     <value>true</value>
>   </property>
>
> Any more help would be greatly appreciated.
>
> Thanks,
> Dave
>
>
>
>
> On 30/10/2013 10:50, Allan Wilson wrote:
>>
>> Hi David
>>
>> How does your block replica count compare to the number of datanodes in
>> your cluster?
>>
>> Anyway...I found this in the online doc.  You may want to use the NEVER
>> policy.
>>
>> dfs.client.block.write.replace-datanode-on-failure.enable true If there is
>> a datanode/network failure in the write pipeline, DFSClient will try to
>> remove the failed datanode from the pipeline and then continue writing
>> with
>> the remaining datanodes. As a result, the number of datanodes in the
>> pipeline is decreased. The feature is to add new datanodes to the
>> pipeline.
>> This is a site-wide property to enable/disable the feature. When the
>> cluster size is extremely small, e.g. 3 nodes or less, cluster
>> administrators may want to set the policy to NEVER in the default
>> configuration file or disable this feature. Otherwise, users may
>> experience
>> an unusually high rate of pipeline failures since it is impossible to find
>> new datanodes for replacement. See also
>> dfs.client.block.write.replace-datanode-on-failure.policy
>> dfs.client.block.write.replace-datanode-on-failure.policy DEFAULT This
>> property is used only if the value of
>> dfs.client.block.write.replace-datanode-on-failure.enable is true. ALWAYS:
>> always add a new datanode when an existing datanode is removed. NEVER:
>> never add a new datanode. DEFAULT: Let r be the replication number. Let n
>> be the number of existing datanodes. Add a new datanode only if r is
>> greater than or equal to 3 and either (1) floor(r/2) is greater than or
>> equal to n; or (2) r is greater than n and the block is hflushed/appended.
>>
>> Allan
>> On Oct 30, 2013 5:52 AM, "David Mankellow" <dm...@swiftserve.com>
>> wrote:
>>
>>> Hi all,
>>>
>>> We are having difficulty writing any logs to a HDFS cluster of less than
>>> 3
>>> nodes. This has been since the update between cdh4.2 and 4.3 (4.4 is also
>>> the same). Has anything changed that may make this occur and is there
>>> anything that can be done to rectify the situation, so we can use a
>>> single
>>> datanode once more?
>>>
>>> The error log contains errors about "lease recovery" and "Failed to add a
>>> datanode".
>>>
>>> Here is an example stack trace:
>>>
>>> java.io.IOException: Failed to add a datanode.  User may turn off this
>>> feature by setting
>>> dfs.client.block.write.**replace-datanode-on-failure.**policy
>>> in configuration, where the current policy is DEFAULT.  (Nodes: current=[
>>> 5.9.130.139:50010, 5.9.130.140:50010], original=[5.9.130.139:50010,
>>> 5.9.130.140:50010])
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> findNewDatanode(**DFSOutputStream.java:816)
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> addDatanode2ExistingPipeline(**DFSOutputStream.java:876)
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> setupPipelineForAppendOrRecove**ry(DFSOutputStream.java:982)
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> run(DFSOutputStream.java:493)
>>> FSDataOutputStream#close error:
>>> java.io.IOException: Failed to add a datanode.  User may turn off this
>>> feature by setting
>>> dfs.client.block.write.**replace-datanode-on-failure.**policy
>>> in configuration, where the current policy is DEFAULT.  (Nodes: current=[
>>> 5.9.130.139:50010, 5.9.130.140:50010], original=[5.9.130.139:50010,
>>> 5.9.130.140:50010])
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> findNewDatanode(**DFSOutputStream.java:816)
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> addDatanode2ExistingPipeline(**DFSOutputStream.java:876)
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> setupPipelineForAppendOrRecove**ry(DFSOutputStream.java:982)
>>>      at org.apache.hadoop.hdfs.**DFSOutputStream$DataStreamer.**
>>> run(DFSOutputStream.java:493)
>>> hdfsOpenFile(hdfs://storage1.**testing.swiftserve.com:9000/**
>>>
>>> scribe/logs/test/log1.testing.**swiftserve.com/test-2013-10-**14_00000<http://storage1.testing.swiftserve.com:9000/scribe/logs/test/log1.testing.swiftserve.com/test-2013-10-14_00000>):
>>>
>>> FileSystem#append((Lorg/**apache/hadoop/fs/Path;)Lorg/**apache/hadoop/fs/*
>>> *FSDataOutputStream;) error:
>>>
>>> org.apache.hadoop.ipc.**RemoteException(org.apache.**hadoop.hdfs.protocol.
>>> **AlreadyBeingCreatedException): Failed to create file
>>> [/scribe/logs/test/
>>>
>>> log1.**testing.swiftserve.com/test-**2013-10-14_00000<http://log1.testing.swiftserve.com/test-2013-10-14_00000>]
>>> for [DFSClient_NONMAPREDUCE_**1056562813_1] on client [5.9.130.136],
>>> because this file is already being created by
>>> [DFSClient_NONMAPREDUCE_**2007800327_1]
>>> on [5.9.130.136]
>>>      at org.apache.hadoop.hdfs.server.**namenode.FSNamesystem.**
>>> recoverLeaseInternal(**FSNamesystem.java:2062)
>>>      at org.apache.hadoop.hdfs.server.**namenode.FSNamesystem.**
>>> startFileInternal(**FSNamesystem.java:1862)
>>>      at org.apache.hadoop.hdfs.server.**namenode.FSNamesystem.**
>>> appendFileInt(FSNamesystem.**java:2105)
>>>      at org.apache.hadoop.hdfs.server.**namenode.FSNamesystem.**
>>> appendFile(FSNamesystem.java:**2081)
>>>      at org.apache.hadoop.hdfs.server.**namenode.NameNodeRpcServer.**
>>> append(NameNodeRpcServer.java:**434)
>>>      at org.apache.hadoop.hdfs.**protocolPB.**
>>> ClientNamenodeProtocolServerSi**deTranslatorPB.append(**
>>> ClientNamenodeProtocolServerSi**deTranslatorPB.java:224)
>>>      at org.apache.hadoop.hdfs.**protocol.proto.**
>>> ClientNamenodeProtocolProtos$**ClientNamenodeProtocol$2.**
>>> callBlockingMethod(**ClientNamenodeProtocolProtos.**java:44944)
>>>      at org.apache.hadoop.ipc.**ProtobufRpcEngine$Server$**
>>> ProtoBufRpcInvoker.call(**ProtobufRpcEngine.java:453)
>>>      at org.apache.hadoop.ipc.RPC$**Server.call(RPC.java:1002)
>>>      at org.apache.hadoop.ipc.Server$**Handler$1.run(Server.java:**1701)
>>>      at org.apache.hadoop.ipc.Server$**Handler$1.run(Server.java:**1697)
>>>      at java.security.**AccessController.doPrivileged(**Native Method)
>>>      at javax.security.auth.Subject.**doAs(Subject.java:396)
>>>      at org.apache.hadoop.security.**UserGroupInformation.doAs(**
>>> UserGroupInformation.java:**1408)
>>>      at org.apache.hadoop.ipc.Server$**Handler.run(Server.java:1695)
>>>
>>>      at org.apache.hadoop.ipc.Client.**call(Client.java:1231)
>>>      at org.apache.hadoop.ipc.**ProtobufRpcEngine$Invoker.**
>>> invoke(ProtobufRpcEngine.java:**202)
>>>      at $Proxy9.append(Unknown Source)
>>>      at sun.reflect.**GeneratedMethodAccessor6.**invoke(Unknown Source)
>>>      at sun.reflect.**DelegatingMethodAccessorImpl.**invoke(**
>>> DelegatingMethodAccessorImpl.**java:25)
>>>      at java.lang.reflect.Method.**invoke(Method.java:597)
>>>      at
>>> org.apache.hadoop.io.retry.**RetryInvocationHandler.**invokeMethod(
>>> **RetryInvocationHandler.java:**164)
>>>      at org.apache.hadoop.io.retry.**RetryInvocationHandler.invoke(**
>>> RetryInvocationHandler.java:**83)
>>>      at $Proxy9.append(Unknown Source)
>>>      at org.apache.hadoop.hdfs.**protocolPB.**
>>> ClientNamenodeProtocolTranslat**orPB.append(**
>>> ClientNamenodeProtocolTranslat**orPB.java:210)
>>>      at org.apache.hadoop.hdfs.**DFSClient.callAppend(**
>>> DFSClient.java:1352)
>>>      at org.apache.hadoop.hdfs.**DFSClient.append(DFSClient.**java:1391)
>>>      at org.apache.hadoop.hdfs.**DFSClient.append(DFSClient.**java:1379)
>>>      at org.apache.hadoop.hdfs.**DistributedFileSystem.append(**
>>> DistributedFileSystem.java:**257)
>>>      at org.apache.hadoop.hdfs.**DistributedFileSystem.append(**
>>> DistributedFileSystem.java:81)
>>>      at org.apache.hadoop.fs.**FileSystem.append(FileSystem.**java:1106)
>>>
>>> The exception suggests we have already set " dfs.client.block.write.**
>>> replace-datanode-on-failure.**policy" to "NEVER" but hadoop ignores it.
>>>
>>> Any help would be appreciated.
>>>
>>> Thanks,
>>> David Mankellow
>>>
>