Skip to content
Closed
Show file tree
Hide file tree
Changes from 6 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -147,6 +147,17 @@ public void writeTo(ByteBuffer buffer) {
buffer.position(pos + numBytes);
}

public void writeTo(OutputStream out) throws IOException {
Copy link
Contributor

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

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed.

Copy link
Author

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.

if (base instanceof byte[] &&
offset >= BYTE_ARRAY_OFFSET &&
offset < (long) Integer.MAX_VALUE) {
// the offset includes the object header... but it is only needed for unsafe copies
out.write((byte[]) base, (int) offset - BYTE_ARRAY_OFFSET, numBytes);
} else {
out.write(getBytes());
}
}

/**
* Returns the number of bytes for a code point with the first byte as `b`
* @param b The first byte of a code point
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,7 @@ import java.io.IOException

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.io.{NullWritable, Text}
import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext}
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat
import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}

import org.apache.spark.TaskContext
import org.apache.spark.ml.feature.LabeledPoint
Expand All @@ -35,7 +33,6 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.text.TextOutputWriter
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
import org.apache.spark.util.SerializableConfiguration
Expand All @@ -46,30 +43,21 @@ private[libsvm] class LibSVMOutputWriter(
context: TaskAttemptContext)
extends OutputWriter {

private[this] val buffer = new Text()

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 writer = CodecStreams.createOutputStreamWriter(context, new Path(path))

override def write(row: Row): Unit = {
val label = row.get(0)
val vector = row.get(1).asInstanceOf[Vector]
val sb = new StringBuilder(label.toString)
writer.write(label.toString)
vector.foreachActive { case (i, v) =>
sb += ' '
sb ++= s"${i + 1}:$v"
writer.write(s" ${i + 1}:$v")
}
buffer.set(sb.mkString)
recordWriter.write(NullWritable.get(), buffer)

writer.write('\n')
}

override def close(): Unit = {
recordWriter.close(context)
writer.close()
}
}

Expand Down Expand Up @@ -136,7 +124,7 @@ private[libsvm] class LibSVMFileFormat extends TextBasedFileFormat with DataSour
}

override def getFileExtension(context: TaskAttemptContext): String = {
".libsvm" + TextOutputWriter.getCompressionExtension(context)
".libsvm" + CodecStreams.getCompressionExtension(context)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -194,4 +194,8 @@ private[sql] class JacksonGenerator(
writeFields(row, schema, rootFieldWriters)
}
}

def writeLineEnding(): Unit = {
gen.writeRaw('\n')
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TextOutputStream actually writes the UTF-8 version of a newline; don't know if that is relevant or not:

"\n".getBytes(StandardCharsets.UTF_8);

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

7-bit ASCII is a subset of UTF-8, \n is the same in both.

Copy link
Contributor

Choose a reason for hiding this comment

The 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 0x0a then

Copy link
Author

Choose a reason for hiding this comment

The 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.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

will "probably" be thrown; object stores have issues there

Copy link
Author

Choose a reason for hiding this comment

The 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:

  /**
   * Create an FSDataOutputStream at the indicated Path.
   * @param f the file to create
   * @param overwrite if a file with this name already exists, then if true,
   *   the file will be overwritten, and if false an exception will be thrown.
   */

*/
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
Expand Up @@ -17,7 +17,8 @@

package org.apache.spark.sql.execution.datasources.csv

import java.io.{CharArrayWriter, StringReader}
import java.io.{CharArrayWriter, OutputStream, StringReader}
import java.nio.charset.StandardCharsets

import com.univocity.parsers.csv._

Expand Down Expand Up @@ -64,7 +65,10 @@ private[csv] class CsvReader(params: CSVOptions) {
* @param params Parameters object for configuration
* @param headers headers for columns
*/
private[csv] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) extends Logging {
private[csv] class LineCsvWriter(
params: CSVOptions,
headers: Seq[String],
output: OutputStream) extends Logging {
private val writerSettings = new CsvWriterSettings
private val format = writerSettings.getFormat

Expand All @@ -80,21 +84,14 @@ private[csv] class LineCsvWriter(params: CSVOptions, headers: Seq[String]) exten
writerSettings.setHeaders(headers: _*)
writerSettings.setQuoteEscapingEnabled(params.escapeQuotes)

private val buffer = new CharArrayWriter()
private val writer = new CsvWriter(buffer, writerSettings)
private val writer = new CsvWriter(output, StandardCharsets.UTF_8, writerSettings)

def writeRow(row: Seq[String], includeHeader: Boolean): Unit = {
if (includeHeader) {
writer.writeHeaders()
}
writer.writeRow(row.toArray: _*)
}

def flush(): String = {
writer.flush()
val lines = buffer.toString.stripLineEnd
buffer.reset()
lines
writer.writeRow(row: _*)
}

def close(): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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)
}
}

Expand All @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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)
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The uniVocity CSV writer converts every column to a String before writing so it's (probably?) not possible to further optimize this without doing a whole bunch of work. I only did a quick scan through their code though.

Copy link
Contributor

Choose a reason for hiding this comment

The 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()
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,12 @@

package org.apache.spark.sql.execution.datasources.json

import java.io.CharArrayWriter

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.io.{LongWritable, NullWritable, Text}
import org.apache.hadoop.io.{LongWritable, Text}
import org.apache.hadoop.mapred.{JobConf, TextInputFormat}
import org.apache.hadoop.mapreduce.{Job, RecordWriter, TaskAttemptContext}
import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}
import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat

import org.apache.spark.TaskContext
import org.apache.spark.internal.Logging
Expand All @@ -35,7 +32,6 @@ import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JacksonParser, JSONOptions}
import org.apache.spark.sql.catalyst.util.CompressionCodecs
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.text.TextOutputWriter
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.SerializableConfiguration
Expand Down Expand Up @@ -90,7 +86,7 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister {
}

override def getFileExtension(context: TaskAttemptContext): String = {
".json" + TextOutputWriter.getCompressionExtension(context)
".json" + CodecStreams.getCompressionExtension(context)
}
}
}
Expand Down Expand Up @@ -163,33 +159,20 @@ private[json] class JsonOutputWriter(
context: TaskAttemptContext)
extends OutputWriter with Logging {

private[this] val writer = new CharArrayWriter()
private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path))

// create the Generator without separator inserted between 2 records
private[this] val gen = new JacksonGenerator(dataSchema, writer, options)
private[this] val result = new Text()

private val recordWriter: RecordWriter[NullWritable, Text] = {
new TextOutputFormat[NullWritable, Text]() {
override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
new Path(path)
}
}.getRecordWriter(context)
}

override def write(row: Row): Unit = throw new UnsupportedOperationException("call writeInternal")

override protected[sql] def writeInternal(row: InternalRow): Unit = {
gen.write(row)
gen.flush()

result.set(writer.toString)
writer.reset()

recordWriter.write(NullWritable.get(), result)
gen.writeLineEnding()
}

override def close(): Unit = {
gen.close()
recordWriter.close(context)
writer.close()
}
}
Loading