@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.planning._
2727import org .apache .spark .sql .catalyst .plans ._
2828import org .apache .spark .sql .catalyst .plans .logical .LogicalPlan
2929import org .apache .spark .sql .catalyst .types .StringType
30- import org .apache .spark .sql .execution .{DescribeCommand , OutputFaker , SparkPlan }
30+ import org .apache .spark .sql .execution .{DescribeCommand , OutputFaker , SparkPlan , PhysicalRDD }
3131import org .apache .spark .sql .hive
3232import org .apache .spark .sql .hive .execution ._
3333import org .apache .spark .sql .parquet .ParquetRelation
@@ -104,53 +104,61 @@ private[hive] trait HiveStrategies {
104104 case a : AttributeReference => UnresolvedAttribute (a.name)
105105 })
106106
107- if (relation.hiveQlTable.isPartitioned) {
108- val rawPredicate = pruningPredicates.reduceOption(And ).getOrElse(Literal (true ))
109- // Translate the predicate so that it automatically casts the input values to the correct
110- // data types during evaluation
111- val castedPredicate = rawPredicate transform {
112- case a : AttributeReference =>
113- val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId)
114- val key = relation.partitionKeys(idx)
115- Cast (BoundReference (idx, StringType , nullable = true ), key.dataType)
116- }
117-
118- val inputData = new GenericMutableRow (relation.partitionKeys.size)
119- val pruningCondition =
120- if (codegenEnabled) {
121- GeneratePredicate (castedPredicate)
122- } else {
123- InterpretedPredicate (castedPredicate)
107+ try {
108+ if (relation.hiveQlTable.isPartitioned) {
109+ val rawPredicate = pruningPredicates.reduceOption(And ).getOrElse(Literal (true ))
110+ // Translate the predicate so that it automatically casts the input values to the
111+ // correct data types during evaluation.
112+ val castedPredicate = rawPredicate transform {
113+ case a : AttributeReference =>
114+ val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId)
115+ val key = relation.partitionKeys(idx)
116+ Cast (BoundReference (idx, StringType , nullable = true ), key.dataType)
124117 }
125118
126- val partitions = relation.hiveQlPartitions.filter { part =>
127- val partitionValues = part.getValues
128- var i = 0
129- while (i < partitionValues.size()) {
130- inputData(i) = partitionValues(i)
131- i += 1
119+ val inputData = new GenericMutableRow (relation.partitionKeys.size)
120+ val pruningCondition =
121+ if (codegenEnabled) {
122+ GeneratePredicate (castedPredicate)
123+ } else {
124+ InterpretedPredicate (castedPredicate)
125+ }
126+
127+ val partitions = relation.hiveQlPartitions.filter { part =>
128+ val partitionValues = part.getValues
129+ var i = 0
130+ while (i < partitionValues.size()) {
131+ inputData(i) = partitionValues(i)
132+ i += 1
133+ }
134+ pruningCondition(inputData)
132135 }
133- pruningCondition(inputData)
134- }
135136
136- hiveContext
137- .parquetFile(partitions.map(_.getLocation).mkString(" ," ))
138- .addPartitioningAttributes(relation.partitionKeys)
139- .lowerCase
140- .where(unresolvedOtherPredicates)
141- .select(unresolvedProjection:_* )
142- .queryExecution
143- .executedPlan
144- .fakeOutput(projectList.map(_.toAttribute)):: Nil
145- } else {
146- hiveContext
147- .parquetFile(relation.hiveQlTable.getDataLocation.toString)
148- .lowerCase
149- .where(unresolvedOtherPredicates)
150- .select(unresolvedProjection:_* )
151- .queryExecution
152- .executedPlan
153- .fakeOutput(projectList.map(_.toAttribute)) :: Nil
137+ hiveContext
138+ .parquetFile(partitions.map(_.getLocation).mkString(" ," ))
139+ .addPartitioningAttributes(relation.partitionKeys)
140+ .lowerCase
141+ .where(unresolvedOtherPredicates)
142+ .select(unresolvedProjection : _* )
143+ .queryExecution
144+ .executedPlan
145+ .fakeOutput(projectList.map(_.toAttribute)) :: Nil
146+ } else {
147+ hiveContext
148+ .parquetFile(relation.hiveQlTable.getDataLocation.toString)
149+ .lowerCase
150+ .where(unresolvedOtherPredicates)
151+ .select(unresolvedProjection : _* )
152+ .queryExecution
153+ .executedPlan
154+ .fakeOutput(projectList.map(_.toAttribute)) :: Nil
155+ }
156+ } catch {
157+ // parquetFile will throw an exception when there is no data.
158+ // TODO: Remove this hack for Spark 1.3.
159+ case iae : java.lang.IllegalArgumentException
160+ if iae.getMessage.contains(" Can not create a Path from an empty string" ) =>
161+ PhysicalRDD (plan.output, sparkContext.emptyRDD[Row ]) :: Nil
154162 }
155163 case _ => Nil
156164 }
0 commit comments