Github user lw-lin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/13575#discussion_r66466502
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
 ---
    @@ -143,39 +146,99 @@ object CSVRelation extends Logging {
           if (nonEmptyLines.hasNext) nonEmptyLines.drop(1)
         }
       }
    +
    +  /**
    +   * Setup writing configurations into the given [[Configuration]], and 
then return the
    +   * wrapped [[CSVOptions]].
    +   * Both continuous-queries writing process and non-continuous-queries 
writing process will
    +   * call this function.
    +   */
    +  private[csv] def prepareConfForWriting(
    +      conf: Configuration,
    +      options: Map[String, String]): CSVOptions = {
    +    val csvOptions = new CSVOptions(options)
    +    csvOptions.compressionCodec.foreach { codec =>
    +      CompressionCodecs.setCodecConfiguration(conf, codec)
    +    }
    +    csvOptions
    +  }
     }
     
    -private[sql] class CSVOutputWriterFactory(params: CSVOptions) extends 
OutputWriterFactory {
    +/**
    + * A factory for generating OutputWriters for writing csv files. This is 
implemented different
    + * from the 'batch' CSVOutputWriter as this does not use any 
[[OutputCommitter]]. It simply
    + * writes the data to the path used to generate the output writer. Callers 
of this factory
    + * has to ensure which files are to be considered as committed.
    + */
    +private[csv] class StreamingCSVOutputWriterFactory(
    +  sqlConf: SQLConf,
    +  dataSchema: StructType,
    +  hadoopConf: Configuration,
    +  options: Map[String, String]) extends StreamingOutputWriterFactory {
    +
    +  private val (csvOptions: CSVOptions, serializableConf: 
SerializableConfiguration) = {
    +    val conf = Job.getInstance(hadoopConf).getConfiguration
    +    val csvOptions = CSVRelation.prepareConfForWriting(conf, options)
    +    (csvOptions, new SerializableConfiguration(conf))
    +  }
    +
    +  /**
    +   * Returns a [[OutputWriter]] that writes data to the give path without 
using an
    +   * [[OutputCommitter]].
    +   */
    +  override private[sql] def newWriter(path: String): OutputWriter = {
    +    val hadoopTaskAttempId = new TaskAttemptID(new TaskID(new JobID, 
TaskType.MAP, 0), 0)
    +    val hadoopAttemptContext =
    +      new TaskAttemptContextImpl(serializableConf.value, 
hadoopTaskAttempId)
    +    // Returns a 'streaming' CSVOutputWriter
    +    new CSVOutputWriterBase(dataSchema, hadoopAttemptContext, csvOptions) {
    +      override private[csv] val recordWriter: RecordWriter[NullWritable, 
Text] =
    +        createNoCommitterTextRecordWriter(
    +          path,
    +          hadoopAttemptContext,
    +          (c: TaskAttemptContext, ext: String) => { new 
Path(s"$path.csv$ext") })
    +    }
    +  }
    +}
    +
    +private[csv] class BatchCSVOutputWriterFactory(params: CSVOptions) extends 
OutputWriterFactory {
       override def newInstance(
           path: String,
           bucketId: Option[Int],
           dataSchema: StructType,
           context: TaskAttemptContext): OutputWriter = {
         if (bucketId.isDefined) sys.error("csv doesn't support bucketing")
    -    new CsvOutputWriter(path, dataSchema, context, params)
    +    // Returns a 'batch' CSVOutputWriter
    +    new CSVOutputWriterBase(dataSchema, context, params) {
    +      private[csv] override val recordWriter: RecordWriter[NullWritable, 
Text] = {
    +        new TextOutputFormat[NullWritable, Text]() {
    +          override def getDefaultWorkFile(context: TaskAttemptContext, 
extension: String): Path = {
    +            val conf = context.getConfiguration
    +            val uniqueWriteJobId = 
conf.get(CreateDataSourceTableUtils.DATASOURCE_WRITEJOBUUID)
    +            val taskAttemptId = context.getTaskAttemptID
    +            val split = taskAttemptId.getTaskID.getId
    +            new Path(path, 
f"part-r-$split%05d-$uniqueWriteJobId.csv$extension")
    +          }
    +        }.getRecordWriter(context)
    +      }
    +    }
       }
     }
     
    -private[sql] class CsvOutputWriter(
    -    path: String,
    +/**
    + * Base CSVOutputWriter class for 'batch' CSVOutputWriter and 'streaming' 
CSVOutputWriter. The
    + * writing logic to a single file resides in this base class.
    + */
    +private[csv] abstract class CSVOutputWriterBase(
    --- End diff --
    
    This `CSVOutputWriterBase` is basically the original `CsvOutputWriter`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to