You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by lw-lin <gi...@git.apache.org> on 2016/06/09 10:32:28 UTC

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

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):
    ```
                    <OutputWriter>
                          ↑
                 TextOutputWriterBase
                     ↗          ↖
    BatchTextOutputWriter   StreamingTextOutputWriter
    ```
    ```
                        <OutputWriterFactory>
                            ↗          ↖
    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 <lw...@gmail.com>
Date:   2016-06-05T09:03:04Z

    Add csv, json, text

commit bc28f4112ca9eca6a9f1602a891dd0388fa3185c
Author: Liwei Lin <lw...@gmail.com>
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by lw-lin <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by lw-lin <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by lw-lin <gi...@git.apache.org>.
Github user lw-lin commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    @marmbrus @tdas @zsxwing , would you mind taking a look? Thanks!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    **[Test build #60230 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60230/consoleFull)** for PR 13575 at commit [`bc28f41`](https://github.com/apache/spark/commit/bc28f4112ca9eca6a9f1602a891dd0388fa3185c).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by lw-lin <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by ScrapCodes <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by lw-lin <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by lw-lin <gi...@git.apache.org>.
Github user lw-lin commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    No problem here. Spark-17924 is super great (I've been watching it all the time)  and thank you for bringing that up!


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by lw-lin <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    **[Test build #60230 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/60230/consoleFull)** for PR 13575 at commit [`bc28f41`](https://github.com/apache/spark/commit/bc28f4112ca9eca6a9f1602a891dd0388fa3185c).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by lw-lin <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    @lw-lin sorry we haven't visited this pr in the past, but the goal of this pr will be accomplished by https://issues.apache.org/jira/browse/SPARK-17924 (as a side effect). It reduces a lot of duplicated code as well.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by lw-lin <gi...@git.apache.org>.
Github user lw-lin commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    No problem here. Spark-17924 is super great (I've been watching it all the time)  and @rxin thank you for bringing that up! :-D


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark issue #13575: [SPARK-15472][SQL] Add support for writing in `csv`, `js...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/13575
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/60230/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


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

Posted by lw-lin <gi...@git.apache.org>.
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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