Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.parquet

import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext}
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter

import parquet.Log
import parquet.hadoop.util.ContextUtil
import parquet.hadoop.{ParquetFileReader, ParquetFileWriter, ParquetOutputCommitter, ParquetOutputFormat}

private[parquet] class DirectParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext)
extends ParquetOutputCommitter(outputPath, context) {
val LOG = Log.getLog(classOf[ParquetOutputCommitter])

override def getWorkPath(): Path = outputPath
override def abortTask(taskContext: TaskAttemptContext): Unit = {}
override def commitTask(taskContext: TaskAttemptContext): Unit = {}
override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true
override def setupJob(jobContext: JobContext): Unit = {}
override def setupTask(taskContext: TaskAttemptContext): Unit = {}

override def commitJob(jobContext: JobContext) {
val configuration = ContextUtil.getConfiguration(jobContext)
val fileSystem = outputPath.getFileSystem(configuration)

if (configuration.getBoolean(ParquetOutputFormat.ENABLE_JOB_SUMMARY, true)) {
try {
val outputStatus = fileSystem.getFileStatus(outputPath)
val footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus)
try {
ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers)
} catch {
case e: Exception => {
LOG.warn("could not write summary file for " + outputPath, e)
val metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE)
if (fileSystem.exists(metadataPath)) {
fileSystem.delete(metadataPath, true)
}
}
}
} catch {
case e: Exception => LOG.warn("could not write summary file for " + outputPath, e)
}
}

if (configuration.getBoolean("mapreduce.fileoutputcommitter.marksuccessfuljobs", true)) {
try {
val successPath = new Path(outputPath, FileOutputCommitter.SUCCEEDED_FILE_NAME)
fileSystem.create(successPath).close()
} catch {
case e: Exception => LOG.warn("could not write success file for " + outputPath, e)
}
}
}
}

Original file line number Diff line number Diff line change
Expand Up @@ -381,6 +381,7 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int)
extends parquet.hadoop.ParquetOutputFormat[Row] {
// override to accept existing directories as valid output directory
override def checkOutputSpecs(job: JobContext): Unit = {}
var committer: OutputCommitter = null

// override to choose output filename so not overwrite existing ones
override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
Expand All @@ -403,6 +404,26 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int)
private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = {
context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID]
}

// override to create output committer from configuration
override def getOutputCommitter(context: TaskAttemptContext): OutputCommitter = {
if (committer == null) {
val output = getOutputPath(context)
val cls = context.getConfiguration.getClass("spark.sql.parquet.output.committer.class",
classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter])
val ctor = cls.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext])
committer = ctor.newInstance(output, context).asInstanceOf[ParquetOutputCommitter]
}
committer
}

// FileOutputFormat.getOutputPath takes JobConf in hadoop-1 but JobContext in hadoop-2
private def getOutputPath(context: TaskAttemptContext): Path = {
context.getConfiguration().get("mapred.output.dir") match {
case null => null
case name => new Path(name)
}
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -329,6 +329,66 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest {
checkAnswer(parquetFile(file), (data ++ newData).map(Row.fromTuple))
}
}

test("SPARK-6315 regression test") {
// Spark 1.1 and prior versions write Spark schema as case class string into Parquet metadata.
// This has been deprecated by JSON format since 1.2. Notice that, 1.3 further refactored data
// types API, and made StructType.fields an array. This makes the result of StructType.toString
// different from prior versions: there's no "Seq" wrapping the fields part in the string now.
val sparkSchema =
"StructType(Seq(StructField(a,BooleanType,false),StructField(b,IntegerType,false)))"

// The Parquet schema is intentionally made different from the Spark schema. Because the new
// Parquet data source simply falls back to the Parquet schema once it fails to parse the Spark
// schema. By making these two different, we are able to assert the old style case class string
// is parsed successfully.
val parquetSchema = MessageTypeParser.parseMessageType(
"""message root {
| required int32 c;
|}
""".stripMargin)

withTempPath { location =>
val extraMetadata = Map(RowReadSupport.SPARK_METADATA_KEY -> sparkSchema.toString)
val fileMetadata = new FileMetaData(parquetSchema, extraMetadata, "Spark")
val path = new Path(location.getCanonicalPath)

ParquetFileWriter.writeMetadataFile(
sparkContext.hadoopConfiguration,
path,
new Footer(path, new ParquetMetadata(fileMetadata, Nil)) :: Nil)

assertResult(parquetFile(path.toString).schema) {
StructType(
StructField("a", BooleanType, nullable = false) ::
StructField("b", IntegerType, nullable = false) ::
Nil)
}
}
}

test("SPARK-6352 DirectParquetOutputCommitter") {
// Write to a parquet file and let it fail.
// _temporary should be missing if direct output committer works.
try {
configuration.set("spark.sql.parquet.output.committer.class",
"org.apache.spark.sql.parquet.DirectParquetOutputCommitter")
sqlContext.udf.register("div0", (x: Int) => x / 0)
withTempPath { dir =>
intercept[org.apache.spark.SparkException] {
sqlContext.sql("select div0(1)").saveAsParquetFile(dir.getCanonicalPath)
}
val path = new Path(dir.getCanonicalPath, "_temporary")
val fs = path.getFileSystem(configuration)
assert(!fs.exists(path))
}
}
finally {
configuration.set("spark.sql.parquet.output.committer.class",
"parquet.hadoop.ParquetOutputCommitter")
}
}
>>>>>>> e13cd86... [SPARK-6352] [SQL] Custom parquet output committer
}

class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll {
Expand Down