You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "Andrew Sherman (JIRA)" <ji...@apache.org> on 2017/09/27 22:34:00 UTC

[jira] [Commented] (HIVE-16395) ConcurrentModificationException on config object in HoS

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

Andrew Sherman commented on HIVE-16395:
---------------------------------------

In the case of the reported bug S3AUtils.propagateBucketOptions() is cloning a Configuration, and then iterating over the properties in the source Configuration which is where the exception happend. So we could fix this particular bug (in Hadoop) by having S3AUtils.propagateBucketOptions() iterate over the clone it has just made, adding any new properties after the operation has finished. I have code that demonstrates the problem, and a fix.

The more general fix is to clone the JobConf. I think we would do this by setting in spark.hadoop.cloneConf to true in HiveSparkClientFactory.

I did some toy benchmarks on cloning a Configuration using 
{noformat}
Configuration clone = new Configuration(original);
{noformat}
The time it takes depends on the size of the Configuration.
* A Configuration with 1000 properties takes less than 1 ms.
* A Configuration with 10000 properties takes ~ 3.5 ms.

What do you think is the best approach [~lirui] [~stakiar] ?

> 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.4.14#64029)