spark-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Denis Gabaydulin (JIRA)" <j...@apache.org>
Subject [jira] [Comment Edited] (SPARK-15905) Driver hung while writing to console progress bar
Date Thu, 26 Oct 2017 08:43:00 GMT

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

Denis Gabaydulin edited comment on SPARK-15905 at 10/26/17 8:42 AM:
--------------------------------------------------------------------

Not sure I've got the same issue. But I have at least two threads which are blocked on a logger.
A first is main (where I called a unpresist() method).

{noformat}
Thread 30581: (state = BLOCKED)
 - org.apache.log4j.Category.callAppenders(org.apache.log4j.spi.LoggingEvent) @bci=12, line=204
(Compiled frame)
 - org.apache.log4j.Category.forcedLog(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=14, line=391 (Compiled frame)
 - org.apache.log4j.Category.log(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=34, line=856 (Compiled frame)
 - org.slf4j.impl.Log4jLoggerAdapter.warn(java.lang.String, java.lang.Throwable) @bci=12,
line=479 (Interpreted frame)
 - org.apache.spark.internal.Logging$class.logWarning(org.apache.spark.internal.Logging, scala.Function0,
java.lang.Throwable) @bci=30, line=87 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.logWarning(scala.Function0, java.lang.Throwable) @bci=3,
line=30 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, org.apache.spark.rpc.RpcTimeout,
scala.reflect.ClassTag) @bci=32, line=111 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, scala.reflect.ClassTag)
@bci=7, line=78 (Compiled frame)
 - org.apache.spark.storage.BlockManagerMaster.removeRdd(int, boolean) @bci=21, line=119 (Compiled
frame)
 - org.apache.spark.SparkContext.unpersistRDD(int, boolean) @bci=12, line=1705 (Compiled frame)
 - org.apache.spark.rdd.RDD.unpersist(boolean) @bci=21, line=216 (Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager$$anonfun$uncacheQuery$1.apply$mcZ$sp() @bci=70,
line=116 (Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager$$anonfun$uncacheQuery$1.apply() @bci=1, line=111
(Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager$$anonfun$uncacheQuery$1.apply() @bci=1, line=111
(Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager.writeLock(scala.Function0) @bci=13, line=65
(Compiled frame)
 - org.apache.spark.sql.execution.CacheManager.uncacheQuery(org.apache.spark.sql.Dataset,
boolean) @bci=11, line=111 (Interpreted frame)
 - org.apache.spark.sql.Dataset.unpersist(boolean) @bci=12, line=2526 (Interpreted frame)
 - org.apache.spark.sql.Dataset.unpersist() @bci=2, line=2536 (Interpreted frame)
 - ru.ok.dwh.analytics.user.kpi.service.KpiBaseMetricDailyAggregator.complete(boolean) @bci=4,
line=68 (Interpreted frame)
 - ru.ok.dwh.analytics.service.v2.BaseSparkDatasetTransformation.complete() @bci=2, line=70
(Interpreted frame)
 - ru.ok.dwh.analytics.application.StandardApplication.run(java.lang.String[]) @bci=232, line=109
(Interpreted frame)
 - ru.ok.dwh.analytics.application.kpi.KpiVideoBaseMetricApp.main(java.lang.String[]) @bci=51,
line=53 (Interpreted frame)
 - sun.reflect.NativeMethodAccessorImpl.invoke0(java.lang.reflect.Method, java.lang.Object,
java.lang.Object[]) @bci=0 (Interpreted frame)
 - sun.reflect.NativeMethodAccessorImpl.invoke(java.lang.Object, java.lang.Object[]) @bci=100,
line=62 (Interpreted frame)
 - sun.reflect.DelegatingMethodAccessorImpl.invoke(java.lang.Object, java.lang.Object[]) @bci=6,
line=43 (Interpreted frame)
 - java.lang.reflect.Method.invoke(java.lang.Object, java.lang.Object[]) @bci=56, line=498
(Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(scala.collection.Seq,
scala.collection.Seq, scala.collection.mutable.Map, java.lang.String, boolean) @bci=663, line=738
(Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.doRunMain$1(org.apache.spark.deploy.SparkSubmitArguments,
scala.collection.Seq, scala.collection.Seq, scala.collection.mutable.Map, java.lang.String)
@bci=18, line=187 (Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.submit(org.apache.spark.deploy.SparkSubmitArguments)
@bci=245, line=212 (Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.main(java.lang.String[]) @bci=76, line=126 (Interpreted
frame)
 - org.apache.spark.deploy.SparkSubmit.main(java.lang.String[]) @bci=4 (Interpreted frame)
{noformat}

And, a couple of spark internal methods

{noformat}
Thread 30910: (state = BLOCKED)
 - org.apache.log4j.Category.callAppenders(org.apache.log4j.spi.LoggingEvent) @bci=12, line=204
(Compiled frame)
 - org.apache.log4j.Category.forcedLog(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=14, line=391 (Compiled frame)
 - org.apache.log4j.Category.log(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=34, line=856 (Compiled frame)
 - org.slf4j.impl.Log4jLoggerAdapter.warn(java.lang.String) @bci=12, line=401 (Compiled frame)
 - org.apache.spark.internal.Logging$class.logWarning(org.apache.spark.internal.Logging, scala.Function0)
@bci=29, line=66 (Compiled frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv.logWarning(scala.Function0) @bci=2, line=43 (Interpreted
frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv.org$apache$spark$rpc$netty$NettyRpcEnv$$onSuccess$1(java.lang.Object,
scala.concurrent.Promise) @bci=63, line=214 (Interpreted frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$ask$2.apply(scala.util.Try) @bci=30, line=222
(Interpreted frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$ask$2.apply(java.lang.Object) @bci=5, line=221
(Interpreted frame)
 - scala.concurrent.impl.CallbackRunnable.run() @bci=26, line=32 (Compiled frame)
 - org.spark_project.guava.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(java.lang.Runnable)
@bci=5, line=293 (Compiled frame)
 - scala.concurrent.impl.ExecutionContextImpl$$anon$1.execute(java.lang.Runnable) @bci=5,
line=136 (Compiled frame)
 - scala.concurrent.impl.CallbackRunnable.executeWithValue(scala.util.Try) @bci=28, line=40
(Compiled frame)
 - scala.concurrent.impl.Promise$DefaultPromise.tryComplete(scala.util.Try) @bci=62, line=248
(Compiled frame)
 - scala.concurrent.Promise$class.complete(scala.concurrent.Promise, scala.util.Try) @bci=2,
line=55 (Compiled frame)
 - scala.concurrent.impl.Promise$DefaultPromise.complete(scala.util.Try) @bci=2, line=153
(Compiled frame)
 - scala.concurrent.Promise$class.success(scala.concurrent.Promise, java.lang.Object) @bci=9,
line=86 (Compiled frame)
 - scala.concurrent.impl.Promise$DefaultPromise.success(java.lang.Object) @bci=2, line=153
(Compiled frame)
 - org.apache.spark.rpc.netty.LocalNettyRpcCallContext.send(java.lang.Object) @bci=5, line=50
(Compiled frame)
 - org.apache.spark.rpc.netty.NettyRpcCallContext.reply(java.lang.Object) @bci=2, line=32
(Compiled frame)
 - org.apache.spark.storage.BlockManagerMasterEndpoint$$anonfun$receiveAndReply$1.applyOrElse(java.lang.Object,
scala.Function1) @bci=859, line=125 (Compiled frame)
 - org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp() @bci=65, line=105 (Compiled
frame)
 - org.apache.spark.rpc.netty.Inbox.safelyCall(org.apache.spark.rpc.RpcEndpoint, scala.Function0)
@bci=1, line=205 (Compiled frame)
 - org.apache.spark.rpc.netty.Inbox.process(org.apache.spark.rpc.netty.Dispatcher) @bci=85,
line=101 (Compiled frame)
 - org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run() @bci=133, line=213 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
@bci=95, line=1142 (Interpreted frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=617 (Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=745 (Interpreted frame)
{noformat}



{noformat}
Thread 22827: (state = BLOCKED)
 - java.nio.CharBuffer.wrap(char[], int, int) @bci=0, line=373 (Compiled frame)
 - sun.nio.cs.StreamEncoder.implWrite(char[], int, int) @bci=3, line=265 (Compiled frame)
 - sun.nio.cs.StreamEncoder.write(char[], int, int) @bci=60, line=125 (Compiled frame)
 - sun.nio.cs.StreamEncoder.write(java.lang.String, int, int) @bci=33, line=135 (Compiled
frame)
 - java.io.OutputStreamWriter.write(java.lang.String, int, int) @bci=7, line=220 (Compiled
frame)
 - java.io.Writer.write(java.lang.String) @bci=7, line=157 (Compiled frame)
 - org.apache.log4j.helpers.QuietWriter.write(java.lang.String) @bci=9, line=48 (Compiled
frame)
 - org.apache.log4j.WriterAppender.subAppend(org.apache.log4j.spi.LoggingEvent) @bci=54, line=317
(Compiled frame)
 - org.apache.log4j.WriterAppender.append(org.apache.log4j.spi.LoggingEvent) @bci=10, line=162
(Compiled frame)
 - org.apache.log4j.AppenderSkeleton.doAppend(org.apache.log4j.spi.LoggingEvent) @bci=106,
line=251 (Compiled frame)
 - org.apache.log4j.helpers.AppenderAttachableImpl.appendLoopOnAppenders(org.apache.log4j.spi.LoggingEvent)
@bci=41, line=66 (Compiled frame)
 - org.apache.log4j.Category.callAppenders(org.apache.log4j.spi.LoggingEvent) @bci=26, line=206
(Compiled frame)
 - org.apache.log4j.Category.forcedLog(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=14, line=391 (Compiled frame)
 - org.apache.log4j.Category.log(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=34, line=856 (Compiled frame)
 - org.slf4j.impl.Log4jLoggerAdapter.warn(java.lang.String, java.lang.Throwable) @bci=12,
line=479 (Interpreted frame)
 - org.apache.spark.internal.Logging$class.logWarning(org.apache.spark.internal.Logging, scala.Function0,
java.lang.Throwable) @bci=30, line=87 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.logWarning(scala.Function0, java.lang.Throwable) @bci=3,
line=30 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, org.apache.spark.rpc.RpcTimeout,
scala.reflect.ClassTag) @bci=32, line=111 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, scala.reflect.ClassTag)
@bci=7, line=78 (Compiled frame)
 - org.apache.spark.storage.BlockManagerMaster.updateBlockInfo(org.apache.spark.storage.BlockManagerId,
org.apache.spark.storage.BlockId, org.apache.spark.storage.StorageLevel, long, long) 
@bci=24, line=75 (Compiled frame)
 - org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$tryToReportBlockStatus(org.apache.spark.storage.BlockId,
org.apache.spark.storage.BlockStatus, long) @bci=37,
 line=387 (Compiled frame)
 - org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$reportBlockStatus(org.apache.spark.storage.BlockId,
org.apache.spark.storage.BlockStatus, long) @bci=4, line=
366 (Compiled frame)
 - org.apache.spark.storage.BlockManager.removeBlockInternal(org.apache.spark.storage.BlockId,
boolean) @bci=65, line=1345 (Compiled frame)
 - org.apache.spark.storage.BlockManager.removeBlock(org.apache.spark.storage.BlockId, boolean)
@bci=102, line=1327 (Compiled frame)
 - org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(org.apache.spark.storage.BroadcastBlockId)
@bci=9, line=1313 (Compiled frame)
 - org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(java.lang.Object)
@bci=5, line=1313 (Compiled frame)
 - scala.collection.Iterator$class.foreach(scala.collection.Iterator, scala.Function1) @bci=16,
line=893 (Compiled frame)
 - scala.collection.AbstractIterator.foreach(scala.Function1) @bci=2, line=1336 (Compiled
frame)
 - org.apache.spark.storage.BlockManager.removeBroadcast(long, boolean) @bci=60, line=1313
(Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply$mcI$sp()
@bci=15, line=66 (Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply()
@bci=1, line=66 (Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply()
@bci=1, line=66 (Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$1.apply() @bci=19, line=82
(Compiled frame)
 - scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1() @bci=8, line=24
(Compiled frame)
 - scala.concurrent.impl.Future$PromiseCompletingRunnable.run() @bci=5, line=24 (Compiled
frame)
 - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
@bci=95, line=1142 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=617 (Compiled frame)
 - java.lang.Thread.run() @bci=11, line=745 (Compiled frame)
{noformat}



was (Author: sherman81):
Not sure I've got the same issue. But I have at least two threads which are blocked on a logger.
A first is main (where I called a unpresist() method).

{noformat}
Thread 30581: (state = BLOCKED)
 - org.apache.log4j.Category.callAppenders(org.apache.log4j.spi.LoggingEvent) @bci=12, line=204
(Compiled frame)
 - org.apache.log4j.Category.forcedLog(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=14, line=391 (Compiled frame)
 - org.apache.log4j.Category.log(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=34, line=856 (Compiled frame)
 - org.slf4j.impl.Log4jLoggerAdapter.warn(java.lang.String, java.lang.Throwable) @bci=12,
line=479 (Interpreted frame)
 - org.apache.spark.internal.Logging$class.logWarning(org.apache.spark.internal.Logging, scala.Function0,
java.lang.Throwable) @bci=30, line=87 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.logWarning(scala.Function0, java.lang.Throwable) @bci=3,
line=30 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, org.apache.spark.rpc.RpcTimeout,
scala.reflect.ClassTag) @bci=32, line=111 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, scala.reflect.ClassTag)
@bci=7, line=78 (Compiled frame)
 - org.apache.spark.storage.BlockManagerMaster.removeRdd(int, boolean) @bci=21, line=119 (Compiled
frame)
 - org.apache.spark.SparkContext.unpersistRDD(int, boolean) @bci=12, line=1705 (Compiled frame)
 - org.apache.spark.rdd.RDD.unpersist(boolean) @bci=21, line=216 (Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager$$anonfun$uncacheQuery$1.apply$mcZ$sp() @bci=70,
line=116 (Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager$$anonfun$uncacheQuery$1.apply() @bci=1, line=111
(Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager$$anonfun$uncacheQuery$1.apply() @bci=1, line=111
(Interpreted frame)
 - org.apache.spark.sql.execution.CacheManager.writeLock(scala.Function0) @bci=13, line=65
(Compiled frame)
 - org.apache.spark.sql.execution.CacheManager.uncacheQuery(org.apache.spark.sql.Dataset,
boolean) @bci=11, line=111 (Interpreted frame)
 - org.apache.spark.sql.Dataset.unpersist(boolean) @bci=12, line=2526 (Interpreted frame)
 - org.apache.spark.sql.Dataset.unpersist() @bci=2, line=2536 (Interpreted frame)
 - ru.ok.dwh.analytics.user.kpi.service.KpiBaseMetricDailyAggregator.complete(boolean) @bci=4,
line=68 (Interpreted frame)
 - ru.ok.dwh.analytics.service.v2.BaseSparkDatasetTransformation.complete() @bci=2, line=70
(Interpreted frame)
 - ru.ok.dwh.analytics.application.StandardApplication.run(java.lang.String[]) @bci=232, line=109
(Interpreted frame)
 - ru.ok.dwh.analytics.application.kpi.KpiVideoBaseMetricApp.main(java.lang.String[]) @bci=51,
line=53 (Interpreted frame)
 - sun.reflect.NativeMethodAccessorImpl.invoke0(java.lang.reflect.Method, java.lang.Object,
java.lang.Object[]) @bci=0 (Interpreted frame)
 - sun.reflect.NativeMethodAccessorImpl.invoke(java.lang.Object, java.lang.Object[]) @bci=100,
line=62 (Interpreted frame)
 - sun.reflect.DelegatingMethodAccessorImpl.invoke(java.lang.Object, java.lang.Object[]) @bci=6,
line=43 (Interpreted frame)
 - java.lang.reflect.Method.invoke(java.lang.Object, java.lang.Object[]) @bci=56, line=498
(Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(scala.collection.Seq,
scala.collection.Seq, scala.collection.mutable.Map, java.lang.String, boolean) @bci=663, line=738
(Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.doRunMain$1(org.apache.spark.deploy.SparkSubmitArguments,
scala.collection.Seq, scala.collection.Seq, scala.collection.mutable.Map, java.lang.String)
@bci=18, line=187 (Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.submit(org.apache.spark.deploy.SparkSubmitArguments)
@bci=245, line=212 (Interpreted frame)
 - org.apache.spark.deploy.SparkSubmit$.main(java.lang.String[]) @bci=76, line=126 (Interpreted
frame)
 - org.apache.spark.deploy.SparkSubmit.main(java.lang.String[]) @bci=4 (Interpreted frame)
{noformat}

And, a couple of spark internal methods

{noforamt}
Thread 30910: (state = BLOCKED)
 - org.apache.log4j.Category.callAppenders(org.apache.log4j.spi.LoggingEvent) @bci=12, line=204
(Compiled frame)
 - org.apache.log4j.Category.forcedLog(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=14, line=391 (Compiled frame)
 - org.apache.log4j.Category.log(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=34, line=856 (Compiled frame)
 - org.slf4j.impl.Log4jLoggerAdapter.warn(java.lang.String) @bci=12, line=401 (Compiled frame)
 - org.apache.spark.internal.Logging$class.logWarning(org.apache.spark.internal.Logging, scala.Function0)
@bci=29, line=66 (Compiled frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv.logWarning(scala.Function0) @bci=2, line=43 (Interpreted
frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv.org$apache$spark$rpc$netty$NettyRpcEnv$$onSuccess$1(java.lang.Object,
scala.concurrent.Promise) @bci=63, line=214 (Interpreted frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$ask$2.apply(scala.util.Try) @bci=30, line=222
(Interpreted frame)
 - org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$ask$2.apply(java.lang.Object) @bci=5, line=221
(Interpreted frame)
 - scala.concurrent.impl.CallbackRunnable.run() @bci=26, line=32 (Compiled frame)
 - org.spark_project.guava.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(java.lang.Runnable)
@bci=5, line=293 (Compiled frame)
 - scala.concurrent.impl.ExecutionContextImpl$$anon$1.execute(java.lang.Runnable) @bci=5,
line=136 (Compiled frame)
 - scala.concurrent.impl.CallbackRunnable.executeWithValue(scala.util.Try) @bci=28, line=40
(Compiled frame)
 - scala.concurrent.impl.Promise$DefaultPromise.tryComplete(scala.util.Try) @bci=62, line=248
(Compiled frame)
 - scala.concurrent.Promise$class.complete(scala.concurrent.Promise, scala.util.Try) @bci=2,
line=55 (Compiled frame)
 - scala.concurrent.impl.Promise$DefaultPromise.complete(scala.util.Try) @bci=2, line=153
(Compiled frame)
 - scala.concurrent.Promise$class.success(scala.concurrent.Promise, java.lang.Object) @bci=9,
line=86 (Compiled frame)
 - scala.concurrent.impl.Promise$DefaultPromise.success(java.lang.Object) @bci=2, line=153
(Compiled frame)
 - org.apache.spark.rpc.netty.LocalNettyRpcCallContext.send(java.lang.Object) @bci=5, line=50
(Compiled frame)
 - org.apache.spark.rpc.netty.NettyRpcCallContext.reply(java.lang.Object) @bci=2, line=32
(Compiled frame)
 - org.apache.spark.storage.BlockManagerMasterEndpoint$$anonfun$receiveAndReply$1.applyOrElse(java.lang.Object,
scala.Function1) @bci=859, line=125 (Compiled frame)
 - org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp() @bci=65, line=105 (Compiled
frame)
 - org.apache.spark.rpc.netty.Inbox.safelyCall(org.apache.spark.rpc.RpcEndpoint, scala.Function0)
@bci=1, line=205 (Compiled frame)
 - org.apache.spark.rpc.netty.Inbox.process(org.apache.spark.rpc.netty.Dispatcher) @bci=85,
line=101 (Compiled frame)
 - org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run() @bci=133, line=213 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
@bci=95, line=1142 (Interpreted frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=617 (Interpreted frame)
 - java.lang.Thread.run() @bci=11, line=745 (Interpreted frame)
{noformat}




{noformat}
Thread 22827: (state = BLOCKED)
 - java.nio.CharBuffer.wrap(char[], int, int) @bci=0, line=373 (Compiled frame)
 - sun.nio.cs.StreamEncoder.implWrite(char[], int, int) @bci=3, line=265 (Compiled frame)
 - sun.nio.cs.StreamEncoder.write(char[], int, int) @bci=60, line=125 (Compiled frame)
 - sun.nio.cs.StreamEncoder.write(java.lang.String, int, int) @bci=33, line=135 (Compiled
frame)
 - java.io.OutputStreamWriter.write(java.lang.String, int, int) @bci=7, line=220 (Compiled
frame)
 - java.io.Writer.write(java.lang.String) @bci=7, line=157 (Compiled frame)
 - org.apache.log4j.helpers.QuietWriter.write(java.lang.String) @bci=9, line=48 (Compiled
frame)
 - org.apache.log4j.WriterAppender.subAppend(org.apache.log4j.spi.LoggingEvent) @bci=54, line=317
(Compiled frame)
 - org.apache.log4j.WriterAppender.append(org.apache.log4j.spi.LoggingEvent) @bci=10, line=162
(Compiled frame)
 - org.apache.log4j.AppenderSkeleton.doAppend(org.apache.log4j.spi.LoggingEvent) @bci=106,
line=251 (Compiled frame)
 - org.apache.log4j.helpers.AppenderAttachableImpl.appendLoopOnAppenders(org.apache.log4j.spi.LoggingEvent)
@bci=41, line=66 (Compiled frame)
 - org.apache.log4j.Category.callAppenders(org.apache.log4j.spi.LoggingEvent) @bci=26, line=206
(Compiled frame)
 - org.apache.log4j.Category.forcedLog(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=14, line=391 (Compiled frame)
 - org.apache.log4j.Category.log(java.lang.String, org.apache.log4j.Priority, java.lang.Object,
java.lang.Throwable) @bci=34, line=856 (Compiled frame)
 - org.slf4j.impl.Log4jLoggerAdapter.warn(java.lang.String, java.lang.Throwable) @bci=12,
line=479 (Interpreted frame)
 - org.apache.spark.internal.Logging$class.logWarning(org.apache.spark.internal.Logging, scala.Function0,
java.lang.Throwable) @bci=30, line=87 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.logWarning(scala.Function0, java.lang.Throwable) @bci=3,
line=30 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, org.apache.spark.rpc.RpcTimeout,
scala.reflect.ClassTag) @bci=32, line=111 (Interpreted frame)
 - org.apache.spark.rpc.RpcEndpointRef.askWithRetry(java.lang.Object, scala.reflect.ClassTag)
@bci=7, line=78 (Compiled frame)
 - org.apache.spark.storage.BlockManagerMaster.updateBlockInfo(org.apache.spark.storage.BlockManagerId,
org.apache.spark.storage.BlockId, org.apache.spark.storage.StorageLevel, long, long) 
@bci=24, line=75 (Compiled frame)
 - org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$tryToReportBlockStatus(org.apache.spark.storage.BlockId,
org.apache.spark.storage.BlockStatus, long) @bci=37,
 line=387 (Compiled frame)
 - org.apache.spark.storage.BlockManager.org$apache$spark$storage$BlockManager$$reportBlockStatus(org.apache.spark.storage.BlockId,
org.apache.spark.storage.BlockStatus, long) @bci=4, line=
366 (Compiled frame)
 - org.apache.spark.storage.BlockManager.removeBlockInternal(org.apache.spark.storage.BlockId,
boolean) @bci=65, line=1345 (Compiled frame)
 - org.apache.spark.storage.BlockManager.removeBlock(org.apache.spark.storage.BlockId, boolean)
@bci=102, line=1327 (Compiled frame)
 - org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(org.apache.spark.storage.BroadcastBlockId)
@bci=9, line=1313 (Compiled frame)
 - org.apache.spark.storage.BlockManager$$anonfun$removeBroadcast$2.apply(java.lang.Object)
@bci=5, line=1313 (Compiled frame)
 - scala.collection.Iterator$class.foreach(scala.collection.Iterator, scala.Function1) @bci=16,
line=893 (Compiled frame)
 - scala.collection.AbstractIterator.foreach(scala.Function1) @bci=2, line=1336 (Compiled
frame)
 - org.apache.spark.storage.BlockManager.removeBroadcast(long, boolean) @bci=60, line=1313
(Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply$mcI$sp()
@bci=15, line=66 (Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply()
@bci=1, line=66 (Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$receiveAndReply$1$$anonfun$applyOrElse$4.apply()
@bci=1, line=66 (Compiled frame)
 - org.apache.spark.storage.BlockManagerSlaveEndpoint$$anonfun$1.apply() @bci=19, line=82
(Compiled frame)
 - scala.concurrent.impl.Future$PromiseCompletingRunnable.liftedTree1$1() @bci=8, line=24
(Compiled frame)
 - scala.concurrent.impl.Future$PromiseCompletingRunnable.run() @bci=5, line=24 (Compiled
frame)
 - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker)
@bci=95, line=1142 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=617 (Compiled frame)
 - java.lang.Thread.run() @bci=11, line=745 (Compiled frame)
{noformat}


> Driver hung while writing to console progress bar
> -------------------------------------------------
>
>                 Key: SPARK-15905
>                 URL: https://issues.apache.org/jira/browse/SPARK-15905
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 1.6.1
>            Reporter: Tejas Patil
>            Priority: Minor
>
> This leads to driver being not able to get heartbeats from its executors and job being
stuck. After looking at the locking dependency amongst the driver threads per the jstack,
this is where the driver seems to be stuck.
> {noformat}
> "refresh progress" #113 daemon prio=5 os_prio=0 tid=0x00007f7986cbc800 nid=0x7887d runnable
[0x00007f6d3507a000]
>    java.lang.Thread.State: RUNNABLE
>         at java.io.FileOutputStream.writeBytes(Native Method)
>         at java.io.FileOutputStream.write(FileOutputStream.java:326)
>         at java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:82)
>         at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:140)
>         - locked <0x00007f6eb81dd290> (a java.io.BufferedOutputStream)
>         at java.io.PrintStream.write(PrintStream.java:482)
>        - locked <0x00007f6eb81dd258> (a java.io.PrintStream)
>         at sun.nio.cs.StreamEncoder.writeBytes(StreamEncoder.java:221)
>         at sun.nio.cs.StreamEncoder.implFlushBuffer(StreamEncoder.java:291)
>         at sun.nio.cs.StreamEncoder.flushBuffer(StreamEncoder.java:104)
>         - locked <0x00007f6eb81dd400> (a java.io.OutputStreamWriter)
>         at java.io.OutputStreamWriter.flushBuffer(OutputStreamWriter.java:185)
>         at java.io.PrintStream.write(PrintStream.java:527)
>         - locked <0x00007f6eb81dd258> (a java.io.PrintStream)
>         at java.io.PrintStream.print(PrintStream.java:669)
>         at org.apache.spark.ui.ConsoleProgressBar.show(ConsoleProgressBar.scala:99)
>         at org.apache.spark.ui.ConsoleProgressBar.org$apache$spark$ui$ConsoleProgressBar$$refresh(ConsoleProgressBar.scala:69)
>         - locked <0x00007f6ed33b48a0> (a org.apache.spark.ui.ConsoleProgressBar)
>         at org.apache.spark.ui.ConsoleProgressBar$$anon$1.run(ConsoleProgressBar.scala:53)
>         at java.util.TimerThread.mainLoop(Timer.java:555)
>         at java.util.TimerThread.run(Timer.java:505)
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

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


Mime
View raw message