You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "John Myers (JIRA)" <ji...@apache.org> on 2017/01/19 17:24:26 UTC

[jira] [Commented] (SPARK-18406) Race between end-of-task and completion iterator read lock release

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

John Myers commented on SPARK-18406:
------------------------------------

Similar issue in doing basic RDD operations (like checking for empty RDD's) within streaming jobs:

17/01/19 16:43:45 WARN BlockManager: Block input-0-1484840671538 replicated to only 0 peer(s) instead of 1 peers
17/01/19 16:43:46 WARN MetricsHelper: No metrics scope set in thread RecurringTimer - Kinesis Checkpointer - Worker localhost:65fe618d-c0a7-4fea-b710-0f1b5c6498f2, getMetricsScope returning NullMetricsScope.
17/01/19 16:44:51 ERROR Executor: Exception in task 0.0 in stage 212.0 (TID 4180)
java.lang.AssertionError: assertion failed
	at scala.Predef$.assert(Predef.scala:156)
	at org.apache.spark.storage.BlockInfo.checkInvariants(BlockInfoManager.scala:84)
	at org.apache.spark.storage.BlockInfo.readerCount_$eq(BlockInfoManager.scala:66)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2$$anonfun$apply$2.apply(BlockInfoManager.scala:362)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2$$anonfun$apply$2.apply(BlockInfoManager.scala:361)
	at scala.Option.foreach(Option.scala:257)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2.apply(BlockInfoManager.scala:361)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2.apply(BlockInfoManager.scala:356)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
	at org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:356)
	at org.apache.spark.storage.BlockManager.releaseAllLocksForTask(BlockManager.scala:646)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:281)
	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)
17/01/19 16:44:51 WARN TaskSetManager: Lost task 0.0 in stage 212.0 (TID 4180, localhost): java.lang.AssertionError: assertion failed
	at scala.Predef$.assert(Predef.scala:156)
	at org.apache.spark.storage.BlockInfo.checkInvariants(BlockInfoManager.scala:84)
	at org.apache.spark.storage.BlockInfo.readerCount_$eq(BlockInfoManager.scala:66)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2$$anonfun$apply$2.apply(BlockInfoManager.scala:362)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2$$anonfun$apply$2.apply(BlockInfoManager.scala:361)
	at scala.Option.foreach(Option.scala:257)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2.apply(BlockInfoManager.scala:361)
	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2.apply(BlockInfoManager.scala:356)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
	at org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:356)
	at org.apache.spark.storage.BlockManager.releaseAllLocksForTask(BlockManager.scala:646)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:281)
	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)

17/01/19 16:44:51 ERROR TaskSetManager: Task 0 in stage 212.0 failed 1 times; aborting job
17/01/19 16:44:51 ERROR JobScheduler: Error running job streaming job 1484844270000 ms.0
org.apache.spark.SparkException: An exception was raised by Python:
Traceback (most recent call last):
  File "/root/spark/python/lib/pyspark.zip/pyspark/streaming/util.py", line 65, in call
    r = self.func(t, *rdds)
  File "/root/spark/python/lib/pyspark.zip/pyspark/streaming/dstream.py", line 159, in <lambda>
    func = lambda t, rdd: old_func(rdd)
  File "/root/streamtest.py", line 519, in multiplex
    if not flow_rdd.isEmpty():
  File "/root/spark/python/lib/pyspark.zip/pyspark/rdd.py", line 1343, in isEmpty
    return self.getNumPartitions() == 0 or len(self.take(1)) == 0
  File "/root/spark/python/lib/pyspark.zip/pyspark/rdd.py", line 1310, in take
    res = self.context.runJob(self, takeUpToNumLeft, p)
  File "/root/spark/python/lib/pyspark.zip/pyspark/context.py", line 933, in runJob
    port = self._jvm.PythonRDD.runJob(self._jsc.sc(), mappedRDD._jrdd, partitions)
  File "/root/spark/python/lib/py4j-0.10.3-src.zip/py4j/java_gateway.py", line 1133, in __call__
    answer, self.gateway_client, self.target_id, self.name)
  File "/root/spark/python/lib/pyspark.zip/pyspark/sql/utils.py", line 63, in deco
    return f(*a, **kw)
  File "/root/spark/python/lib/py4j-0.10.3-src.zip/py4j/protocol.py", line 319, in get_return_value
    format(target_id, ".", name), value)
