Skip to content

Commit 9d17607

Browse files
committed
Adds the contract that OutputWriter should have zero-arg constructor
Also adds more arguments to OutputWriter.init() for initialization.
1 parent 5de194a commit 9d17607

File tree

1 file changed

+20
-4
lines changed

1 file changed

+20
-4
lines changed

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

Lines changed: 20 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,8 @@
1717

1818
package org.apache.spark.sql.sources
1919

20+
import org.apache.hadoop.conf.Configuration
21+
2022
import org.apache.spark.annotation.{DeveloperApi, Experimental}
2123
import org.apache.spark.rdd.RDD
2224
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
@@ -245,12 +247,26 @@ trait CatalystScan {
245247
/**
246248
* ::Experimental::
247249
* [[OutputWriter]] is used together with [[FSBasedRelation]] for persisting rows to the
248-
* underlying file system. An [[OutputWriter]] instance is created when a new output file is
249-
* opened. This instance is used to persist rows to this single output file.
250+
* underlying file system. Subclasses of [[OutputWriter]] must provide a zero-argument constructor.
251+
* An [[OutputWriter]] instance is created and initialized when a new output file is opened on
252+
* executor side. This instance is used to persist rows to this single output file.
250253
*/
251254
@Experimental
252255
abstract class OutputWriter {
253-
def init(): Unit = ()
256+
/**
257+
* Initializes this [[OutputWriter]] before any rows are persisted.
258+
*
259+
* @param path The file path to which this [[OutputWriter]] is supposed to write.
260+
* @param dataSchema Schema of the rows to be written. Partition columns are not included in the
261+
* schema if the corresponding relation is partitioned.
262+
* @param options Data source options inherited from driver side.
263+
* @param conf Hadoop configuration inherited from driver side.
264+
*/
265+
def init(
266+
path: String,
267+
dataSchema: StructType,
268+
options: java.util.Map[String, String],
269+
conf: Configuration): Unit = ()
254270

255271
/**
256272
* Persists a single row. Invoked on the executor side. When writing to dynamically partitioned
@@ -341,5 +357,5 @@ abstract class FSBasedRelation extends BaseRelation {
341357
* This method is responsible for producing a new [[OutputWriter]] for each newly opened output
342358
* file on the executor side.
343359
*/
344-
def newOutputWriter(path: String): OutputWriter
360+
def outputWriterClass: Class[_ <: OutputWriter]
345361
}

0 commit comments

Comments
 (0)