[
https://issues.apache.org/jira/browse/HIVE-20695?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16639991#comment-16639991
]
Yongzhi Chen commented on HIVE-20695:
-------------------------------------
RemoteHiveSparkClient.addJars is not thread safe, the jarUri is added to
localJars without upload to hdfs finish. Which cause other thread think the jar
already uploaded and calls JobHandle<Serializable> jobHandle =
remoteClient.submit(job); without the jars fully ready.
{noformat}
private void addJars(String addedJars) throws IOException {
for (String addedJar : CSV_SPLITTER.split(Strings.nullToEmpty(addedJars))) {
try {
URI jarUri = FileUtils.getURI(addedJar);
if (jarUri != null && !localJars.contains(jarUri)) {
localJars.add(jarUri);
if (SparkUtilities.needUploadToHDFS(jarUri, sparkConf)) {
jarUri = SparkUtilities.uploadToHDFS(jarUri, hiveConf);
}
remoteClient.addJar(jarUri);
}
} catch (URISyntaxException e) {
LOG.warn("Failed to add jar:" + addedJar, e);
}
}
}
{noformat}
> HoS Query fails with hive.exec.parallel=true
> --------------------------------------------
>
> Key: HIVE-20695
> URL: https://issues.apache.org/jira/browse/HIVE-20695
> Project: Hive
> Issue Type: Bug
> Components: Spark
> Affects Versions: 1.2.1
> Reporter: Yongzhi Chen
> Assignee: Yongzhi Chen
> Priority: Major
>
> Hive queries which fail when running a HiveOnSpark job:
> {noformat}
> ERROR : Failed to execute spark task, with exception
> 'java.lang.Exception(Failed to submit Spark work, please retry later)'
> java.lang.Exception: Failed to submit Spark work, please retry later
> at
> org.apache.hadoop.hive.ql.exec.spark.RemoteHiveSparkClient.execute(RemoteHiveSparkClient.java:186)
> at
> org.apache.hadoop.hive.ql.exec.spark.session.SparkSessionImpl.submit(SparkSessionImpl.java:71)
> at
> org.apache.hadoop.hive.ql.exec.spark.SparkTask.execute(SparkTask.java:107)
> at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:214)
> at
> org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:99)
> at org.apache.hadoop.hive.ql.exec.TaskRunner.run(TaskRunner.java:79)
> Caused by:
> org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException):
> No lease on
> /tmp/hive/dbname/_spark_session_dir/e202c452-8793-4e4e-ad55-61e3d4965c69/somename.jar
> (inode 725730760): File does not exist. [Lease. Holder:
> DFSClient_NONMAPREDUCE_-1981084042_486659, pending creates: 7]
> at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkLease(FSNamesystem.java:3755)
> at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.analyzeFileState(FSNamesystem.java:3556)
> at
> org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getAdditionalBlock(FSNamesystem.java:3412)
> at
> org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.addBlock(NameNodeRpcServer.java:688)
> {noformat}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)