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

Sahil Takiar commented on HIVE-16395:
-------------------------------------

Think I found the issue. By default, Spark gives Tasks the same {{JobConf}} 
object to use (ref 
[HadoopRDD.scala|https://github.com/apache/spark/blob/branch-2.0/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala#L143]).
 In this case, each {{CombineHiveInputFormat.getRecordReader}} inside an 
executor is given the same {{JobConf}} object. This can lead to a 
{{ConcurrentModificationException}} if one of the tasks mutates the {{JobConf}} 
while another task is iterating over it.

Similar issues have been reported in Spark too: SPARK-2546

I dug through the code a bit and Hive modifies the {{JobConf}} object in at 
least one place: {{ColumnProjectionUtils.appendReadColumns}}. Its probably 
modified in other places too, and SPARK-2546 suggests that some 
{{RecordReader}} implementations mutate it too.

The solution to SPARK-2546 was to add a config called 
{{spark.hadoop.cloneConf}} that is set to {{false}} by default. When set to 
{{true}} it clones a new {{JobConf}} object for each Spark Task, which avoids 
any thread safety issues (the [PR|https://github.com/apache/spark/pull/2684] 
for SPARK-2546 has a good explanation of the change). It's set to {{false}} by 
default for performance considerations.

So for this JIRA we could:

1: Close this and tell users if they hit this issue just set 
{{spark.hadoop.cloneConf}} to {{true}}
2: Given that we know Hive will mutate {{JobConf}} objects, maybe we should set 
{{spark.hadoop.cloneConf}} to {{true}} for HoS, we can profile the perf impact
3: ??

[~lirui], [~xuefuz] any thoughts on this?

> ConcurrentModificationException on config object in HoS
> -------------------------------------------------------
>
>                 Key: HIVE-16395
>                 URL: https://issues.apache.org/jira/browse/HIVE-16395
>             Project: Hive
>          Issue Type: Task
>          Components: Spark
>            Reporter: Sahil Takiar
>            Assignee: Sahil Takiar
>
> Looks like this is happening inside spark executors, looks to be some race 
> condition when modifying {{Configuration}} objects.
> Stack-Trace:
> {code}
> java.io.IOException: java.lang.reflect.InvocationTargetException
>       at 
> org.apache.hadoop.hive.io.HiveIOExceptionHandlerChain.handleRecordReaderCreationException(HiveIOExceptionHandlerChain.java:97)
>       at 
> org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(HiveIOExceptionHandlerUtil.java:57)
>       at 
> org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:267)
>       at 
> org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.<init>(HadoopShimsSecure.java:213)
>       at 
> org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileInputFormatShim.getRecordReader(HadoopShimsSecure.java:334)
>       at 
> org.apache.hadoop.hive.ql.io.CombineHiveInputFormat.getRecordReader(CombineHiveInputFormat.java:682)
>       at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:240)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:211)
>       at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:87)
>       at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:306)
>       at org.apache.spark.rdd.RDD.iterator(RDD.scala:270)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73)
>       at 
> org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41)
>       at org.apache.spark.scheduler.Task.run(Task.scala:89)
>       at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:242)
>       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)
> Caused by: java.lang.reflect.InvocationTargetException
>       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>       at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
>       at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>       at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
>       at 
> org.apache.hadoop.hive.shims.HadoopShimsSecure$CombineFileRecordReader.initNextRecordReader(HadoopShimsSecure.java:253)
>       ... 21 more
> Caused by: java.util.ConcurrentModificationException
>       at java.util.Hashtable$Enumerator.next(Hashtable.java:1167)
>       at 
> org.apache.hadoop.conf.Configuration.iterator(Configuration.java:2455)
>       at 
> org.apache.hadoop.fs.s3a.S3AUtils.propagateBucketOptions(S3AUtils.java:716)
>       at 
> org.apache.hadoop.fs.s3a.S3AFileSystem.initialize(S3AFileSystem.java:181)
>       at 
> org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2815)
>       at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:98)
>       at 
> org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2852)
>       at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2834)
>       at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:387)
>       at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)
>       at 
> org.apache.hadoop.mapred.LineRecordReader.<init>(LineRecordReader.java:108)
>       at 
> org.apache.hadoop.mapred.TextInputFormat.getRecordReader(TextInputFormat.java:67)
>       at 
> org.apache.hadoop.hive.ql.io.CombineHiveRecordReader.<init>(CombineHiveRecordReader.java:68)
>       ... 26 more
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to