You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Rostyslav Sotnychenko <r....@gmail.com> on 2017/01/13 10:44:04 UTC

Both Spark AM and Client are trying to delete Staging Directory

Hi all!

I am a bit confused why Spark AM and Client are both trying to delete
Staging Directory.

https://github.com/apache/spark/blob/branch-2.1/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L1110
https://github.com/apache/spark/blob/branch-2.1/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala#L233

As you can see, in case if a job was running on YARN in Cluster deployment
mode, both AM and Client will try to delete Staging directory if job
succeeded and eventually one of them will fail to do this, because the
other one already deleted the directory.

Shouldn't we add some check to Client?


Thanks,
Rostyslav

Re: Both Spark AM and Client are trying to delete Staging Directory

Posted by Steve Loughran <st...@hortonworks.com>.
On 16 Jan 2017, at 12:51, Rostyslav Sotnychenko <r....@gmail.com>> wrote:

Thanks all!

I was using another DFS instead of HDFS, which was logging an error when fs.delete got called on non-existing path.


really? Whose DFS, if you don't mind me asking? I'm surprised they logged that delete() of a missing path, as it's not entirely uncommon to happen during cleanup

In Spark 2.0.1 which I was using previously, everything was working fine because existence of an additional check that was made prior to deleting. However that check got removed in 2.1 (SPARK-16736<https://issues.apache.org/jira/browse/SPARK-16736>, commit<https://github.com/apache/spark/pull/14371/files#diff-b050df3f55b82065803d6e83453b9706>), so I started seeing an error from my DFS.

Its not a problem in any way (i.e. it does not affect Spark job in any way), so everything is fine. I just wanted to make sure its not a Spark issue.


Thanks,
Rostyslav



No, not a problem. Really that whole idea of having delete() return true/false is pretty useless: nobody really knows what it means when it returns false. It should just have been void() wth an exception thrown if something actually failed. That's essentially what they all do, though I've never seen any which complains about the situation.

mkdirs(), now there's one to fear. Not even the java.io<http://java.io> API clearly defines what "false" coming back from there means, as it can mean both "ther'es a directory there, so I didnt' do any work", or "there's a file/symlnk/mount point/device there which is a probably a serious problem"

Re: Both Spark AM and Client are trying to delete Staging Directory

Posted by Rostyslav Sotnychenko <r....@gmail.com>.
Thanks all!

I was using another DFS instead of HDFS, which was logging an error when
fs.delete got called on non-existing path.
In Spark 2.0.1 which I was using previously, everything was working fine
because existence of an additional check that was made prior to deleting.
However that check got removed in 2.1 (SPARK-16736
<https://issues.apache.org/jira/browse/SPARK-16736>, commit
<https://github.com/apache/spark/pull/14371/files#diff-b050df3f55b82065803d6e83453b9706>),
so I started seeing an error from my DFS.

Its not a problem in any way (i.e. it does not affect Spark job in any
way), so everything is fine. I just wanted to make sure its not a Spark
issue.


Thanks,
Rostyslav

On Sun, Jan 15, 2017 at 3:19 PM, Liang-Chi Hsieh <vi...@gmail.com> wrote:

>
> Hi,
>
> Will it be a problem if the staging directory is already deleted? Because
> even the directory doesn't exist, fs.delete(stagingDirPath, true) won't
> cause failure but just return false.
>
>
> Rostyslav Sotnychenko wrote
> > Hi all!
> >
> > I am a bit confused why Spark AM and Client are both trying to delete
> > Staging Directory.
> >
> > https://github.com/apache/spark/blob/branch-2.1/yarn/
> src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L1110
> > https://github.com/apache/spark/blob/branch-2.1/yarn/
> src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala#L233
> >
> > As you can see, in case if a job was running on YARN in Cluster
> deployment
> > mode, both AM and Client will try to delete Staging directory if job
> > succeeded and eventually one of them will fail to do this, because the
> > other one already deleted the directory.
> >
> > Shouldn't we add some check to Client?
> >
> >
> > Thanks,
> > Rostyslav
>
>
>
>
>
> -----
> Liang-Chi Hsieh | @viirya
> Spark Technology Center
> http://www.spark.tc/
> --
> View this message in context: http://apache-spark-
> developers-list.1001551.n3.nabble.com/Both-Spark-AM-and-
> Client-are-trying-to-delete-Staging-Directory-tp20588p20600.html
> Sent from the Apache Spark Developers List mailing list archive at
> Nabble.com.
>
> ---------------------------------------------------------------------
> To unsubscribe e-mail: dev-unsubscribe@spark.apache.org
>
>

Re: Both Spark AM and Client are trying to delete Staging Directory

Posted by Liang-Chi Hsieh <vi...@gmail.com>.
Hi,

Will it be a problem if the staging directory is already deleted? Because
even the directory doesn't exist, fs.delete(stagingDirPath, true) won't
cause failure but just return false.


Rostyslav Sotnychenko wrote
> Hi all!
> 
> I am a bit confused why Spark AM and Client are both trying to delete
> Staging Directory.
> 
> https://github.com/apache/spark/blob/branch-2.1/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala#L1110
> https://github.com/apache/spark/blob/branch-2.1/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala#L233
> 
> As you can see, in case if a job was running on YARN in Cluster deployment
> mode, both AM and Client will try to delete Staging directory if job
> succeeded and eventually one of them will fail to do this, because the
> other one already deleted the directory.
> 
> Shouldn't we add some check to Client?
> 
> 
> Thanks,
> Rostyslav





-----
Liang-Chi Hsieh | @viirya 
Spark Technology Center 
http://www.spark.tc/ 
--
View this message in context: http://apache-spark-developers-list.1001551.n3.nabble.com/Both-Spark-AM-and-Client-are-trying-to-delete-Staging-Directory-tp20588p20600.html
Sent from the Apache Spark Developers List mailing list archive at Nabble.com.

---------------------------------------------------------------------
To unsubscribe e-mail: dev-unsubscribe@spark.apache.org