[jira] [Commented] (HADOOP-15469) S3A directory committer commit job fails if _temporary directory created under dest
[ https://issues.apache.org/jira/browse/HADOOP-15469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479527#comment-16479527 ] Hudson commented on HADOOP-15469: - SUCCESS: Integrated in Jenkins build Hadoop-trunk-Commit #14220 (See [https://builds.apache.org/job/Hadoop-trunk-Commit/14220/]) HADOOP-15469. S3A directory committer commit job fails if _temporary (stevel: rev cc3600aabdca6e8b14c9fe02fe54073bf4ef7685) * (edit) hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java * (edit) hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java > S3A directory committer commit job fails if _temporary directory created > under dest > --- > > Key: HADOOP-15469 > URL: https://issues.apache.org/jira/browse/HADOOP-15469 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.1.0 > Environment: spark test runs >Reporter: Steve Loughran >Assignee: Steve Loughran >Priority: Major > Fix For: 3.1.1 > > Attachments: HADOOP-15469-001.patch > > > The directory staging committer fails in commit job if any temporary > files/dirs have been created. Spark work can create such a dir for placement > of absolute files. > This is because commitJob() looks for the dest dir existing, not containing > non-hidden files. > As the comment says, "its kind of superfluous". More specifically, it means > jobs which would commit with the classic committer & overwrite=false will fail > Proposed fix: remove the check -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-15469) S3A directory committer commit job fails if _temporary directory created under dest
[ https://issues.apache.org/jira/browse/HADOOP-15469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479458#comment-16479458 ] Aaron Fabbri commented on HADOOP-15469: --- And even before change, files could arrive in job output directory during the commit process. So the window is just larger, right? My interpretation is: the job driver / app. master is enforcing the "at most once commit" anyways, so this is more of a sanity check. I think the docs already spell out the conflict behavior as happening in job setup. +1 LGTM I will apply the patch and run tests while i'm in meetings today, but am ok with you committing this now based on it being a fairly small change. Will shout if I see any issues. > S3A directory committer commit job fails if _temporary directory created > under dest > --- > > Key: HADOOP-15469 > URL: https://issues.apache.org/jira/browse/HADOOP-15469 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.1.0 > Environment: spark test runs >Reporter: Steve Loughran >Assignee: Steve Loughran >Priority: Major > Attachments: HADOOP-15469-001.patch > > > The directory staging committer fails in commit job if any temporary > files/dirs have been created. Spark work can create such a dir for placement > of absolute files. > This is because commitJob() looks for the dest dir existing, not containing > non-hidden files. > As the comment says, "its kind of superfluous". More specifically, it means > jobs which would commit with the classic committer & overwrite=false will fail > Proposed fix: remove the check -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-15469) S3A directory committer commit job fails if _temporary directory created under dest
[ https://issues.apache.org/jira/browse/HADOOP-15469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16479174#comment-16479174 ] Steve Loughran commented on HADOOP-15469: - Risk of change? only if you execute a job where at job setup all was good and yet at job complete something not _temporary has arrived. Its just a safety check , but one which isn't handling things. If we wanted to retain it, you could do something like if (exists(dest)) ls dest, filter out temp _* and .* files then fail iff the filtered list is non-empty > S3A directory committer commit job fails if _temporary directory created > under dest > --- > > Key: HADOOP-15469 > URL: https://issues.apache.org/jira/browse/HADOOP-15469 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.1.0 > Environment: spark test runs >Reporter: Steve Loughran >Assignee: Steve Loughran >Priority: Major > Attachments: HADOOP-15469-001.patch > > > The directory staging committer fails in commit job if any temporary > files/dirs have been created. Spark work can create such a dir for placement > of absolute files. > This is because commitJob() looks for the dest dir existing, not containing > non-hidden files. > As the comment says, "its kind of superfluous". More specifically, it means > jobs which would commit with the classic committer & overwrite=false will fail > Proposed fix: remove the check -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-15469) S3A directory committer commit job fails if _temporary directory created under dest
[ https://issues.apache.org/jira/browse/HADOOP-15469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16478057#comment-16478057 ] Aaron Fabbri commented on HADOOP-15469: --- Interesting. The argument for this seems to be (1) this is a case that works with FileOutputCommitter and (2) this does not harm any important uses of job commit conflict resolution. The current docs seem to be congruent with this: {quote} The Directory Committer uses the entire directory tree for conflict resolution. If any file exists at the destination it will fail in job setup; if the resolution mechanism is "replace" then all existing files will be deleted. {quote} I didn't notice any docs that really need updating here. Any risks of this change? I'm not thinking of any. > S3A directory committer commit job fails if _temporary directory created > under dest > --- > > Key: HADOOP-15469 > URL: https://issues.apache.org/jira/browse/HADOOP-15469 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.1.0 > Environment: spark test runs >Reporter: Steve Loughran >Assignee: Steve Loughran >Priority: Major > Attachments: HADOOP-15469-001.patch > > > The directory staging committer fails in commit job if any temporary > files/dirs have been created. Spark work can create such a dir for placement > of absolute files. > This is because commitJob() looks for the dest dir existing, not containing > non-hidden files. > As the comment says, "its kind of superfluous". More specifically, it means > jobs which would commit with the classic committer & overwrite=false will fail > Proposed fix: remove the check -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-15469) S3A directory committer commit job fails if _temporary directory created under dest
[ https://issues.apache.org/jira/browse/HADOOP-15469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16476170#comment-16476170 ] genericqa commented on HADOOP-15469: | (/) *{color:green}+1 overall{color}* | \\ \\ || Vote || Subsystem || Runtime || Comment || | {color:blue}0{color} | {color:blue} reexec {color} | {color:blue} 0m 23s{color} | {color:blue} Docker mode activated. {color} | || || || || {color:brown} Prechecks {color} || | {color:green}+1{color} | {color:green} @author {color} | {color:green} 0m 0s{color} | {color:green} The patch does not contain any @author tags. {color} | | {color:green}+1{color} | {color:green} test4tests {color} | {color:green} 0m 0s{color} | {color:green} The patch appears to include 1 new or modified test files. {color} | || || || || {color:brown} trunk Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 24m 53s{color} | {color:green} trunk passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 28s{color} | {color:green} trunk passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 18s{color} | {color:green} trunk passed {color} | | {color:green}+1{color} | {color:green} mvnsite {color} | {color:green} 0m 32s{color} | {color:green} trunk passed {color} | | {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 10m 58s{color} | {color:green} branch has no errors when building and testing our client artifacts. {color} | | {color:green}+1{color} | {color:green} findbugs {color} | {color:green} 0m 38s{color} | {color:green} trunk passed {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 20s{color} | {color:green} trunk passed {color} | || || || || {color:brown} Patch Compile Tests {color} || | {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 0m 33s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} compile {color} | {color:green} 0m 26s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javac {color} | {color:green} 0m 26s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} checkstyle {color} | {color:green} 0m 15s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} mvnsite {color} | {color:green} 0m 29s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} whitespace {color} | {color:green} 0m 0s{color} | {color:green} The patch has no whitespace issues. {color} | | {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 11m 41s{color} | {color:green} patch has no errors when building and testing our client artifacts. {color} | | {color:green}+1{color} | {color:green} findbugs {color} | {color:green} 0m 46s{color} | {color:green} the patch passed {color} | | {color:green}+1{color} | {color:green} javadoc {color} | {color:green} 0m 19s{color} | {color:green} the patch passed {color} | || || || || {color:brown} Other Tests {color} || | {color:green}+1{color} | {color:green} unit {color} | {color:green} 4m 44s{color} | {color:green} hadoop-aws in the patch passed. {color} | | {color:green}+1{color} | {color:green} asflicense {color} | {color:green} 0m 21s{color} | {color:green} The patch does not generate ASF License warnings. {color} | | {color:black}{color} | {color:black} {color} | {color:black} 58m 12s{color} | {color:black} {color} | \\ \\ || Subsystem || Report/Notes || | Docker | Client=17.05.0-ce Server=17.05.0-ce Image:yetus/hadoop:abb62dd | | JIRA Issue | HADOOP-15469 | | JIRA Patch URL | https://issues.apache.org/jira/secure/attachment/12923511/HADOOP-15469-001.patch | | Optional Tests | asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle | | uname | Linux c4406e655832 3.13.0-137-generic #186-Ubuntu SMP Mon Dec 4 19:09:19 UTC 2017 x86_64 x86_64 x86_64 GNU/Linux | | Build tool | maven | | Personality | /testptch/patchprocess/precommit/personality/provided.sh | | git revision | trunk / 07d8505 | | maven | version: Apache Maven 3.3.9 | | Default Java | 1.8.0_162 | | findbugs | v3.1.0-RC1 | | Test Results | https://builds.apache.org/job/PreCommit-HADOOP-Build/14631/testReport/ | | Max. process+thread count | 301 (vs. ulimit of 1) | | modules | C: hadoop-tools/hadoop-aws U: hadoop-tools/hadoop-aws | | Console output | https://builds.apache.org/job/PreCommit-HADOOP-Build/14631/console | | Powered by | Apache Yetus 0.8.0-SNAPSHOT http://yetus.apache.org | This message was automatically generated. > S3A directory committer commit job fails if _temporary directory created > under dest > --
[jira] [Commented] (HADOOP-15469) S3A directory committer commit job fails if _temporary directory created under dest
[ https://issues.apache.org/jira/browse/HADOOP-15469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16476055#comment-16476055 ] Steve Loughran commented on HADOOP-15469: - Patch 001. Remove test, fix mock tests to verify that the behaviour has now changed. There's still an existence check in job setup, we just don't overreact otherwise. The alternative would be a more complex & brittle scan for >1 non-temp entry. I think that's overkill Testing: s3 ireland. First run failed with HADOOP-14946 over a slow network & many workers; rerun made it go away. FYI, [~rdblue] > S3A directory committer commit job fails if _temporary directory created > under dest > --- > > Key: HADOOP-15469 > URL: https://issues.apache.org/jira/browse/HADOOP-15469 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.1.0 > Environment: spark test runs >Reporter: Steve Loughran >Assignee: Steve Loughran >Priority: Major > Attachments: HADOOP-15469-001.patch > > > The directory staging committer fails in commit job if any temporary > files/dirs have been created. Spark work can create such a dir for placement > of absolute files. > This is because commitJob() looks for the dest dir existing, not containing > non-hidden files. > As the comment says, "its kind of superfluous". More specifically, it means > jobs which would commit with the classic committer & overwrite=false will fail > Proposed fix: remove the check -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: common-issues-h...@hadoop.apache.org
[jira] [Commented] (HADOOP-15469) S3A directory committer commit job fails if _temporary directory created under dest
[ https://issues.apache.org/jira/browse/HADOOP-15469?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16475949#comment-16475949 ] Steve Loughran commented on HADOOP-15469: - stack. * The job setup has already done the path doesn't exists check; this is job commit * I'm running s3guard in debug, and it says the only path which exists underneath is _temporary, and even that is empty. {code} org.apache.hadoop.fs.PathExistsException: `s3a://hwdev-steve-new/spark_committer/orc': Destination path exists and committer conflict resolution mode is "fail" at org.apache.hadoop.fs.s3a.commit.staging.DirectoryStagingCommitter.preCommitJob(DirectoryStagingCommitter.java:99) at org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter.commitJob(AbstractS3ACommitter.java:576) at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.commitJob(HadoopMapReduceCommitProtocol.scala:166) at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:213) at org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand.run(InsertIntoHadoopFsRelationCommand.scala:154) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:104) at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:102) at org.apache.spark.sql.execution.command.DataWritingCommandExec.doExecute(commands.scala:122) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:131) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:127) at org.apache.spark.sql.execution.SparkPlan$$anonfun$executeQuery$1.apply(SparkPlan.scala:155) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.sql.execution.SparkPlan.executeQuery(SparkPlan.scala:152) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:127) at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:80) at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:80) at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:654) at org.apache.spark.sql.DataFrameWriter$$anonfun$runCommand$1.apply(DataFrameWriter.scala:654) at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:77) at org.apache.spark.sql.DataFrameWriter.runCommand(DataFrameWriter.scala:654) at org.apache.spark.sql.DataFrameWriter.saveToV1Source(DataFrameWriter.scala:273) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:267) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:225) 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 py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244) at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357) at py4j.Gateway.invoke(Gateway.java:282) at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132) at py4j.commands.CallCommand.execute(CallCommand.java:79) at py4j.GatewayConnection.run(GatewayConnection.java:214) at java.lang.Thread.run(Thread.java:748) {code} > S3A directory committer commit job fails if _temporary directory created > under dest > --- > > Key: HADOOP-15469 > URL: https://issues.apache.org/jira/browse/HADOOP-15469 > Project: Hadoop Common > Issue Type: Sub-task > Components: fs/s3 >Affects Versions: 3.1.0 > Environment: spark test runs >Reporter: Steve Loughran >Assignee: Steve Loughran >Priority: Major > > The directory staging committer fails in commit job if any temporary > files/dirs have been created. Spark work can create such a dir for placement > of absolute files. > This is because commitJob() looks for the dest dir existing, not containing > non-hidden files. > As the comment says, "its kind of superfluous". More specifically, it means > jobs which would commit with the classic committer & overwrite=false will fail > Proposed fix: remove the check -- This message was sent by Atlassian JIRA (v7.6.3#76005) - To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org For additi