[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-16 Thread lw-lin
Github user lw-lin closed the pull request at:

https://github.com/apache/spark/pull/13575


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



[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13575#discussion_r66467191
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala
 ---
@@ -120,24 +109,31 @@ class TextFileFormat extends FileFormat with 
DataSourceRegister {
   }
 }
   }
+
+  override def buildWriter(
+  sqlContext: SQLContext,
+  dataSchema: StructType,
+  options: Map[String, String]): OutputWriterFactory = {
+verifySchema(dataSchema)
+new StreamingTextOutputWriterFactory(
+  sqlContext.conf,
+  dataSchema,
+  sqlContext.sparkContext.hadoopConfiguration,
+  options)
+  }
 }
 
-class TextOutputWriter(path: String, dataSchema: StructType, context: 
TaskAttemptContext)
+/**
+ * Base TextOutputWriter class for 'batch' TextOutputWriter and 
'streaming' TextOutputWriter. The
+ * writing logic to a single file resides in this base class.
+ */
+private[text] abstract class TextOutputWriterBase(context: 
TaskAttemptContext)
--- End diff --

This `TextOutputWriterBase` is basically the original `TextOutputWriter`.


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



[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13575#discussion_r66467095
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
 ---
@@ -488,7 +488,12 @@ private[sql] class ParquetOutputWriterFactory(
 // Custom ParquetOutputFormat that disable use of committer and writes 
to the given path
 val outputFormat = new ParquetOutputFormat[InternalRow]() {
   override def getOutputCommitter(c: TaskAttemptContext): 
OutputCommitter = { null }
-  override def getDefaultWorkFile(c: TaskAttemptContext, ext: String): 
Path = { new Path(path) }
+  override def getDefaultWorkFile(c: TaskAttemptContext, ext: String): 
Path = {
+// It has the `.parquet` extension at the end because 
(de)compression tools
+// such as gunzip would not be able to decompress this as the 
compression
+// is not applied on this whole file but on each "page" in Parquet 
format.
+new Path(s"$path$ext")
+  }
--- End diff --

This patch appends an extension to the assigned `path`; new `path` would be 
like `some_path.gz.parquet`.


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



[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13575#discussion_r66466672
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala
 ---
@@ -146,16 +173,53 @@ class JsonFileFormat extends FileFormat with 
DataSourceRegister {
 }
   }
 
-  override def toString: String = "JSON"
-
   override def hashCode(): Int = getClass.hashCode()
 
   override def equals(other: Any): Boolean = 
other.isInstanceOf[JsonFileFormat]
 }
 
-private[json] class JsonOutputWriter(
-path: String,
-bucketId: Option[Int],
+/**
+ * A factory for generating [[OutputWriter]]s for writing json files. This 
is implemented different
+ * from the 'batch' JsonOutputWriter 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[json] class StreamingJsonOutputWriterFactory(
+sqlConf: SQLConf,
+dataSchema: StructType,
+hadoopConf: Configuration,
+options: Map[String, String]) extends StreamingOutputWriterFactory {
+
+  private val serializableConf = {
+val conf = Job.getInstance(hadoopConf).getConfiguration
+JsonFileFormat.prepareConfForWriting(conf, options)
+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' JsonOutputWriter
+new JsonOutputWriterBase(dataSchema, hadoopAttemptContext) {
+  override private[json] val recordWriter: RecordWriter[NullWritable, 
Text] =
+createNoCommitterTextRecordWriter(
+  path,
+  hadoopAttemptContext,
+  (c: TaskAttemptContext, ext: String) => { new 
Path(s"$path.json$ext") })
+}
+  }
+}
+
+/**
+ * Base JsonOutputWriter class for 'batch' JsonOutputWriter and 
'streaming' JsonOutputWriter. The
+ * writing logic to a single file resides in this base class.
+ */
+private[json] abstract class JsonOutputWriterBase(
--- End diff --

This `JsonOutputWriterBase` is basically the original `JsonOutputWriter`


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



[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread lw-lin
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, 

[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13575#discussion_r66466310
  
--- 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
+  }
--- End diff --

These mostly are moved from `CSVFileFormat.prepareWrite()` to here


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



[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread lw-lin
Github user lw-lin commented on a diff in the pull request:

https://github.com/apache/spark/pull/13575#discussion_r66465201
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -246,7 +247,12 @@ case class DataSource(
   case s: StreamSinkProvider =>
 s.createSink(sparkSession.sqlContext, options, partitionColumns, 
outputMode)
 
-  case parquet: parquet.ParquetFileFormat =>
+  // TODO: Remove the `isInstanceOf` check when other formats have 
been ported
+  case fileFormat: FileFormat
+if (fileFormat.isInstanceOf[CSVFileFormat]
+  || fileFormat.isInstanceOf[JsonFileFormat]
--- End diff --

@ScrapCodes , thanks! But I'm afraid that syntax would raise a compilation 
error:
```
[ERROR] .../datasources/DataSource.scala:250: illegal variable in pattern 
alternative
[ERROR]   case fileFormat: CSVFileFormat | JsonFileFormat | 
ParquetFileFormat | TextFileFormat =>
[ERROR]^
```
A work-around can be the following, but I found it somewhat less intuitive:
```scala
case fileFormat@(_: CSVFileFormat |
 _: JsonFileFormat |
 _: ParquetFileFormat |
 _: TextFileFormat) =>
  // other code
  ... fileFormat.asInstanceOf[FileFormat] ...
```


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



[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread ScrapCodes
Github user ScrapCodes commented on a diff in the pull request:

https://github.com/apache/spark/pull/13575#discussion_r66433407
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -246,7 +247,12 @@ case class DataSource(
   case s: StreamSinkProvider =>
 s.createSink(sparkSession.sqlContext, options, partitionColumns, 
outputMode)
 
-  case parquet: parquet.ParquetFileFormat =>
+  // TODO: Remove the `isInstanceOf` check when other formats have 
been ported
+  case fileFormat: FileFormat
+if (fileFormat.isInstanceOf[CSVFileFormat]
+  || fileFormat.isInstanceOf[JsonFileFormat]
--- End diff --

I think there is a better syntax to achieve this.
```scala 
case fileFormat: CSVFileFormat | JsonFileFormat | .. =>
``` 


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



[GitHub] spark pull request #13575: [SPARK-15472][SQL] Add support for writing in `cs...

2016-06-09 Thread lw-lin
GitHub user lw-lin opened a pull request:

https://github.com/apache/spark/pull/13575

[SPARK-15472][SQL] Add support for writing in `csv`, `json`, `text` formats 
in Structured Streaming

## What changes were proposed in this pull request?

This patch adds support for writing in `csv`, `json`, `text` formats in 
Structured Streaming:

**1. at a high level, this patch forms the following hierarchy**(`text` as 
an example):
```

  ↑
 TextOutputWriterBase
 ↗  ↖
BatchTextOutputWriter   StreamingTextOutputWriter
```
```

↗  ↖
BatchTextOutputWriterFactory   StreamingOutputWriterFactory
  ↑
  StreamingTextOutputWriterFactory
```
The `StreamingTextOutputWriter` and other 'streaming' output writers would 
write data **without** using an `OutputCommitter`. This was the same approach 
taken by [SPARK-14716](https://github.com/apache/spark/pull/12409).

**2. to support compression, this patch attaches an extension to the path 
assigned by `FileStreamSink`**, which is slightly different from 
[SPARK-14716](https://github.com/apache/spark/pull/12409). For example, if we 
are writing out using the `gzip` compression and `FileStreamSink` assigns path 
`${uuid}` to a text writer, then in the end the file written out will be 
`${uuid}.txt.gz` -- so that when we read the file back, we'll correctly 
interpret it as `gzip` compressed.

## How was this patch tested?

`FileStreamSinkSuite` is expanded much more to cover the added `csv`, 
`json`, `text` formats:

```scala
test(" csv - unpartitioned data - codecs: none/gzip")
test("json - unpartitioned data - codecs: none/gzip")
test("text - unpartitioned data - codecs: none/gzip")

test(" csv - partitioned data - codecs: none/gzip")
test("json - partitioned data - codecs: none/gzip")
test("text - partitioned data - codecs: none/gzip")

test(" csv - unpartitioned writing and batch reading - codecs: none/gzip")
test("json - unpartitioned writing and batch reading - codecs: none/gzip")
test("text - unpartitioned writing and batch reading - codecs: none/gzip")

test(" csv - partitioned writing and batch reading - codecs: none/gzip")
test("json - partitioned writing and batch reading - codecs: none/gzip")
test("text - partitioned writing and batch reading - codecs: none/gzip")
```

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/lw-lin/spark add-csv-json-text-in-ss

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/13575.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #13575


commit c70083e9f76c20f6bf48e7ec821452f9bf63783a
Author: Liwei Lin 
Date:   2016-06-05T09:03:04Z

Add csv, json, text

commit bc28f4112ca9eca6a9f1602a891dd0388fa3185c
Author: Liwei Lin 
Date:   2016-06-09T03:31:59Z

Fix parquet extension




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