Tony Wu created HDFS-9290:
-----------------------------
Summary: DFSClient#callAppend() is not backward compatible for
slightly older NameNodes
Key: HDFS-9290
URL: https://issues.apache.org/jira/browse/HDFS-9290
Project: Hadoop HDFS
Issue Type: Bug
Affects Versions: 2.7.1
Reporter: Tony Wu
Assignee: Tony Wu
Priority: Minor
HDFS-7210 combined 2 RPC calls used at file append into a single one.
Specifically {{getFileInfo()}} is combined with {{append()}}. While backward
compatibility for older client is handled by the new NameNode (protobuf). Newer
client's {{append()}} call does not work with older NameNodes. One will run
into an exception like the following:
{code:java}
java.lang.NullPointerException
at
org.apache.hadoop.hdfs.DFSOutputStream.isLazyPersist(DFSOutputStream.java:1741)
at
org.apache.hadoop.hdfs.DFSOutputStream.getChecksum4Compute(DFSOutputStream.java:1550)
at
org.apache.hadoop.hdfs.DFSOutputStream.<init>(DFSOutputStream.java:1560)
at
org.apache.hadoop.hdfs.DFSOutputStream.<init>(DFSOutputStream.java:1670)
at
org.apache.hadoop.hdfs.DFSOutputStream.newStreamForAppend(DFSOutputStream.java:1717)
at org.apache.hadoop.hdfs.DFSClient.callAppend(DFSClient.java:1861)
at org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:1922)
at org.apache.hadoop.hdfs.DFSClient.append(DFSClient.java:1892)
at
org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:340)
at
org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:336)
at
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at
org.apache.hadoop.hdfs.DistributedFileSystem.append(DistributedFileSystem.java:336)
at
org.apache.hadoop.hdfs.DistributedFileSystem.append(DistributedFileSystem.java:318)
at org.apache.hadoop.fs.FileSystem.append(FileSystem.java:1164)
{code}
The cause is that the new client code is expecting both the last block and file
info in the same RPC but the old NameNode only replied with the first. The
exception itself does not reflect this cause and one will have to look at the
HDFS source code to really understand what happened.
We can have the client detect it's talking to a old NameNode and send an extra
{{getFileInfo()}} RPC. At the very least we can improve the exception being
thrown to accurately reflect the failure.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)