Skip to content

Commit 8719c26

Browse files
committed
[SPARK-10063][SQL] Remove DirectParquetOutputCommitter
1 parent 21d5ca1 commit 8719c26

5 files changed

Lines changed: 5 additions & 190 deletions

File tree

docs/sql-programming-guide.md

Lines changed: 0 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -1466,37 +1466,6 @@ Configuration of Parquet can be done using the `setConf` method on `SQLContext`
14661466
support.
14671467
</td>
14681468
</tr>
1469-
<tr>
1470-
<td><code>spark.sql.parquet.output.committer.class</code></td>
1471-
<td><code>org.apache.parquet.hadoop.<br />ParquetOutputCommitter</code></td>
1472-
<td>
1473-
<p>
1474-
The output committer class used by Parquet. The specified class needs to be a subclass of
1475-
<code>org.apache.hadoop.<br />mapreduce.OutputCommitter</code>. Typically, it's also a
1476-
subclass of <code>org.apache.parquet.hadoop.ParquetOutputCommitter</code>.
1477-
</p>
1478-
<p>
1479-
<b>Note:</b>
1480-
<ul>
1481-
<li>
1482-
This option is automatically ignored if <code>spark.speculation</code> is turned on.
1483-
</li>
1484-
<li>
1485-
This option must be set via Hadoop <code>Configuration</code> rather than Spark
1486-
<code>SQLConf</code>.
1487-
</li>
1488-
<li>
1489-
This option overrides <code>spark.sql.sources.<br />outputCommitterClass</code>.
1490-
</li>
1491-
</ul>
1492-
</p>
1493-
<p>
1494-
Spark SQL comes with a builtin
1495-
<code>org.apache.spark.sql.<br />parquet.DirectParquetOutputCommitter</code>, which can be more
1496-
efficient then the default Parquet output committer when writing data to S3.
1497-
</p>
1498-
</td>
1499-
</tr>
15001469
<tr>
15011470
<td><code>spark.sql.parquet.mergeSchema</code></td>
15021471
<td><code>false</code></td>
@@ -2165,8 +2134,6 @@ options.
21652134
- In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains
21662135
unchanged.
21672136
- The canonical name of SQL/DataFrame functions are now lower case (e.g. sum vs SUM).
2168-
- It has been determined that using the DirectOutputCommitter when speculation is enabled is unsafe
2169-
and thus this output committer will not be used when speculation is on, independent of configuration.
21702137
- JSON data source will not automatically load new files that are created by other applications
21712138
(i.e. files that are not inserted to the dataset through Spark SQL).
21722139
For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore),

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/WriterContainer.scala

Lines changed: 5 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -129,16 +129,17 @@ private[sql] abstract class BaseWriterContainer(
129129
outputWriterFactory.newInstance(path, bucketId, dataSchema, taskAttemptContext)
130130
} catch {
131131
case e: org.apache.hadoop.fs.FileAlreadyExistsException =>
132-
if (outputCommitter.isInstanceOf[parquet.DirectParquetOutputCommitter]) {
133-
// Spark-11382: DirectParquetOutputCommitter is not idempotent, meaning on retry
132+
if (outputCommitter.getClass.getName.contains("Direct")) {
133+
// SPARK-11382: DirectParquetOutputCommitter is not idempotent, meaning on retry
134134
// attempts, the task will fail because the output file is created from a prior attempt.
135135
// This often means the most visible error to the user is misleading. Augment the error
136136
// to tell the user to look for the actual error.
137137
throw new SparkException("The output file already exists but this could be due to a " +
138138
"failure from an earlier attempt. Look through the earlier logs or stage page for " +
139-
"the first error.\n File exists error: " + e)
139+
"the first error.\n File exists error: " + e.getLocalizedMessage, e)
140+
} else {
141+
throw e
140142
}
141-
throw e
142143
}
143144
}
144145

