Skip to content

Commit 54c3d7b

Browse files
committed
Enforces that FileOutputFormat must be used
1 parent be0c268 commit 54c3d7b

File tree

3 files changed

+28
-21
lines changed

3 files changed

+28
-21
lines changed

sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala

Lines changed: 14 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@ import scala.collection.mutable
2424
import org.apache.hadoop.conf.Configuration
2525
import org.apache.hadoop.fs.Path
2626
import org.apache.hadoop.mapreduce._
27-
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
27+
import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter, FileOutputFormat}
2828
import org.apache.hadoop.util.Shell
2929
import parquet.hadoop.util.ContextUtil
3030

@@ -211,7 +211,7 @@ private[sql] abstract class BaseWriterContainer(
211211
@transient private val jobContext: JobContext = job
212212

213213
// The following fields are initialized and used on both driver and executor side.
214-
@transient private var outputCommitter: OutputCommitter = _
214+
@transient protected var outputCommitter: FileOutputCommitter = _
215215
@transient private var jobId: JobID = _
216216
@transient private var taskId: TaskID = _
217217
@transient private var taskAttemptId: TaskAttemptID = _
@@ -235,7 +235,11 @@ private[sql] abstract class BaseWriterContainer(
235235
setupConf()
236236
taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId)
237237
val outputFormat = relation.outputFormatClass.newInstance()
238-
outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext)
238+
outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) match {
239+
case c: FileOutputCommitter => c
240+
case _ => sys.error(
241+
s"Output committer must be ${classOf[FileOutputCommitter].getName} or its subclasses")
242+
}
239243
outputCommitter.setupJob(jobContext)
240244
}
241245

