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

$iddhe$h Divekar commented on HADOOP-11487:
-------------------------------------------

Hi,
We are processing data on US west and still seeing consistency issue.
As per forums US west should not be having consistency issue but we
are doing update of a table. Not sure if 'read-after-write' consistency 
will take care of 'read-after-update' consistency also.

Will 9565 help us here.

Below is the back trace of the issue we are seeing.
org.apache.spark.SparkException: Job aborted.
        at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1.apply$mcV$sp(InsertIntoHadoopFsRelation.scala:154)
        at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1.apply(InsertIntoHadoopFsRelation.scala:106)
        at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1.apply(InsertIntoHadoopFsRelation.scala:106)
        at 
org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:56)
        at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation.run(InsertIntoHadoopFsRelation.scala:106)
        at 
org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:58)
        at 
org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:56)
        at 
org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:70)
        at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:132)
        at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:130)
        at 
org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
        at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:130)
        at 
org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:55)
        at 
org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:55)
        at 
org.apache.spark.sql.DataFrameWriter.insertInto(DataFrameWriter.scala:189)
        at 
org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:239)
        at 
org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:221)
        at com.foo.vAnalytics.xyz_load$.main(xyz_load.scala:130)
        at com.foo.vAnalytics.xyz_load.main(xyz_load.scala)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at 
org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:731)
        at 
org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:181)
        at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:206)
        at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:121)
        at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
        at org.apache.oozie.action.hadoop.SparkMain.runSpark(SparkMain.java:104)
        at org.apache.oozie.action.hadoop.SparkMain.run(SparkMain.java:95)
        at org.apache.oozie.action.hadoop.LauncherMain.run(LauncherMain.java:47)
        at org.apache.oozie.action.hadoop.SparkMain.main(SparkMain.java:38)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at 
org.apache.oozie.action.hadoop.LauncherMapper.map(LauncherMapper.java:236)
        at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:54)
        at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:430)
        at org.apache.hadoop.mapred.MapTask.run(MapTask.java:342)
        at 
org.apache.hadoop.mapred.LocalContainerLauncher$SubtaskRunner.runSubtask(LocalContainerLauncher.java:317)
        at 
org.apache.hadoop.mapred.LocalContainerLauncher$SubtaskRunner.run(LocalContainerLauncher.java:232)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.io.FileNotFoundException: File 
s3n://foo-hive/warehouse/fooabcxyz0719/_temporary/0/task_201607210010_0005_m_000041
 does not exist.
        at 
org.apache.hadoop.fs.s3native.NativeS3FileSystem.listStatus(NativeS3FileSystem.java:506)
        at 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.mergePaths(FileOutputCommitter.java:360)
        at 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitJob(FileOutputCommitter.java:310)
        at 
org.apache.parquet.hadoop.ParquetOutputCommitter.commitJob(ParquetOutputCommitter.java:46)
        at 
org.apache.spark.sql.execution.datasources.BaseWriterContainer.commitJob(WriterContainer.scala:230)
        at 
org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1.apply$mcV$sp(InsertIntoHadoopFsRelation.scala:149)
        ... 42 more

2016-07-21 00:22:54,370  WARN ParameterVerifier:523 - 
SERVER[ip-10-0-0-136.us-west-2.compute.internal] USER[root] GROUP[-] TOKEN[] 
APP[spark-coord] JOB[0000096-160719225831887-oozie-root-C] 
ACTION[0000096-1607192\
25831887-oozie-root-C@3] The application does not define formal parameters in 
its XML definition


> FileNotFound on distcp to s3n/s3a due to creation inconsistency 
> ----------------------------------------------------------------
>
>                 Key: HADOOP-11487
>                 URL: https://issues.apache.org/jira/browse/HADOOP-11487
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs, fs/s3
>    Affects Versions: 2.7.2
>            Reporter: Paulo Motta
>
> I'm trying to copy a large amount of files from HDFS to S3 via distcp and I'm 
> getting the following exception:
> {code:java}
> 2015-01-16 20:53:18,187 ERROR [main] 
> org.apache.hadoop.tools.mapred.CopyMapper: Failure in copying 
> hdfs://10.165.35.216/hdfsFolder/file.gz to s3n://s3-bucket/file.gz
> java.io.FileNotFoundException: No such file or directory 
> 's3n://s3-bucket/file.gz'
>       at 
> org.apache.hadoop.fs.s3native.NativeS3FileSystem.getFileStatus(NativeS3FileSystem.java:445)
>       at 
> org.apache.hadoop.tools.util.DistCpUtils.preserve(DistCpUtils.java:187)
>       at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:233)
>       at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:45)
>       at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
>       at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:764)
>       at org.apache.hadoop.mapred.MapTask.run(MapTask.java:340)
>       at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:167)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at javax.security.auth.Subject.doAs(Subject.java:422)
>       at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
>       at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:162)
> 2015-01-16 20:53:18,276 WARN [main] org.apache.hadoop.mapred.YarnChild: 
> Exception running child : java.io.FileNotFoundException: No such file or 
> directory 's3n://s3-bucket/file.gz'
>       at 
> org.apache.hadoop.fs.s3native.NativeS3FileSystem.getFileStatus(NativeS3FileSystem.java:445)
>       at 
> org.apache.hadoop.tools.util.DistCpUtils.preserve(DistCpUtils.java:187)
>       at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:233)
>       at org.apache.hadoop.tools.mapred.CopyMapper.map(CopyMapper.java:45)
>       at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:145)
>       at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:764)
>       at org.apache.hadoop.mapred.MapTask.run(MapTask.java:340)
>       at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:167)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at javax.security.auth.Subject.doAs(Subject.java:422)
>       at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1548)
>       at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:162)
> {code}
> However, when I try hadoop fs -ls s3n://s3-bucket/file.gz the file is there. 
> So probably due to Amazon's S3 eventual consistency the job failure.
> In my opinion, in order to fix this problem NativeS3FileSystem.getFileStatus 
> must use fs.s3.maxRetries property in order to avoid failures like this.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to