Skip to content

Commit 3386e4f

Browse files
committed
Merge pull request apache#58 from AndreSchumacher/parquet_fixes
Parquet fixes
2 parents 7386a9f + aeaef54 commit 3386e4f

8 files changed

Lines changed: 244 additions & 151 deletions

File tree

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,8 @@ trait MutableRow extends Row {
6363
def setBoolean(ordinal: Int, value: Boolean)
6464
def setShort(ordinal: Int, value: Short)
6565
def setByte(ordinal: Int, value: Byte)
66-
def setFloat(ordinal: Int, value: Byte)
66+
def setFloat(ordinal: Int, value: Float)
67+
def setString(ordinal: Int, value: String)
6768

6869
/**
6970
* EXPERIMENTAL
@@ -152,7 +153,7 @@ class GenericRow(protected[catalyst] val values: Array[Any]) extends Row {
152153
}
153154

154155
def getString(i: Int): String = {
155-
if (values(i) == null) sys.error("Failed to check null bit for primitive byte value.")
156+
if (values(i) == null) sys.error("Failed to check null bit for primitive String value.")
156157
values(i).asInstanceOf[String]
157158
}
158159

@@ -168,9 +169,10 @@ class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow {
168169
override def setBoolean(ordinal: Int,value: Boolean): Unit = { values(ordinal) = value }
169170
override def setByte(ordinal: Int,value: Byte): Unit = { values(ordinal) = value }
170171
override def setDouble(ordinal: Int,value: Double): Unit = { values(ordinal) = value }
171-
override def setFloat(ordinal: Int,value: Byte): Unit = { values(ordinal) = value }
172+
override def setFloat(ordinal: Int,value: Float): Unit = { values(ordinal) = value }
172173
override def setInt(ordinal: Int,value: Int): Unit = { values(ordinal) = value }
173174
override def setLong(ordinal: Int,value: Long): Unit = { values(ordinal) = value }
175+
override def setString(ordinal: Int,value: String): Unit = { values(ordinal) = value }
174176

175177
override def setNullAt(i: Int): Unit = { values(i) = null }
176178

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

Lines changed: 22 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -77,7 +77,7 @@ import scala.collection.JavaConversions._
7777
*
7878
* scala> val query = TestHive.parseSql(query_string).transform {
7979
* | case relation @ UnresolvedRelation(databaseName, name, alias) =>
80-
* | if(name == "psrc") ParquetRelation(name, filename)
80+
* | if (name == "psrc") ParquetRelation(name, filename)
8181
* | else relation
8282
* | }
8383
* query: org.apache.spark.sql.catalyst.plans.logical.LogicalPlan =
@@ -119,7 +119,7 @@ case class ParquetRelation(val tableName: String, val path: String) extends Base
119119
object ParquetRelation {
120120

121121
// The element type for the RDDs that this relation maps to.
122-
type RowType = org.apache.spark.sql.catalyst.expressions.GenericRow
122+
type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow
123123

124124
/**
125125
* Creates a new ParquetRelation and underlying Parquetfile for the given
@@ -138,7 +138,7 @@ object ParquetRelation {
138138
child: LogicalPlan,
139139
conf: Configuration,
140140
tableName: Option[String]): ParquetRelation = {
141-
if(!child.resolved) {
141+
if (!child.resolved) {
142142
throw new UnresolvedException[LogicalPlan](
143143
child,
144144
"Attempt to create Parquet table from unresolved child (when schema is not available)")
@@ -153,13 +153,13 @@ object ParquetRelation {
153153
private def checkPath(pathStr: String, conf: Configuration): Path = {
154154
val path = new Path(pathStr)
155155
val fs = path.getFileSystem(conf)
156-
if(fs.exists(path) &&
156+
if (fs.exists(path) &&
157157
!fs.getFileStatus(path)
158158
.getPermission
159159
.getUserAction
160160
.implies(FsAction.READ_WRITE)) {
161161
throw new IOException(
162-
s"Unable to create ParquetRelation: path ${path.toString} not read-writable")
162+
s"Unable to create ParquetRelation: path $path not read-writable")
163163
}
164164
path
165165
}
@@ -176,9 +176,13 @@ object ParquetTypesConverter {
176176
case ParquetPrimitiveTypeName.FLOAT => FloatType
177177
case ParquetPrimitiveTypeName.INT32 => IntegerType
178178
case ParquetPrimitiveTypeName.INT64 => LongType
179-
case ParquetPrimitiveTypeName.INT96 => LongType // TODO: is there an equivalent?
179+
case ParquetPrimitiveTypeName.INT96 => {
180+
// TODO: add BigInteger type? TODO(andre) use DecimalType instead????
181+
sys.error("Warning: potential loss of precision: converting INT96 to long")
182+
LongType
183+
}
180184
case _ => sys.error(
181-
s"Unsupported parquet datatype ${parquetType.asInstanceOf[Enum[String]].toString()}")
185+
s"Unsupported parquet datatype $parquetType")
182186
}
183187

184188
def fromDataType(ctype: DataType): ParquetPrimitiveTypeName = ctype match {
@@ -189,7 +193,7 @@ object ParquetTypesConverter {
189193
case FloatType => ParquetPrimitiveTypeName.FLOAT
190194
case IntegerType => ParquetPrimitiveTypeName.INT32
191195
case LongType => ParquetPrimitiveTypeName.INT64
192-
case _ => sys.error(s"Unsupported datatype ${ctype.toString}")
196+
case _ => sys.error(s"Unsupported datatype $ctype")
193197
}
194198

195199
def consumeType(consumer: RecordConsumer, ctype: DataType, record: Row, index: Int): Unit = {
@@ -204,7 +208,7 @@ object ParquetTypesConverter {
204208
case DoubleType => consumer.addDouble(record.getDouble(index))
205209
case FloatType => consumer.addFloat(record.getFloat(index))
206210
case BooleanType => consumer.addBoolean(record.getBoolean(index))
207-
case _ => sys.error(s"Unsupported datatype ${ctype.toString}, cannot write to consumer")
211+
case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer")
208212
}
209213
}
210214

@@ -232,18 +236,18 @@ object ParquetTypesConverter {
232236
def writeMetaData(attributes: Seq[Attribute], path: Path, conf: Configuration) {
233237
val fileSystem = FileSystem.get(conf)
234238

235-
if(fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) {
236-
throw new IOException(s"Expected to write to directory ${path.toString} but found file")
239+
if (fileSystem.exists(path) && !fileSystem.getFileStatus(path).isDir) {
240+
throw new IOException(s"Expected to write to directory $path but found file")
237241
}
238242

239243
val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE)
240244

241-
if(fileSystem.exists(metadataPath)) {
245+
if (fileSystem.exists(metadataPath)) {
242246
try {
243247
fileSystem.delete(metadataPath, true)
244248
} catch {
245249
case e: IOException =>
246-
throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE:\n${e.toString}")
250+
throw new IOException(s"Unable to delete previous PARQUET_METADATA_FILE at $metadataPath")
247251
}
248252
}
249253

@@ -255,18 +259,13 @@ object ParquetTypesConverter {
255259
ParquetTypesConverter.convertFromAttributes(attributes)
256260
val metaData: FileMetaData = new FileMetaData(
257261
parquetSchema,
258-
new java.util.HashMap[String, String](),
259-
"Shark")
262+
extraMetadata,
263+
"Spark")
260264

261265
ParquetFileWriter.writeMetadataFile(
262266
conf,
263267
path,
264-
new Footer(
265-
path,
266-
new ParquetMetadata(
267-
metaData,
268-
Nil)
269-
) :: Nil)
268+
new Footer(path, new ParquetMetadata(metaData, Nil)) :: Nil)
270269
}
271270

272271
/**
@@ -283,11 +282,11 @@ object ParquetTypesConverter {
283282

284283
val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE)
285284

286-
if(fs.exists(metadataPath) && fs.isFile(metadataPath)) {
285+
if (fs.exists(metadataPath) && fs.isFile(metadataPath)) {
287286
// TODO: improve exception handling, etc.
288287
ParquetFileReader.readFooter(conf, metadataPath)
289288
} else {
290-
if(!fs.exists(path) || !fs.isFile(path)) {
289+
if (!fs.exists(path) || !fs.isFile(path)) {
291290
throw new FileNotFoundException(
292291
s"Could not find file ${path.toString} when trying to read metadata")
293292
}

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

Lines changed: 77 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -15,34 +15,35 @@
1515
* limitations under the License.
1616
*/
1717

18-
package org.apache.spark.sql
19-
package parquet
18+
package org.apache.spark.sql.parquet
2019

21-
import _root_.parquet.io.InvalidRecordException
22-
import _root_.parquet.schema.MessageType
23-
import _root_.parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat}
24-
import _root_.parquet.hadoop.util.ContextUtil
20+
import parquet.io.InvalidRecordException
21+
import parquet.schema.MessageType
22+
import parquet.hadoop.{ParquetOutputFormat, ParquetInputFormat}
23+
import parquet.hadoop.util.ContextUtil
2524

