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

Hari Shreedharan commented on SPARK-4790:
-----------------------------------------

Ah, so this is the issue:
tracker3.cleanupOldBatches(batchTime2) eventually ends up calling deleteFiles() 
in WriteAheadLogManager which is run in another thread. The thread may or may 
not be done before the getWrittenLogData call is made. So if the deletion 
happens in between the list call and the getFileStatus call happening in HDFS, 
this race conditions gets hit.

> Flaky test in ReceivedBlockTrackerSuite: "block addition, block to batch 
> allocation, and cleanup with write ahead log"
> ----------------------------------------------------------------------------------------------------------------------
>
>                 Key: SPARK-4790
>                 URL: https://issues.apache.org/jira/browse/SPARK-4790
>             Project: Spark
>          Issue Type: Bug
>          Components: Streaming
>    Affects Versions: 1.3.0
>            Reporter: Josh Rosen
>            Assignee: Tathagata Das
>              Labels: flaky-test
>
> Found another flaky streaming test, 
> "org.apache.spark.streaming.ReceivedBlockTrackerSuite.block addition, block 
> to batch allocation and cleanup with write ahead log":
> {code}
> Error Message
> File /tmp/1418069118106-0/receivedBlockMetadata/log-0-1000 does not exist.
> Stacktrace
> sbt.ForkMain$ForkError: File 
> /tmp/1418069118106-0/receivedBlockMetadata/log-0-1000 does not exist.
>       at 
> org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:397)
>       at 
> org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:324)
>       at 
> org.apache.spark.streaming.util.WriteAheadLogSuite$.getLogFilesInDirectory(WriteAheadLogSuite.scala:344)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite.getWriteAheadLogFiles(ReceivedBlockTrackerSuite.scala:248)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite$$anonfun$4.apply$mcV$sp(ReceivedBlockTrackerSuite.scala:173)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite$$anonfun$4.apply(ReceivedBlockTrackerSuite.scala:96)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite$$anonfun$4.apply(ReceivedBlockTrackerSuite.scala:96)
>       at 
> org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
>       at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
>       at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
>       at org.scalatest.Transformer.apply(Transformer.scala:22)
>       at org.scalatest.Transformer.apply(Transformer.scala:20)
>       at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
>       at org.scalatest.Suite$class.withFixture(Suite.scala:1122)
>       at org.scalatest.FunSuite.withFixture(FunSuite.scala:1555)
>       at 
> org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
>       at 
> org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
>       at 
> org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
>       at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
>       at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite.org$scalatest$BeforeAndAfter$$super$runTest(ReceivedBlockTrackerSuite.scala:41)
>       at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite.runTest(ReceivedBlockTrackerSuite.scala:41)
>       at 
> org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
>       at 
> org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
>       at 
> org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
>       at 
> org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
>       at scala.collection.immutable.List.foreach(List.scala:318)
>       at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
>       at 
> org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
>       at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
>       at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
>       at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
>       at org.scalatest.Suite$class.run(Suite.scala:1424)
>       at 
> org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
>       at 
> org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
>       at 
> org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
>       at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
>       at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite.org$scalatest$BeforeAndAfter$$super$run(ReceivedBlockTrackerSuite.scala:41)
>       at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241)
>       at 
> org.apache.spark.streaming.ReceivedBlockTrackerSuite.run(ReceivedBlockTrackerSuite.scala:41)
>       at 
> org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
>       at 
> org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
>       at sbt.ForkMain$Run$2.call(ForkMain.java:294)
>       at sbt.ForkMain$Run$2.call(ForkMain.java:284)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>       at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>       at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>       at java.lang.Thread.run(Thread.java:745)
> {code}



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

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

Reply via email to