@@ -244,7 +248,11 @@ private[sql] abstract class BaseWriterContainer(
244248
setupConf()
245249
taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId)
246250
val outputFormat = outputFormatClass.newInstance()
247-
outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext)
251+
outputCommitter = outputFormat.getOutputCommitter(taskAttemptContext) match {
252+
case c: FileOutputCommitter => c
253+
case _ => sys.error(
254+
s"Output committer must be ${classOf[FileOutputCommitter].getName} or its subclasses")
255+
}
248256
outputCommitter.setupTask(taskAttemptContext)
249257
initWriters()
250258
}
@@ -298,7 +306,7 @@ private[sql] class DefaultWriterContainer(
298306

299307
override protected def initWriters(): Unit = {
300308
writer = relation.outputWriterClass.newInstance()
301-
writer.init(outputPath, dataSchema, taskAttemptContext)
309+
writer.init(outputCommitter.getWorkPath.toString, dataSchema, taskAttemptContext)
302310
}
303311

304312
override def outputWriterForRow(row: Row): OutputWriter = writer
@@ -340,7 +348,7 @@ private[sql] class DynamicPartitionWriterContainer(
340348
}.mkString
341349

342350
outputWriters.getOrElseUpdate(partitionPath, {
343-
val path = new Path(outputPath, partitionPath.stripPrefix(Path.SEPARATOR))
351+
val path = new Path(outputCommitter.getWorkPath, partitionPath.stripPrefix(Path.SEPARATOR))
344352
val writer = outputWriterClass.newInstance()
345353
writer.init(path.toString, dataSchema, taskAttemptContext)
346354
writer

sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -19,8 +19,8 @@ package org.apache.spark.sql.sources
1919

2020
import org.apache.hadoop.conf.Configuration
2121
import org.apache.hadoop.fs.{FileStatus, Path}
22-
import org.apache.hadoop.mapreduce.{TaskAttemptContext, OutputFormat, OutputCommitter}
23-
import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter
22+
import org.apache.hadoop.mapreduce.TaskAttemptContext
23+
import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat
2424

2525
import org.apache.spark.annotation.{DeveloperApi, Experimental}
2626
import org.apache.spark.deploy.SparkHadoopUtil
@@ -411,7 +411,7 @@ abstract class FSBasedRelation private[sql](
411411
buildScan(requiredColumns, inputPaths)
412412
}
413413

414-
def outputFormatClass: Class[_ <: OutputFormat[Void, Row]]
414+
def outputFormatClass: Class[_ <: FileOutputFormat[Void, Row]]
415415

416416
/**
417417
* This method is responsible for producing a new [[OutputWriter]] for each newly opened output

sql/hive/src/test/scala/org/apache/spark/sql/sources/FSBasedRelationSuite.scala

Lines changed: 11 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -20,10 +20,9 @@ package org.apache.spark.sql.sources
2020
import scala.collection.mutable
2121

2222
import com.google.common.base.Objects
23-
import org.apache.hadoop.conf.Configuration
2423
import org.apache.hadoop.fs.{FileSystem, Path}
25-
import org.apache.hadoop.mapreduce.{TaskAttemptContext, OutputFormat}
26-
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat
24+
import org.apache.hadoop.mapreduce.TaskAttemptContext
25+
import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat, TextOutputFormat}
2726
import org.scalatest.BeforeAndAfter
2827

2928
import org.apache.spark.rdd.RDD
@@ -110,7 +109,7 @@ class SimpleFSBasedRelation
110109

111110
override def outputWriterClass: Class[_ <: OutputWriter] = classOf[SimpleOutputWriter]
112111

113-
override def outputFormatClass: Class[_ <: OutputFormat[Void, Row]] = {
112+
override def outputFormatClass: Class[_ <: FileOutputFormat[Void, Row]] = {
114113
// This is just a mock, not used within this test suite.
115114
classOf[TextOutputFormat[Void, Row]]
116115
}
@@ -268,7 +267,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
268267
val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i")
269268

270269
TestResult.synchronized {
271-
assert(TestResult.writerPaths.size === 2)
270+
assert(TestResult.writerPaths.size === 4)
272271
assert(TestResult.writtenRows === expectedRows.toSet)
273272
}
274273
}
@@ -295,7 +294,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
295294
val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i")
296295

297296
TestResult.synchronized {
298-
assert(TestResult.writerPaths.size === 2)
297+
assert(TestResult.writerPaths.size === 4)
299298
assert(TestResult.writtenRows === expectedRows.toSet)
300299
}
301300
}
@@ -328,7 +327,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
328327
val expectedRows = for (i <- 1 to 3; _ <- 1 to 4) yield Row(i, s"val_$i")
329328

330329
TestResult.synchronized {
331-
assert(TestResult.writerPaths.size === 4)
330+
assert(TestResult.writerPaths.size === 8)
332331
assert(TestResult.writtenRows === expectedRows.toSet)
333332
}
334333
}
@@ -381,7 +380,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
381380
val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i")
382381

383382
TestResult.synchronized {
384-
assert(TestResult.writerPaths.size === 2)
383+
assert(TestResult.writerPaths.size === 4)
385384
assert(TestResult.writtenRows === expectedRows.toSet)
386385
}
387386

@@ -443,7 +442,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
443442
val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i")
444443

445444
TestResult.synchronized {
446-
assert(TestResult.writerPaths.size === 2)
445+
assert(TestResult.writerPaths.size === 4)
447446
assert(TestResult.writtenRows === expectedRows.toSet)
448447
}
449448

@@ -472,7 +471,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
472471
val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i")
473472

474473
TestResult.synchronized {
475-
assert(TestResult.writerPaths.size === 2)
474+
assert(TestResult.writerPaths.size === 4)
476475
assert(TestResult.writtenRows === expectedRows.toSet)
477476
}
478477

@@ -493,7 +492,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
493492
val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i")
494493

495494
TestResult.synchronized {
496-
assert(TestResult.writerPaths.size === 2)
495+
assert(TestResult.writerPaths.size === 8)
497496
assert(TestResult.writtenRows === expectedRows.toSet)
498497
}
499498

@@ -535,7 +534,7 @@ class FSBasedRelationSuite extends QueryTest with BeforeAndAfter {
535534
val expectedRows = for (i <- 1 to 3; _ <- 1 to 2) yield Row(i, s"val_$i")
536535

537536
TestResult.synchronized {
538-
assert(TestResult.writerPaths.size === 2)
537+
assert(TestResult.writerPaths.size === 4)
539538
assert(TestResult.writtenRows === expectedRows.toSet)
540539
}
541540

0 commit comments

Comments
 (0)