Skip to content

Commit 513ef82

Browse files
committed
[SPARK-4552][SQL] Avoid exception when reading empty parquet data through Hive
This is a very small fix that catches one specific exception and returns an empty table. #3441 will address this in a more principled way. Author: Michael Armbrust <[email protected]> Closes #3586 from marmbrus/fixEmptyParquet and squashes the following commits: 2781d9f [Michael Armbrust] Handle empty lists for newParquet 04dd376 [Michael Armbrust] Avoid exception when reading empty parquet data through Hive
1 parent 90ec643 commit 513ef82

File tree

3 files changed

+62
-45
lines changed

3 files changed

+62
-45
lines changed

sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -191,7 +191,10 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext)
191191
val selectedPartitions = partitions.filter(p => partitionFilters.forall(_(p)))
192192
val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration)
193193
val selectedFiles = selectedPartitions.flatMap(_.files).map(f => fs.makeQualified(f.getPath))
194-
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles:_*)
194+
// FileInputFormat cannot handle empty lists.
195+
if (selectedFiles.nonEmpty) {
196+
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles: _*)
197+
}
195198

196199
// Push down filters when possible
197200
predicates

sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala

Lines changed: 52 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.planning._
2727
import org.apache.spark.sql.catalyst.plans._
2828
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
2929
import 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}
3131
import org.apache.spark.sql.hive
3232
import org.apache.spark.sql.hive.execution._
3333
import 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
}

sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -218,6 +218,12 @@ abstract class ParquetTest extends QueryTest with BeforeAndAfterAll {
218218
10)
219219
}
220220

221+
test(s"non-existant partition $table") {
222+
checkAnswer(
223+
sql(s"SELECT COUNT(*) FROM $table WHERE p = 1000"),
224+
0)
225+
}
226+
221227
test(s"multi-partition pruned count $table") {
222228
checkAnswer(
223229
sql(s"SELECT COUNT(*) FROM $table WHERE p IN (1,2,3)"),

0 commit comments

Comments
 (0)