2625
import org.apache.spark.rdd.RDD
27-
import org.apache.spark.api.java.JavaPairRDD
28-
import org.apache.spark.SparkContext
26+
import org.apache.spark.{TaskContext, SerializableWritable, SparkContext}
2927
import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression}
3028
import org.apache.spark.sql.execution.{SparkPlan, UnaryNode, LeafNode}
3129

32-
import org.apache.hadoop.mapreduce.Job
30+
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
31+
import org.apache.hadoop.mapreduce._
3332
import org.apache.hadoop.conf.Configuration
3433
import org.apache.hadoop.fs.Path
3534

3635
import java.io.IOException
36+
import java.text.SimpleDateFormat
37+
import java.util.Date
3738

3839
/**
3940
* Parquet table scan operator. Imports the file that backs the given
4041
* [[ParquetRelation]] as a RDD[Row].
4142
*/
4243
case class ParquetTableScan(
43-
output: Seq[Attribute],
44-
relation: ParquetRelation,
45-
columnPruningPred: Option[Expression])(
44+
@transient output: Seq[Attribute],
45+
@transient relation: ParquetRelation,
46+
@transient columnPruningPred: Option[Expression])(
4647
@transient val sc: SparkContext)
4748
extends LeafNode {
4849

@@ -56,12 +57,18 @@ case class ParquetTableScan(
5657
RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA,
5758
ParquetTypesConverter.convertFromAttributes(output).toString)
5859
// TODO: think about adding record filters
60+
/* Comments regarding record filters: it would be nice to push down as much filtering
61+
to Parquet as possible. However, currently it seems we cannot pass enough information
62+
to materialize an (arbitrary) Catalyst [[Predicate]] inside Parquet's
63+
``FilteredRecordReader`` (via Configuration, for example). Simple
64+
filter-rows-by-column-values however should be supported.
65+
*/
5966
sc.newAPIHadoopFile(
6067
relation.path,
6168
classOf[ParquetInputFormat[Row]],
6269
classOf[Void], classOf[Row],
6370
conf)
64-
.map(_._2)
71+
.map(_._2)
6572
}
6673