Py4JJavaError: An error occurred while calling z:org.apache.spark.api.python.PythonRDD.runJob.

> Race between end-of-task and completion iterator read lock release
> ------------------------------------------------------------------
>
>                 Key: SPARK-18406
>                 URL: https://issues.apache.org/jira/browse/SPARK-18406
>             Project: Spark
>          Issue Type: Bug
>          Components: Block Manager, Spark Core
>    Affects Versions: 2.0.0, 2.0.1
>            Reporter: Josh Rosen
>
> The following log comes from a production streaming job where executors periodically die due to uncaught exceptions during block release:
> {code}
> 16/11/07 17:11:06 INFO CoarseGrainedExecutorBackend: Got assigned task 7921
> 16/11/07 17:11:06 INFO Executor: Running task 0.0 in stage 2390.0 (TID 7921)
> 16/11/07 17:11:06 INFO CoarseGrainedExecutorBackend: Got assigned task 7922
> 16/11/07 17:11:06 INFO Executor: Running task 1.0 in stage 2390.0 (TID 7922)
> 16/11/07 17:11:06 INFO CoarseGrainedExecutorBackend: Got assigned task 7923
> 16/11/07 17:11:06 INFO Executor: Running task 2.0 in stage 2390.0 (TID 7923)
> 16/11/07 17:11:06 INFO TorrentBroadcast: Started reading broadcast variable 2721
> 16/11/07 17:11:06 INFO CoarseGrainedExecutorBackend: Got assigned task 7924
> 16/11/07 17:11:06 INFO Executor: Running task 3.0 in stage 2390.0 (TID 7924)
> 16/11/07 17:11:06 INFO MemoryStore: Block broadcast_2721_piece0 stored as bytes in memory (estimated size 5.0 KB, free 4.9 GB)
> 16/11/07 17:11:06 INFO TorrentBroadcast: Reading broadcast variable 2721 took 3 ms
> 16/11/07 17:11:06 INFO MemoryStore: Block broadcast_2721 stored as values in memory (estimated size 9.4 KB, free 4.9 GB)
> 16/11/07 17:11:06 INFO BlockManager: Found block rdd_2741_1 locally
> 16/11/07 17:11:06 INFO BlockManager: Found block rdd_2741_3 locally
> 16/11/07 17:11:06 INFO BlockManager: Found block rdd_2741_2 locally
> 16/11/07 17:11:06 INFO BlockManager: Found block rdd_2741_4 locally
> 16/11/07 17:11:06 INFO PythonRunner: Times: total = 2, boot = -566, init = 567, finish = 1
> 16/11/07 17:11:06 INFO PythonRunner: Times: total = 7, boot = -540, init = 541, finish = 6
> 16/11/07 17:11:06 INFO Executor: Finished task 2.0 in stage 2390.0 (TID 7923). 1429 bytes result sent to driver
> 16/11/07 17:11:06 INFO PythonRunner: Times: total = 8, boot = -532, init = 533, finish = 7
> 16/11/07 17:11:06 INFO Executor: Finished task 3.0 in stage 2390.0 (TID 7924). 1429 bytes result sent to driver
> 16/11/07 17:11:06 ERROR Executor: Exception in task 0.0 in stage 2390.0 (TID 7921)
> java.lang.AssertionError: assertion failed
> 	at scala.Predef$.assert(Predef.scala:165)
> 	at org.apache.spark.storage.BlockInfo.checkInvariants(BlockInfoManager.scala:84)
> 	at org.apache.spark.storage.BlockInfo.readerCount_$eq(BlockInfoManager.scala:66)
> 	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2$$anonfun$apply$2.apply(BlockInfoManager.scala:362)
> 	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2$$anonfun$apply$2.apply(BlockInfoManager.scala:361)
> 	at scala.Option.foreach(Option.scala:236)
> 	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2.apply(BlockInfoManager.scala:361)
> 	at org.apache.spark.storage.BlockInfoManager$$anonfun$releaseAllLocksForTask$2.apply(BlockInfoManager.scala:356)
> 	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> 	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
> 	at org.apache.spark.storage.BlockInfoManager.releaseAllLocksForTask(BlockInfoManager.scala:356)
> 	at org.apache.spark.storage.BlockManager.releaseAllLocksForTask(BlockManager.scala:646)
> 	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:281)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> 16/11/07 17:11:06 INFO CoarseGrainedExecutorBackend: Got assigned task 7925
> 16/11/07 17:11:06 INFO Executor: Running task 0.1 in stage 2390.0 (TID 7925)
> 16/11/07 17:11:06 INFO BlockManager: Found block rdd_2741_1 locally
> 16/11/07 17:11:06 INFO PythonRunner: Times: total = 41, boot = -536, init = 576, finish = 1
> 16/11/07 17:11:06 INFO Executor: Finished task 1.0 in stage 2390.0 (TID 7922). 1429 bytes result sent to driver
> 16/11/07 17:11:06 ERROR Utils: Uncaught exception in thread stdout writer for /databricks/python/bin/python
> java.lang.AssertionError: assertion failed: Block rdd_2741_1 is not locked for reading
> 	at scala.Predef$.assert(Predef.scala:179)
> 	at org.apache.spark.storage.BlockInfoManager.unlock(BlockInfoManager.scala:294)
> 	at org.apache.spark.storage.BlockManager.releaseLock(BlockManager.scala:630)
> 	at org.apache.spark.storage.BlockManager$$anonfun$1.apply$mcV$sp(BlockManager.scala:434)
> 	at org.apache.spark.util.CompletionIterator$$anon$1.completion(CompletionIterator.scala:46)
> 	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:35)
> 	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
> 	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> 	at org.apache.spark.InterruptibleIterator.foreach(InterruptibleIterator.scala:28)
> 	at org.apache.spark.api.python.PythonRDD$.writeIteratorToStream(PythonRDD.scala:504)
> 	at org.apache.spark.api.python.PythonRunner$WriterThread$$anonfun$run$3.apply(PythonRDD.scala:328)
> 	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1882)
> 	at org.apache.spark.api.python.PythonRunner$WriterThread.run(PythonRDD.scala:269)
> 16/11/07 17:11:06 ERROR SparkUncaughtExceptionHandler: Uncaught exception in thread Thread[stdout writer for /databricks/python/bin/python,5,main]
> java.lang.AssertionError: assertion failed: Block rdd_2741_1 is not locked for reading
> 	at scala.Predef$.assert(Predef.scala:179)
> 	at org.apache.spark.storage.BlockInfoManager.unlock(BlockInfoManager.scala:294)
> 	at org.apache.spark.storage.BlockManager.releaseLock(BlockManager.scala:630)
> 	at org.apache.spark.storage.BlockManager$$anonfun$1.apply$mcV$sp(BlockManager.scala:434)
> 	at org.apache.spark.util.CompletionIterator$$anon$1.completion(CompletionIterator.scala:46)
> 	at org.apache.spark.util.CompletionIterator.hasNext(CompletionIterator.scala:35)
> 	at org.apache.spark.InterruptibleIterator.hasNext(InterruptibleIterator.scala:39)
> 	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
> 	at org.apache.spark.InterruptibleIterator.foreach(InterruptibleIterator.scala:28)
> 	at org.apache.spark.api.python.PythonRDD$.writeIteratorToStream(PythonRDD.scala:504)
> 	at org.apache.spark.api.python.PythonRunner$WriterThread$$anonfun$run$3.apply(PythonRDD.scala:328)
> 	at org.apache.spark.util.Utils$.logUncaughtExceptions(Utils.scala:1882)
> 	at org.apache.spark.api.python.PythonRunner$WriterThread.run(PythonRDD.scala
> {code}
> I think that there's some sort of internal race condition between a task finishing (TID 7921) and automatically releasing locks and between some "automatically release locks on hitting the end of an iterator" logic running in a separate thread. The log above came from a production streaming job where executors periodically died with this type of error.



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

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