We've fixed the single case for "onJobStart", please check SPARK-34731 [1].
The patch will be available in Spark 3.1.2 / 3.2.0, but if someone reports
the same for lower version lines I think we could port back to lower
version lines as well.

1. https://issues.apache.org/jira/browse/SPARK-34731

On Fri, Mar 19, 2021 at 5:00 AM Mich Talebzadeh <mich.talebza...@gmail.com>
wrote:

>
> Recall this was the error
>
> 21/03/18 16:53:38 ERROR org.apache.spark.scheduler.AsyncEventQueue:
> Listener EventLoggingListener threw an exception
>
> java.util.ConcurrentModificationException
>
>         at java.util.Hashtable$Enumerator.next(Hashtable.java:1387)
>
> I resolved this error message by setting:
>
> spark.conf.set("spark.eventLog.enabled", "false")
>
> This happens when Spark writes to Google BigQuery table. The error is not
> a show stopper
>
> There is also a recent reference to event logging listener exception in
> this jira
>
> Getting event logging listener exception · Issue #439 · delta-io/delta
> (github.com) <https://github.com/delta-io/delta/issues/439>
>
>
> HTH
>
>
>    view my Linkedin profile
> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>
>
>
> *Disclaimer:* Use it at your own risk. Any and all responsibility for any
> loss, damage or destruction of data or any other property which may arise
> from relying on this email's technical content is explicitly disclaimed.
> The author will in no case be liable for any monetary damages arising from
> such loss, damage or destruction.
>
>
>
>
> On Thu, 18 Mar 2021 at 17:39, Mich Talebzadeh <mich.talebza...@gmail.com>
> wrote:
>
>> This is an intermittent error. Full error is this
>>
>> 21/03/18 17:35:12 ERROR org.apache.spark.scheduler.AsyncEventQueue:
>> Listener EventLoggingListener threw an exception
>> java.util.ConcurrentModificationException
>>         at java.util.Hashtable$Enumerator.next(Hashtable.java:1387)
>>         at
>> scala.collection.convert.Wrappers$JPropertiesWrapper$$anon$6.next(Wrappers.scala:424)
>>         at
>> scala.collection.convert.Wrappers$JPropertiesWrapper$$anon$6.next(Wrappers.scala:420)
>>         at scala.collection.Iterator.foreach(Iterator.scala:943)
>>         at scala.collection.Iterator.foreach$(Iterator.scala:943)
>>         at scala.collection.AbstractIterator.foreach(Iterator.scala:1431)
>>         at scala.collection.IterableLike.foreach(IterableLike.scala:74)
>>         at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
>>         at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
>>         at scala.collection.mutable.MapLike.toSeq(MapLike.scala:75)
>>         at scala.collection.mutable.MapLike.toSeq$(MapLike.scala:72)
>>         at scala.collection.mutable.AbstractMap.toSeq(Map.scala:84)
>>         at
>> org.apache.spark.scheduler.EventLoggingListener.redactProperties(EventLoggingListener.scala:290)
>>         at
>> org.apache.spark.scheduler.EventLoggingListener.onJobStart(EventLoggingListener.scala:162)
>>         at
>> org.apache.spark.scheduler.SparkListenerBus.doPostEvent(SparkListenerBus.scala:37)
>>         at
>> org.apache.spark.scheduler.SparkListenerBus.doPostEvent$(SparkListenerBus.scala:28)
>>         at
>> org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37)
>>         at
>> org.apache.spark.scheduler.AsyncEventQueue.doPostEvent(AsyncEventQueue.scala:37)
>>         at
>> org.apache.spark.util.ListenerBus.postToAll(ListenerBus.scala:117)
>>         at
>> org.apache.spark.util.ListenerBus.postToAll$(ListenerBus.scala:101)
>>         at
>> org.apache.spark.scheduler.AsyncEventQueue.super$postToAll(AsyncEventQueue.scala:105)
>>         at
>> org.apache.spark.scheduler.AsyncEventQueue.$anonfun$dispatch$1(AsyncEventQueue.scala:105)
>>         at
>> scala.runtime.java8.JFunction0$mcJ$sp.apply(JFunction0$mcJ$sp.java:23)
>>         at scala.util.DynamicVariable.withValue(DynamicVariable.scala:62)
>>         at org.apache.spark.scheduler.AsyncEventQueue.org
>> $apache$spark$scheduler$AsyncEventQueue$$dispatch(AsyncEventQueue.scala:100)
>>         at
>> org.apache.spark.scheduler.AsyncEventQueue$$anon$2.$anonfun$run$1(AsyncEventQueue.scala:96)
>>         at
>> org.apache.spark.util.Utils$.tryOrStopSparkContext(Utils.scala:1381)
>>         at
>> org.apache.spark.scheduler.AsyncEventQueue$$anon$2.run(AsyncEventQueue.scala:96)
>>
>> There is a reference to this error here.
>>
>>
>> [SPARK-32027] EventLoggingListener threw
>> java.util.ConcurrentModificationException - ASF JIRA (apache.org)
>> <https://issues.apache.org/jira/browse/SPARK-32027>
>>
>>
>> HTH
>>
>>
>>    view my Linkedin profile
>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>>
>>
>>
>> *Disclaimer:* Use it at your own risk. Any and all responsibility for
>> any loss, damage or destruction of data or any other property which may
>> arise from relying on this email's technical content is explicitly
>> disclaimed. The author will in no case be liable for any monetary damages
>> arising from such loss, damage or destruction.
>>
>>
>>
>>
>> On Thu, 18 Mar 2021 at 17:02, Mich Talebzadeh <mich.talebza...@gmail.com>
>> wrote:
>>
>>> Hi,
>>>
>>>
>>> Does anyone know about the cause of this error in Spark structured
>>> streaming?
>>>
>>>
>>> Spark version 3.1.1
>>>
>>>
>>>
>>> 21/03/18 16:53:38 ERROR org.apache.spark.scheduler.AsyncEventQueue:
>>> Listener EventLoggingListener threw an exception
>>>
>>> java.util.ConcurrentModificationException
>>>
>>>         at java.util.Hashtable$Enumerator.next(Hashtable.java:1387)
>>>
>>> Thanks
>>>
>>>
>>>    view my Linkedin profile
>>> <https://www.linkedin.com/in/mich-talebzadeh-ph-d-5205b2/>
>>>
>>>
>>>
>>> *Disclaimer:* Use it at your own risk. Any and all responsibility for
>>> any loss, damage or destruction of data or any other property which may
>>> arise from relying on this email's technical content is explicitly
>>> disclaimed. The author will in no case be liable for any monetary damages
>>> arising from such loss, damage or destruction.
>>>
>>>
>>>
>>

Reply via email to