[ https://issues.apache.org/jira/browse/SPARK-28106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
angerszhu updated SPARK-28106: ------------------------------ Attachment: image-2019-06-20-11-51-06-889.png > Spark SQL add jar with wrong hdfs path, SparkContext still add it to jar path > ,and cause Task Failed > ---------------------------------------------------------------------------------------------------- > > Key: SPARK-28106 > URL: https://issues.apache.org/jira/browse/SPARK-28106 > Project: Spark > Issue Type: Improvement > Components: SQL > Affects Versions: 2.2.0, 2.3.0, 2.4.0 > Reporter: angerszhu > Priority: Minor > Attachments: image-2019-06-19-21-23-22-061.png, > image-2019-06-20-11-49-13-691.png, image-2019-06-20-11-50-36-418.png, > image-2019-06-20-11-51-06-889.png > > > When we use SparkSQL, about add jar command, if we add a wrong path of HDFS > such as "add jar hdfs:///home/hadoop/test/test.jar", when execute it: > * In hive case , HiveClientImple call add jar, when runHiveSql() called, it > will cause error but will still run next code , then call > SparkContext.addJar, but this method don't have a path check when path schema > is HDFS , then do other sql, TaskDescribtion will carry jarPath of > SparkContext's registered JarPath. Then it will carry wrong path then cause > error happen > * None hive case, the same, will only check local path but not check hdfs > path. > > {code:java} > 19/06/19 19:55:12 INFO SessionState: converting to local > hdfs://home/hadoop/aaa.jar > Failed to read external resource hdfs://home/hadoop/aaa.jar > 19/06/19 19:55:12 ERROR SessionState: Failed to read external resource > hdfs://home/hadoop/aaa.jar > java.lang.RuntimeException: Failed to read external resource > hdfs://home/hadoop/aaa.jar > at > org.apache.hadoop.hive.ql.session.SessionState.downloadResource(SessionState.java:1288) > atorg.apache.hadoop.hive.ql.session.SessionState.resolveAndDownload(SessionState.java:1242) > at > org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1163) > at > org.apache.hadoop.hive.ql.session.SessionState.add_resources(SessionState.java:1149) > at > org.apache.hadoop.hive.ql.processors.AddResourceProcessor.run(AddResourceProcessor.java:67) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:866) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:835) > at > org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$withHiveState$1.apply(HiveClientImpl.scala:275) > at > org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:213) > at > org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:212) > at > org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:258) > at > org.apache.spark.sql.hive.client.HiveClientImpl.runHive(HiveClientImpl.scala:835) > at > org.apache.spark.sql.hive.client.HiveClientImpl.runSqlHive(HiveClientImpl.scala:825) > at > org.apache.spark.sql.hive.client.HiveClientImpl.addJar(HiveClientImpl.scala:983) > at > org.apache.spark.sql.hive.HiveSessionResourceLoader.addJar(HiveSessionStateBuilder.scala:112) > at > org.apache.spark.sql.execution.command.AddJarCommand.run(resources.scala:40) > at > org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70) > at > org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68) > at > org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79) > at org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:195) > at org.apache.spark.sql.Dataset$$anonfun$6.apply(Dataset.scala:195) > at org.apache.spark.sql.Dataset$$anonfun$53.apply(Dataset.scala:3365) > at > org.apache.spark.sql.execution.SQLExecution$.withCustomJobTag(SQLExecution.scala:119) > at > org.apache.spark.sql.execution.SQLExecution$$anonfun$withNewExecutionId$1.apply(SQLExecution.scala:79) > at > org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:143) > at > org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:73) > at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3364) > at org.apache.spark.sql.Dataset.<init>(Dataset.scala:195) > at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:80) > at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:642) > at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:694) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.org$apache$spark$sql$hive$thriftserver$SparkExecuteStatementOperation$$execute(SparkExecuteStatementOperation.scala:233) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:175) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1$$anon$2.run(SparkExecuteStatementOperation.scala:171) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) > at > org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$anon$1.run(SparkExecuteStatementOperation.scala:185) > at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) > at java.util.concurrent.FutureTask.run(FutureTask.java:266) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748) > Caused by: java.lang.IllegalArgumentException: java.net.UnknownHostException: > home > at > org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:374) > at > org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:312) > at > org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:178) > at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:665) > at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:601) > at > org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:148) > at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2596) > at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:91) > at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2630) > at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2612) > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370) > at > org.apache.hadoop.hive.ql.session.SessionState.downloadResource(SessionState.java:1273) > ... 42 more > Caused by: java.net.UnknownHostException: home > ... 54 more > 19/06/19 19:55:12 INFO SparkContext: Added JAR hdfs://home/hadoop/aaa.jar at > hdfs://home/hadoop/aaa.jar with timestamp 1560945312069 > 19/06/19 19:55:12 INFO HiveThriftServer2Listener: > > {code} > > > !image-2019-06-20-11-49-13-691.png! > > > !image-2019-06-20-11-51-06-889.png! -- This message was sent by Atlassian JIRA (v7.6.3#76005) --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@spark.apache.org For additional commands, e-mail: issues-h...@spark.apache.org