[ 
https://issues.apache.org/jira/browse/HDFS-15079?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17003153#comment-17003153
 ] 

Fei Hui edited comment on HDFS-15079 at 12/25/19 9:09 AM:
----------------------------------------------------------

[~elgoiri]
HDFS-15078 has a test case, it's one case for this.

[~hexiaoqiao]
 Client gets Exception, but the exception is not that router throws. client 
logs as follow
{quote}
java.io.EOFException: End of File Exception between local host is: 
"xx.xx.xx.xx"; destination host is: "xx.xx.xx.xx":8888; : java.io.EOFException; 
For more details see:  http://wiki.apache.org/hadoop/EOFException
        at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
        at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
        at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:792)
        at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:765)
        at org.apache.hadoop.ipc.Client.call(Client.java:1507)
        at org.apache.hadoop.ipc.Client.call(Client.java:1441)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
        at com.sun.proxy.$Proxy19.create(Unknown Source)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:303)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:253)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:101)
        at com.sun.proxy.$Proxy20.create(Unknown Source)
        at 
org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:264)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1727)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1662)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:503)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:499)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:514)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:442)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:979)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:872)
        at 
org.apache.hadoop.mapred.TextOutputFormat.getRecordWriter(TextOutputFormat.java:135)
        at 
org.apache.spark.internal.io.HadoopMapRedWriteConfigUtil.initWriter(SparkHadoopWriter.scala:228)
        at 
org.apache.spark.internal.io.SparkHadoopWriter$.org$apache$spark$internal$io$SparkHadoopWriter$$executeTask(SparkHadoopWriter.scala:122)
        at 
org.apache.spark.internal.io.SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:83)
        at 
org.apache.spark.internal.io.SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:78)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:121)
        at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.EOFException
        at java.io.DataInputStream.readInt(DataInputStream.java:392)
        at 
org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1113)
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:1008)
{quote}

I think maybe consistency is not guaranteed if do not resolve it on nn side.


was (Author: ferhui):
[~elgoiri]
HDFS-15078 has a test case, it's one case for this.

[~hexiaoqiao]
 Client gets Exception, but the exception is not that router throws. client 
logs as follow
{quote}
java.io.EOFException: End of File Exception between local host is: 
"xx.xx.xx.xx"; destination host is: "xx.xx.xx.xx":8888; : java.io.EOFException; 
For more details see:  http://wiki.apache.org/hadoop/EOFException
        at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
        at 
sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
        at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
        at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
        at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:792)
        at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:765)
        at org.apache.hadoop.ipc.Client.call(Client.java:1507)
        at org.apache.hadoop.ipc.Client.call(Client.java:1441)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
        at com.sun.proxy.$Proxy19.create(Unknown Source)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:303)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:253)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:101)
        at com.sun.proxy.$Proxy20.create(Unknown Source)
        at 
org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:264)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1727)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1662)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:503)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:499)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:514)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:442)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:979)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:872)
        at 
org.apache.hadoop.mapred.TextOutputFormat.getRecordWriter(TextOutputFormat.java:135)
        at 
org.apache.spark.internal.io.HadoopMapRedWriteConfigUtil.initWriter(SparkHadoopWriter.scala:228)
        at 
org.apache.spark.internal.io.SparkHadoopWriter$.org$apache$spark$internal$io$SparkHadoopWriter$$executeTask(SparkHadoopWriter.scala:122)
        at 
org.apache.spark.internal.io.SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:83)
        at 
org.apache.spark.internal.io.SparkHadoopWriter$$anonfun$3.apply(SparkHadoopWriter.scala:78)
        at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
        at org.apache.spark.scheduler.Task.run(Task.scala:121)
        at 
org.apache.spark.executor.Executor$TaskRunner$$anonfun$10.apply(Executor.scala:408)
        at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1360)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:414)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.EOFException
        at java.io.DataInputStream.readInt(DataInputStream.java:392)
        at 
org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1113)
        at org.apache.hadoop.ipc.Client$Connection.run(Client.java:1008)
{quote}

I think maybe consistency is not guaranteed if resolve it on nn side.

> RBF: Client maybe get an unexpected result with network anomaly 
> ----------------------------------------------------------------
>
>                 Key: HDFS-15079
>                 URL: https://issues.apache.org/jira/browse/HDFS-15079
>             Project: Hadoop HDFS
>          Issue Type: Sub-task
>          Components: rbf
>    Affects Versions: 3.3.0
>            Reporter: Fei Hui
>            Priority: Critical
>
>  I find there is a critical problem on RBF, HDFS-15078 can resolve it on some 
> Scenarios, but i have no idea about the overall resolution.
> The problem is that
> Client with RBF(r0, r1) create a file HDFS file via r0, it gets Exception and 
> failovers to r1
> r0 has been send create rpc to namenode(1st create)
> Client create a HDFS file via r1(2nd create)
> Client writes the HDFS file and close it finally(3rd close)
> Maybe namenode receiving the rpc in order as follow
> 2nd create
> 3rd close
> 1st create
> And overwrite is true by default, this would make the file had been written 
> an empty file. This is an critical problem 
> We had encountered this problem. There are many hive and spark jobs running 
> on our cluster,   sometimes it occurs



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to