@@ -156,15 +157,6 @@ private[sql] abstract class BaseWriterContainer(
156157
s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " +
157158
"for appending.")
158159
defaultOutputCommitter
159-
} else if (speculationEnabled) {
160-
// When speculation is enabled, it's not safe to use customized output committer classes,
161-
// especially direct output committers (e.g. `DirectParquetOutputCommitter`).
162-
//
163-
// See SPARK-9899 for more details.
164-
logInfo(
165-
s"Using default output committer ${defaultOutputCommitter.getClass.getCanonicalName} " +
166-
"because spark.speculation is configured to be true.")
167-
defaultOutputCommitter
168160
} else {
169161
val configuration = context.getConfiguration
170162
val committerClass = configuration.getClass(

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/DirectParquetOutputCommitter.scala

Lines changed: 0 additions & 88 deletions
This file was deleted.

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

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -76,13 +76,6 @@ private[sql] class DefaultSource
7676

7777
val conf = ContextUtil.getConfiguration(job)
7878

79-
// SPARK-9849 DirectParquetOutputCommitter qualified name should be backward compatible
80-
val committerClassName = conf.get(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key)
81-
if (committerClassName == "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") {
82-
conf.set(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key,
83-
classOf[DirectParquetOutputCommitter].getCanonicalName)
84-
}
85-
8679
val committerClass =
8780
conf.getClass(
8881
SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key,

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala

Lines changed: 0 additions & 49 deletions
Original file line numberDiff line numberDiff line change
@@ -445,55 +445,6 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext {
445445
}
446446
}
447447

448-
testQuietly("SPARK-6352 DirectParquetOutputCommitter") {
449-
val clonedConf = new Configuration(hadoopConfiguration)
450-
451-
// Write to a parquet file and let it fail.
452-
// _temporary should be missing if direct output committer works.
453-
try {
454-
hadoopConfiguration.set("spark.sql.parquet.output.committer.class",
455-
classOf[DirectParquetOutputCommitter].getCanonicalName)
456-
sqlContext.udf.register("div0", (x: Int) => x / 0)
457-
withTempPath { dir =>
458-
intercept[org.apache.spark.SparkException] {
459-
sqlContext.sql("select div0(1) as div0").write.parquet(dir.getCanonicalPath)
460-
}
461-
val path = new Path(dir.getCanonicalPath, "_temporary")
462-
val fs = path.getFileSystem(hadoopConfiguration)
463-
assert(!fs.exists(path))
464-
}
465-
} finally {
466-
// Hadoop 1 doesn't have `Configuration.unset`
467-
hadoopConfiguration.clear()
468-
clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue))
469-
}
470-
}
471-
472-
testQuietly("SPARK-9849 DirectParquetOutputCommitter qualified name backwards compatibility") {
473-
val clonedConf = new Configuration(hadoopConfiguration)
474-
475-
// Write to a parquet file and let it fail.
476-
// _temporary should be missing if direct output committer works.
477-
try {
478-
hadoopConfiguration.set("spark.sql.parquet.output.committer.class",
479-
"org.apache.spark.sql.parquet.DirectParquetOutputCommitter")
480-
sqlContext.udf.register("div0", (x: Int) => x / 0)
481-
withTempPath { dir =>
482-
intercept[org.apache.spark.SparkException] {
483-
sqlContext.sql("select div0(1) as div0").write.parquet(dir.getCanonicalPath)
484-
}
485-
val path = new Path(dir.getCanonicalPath, "_temporary")
486-
val fs = path.getFileSystem(hadoopConfiguration)
487-
assert(!fs.exists(path))
488-
}
489-
} finally {
490-
// Hadoop 1 doesn't have `Configuration.unset`
491-
hadoopConfiguration.clear()
492-
clonedConf.asScala.foreach(entry => hadoopConfiguration.set(entry.getKey, entry.getValue))
493-
}
494-
}
495-
496-
497448
test("SPARK-8121: spark.sql.parquet.output.committer.class shouldn't be overridden") {
498449
withTempPath { dir =>
499450
val clonedConf = new Configuration(hadoopConfiguration)

0 commit comments

Comments
 (0)