@@ -42,7 +42,7 @@ import parquet.schema.MessageType
42
42
43
43
import org .apache .spark .rdd .RDD
44
44
import org .apache .spark .sql .SQLContext
45
- import org .apache .spark .sql .catalyst .expressions .{ Attribute , Expression , Row }
45
+ import org .apache .spark .sql .catalyst .expressions ._
46
46
import org .apache .spark .sql .execution .{LeafNode , SparkPlan , UnaryNode }
47
47
import org .apache .spark .{Logging , SerializableWritable , TaskContext }
48
48
@@ -59,28 +59,38 @@ case class ParquetTableScan(
59
59
// The resolution of Parquet attributes is case sensitive, so we resolve the original attributes
60
60
// by exprId. note: output cannot be transient, see
61
61
// https://issues.apache.org/jira/browse/SPARK-1367
62
- val output = attributes.map { a =>
63
- relation.output
64
- .find(o => o.exprId == a.exprId)
65
- .getOrElse(sys.error(s " Invalid parquet attribute $a in ${relation.output.mkString(" ," )}" ))
66
- }
62
+ val normalOutput =
63
+ attributes
64
+ .filterNot(a => relation.partitioningAttributes.map(_.exprId).contains(a.exprId))
65
+ .flatMap(a => relation.output.find(o => o.exprId == a.exprId))
66
+
67
+ val partOutput =
68
+ attributes.flatMap(a => relation.partitioningAttributes.find(o => o.exprId == a.exprId))
69
+
70
+ def output = partOutput ++ normalOutput
71
+
72
+ assert(normalOutput.size + partOutput.size == attributes.size,
73
+ s " $normalOutput + $partOutput != $attributes, ${relation.output}" )
67
74
68
75
override def execute (): RDD [Row ] = {
69
76
val sc = sqlContext.sparkContext
70
77
val job = new Job (sc.hadoopConfiguration)
71
78
ParquetInputFormat .setReadSupportClass(job, classOf [RowReadSupport ])
72
79
73
80
val conf : Configuration = ContextUtil .getConfiguration(job)
74
- val qualifiedPath = {
75
- val path = new Path (relation.path)
76
- path.getFileSystem(conf).makeQualified(path)
81
+
82
+ relation.path.split(" ," ).foreach { curPath =>
83
+ val qualifiedPath = {
84
+ val path = new Path (curPath)
85
+ path.getFileSystem(conf).makeQualified(path)
86
+ }
87
+ NewFileInputFormat .addInputPath(job, qualifiedPath)
77
88
}
78
- NewFileInputFormat .addInputPath(job, qualifiedPath)
79
89
80
90
// Store both requested and original schema in `Configuration`
81
91
conf.set(
82
92
RowReadSupport .SPARK_ROW_REQUESTED_SCHEMA ,
83
- ParquetTypesConverter .convertToString(output ))
93
+ ParquetTypesConverter .convertToString(normalOutput ))
84
94
conf.set(
85
95
RowWriteSupport .SPARK_ROW_SCHEMA ,
86
96
ParquetTypesConverter .convertToString(relation.output))
@@ -96,13 +106,41 @@ case class ParquetTableScan(
96
106
ParquetFilters .serializeFilterExpressions(columnPruningPred, conf)
97
107
}
98
108
99
- sc.newAPIHadoopRDD(
100
- conf,
101
- classOf [FilteringParquetRowInputFormat ],
102
- classOf [Void ],
103
- classOf [Row ])
104
- .map(_._2)
105
- .filter(_ != null ) // Parquet's record filters may produce null values
109
+ val baseRDD =
110
+ new org.apache.spark.rdd.NewHadoopRDD (
111
+ sc,
112
+ classOf [FilteringParquetRowInputFormat ],
113
+ classOf [Void ],
114
+ classOf [Row ],
115
+ conf)
116
+
117
+ if (partOutput.nonEmpty) {
118
+ baseRDD.mapPartitionsWithInputSplit { case (split, iter) =>
119
+ val partValue = " ([^=]+)=([^=]+)" .r
120
+ val partValues =
121
+ split.asInstanceOf [parquet.hadoop.ParquetInputSplit ]
122
+ .getPath
123
+ .toString
124
+ .split(" /" )
125
+ .flatMap {
126
+ case partValue(key, value) => Some (key -> value)
127
+ case _ => None
128
+ }.toMap
129
+
130
+ val partitionRowValues =
131
+ partOutput.map(a => Cast (Literal (partValues(a.name)), a.dataType).eval(EmptyRow ))
132
+
133
+ new Iterator [Row ] {
134
+ private [this ] val joinedRow = new JoinedRow (Row (partitionRowValues:_* ), null )
135
+
136
+ def hasNext = iter.hasNext
137
+
138
+ def next () = joinedRow.withRight(iter.next()._2)
139
+ }
140
+ }
141
+ } else {
142
+ baseRDD.map(_._2)
143
+ }.filter(_ != null ) // Parquet's record filters may produce null values
106
144
}
107
145
108
146
/**
0 commit comments