Github user tdas commented on a diff in the pull request: https://github.com/apache/spark/pull/13653#discussion_r66888185 --- Diff: sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala --- @@ -0,0 +1,401 @@ +/* + * 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.streaming + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, ForeachWriter} +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.{ForeachSink, MemoryPlan, MemorySink} + +/** + * :: Experimental :: + * Interface used to write a streaming [[Dataset]] to external storage systems (e.g. file systems, + * key-value stores, etc). Use [[Dataset.write]] to access this. + * + * @since 2.0.0 + */ +@Experimental +final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { + + private val df = ds.toDF() + + /** + * :: Experimental :: + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be + * written to the sink + * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time these is some updates + * + * @since 2.0.0 + */ + @Experimental + def outputMode(outputMode: OutputMode): DataStreamWriter[T] = { + this.outputMode = outputMode + this + } + + + /** + * :: Experimental :: + * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. + * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to + * the sink + * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink + * every time these is some updates + * + * @since 2.0.0 + */ + @Experimental + def outputMode(outputMode: String): DataStreamWriter[T] = { + this.outputMode = outputMode.toLowerCase match { + case "append" => + OutputMode.Append + case "complete" => + OutputMode.Complete + case _ => + throw new IllegalArgumentException(s"Unknown output mode $outputMode. " + + "Accepted output modes are 'append' and 'complete'") + } + this + } + + /** + * :: Experimental :: + * Set the trigger for the stream query. The default value is `ProcessingTime(0)` and it will run + * the query as fast as possible. + * + * Scala Example: + * {{{ + * df.write.trigger(ProcessingTime("10 seconds")) + * + * import scala.concurrent.duration._ + * df.write.trigger(ProcessingTime(10.seconds)) + * }}} + * + * Java Example: + * {{{ + * df.write.trigger(ProcessingTime.create("10 seconds")) + * + * import java.util.concurrent.TimeUnit + * df.write.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) + * }}} + * + * @since 2.0.0 + */ + @Experimental + def trigger(trigger: Trigger): DataStreamWriter[T] = { + this.trigger = trigger + this + } + + + /** + * :: Experimental :: + * Specifies the name of the [[ContinuousQuery]] that can be started with `startStream()`. + * This name must be unique among all the currently active queries in the associated SQLContext. + * + * @since 2.0.0 + */ + @Experimental + def queryName(queryName: String): DataStreamWriter[T] = { + this.extraOptions += ("queryName" -> queryName) + this + } + + /** + * :: Experimental :: + * Specifies the underlying output data source. Built-in options include "parquet", "json", etc. + * + * @since 2.0.0 + */ + @Experimental + def format(source: String): DataStreamWriter[T] = { + this.source = source + this + } + + /** + * Partitions the output by the given columns on the file system. If specified, the output is + * laid out on the file system similar to Hive's partitioning scheme. As an example, when we + * partition a dataset by year and then month, the directory layout would look like: + * + * - year=2016/month=01/ + * - year=2016/month=02/ + * + * Partitioning is one of the most widely used techniques to optimize physical data layout. + * It provides a coarse-grained index for skipping unnecessary data reads when queries have + * predicates on the partitioned columns. In order for partitioning to work well, the number + * of distinct values in each column should typically be less than tens of thousands. + * + * This was initially applicable for Parquet but in 1.5+ covers JSON, text, ORC and avro as well. + * + * @since 1.4.0 + */ + @scala.annotation.varargs + def partitionBy(colNames: String*): DataStreamWriter[T] = { + this.partitioningColumns = Option(colNames) + this + } + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: String): DataStreamWriter[T] = { + this.extraOptions += (key -> value) + this + } + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Boolean): DataStreamWriter[T] = option(key, value.toString) + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Long): DataStreamWriter[T] = option(key, value.toString) + + /** + * :: Experimental :: + * Adds an output option for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def option(key: String, value: Double): DataStreamWriter[T] = option(key, value.toString) + + /** + * :: Experimental :: + * (Scala-specific) Adds output options for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def options(options: scala.collection.Map[String, String]): DataStreamWriter[T] = { + this.extraOptions ++= options + this + } + + /** + * :: Experimental :: + * Adds output options for the underlying data source. + * + * @since 2.0.0 + */ + @Experimental + def options(options: java.util.Map[String, String]): DataStreamWriter[T] = { + this.options(options.asScala) + this + } + + + /** + * :: Experimental :: + * Saves the content of the streaming [[DataFrame]] in Parquet format at the specified path. + * This is equivalent to: + * {{{ + * format("parquet").save(path) + * }}} + * + * You can set the following Parquet-specific option(s) for writing Parquet files: + * <li>`compression` (default is the value specified in `spark.sql.parquet.compression.codec`): + * compression codec to use when saving to file. This can be one of the known case-insensitive + * shorten names(none, `snappy`, `gzip`, and `lzo`). This will override + * `spark.sql.parquet.compression.codec`.</li> + * + * @since 2.0.0 + */ + @Experimental + def parquet(path: String): ContinuousQuery = { + format("parquet").save(path) + } + + + /** + * :: Experimental :: + * Starts the execution of the streaming query, which will continually output results to the given + * path as new data arrives. The returned [[ContinuousQuery]] object can be used to interact with + * the stream. + * + * @since 2.0.0 + */ + @Experimental + def save(path: String): ContinuousQuery = { --- End diff -- @marmbrus @rxin @mateiz Should this be `save()`? The general case will be ``` val query = sdf .writeStream .format("myFormat") .save("stringIdentifier") ```
--- 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