[ 
https://issues.apache.org/jira/browse/HADOOP-13574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Alex Zolotko updated HADOOP-13574:
----------------------------------
    Description: 
We recently got the following exception on production:

{noformat}
java.io.FileNotFoundException: Key 
'xxx/_temporary/0/_temporary/attempt_201609010631_0000_m_001128_1128/part-01128'
 does not exist in S3
        at 
org.apache.hadoop.fs.s3native.Jets3tNativeFileSystemStore.handleServiceException(Jets3tNativeFileSystemStore.java:234)
        at 
org.apache.hadoop.fs.s3native.Jets3tNativeFileSystemStore.copy(Jets3tNativeFileSystemStore.java:201)
        at sun.reflect.GeneratedMethodAccessor71.invoke(Unknown Source)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:497)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
        at org.apache.hadoop.fs.s3native.$Proxy13.copy(Unknown Source)
        at 
org.apache.hadoop.fs.s3native.NativeS3FileSystem.rename(NativeS3FileSystem.java:659)
        at 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:435)
        at 
org.apache.hadoop.mapred.FileOutputCommitter.commitTask(FileOutputCommitter.java:172)
        at 
org.apache.hadoop.mapred.OutputCommitter.commitTask(OutputCommitter.java:291)
        at 
org.apache.spark.mapred.SparkHadoopMapRedUtil$.performCommit$1(SparkHadoopMapRedUtil.scala:98)
        at 
org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:124)
        at 
org.apache.spark.SparkHadoopWriter.commit(SparkHadoopWriter.scala:107)
        at 
org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1204)
        at 
org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1183)
{noformat}

FileOutputCommitter.commitTask() does check that the file exists before trying 
to rename it, but due to S3's relaxed consistency guarantees the following 
fs.rename(taskAttemptPath, committedTaskPath) still fails.

Here's an excerpt from the Amazon S3 documentation 
([https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html]):

{quote}
Amazon S3 Data Consistency Model

Amazon S3 provides read-after-write consistency for PUTS of new objects in your 
S3 bucket in all regions with one caveat. The caveat is that if you make a HEAD 
or GET request to the key name (to find if the object exists) before creating 
the object, Amazon S3 provides eventual consistency for read-after-write.
{quote}

The problematic S3 object existence check, that causes S3 to fallback to 
eventual consistency, is in NativeS3FileSystem.create():

{code}
    if (exists(f) && !overwrite) {
      throw new IOException("File already exists:"+f);
    }
{code}

If the "overwrite" parameter is set to "true" (as in our case), calling 
exists(f) is unnecessary and only "upsets" S3.

The proposed fix is to switch the order of the predicates:
{code}
    if (!overwrite && exists(f)) {
      throw new IOException("File already exists:"+f);
    }
{code}

  was:
We recently got the following exception on production:

{noformat}
java.io.FileNotFoundException: Key 
'xxx/_temporary/0/_temporary/attempt_201609010631_0000_m_001128_1128/part-01128'
 does not exist in S3
        at 
org.apache.hadoop.fs.s3native.Jets3tNativeFileSystemStore.handleServiceException(Jets3tNativeFileSystemStore.java:234)
        at 
org.apache.hadoop.fs.s3native.Jets3tNativeFileSystemStore.copy(Jets3tNativeFileSystemStore.java:201)
        at sun.reflect.GeneratedMethodAccessor71.invoke(Unknown Source)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:497)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
        at org.apache.hadoop.fs.s3native.$Proxy13.copy(Unknown Source)
        at 
org.apache.hadoop.fs.s3native.NativeS3FileSystem.rename(NativeS3FileSystem.java:659)
        at 
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:435)
        at 
org.apache.hadoop.mapred.FileOutputCommitter.commitTask(FileOutputCommitter.java:172)
        at 
org.apache.hadoop.mapred.OutputCommitter.commitTask(OutputCommitter.java:291)
        at 
org.apache.spark.mapred.SparkHadoopMapRedUtil$.performCommit$1(SparkHadoopMapRedUtil.scala:98)
        at 
org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:124)
        at 
org.apache.spark.SparkHadoopWriter.commit(SparkHadoopWriter.scala:107)
        at 
org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1204)
        at 
org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1183)
{noformat}

