Skip to content

Commit 4eb04e9

Browse files
author
Anselme Vignon
committed
bugfix SPARK-5775
1 parent 432ceca commit 4eb04e9

File tree

1 file changed

+17
-6
lines changed

1 file changed

+17
-6
lines changed

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

Lines changed: 17 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -144,19 +144,30 @@ case class ParquetTableScan(
144144
new Iterator[Row] {
145145
def hasNext = iter.hasNext
146146
def next() = {
147-
val row = iter.next()._2.asInstanceOf[SpecificMutableRow]
147+
iter.next()._2 match {
148+
case row: SpecificMutableRow => {
149+
//val row = iter.next ()._2.asInstanceOf[SpecificMutableRow]
148150

149-
// Parquet will leave partitioning columns empty, so we fill them in here.
151+
// Parquet will leave partitioning columns empty, so we fill them in here.
150152
var i = 0
151153
while (i < requestedPartitionOrdinals.size) {
152-
row(requestedPartitionOrdinals(i)._2) =
153-
partitionRowValues(requestedPartitionOrdinals(i)._1)
154-
i += 1
154+
row (requestedPartitionOrdinals (i)._2) =
155+
partitionRowValues (requestedPartitionOrdinals (i)._1)
156+
i += 1
155157
}
156158
row
159+
}
160+
case row : Row => {
161+
val rVals = row.to[Array]
162+
var i = 0
163+
while (i < requestedPartitionOrdinals.size) {
164+
rVals.update(requestedPartitionOrdinals (i)._2,partitionRowValues (requestedPartitionOrdinals (i)._1))
165+
}
166+
Row.fromSeq(rVals)
167+
}
157168
}
158169
}
159-
}
170+
}}
160171
} else {
161172
baseRDD.map(_._2)
162173
}

0 commit comments

Comments
 (0)