You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hive.apache.org by "Sahil Takiar (JIRA)" <ji...@apache.org> on 2017/08/02 02:24:00 UTC
[jira] [Updated] (HIVE-17225) HoS DPP throws FileNotFoundException
in HiveInputFormat#init when target work is in the same Spark job
[ https://issues.apache.org/jira/browse/HIVE-17225?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Sahil Takiar updated HIVE-17225:
--------------------------------
Summary: HoS DPP throws FileNotFoundException in HiveInputFormat#init when target work is in the same Spark job (was: FileNotFoundException in HiveInputFormat#init for query HoS DPP query with multiple left semi-joins against the same partition column)
> HoS DPP throws FileNotFoundException in HiveInputFormat#init when target work is in the same Spark job
> ------------------------------------------------------------------------------------------------------
>
> Key: HIVE-17225
> URL: https://issues.apache.org/jira/browse/HIVE-17225
> Project: Hive
> Issue Type: Sub-task
> Components: Spark
> Affects Versions: 3.0.0
> Reporter: Sahil Takiar
> Assignee: Sahil Takiar
>
> Setup:
> {code:sql}
> SET hive.spark.dynamic.partition.pruning=true;
> SET hive.strict.checks.cartesian.product=false;
> SET hive.auto.convert.join=true;
> CREATE TABLE partitioned_table1 (col int) PARTITIONED BY (part_col int);
> CREATE TABLE regular_table1 (col1 int, col2 int);
> CREATE TABLE regular_table2 (col1 int, col2 int);
> ALTER TABLE partitioned_table1 ADD PARTITION (part_col = 1);
> ALTER TABLE partitioned_table1 ADD PARTITION (part_col = 2);
> ALTER TABLE partitioned_table1 ADD PARTITION (part_col = 3);
> INSERT INTO table regular_table1 VALUES (0, 0), (1, 1), (2, 2);
> INSERT INTO table regular_table2 VALUES (0, 0), (1, 1), (2, 2);
> INSERT INTO TABLE partitioned_table1 PARTITION (part_col = 1) VALUES (1), (2), (3);
> INSERT INTO TABLE partitioned_table1 PARTITION (part_col = 2) VALUES (1), (2), (3);
> INSERT INTO TABLE partitioned_table1 PARTITION (part_col = 3) VALUES (1), (2), (3);
> SELECT *
> FROM regular_table1,
> regular_table2,
> partitioned_table1
> WHERE partitioned_table1.part_col IN (SELECT regular_table1.col2
> FROM regular_table1
> WHERE regular_table1.col1 > 0)
> AND partitioned_table1.part_col IN (SELECT regular_table2.col2
> FROM regular_table2
> WHERE regular_table2.col1 > 1);
> {code}
> Exception:
> {code}
> 2017-08-01T13:27:47,483 ERROR [b0d354a8-4cdb-4ba9-acec-27d14926aaf4 main] ql.Driver: FAILED: Execution Error, return code 3 from org.apache.hadoop.hive.ql.exec.spark.SparkTask. java.lang.RuntimeException: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.FileNotFoundException: File file:/Users/stakiar/Documents/idea/apache-hive/itests/qtest-spark/target/tmp/scratchdir/stakiar/b0d354a8-4cdb-4ba9-acec-27d14926aaf4/hive_2017-08-01_13-27-45_553_1088589686371686526-1/-mr-10004/3/5 does not exist
> at org.apache.hadoop.hive.ql.io.HiveInputFormat.init(HiveInputFormat.java:408)
> at org.apache.hadoop.hive.ql.io.CombineHiveInputFormat.getSplits(CombineHiveInputFormat.java:498)
> at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:200)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:248)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:246)
> at scala.Option.getOrElse(Option.scala:121)
> at org.apache.spark.rdd.RDD.partitions(RDD.scala:246)
> at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:35)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:248)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:246)
> at scala.Option.getOrElse(Option.scala:121)
> at org.apache.spark.rdd.RDD.partitions(RDD.scala:246)
> at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:82)
> at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:82)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
> at scala.collection.immutable.List.map(List.scala:285)
> at org.apache.spark.rdd.UnionRDD.getPartitions(UnionRDD.scala:82)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:248)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:246)
> at scala.Option.getOrElse(Option.scala:121)
> at org.apache.spark.rdd.RDD.partitions(RDD.scala:246)
> at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:82)
> at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:82)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
> at scala.collection.immutable.List.map(List.scala:285)
> at org.apache.spark.rdd.UnionRDD.getPartitions(UnionRDD.scala:82)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:248)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:246)
> at scala.Option.getOrElse(Option.scala:121)
> at org.apache.spark.rdd.RDD.partitions(RDD.scala:246)
> at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:82)
> at org.apache.spark.rdd.UnionRDD$$anonfun$1.apply(UnionRDD.scala:82)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
> at scala.collection.immutable.List.foreach(List.scala:381)
> at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
> at scala.collection.immutable.List.map(List.scala:285)
> at org.apache.spark.rdd.UnionRDD.getPartitions(UnionRDD.scala:82)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:248)
> at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:246)
> at scala.Option.getOrElse(Option.scala:121)
> at org.apache.spark.rdd.RDD.partitions(RDD.scala:246)
> at org.apache.spark.rdd.AsyncRDDActions$$anonfun$foreachAsync$1.apply(AsyncRDDActions.scala:127)
> at org.apache.spark.rdd.AsyncRDDActions$$anonfun$foreachAsync$1.apply(AsyncRDDActions.scala:125)
> at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
> at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
> at org.apache.spark.rdd.RDD.withScope(RDD.scala:358)
> at org.apache.spark.rdd.AsyncRDDActions.foreachAsync(AsyncRDDActions.scala:125)
> at org.apache.spark.api.java.JavaRDDLike$class.foreachAsync(JavaRDDLike.scala:731)
> at org.apache.spark.api.java.AbstractJavaRDDLike.foreachAsync(JavaRDDLike.scala:45)
> at org.apache.hadoop.hive.ql.exec.spark.RemoteHiveSparkClient$JobStatusJob.call(RemoteHiveSparkClient.java:351)
> at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:358)
> at org.apache.hive.spark.client.RemoteDriver$JobWrapper.call(RemoteDriver.java:323)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.FileNotFoundException: File file:/Users/stakiar/Documents/idea/apache-hive/itests/qtest-spark/target/tmp/scratchdir/stakiar/b0d354a8-4cdb-4ba9-acec-27d14926aaf4/hive_2017-08-01_13-27-45_553_1088589686371686526-1/-mr-10004/3/5 does not exist
> at org.apache.hadoop.hive.ql.exec.spark.SparkDynamicPartitionPruner.processFiles(SparkDynamicPartitionPruner.java:147)
> at org.apache.hadoop.hive.ql.exec.spark.SparkDynamicPartitionPruner.prune(SparkDynamicPartitionPruner.java:76)
> at org.apache.hadoop.hive.ql.io.HiveInputFormat.init(HiveInputFormat.java:406)
> ... 62 more
> Caused by: java.io.FileNotFoundException: File file:/Users/stakiar/Documents/idea/apache-hive/itests/qtest-spark/target/tmp/scratchdir/stakiar/b0d354a8-4cdb-4ba9-acec-27d14926aaf4/hive_2017-08-01_13-27-45_553_1088589686371686526-1/-mr-10004/3/5 does not exist
> at org.apache.hadoop.fs.RawLocalFileSystem.listStatus(RawLocalFileSystem.java:431)
> at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1517)
> at org.apache.hadoop.fs.FileSystem.listStatus(FileSystem.java:1557)
> at org.apache.hadoop.fs.ChecksumFileSystem.listStatus(ChecksumFileSystem.java:674)
> at org.apache.hadoop.hive.ql.exec.spark.SparkDynamicPartitionPruner.processFiles(SparkDynamicPartitionPruner.java:119)
> ... 64 more
> {code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)