6774
/**
@@ -72,7 +79,7 @@ case class ParquetTableScan(
7279
*/
7380
def pruneColumns(prunedAttributes: Seq[Attribute]): ParquetTableScan = {
7481
val success = validateProjection(prunedAttributes)
75-
if(success) {
82+
if (success) {
7683
ParquetTableScan(prunedAttributes, relation, columnPruningPred)(sc)
7784
} else {
7885
sys.error("Warning: Could not validate Parquet schema projection in pruneColumns")
@@ -102,10 +109,10 @@ case class ParquetTableScan(
102109
}
103110

104111
case class InsertIntoParquetTable(
105-
relation: ParquetRelation,
106-
child: SparkPlan)(
112+
@transient relation: ParquetRelation,
113+
@transient child: SparkPlan)(
107114
@transient val sc: SparkContext)
108-
extends UnaryNode {
115+
extends UnaryNode with SparkHadoopMapReduceUtil {
109116

110117
/**
111118
* Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since
@@ -142,18 +149,64 @@ case class InsertIntoParquetTable(
142149
s"Unable to clear output directory ${fspath.toString} prior"
143150
+ s" to InsertIntoParquetTable:\n${e.toString}")
144151
}
145-
146-
JavaPairRDD.fromRDD(childRdd.map(Tuple2(null, _))).saveAsNewAPIHadoopFile(
147-
relation.path.toString,
148-
classOf[Void],
149-
classOf[ParquetRelation.RowType],
150-
classOf[_root_.parquet.hadoop.ParquetOutputFormat[ParquetRelation.RowType]],
151-
conf)
152+
saveAsHadoopFile(childRdd, relation.path.toString, conf)
152153

153154
// We return the child RDD to allow chaining (alternatively, one could return nothing).
154155
childRdd
155156
}
156157

157158
override def output = child.output
159+
160+
// based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]]
161+
// TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2?
162+
// .. then we could use the default one and could use [[MutablePair]]
163+
// instead of ``Tuple2``
164+
private def saveAsHadoopFile(
165+
rdd: RDD[Row],
166+
path: String,
167+
conf: Configuration) {
168+
val job = new Job(conf)
169+
val keyType = classOf[Void]
170+
val outputFormatType = classOf[parquet.hadoop.ParquetOutputFormat[Row]]
171+
job.setOutputKeyClass(keyType)
172+
job.setOutputValueClass(classOf[Row])
173+
val wrappedConf = new SerializableWritable(job.getConfiguration)
174+
NewFileOutputFormat.setOutputPath(job, new Path(path))
175+
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
176+
val jobtrackerID = formatter.format(new Date())
177+
val stageId = sc.newRddId()
178+
179+
def writeShard(context: TaskContext, iter: Iterator[Row]): Int = {
180+
// Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
181+
// around by taking a mod. We expect that no task will be attempted 2 billion times.
182+
val attemptNumber = (context.attemptId % Int.MaxValue).toInt
183+
/* "reduce task" <split #> <attempt # = spark task #> */
184+
val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId,
185+
attemptNumber)
186+
val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
187+
val format = outputFormatType.newInstance
188+
val committer = format.getOutputCommitter(hadoopContext)
189+
committer.setupTask(hadoopContext)
190+
val writer = format.getRecordWriter(hadoopContext)
191+
while (iter.hasNext) {
192+
val row = iter.next()
193+
writer.write(null, row)
194+
}
195+
writer.close(hadoopContext)
196+
committer.commitTask(hadoopContext)
197+
return 1
198+
}
199+
val jobFormat = outputFormatType.newInstance
200+
/* apparently we need a TaskAttemptID to construct an OutputCommitter;
201+
* however we're only going to use this local OutputCommitter for
202+
* setupJob/commitJob, so we just use a dummy "map" task.
203+
*/
204+
val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0)
205+
val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId)
206+
val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext)
207+
jobCommitter.setupJob(jobTaskContext)
208+
sc.runJob(rdd, writeShard _)
209+
jobCommitter.commitJob(jobTaskContext)
210+
}
158211
}
159212

0 commit comments

Comments
 (0)