diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala index 9708199f07349..e10d3a0b6846c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcRelation.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.orc import java.util.{Objects, Properties} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.io.orc.{OrcInputFormat, OrcOutputFormat, OrcSerde, OrcSplit} import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector @@ -171,9 +171,10 @@ private[sql] case class OrcRelation( maybePartitionSpec) } - override def buildScan(requiredColumns: Array[String], + override def buildScan( + requiredColumns: Array[String], filters: Array[Filter], - inputPaths: Array[String]): RDD[Row] = { + inputPaths: Array[FileStatus]): RDD[Row] = { val output = StructType(requiredColumns.map(dataSchema(_))).toAttributes OrcTableScan(output, this, filters, inputPaths).execute() } @@ -194,7 +195,7 @@ private[orc] case class OrcTableScan( attributes: Seq[Attribute], @transient relation: OrcRelation, filters: Array[Filter], - inputPaths: Array[String]) + inputPaths: Array[FileStatus]) extends Logging with HiveInspectors { @@ -256,7 +257,7 @@ private[orc] case class OrcTableScan( addColumnIds(attributes, relation, conf) if (inputPaths.nonEmpty) { - FileInputFormat.setInputPaths(job, inputPaths.map(new Path(_)): _*) + FileInputFormat.setInputPaths(job, inputPaths.map(_.getPath): _*) } val inputFormatClass =