@@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.orc
2020import java .util .{Objects , Properties }
2121
2222import org .apache .hadoop .conf .Configuration
23- import org .apache .hadoop .fs .Path
23+ import org .apache .hadoop .fs .{ FileStatus , Path }
2424import org .apache .hadoop .hive .conf .HiveConf .ConfVars
2525import org .apache .hadoop .hive .ql .io .orc .{OrcInputFormat , OrcOutputFormat , OrcSerde , OrcSplit }
2626import org .apache .hadoop .hive .serde2 .objectinspector .StructObjectInspector
@@ -171,9 +171,10 @@ private[sql] case class OrcRelation(
171171 maybePartitionSpec)
172172 }
173173
174- override def buildScan (requiredColumns : Array [String ],
174+ override def buildScan (
175+ requiredColumns : Array [String ],
175176 filters : Array [Filter ],
176- inputPaths : Array [String ]): RDD [Row ] = {
177+ inputPaths : Array [FileStatus ]): RDD [Row ] = {
177178 val output = StructType (requiredColumns.map(dataSchema(_))).toAttributes
178179 OrcTableScan (output, this , filters, inputPaths).execute()
179180 }
@@ -194,7 +195,7 @@ private[orc] case class OrcTableScan(
194195 attributes : Seq [Attribute ],
195196 @ transient relation : OrcRelation ,
196197 filters : Array [Filter ],
197- inputPaths : Array [String ])
198+ inputPaths : Array [FileStatus ])
198199 extends Logging
199200 with HiveInspectors {
200201
@@ -256,7 +257,7 @@ private[orc] case class OrcTableScan(
256257 addColumnIds(attributes, relation, conf)
257258
258259 if (inputPaths.nonEmpty) {
259- FileInputFormat .setInputPaths(job, inputPaths.map(new Path (_) ): _* )
260+ FileInputFormat .setInputPaths(job, inputPaths.map(_.getPath ): _* )
260261 }
261262
262263 val inputFormatClass =
0 commit comments