FileOutputCommitter.commitTask() does check that the file exists before trying 
to rename it, but due to S3's relaxed consistency guarantees the following 
fs.rename(taskAttemptPath, committedTaskPath) still fails.

Here's an excerpt from the Amazon S3 documentation 
([https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html]):

{quote}
Amazon S3 Data Consistency Model

Amazon S3 provides read-after-write consistency for PUTS of new objects in your 
S3 bucket in all regions with one caveat. The caveat is that if you make a HEAD 
or GET request to the key name (to find if the object exists) before creating 
the object, Amazon S3 provides eventual consistency for read-after-write.
{quote}

The problematic S3 object existence check, that causes S3 to fallback to 
eventual consistency, is in NativeS3FileSystem.create():

{code}
    if (exists(f) && !overwrite) {
      throw new IOException("File already exists:"+f);
    }
{code}

If the overwrite parameter is set to "true" (as in our case), calling exists(f) 
is unnecessary and only "upsets" S3.

The proposed fix is switch the order of the predicates:
{code}
    if (!overwrite && exists(f)) {
      throw new IOException("File already exists:"+f);
    }
{code}


> Unnecessary file existence check causes problems with S3
> --------------------------------------------------------
>
>                 Key: HADOOP-13574
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13574
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: fs/s3
>            Reporter: Alex Zolotko
>            Priority: Minor
>              Labels: s3
>
> We recently got the following exception on production:
> {noformat}
> java.io.FileNotFoundException: Key 
> 'xxx/_temporary/0/_temporary/attempt_201609010631_0000_m_001128_1128/part-01128'
>  does not exist in S3
>         at 
> org.apache.hadoop.fs.s3native.Jets3tNativeFileSystemStore.handleServiceException(Jets3tNativeFileSystemStore.java:234)
>         at 
> org.apache.hadoop.fs.s3native.Jets3tNativeFileSystemStore.copy(Jets3tNativeFileSystemStore.java:201)
>         at sun.reflect.GeneratedMethodAccessor71.invoke(Unknown Source)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:497)
>         at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:186)
>         at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>         at org.apache.hadoop.fs.s3native.$Proxy13.copy(Unknown Source)
>         at 
> org.apache.hadoop.fs.s3native.NativeS3FileSystem.rename(NativeS3FileSystem.java:659)
>         at 
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:435)
>         at 
> org.apache.hadoop.mapred.FileOutputCommitter.commitTask(FileOutputCommitter.java:172)
>         at 
> org.apache.hadoop.mapred.OutputCommitter.commitTask(OutputCommitter.java:291)
>         at 
> org.apache.spark.mapred.SparkHadoopMapRedUtil$.performCommit$1(SparkHadoopMapRedUtil.scala:98)
>         at 
> org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:124)
>         at 
> org.apache.spark.SparkHadoopWriter.commit(SparkHadoopWriter.scala:107)
>         at 
> org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1204)
>         at 
> org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopDataset$1$$anonfun$13.apply(PairRDDFunctions.scala:1183)
> {noformat}
> FileOutputCommitter.commitTask() does check that the file exists before 
> trying to rename it, but due to S3's relaxed consistency guarantees the 
> following fs.rename(taskAttemptPath, committedTaskPath) still fails.
> Here's an excerpt from the Amazon S3 documentation 
> ([https://docs.aws.amazon.com/AmazonS3/latest/dev/Introduction.html]):
> {quote}
> Amazon S3 Data Consistency Model
> Amazon S3 provides read-after-write consistency for PUTS of new objects in 
> your S3 bucket in all regions with one caveat. The caveat is that if you make 
> a HEAD or GET request to the key name (to find if the object exists) before 
> creating the object, Amazon S3 provides eventual consistency for 
> read-after-write.
> {quote}
> The problematic S3 object existence check, that causes S3 to fallback to 
> eventual consistency, is in NativeS3FileSystem.create():
> {code}
>     if (exists(f) && !overwrite) {
>       throw new IOException("File already exists:"+f);
>     }
> {code}
> If the "overwrite" parameter is set to "true" (as in our case), calling 
> exists(f) is unnecessary and only "upsets" S3.
> The proposed fix is to switch the order of the predicates:
> {code}
>     if (!overwrite && exists(f)) {
>       throw new IOException("File already exists:"+f);
>     }
> {code}



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to