-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18658][SQL] Write text records directly to a FileOutputStream #16089
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 6 commits
56667bd
373d8e0
d349d71
c178dac
7742ee4
5707218
27c102d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -194,4 +194,8 @@ private[sql] class JacksonGenerator( | |
| writeFields(row, schema, rootFieldWriters) | ||
| } | ||
| } | ||
|
|
||
| def writeLineEnding(): Unit = { | ||
| gen.writeRaw('\n') | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 7-bit ASCII is a subset of UTF-8,
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Presumably just a convoluted way to create a byte array containing the byte
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That is my assumption. I'm also assuming that writing a single byte is slightly more efficient than writing an array of a single byte. |
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,74 @@ | ||
| /* | ||
| * 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.execution.datasources | ||
|
|
||
| import java.io.{OutputStream, OutputStreamWriter} | ||
| import java.nio.charset.{Charset, StandardCharsets} | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.io.compress._ | ||
| import org.apache.hadoop.mapreduce.JobContext | ||
| import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat | ||
| import org.apache.hadoop.util.ReflectionUtils | ||
|
|
||
| object CodecStreams { | ||
| private def getCompressionCodec( | ||
| context: JobContext, | ||
| file: Option[Path] = None): Option[CompressionCodec] = { | ||
| if (FileOutputFormat.getCompressOutput(context)) { | ||
| val compressorClass = FileOutputFormat.getOutputCompressorClass( | ||
| context, | ||
| classOf[GzipCodec]) | ||
|
|
||
| Some(ReflectionUtils.newInstance(compressorClass, context.getConfiguration)) | ||
| } else { | ||
| file.flatMap { path => | ||
| val compressionCodecs = new CompressionCodecFactory(context.getConfiguration) | ||
| Option(compressionCodecs.getCodec(path)) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Create a new file and open it for writing. | ||
| * If compression is enabled in the [[JobContext]] the stream will write compressed data to disk. | ||
| * An exception will be thrown if the file already exists. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. will "probably" be thrown; object stores have issues there
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Is this a problem with Hadoop in general? The FileSystem docs also specify this behavior: |
||
| */ | ||
| def createOutputStream(context: JobContext, file: Path): OutputStream = { | ||
| val fs = file.getFileSystem(context.getConfiguration) | ||
| val outputStream: OutputStream = fs.create(file, false) | ||
|
|
||
| getCompressionCodec(context, Some(file)) | ||
| .map(codec => codec.createOutputStream(outputStream)) | ||
| .getOrElse(outputStream) | ||
| } | ||
|
|
||
| def createOutputStreamWriter( | ||
| context: JobContext, | ||
| file: Path, | ||
| charset: Charset = StandardCharsets.UTF_8): OutputStreamWriter = { | ||
| new OutputStreamWriter(createOutputStream(context, file), charset) | ||
| } | ||
|
|
||
| /** Returns the compression codec extension to be used in a file name, e.g. ".gzip"). */ | ||
| def getCompressionExtension(context: JobContext): String = { | ||
| getCompressionCodec(context) | ||
| .map(_.getDefaultExtension) | ||
| .getOrElse("") | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -20,19 +20,15 @@ package org.apache.spark.sql.execution.datasources.csv | |
| import scala.util.control.NonFatal | ||
|
|
||
| import org.apache.hadoop.fs.Path | ||
| import org.apache.hadoop.io.{NullWritable, Text} | ||
| import org.apache.hadoop.mapreduce.RecordWriter | ||
| import org.apache.hadoop.mapreduce.TaskAttemptContext | ||
| import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat | ||
|
|
||
| import org.apache.spark.internal.Logging | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.GenericInternalRow | ||
| import org.apache.spark.sql.catalyst.util.DateTimeUtils | ||
| import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory, PartitionedFile} | ||
| import org.apache.spark.sql.execution.datasources.text.TextOutputWriter | ||
| import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter, OutputWriterFactory, PartitionedFile} | ||
| import org.apache.spark.sql.types._ | ||
|
|
||
| object CSVRelation extends Logging { | ||
|
|
@@ -179,7 +175,7 @@ private[csv] class CSVOutputWriterFactory(params: CSVOptions) extends OutputWrit | |
| } | ||
|
|
||
| override def getFileExtension(context: TaskAttemptContext): String = { | ||
| ".csv" + TextOutputWriter.getCompressionExtension(context) | ||
| ".csv" + CodecStreams.getCompressionExtension(context) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -189,9 +185,6 @@ private[csv] class CsvOutputWriter( | |
| context: TaskAttemptContext, | ||
| params: CSVOptions) extends OutputWriter with Logging { | ||
|
|
||
| // create the Generator without separator inserted between 2 records | ||
| private[this] val text = new Text() | ||
|
|
||
| // A `ValueConverter` is responsible for converting a value of an `InternalRow` to `String`. | ||
| // When the value is null, this converter should not be called. | ||
| private type ValueConverter = (InternalRow, Int) => String | ||
|
|
@@ -200,17 +193,9 @@ private[csv] class CsvOutputWriter( | |
| private val valueConverters: Array[ValueConverter] = | ||
| dataSchema.map(_.dataType).map(makeConverter).toArray | ||
|
|
||
| private val recordWriter: RecordWriter[NullWritable, Text] = { | ||
| new TextOutputFormat[NullWritable, Text]() { | ||
| override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { | ||
| new Path(path) | ||
| } | ||
| }.getRecordWriter(context) | ||
| } | ||
|
|
||
| private val FLUSH_BATCH_SIZE = 1024L | ||
| private var records: Long = 0L | ||
| private val csvWriter = new LineCsvWriter(params, dataSchema.fieldNames.toSeq) | ||
| private var printHeader: Boolean = params.headerFlag | ||
| private val writer = CodecStreams.createOutputStream(context, new Path(path)) | ||
| private val csvWriter = new LineCsvWriter(params, dataSchema.fieldNames.toSeq, writer) | ||
|
|
||
| private def rowToString(row: InternalRow): Seq[String] = { | ||
| var i = 0 | ||
|
|
@@ -245,24 +230,12 @@ private[csv] class CsvOutputWriter( | |
| override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal") | ||
|
|
||
| override protected[sql] def writeInternal(row: InternalRow): Unit = { | ||
| csvWriter.writeRow(rowToString(row), records == 0L && params.headerFlag) | ||
| records += 1 | ||
| if (records % FLUSH_BATCH_SIZE == 0) { | ||
| flush() | ||
| } | ||
| } | ||
|
|
||
| private def flush(): Unit = { | ||
| val lines = csvWriter.flush() | ||
| if (lines.nonEmpty) { | ||
| text.set(lines) | ||
| recordWriter.write(NullWritable.get(), text) | ||
| } | ||
| csvWriter.writeRow(rowToString(row), printHeader) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. we can probably optimize this as well - but not a huge deal in the first pr.
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The uniVocity CSV writer converts every column to a
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yup I asked this question but never had time to follow-up: uniVocity/univocity-parsers#99 |
||
| printHeader = false | ||
| } | ||
|
|
||
| override def close(): Unit = { | ||
| flush() | ||
| csvWriter.close() | ||
| recordWriter.close(context) | ||
| writer.close() | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
always good to have tests for the corner case codepaths here, as they are invariably the official home of off-by-one errors
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agreed.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I've added a few tests for this method.