aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala
blob: f70c7d08a691c26827276c4b2b38bb4c463ff544 (plain) (blame)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
/*
 * 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.writeStream]] 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.writeStream.trigger(ProcessingTime("10 seconds"))
   *
   *   import scala.concurrent.duration._
   *   df.writeStream.trigger(ProcessingTime(10.seconds))
   * }}}
   *
   * Java Example:
   * {{{
   *   df.writeStream().trigger(ProcessingTime.create("10 seconds"))
   *
   *   import java.util.concurrent.TimeUnit
   *   df.writeStream().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 [[StreamingQuery]] that can be started with `start()`.
   * 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" for now.
   *
   * @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 ::
   * Starts the execution of the streaming query, which will continually output results to the given
   * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with
   * the stream.
   *
   * @since 2.0.0
   */
  @Experimental
  def start(path: String): StreamingQuery = {
    option("path", path).start()
  }

  /**
   * :: Experimental ::
   * Starts the execution of the streaming query, which will continually output results to the given
   * path as new data arrives. The returned [[StreamingQuery]] object can be used to interact with
   * the stream.
   *
   * @since 2.0.0
   */
  @Experimental
  def start(): StreamingQuery = {
    if (source == "memory") {
      assertNotPartitioned("memory")
      if (extraOptions.get("queryName").isEmpty) {
        throw new AnalysisException("queryName must be specified for memory sink")
      }

      val sink = new MemorySink(df.schema, outputMode)
      val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink))
      val query = df.sparkSession.sessionState.streamingQueryManager.startQuery(
        extraOptions.get("queryName"),
        extraOptions.get("checkpointLocation"),
        df,
        sink,
        outputMode,
        useTempCheckpointLocation = true,
        recoverFromCheckpointLocation = false,
        trigger = trigger)
      resultDf.createOrReplaceTempView(query.name)
      query
    } else if (source == "foreach") {
      assertNotPartitioned("foreach")
      val sink = new ForeachSink[T](foreachWriter)(ds.exprEnc)
      df.sparkSession.sessionState.streamingQueryManager.startQuery(
        extraOptions.get("queryName"),
        extraOptions.get("checkpointLocation"),
        df,
        sink,
        outputMode,
        useTempCheckpointLocation = true,
        trigger = trigger)
    } else {
      val (useTempCheckpointLocation, recoverFromCheckpointLocation) =
        if (source == "console") {
          (true, false)
        } else {
          (false, true)
        }
      val dataSource =
        DataSource(
          df.sparkSession,
          className = source,
          options = extraOptions.toMap,
          partitionColumns = normalizedParCols.getOrElse(Nil))
      df.sparkSession.sessionState.streamingQueryManager.startQuery(
        extraOptions.get("queryName"),
        extraOptions.get("checkpointLocation"),
        df,
        dataSource.createSink(outputMode),
        outputMode,
        useTempCheckpointLocation = useTempCheckpointLocation,
        recoverFromCheckpointLocation = recoverFromCheckpointLocation,
        trigger = trigger)
    }
  }

  /**
   * :: Experimental ::
   * Starts the execution of the streaming query, which will continually send results to the given
   * [[ForeachWriter]] as as new data arrives. The [[ForeachWriter]] can be used to send the data
   * generated by the [[DataFrame]]/[[Dataset]] to an external system.
   *
   * Scala example:
   * {{{
   *   datasetOfString.writeStream.foreach(new ForeachWriter[String] {
   *
   *     def open(partitionId: Long, version: Long): Boolean = {
   *       // open connection
   *     }
   *
   *     def process(record: String) = {
   *       // write string to connection
   *     }
   *
   *     def close(errorOrNull: Throwable): Unit = {
   *       // close the connection
   *     }
   *   }).start()
   * }}}
   *
   * Java example:
   * {{{
   *  datasetOfString.writeStream().foreach(new ForeachWriter<String>() {
   *
   *    @Override
   *    public boolean open(long partitionId, long version) {
   *      // open connection
   *    }
   *
   *    @Override
   *    public void process(String value) {
   *      // write string to connection
   *    }
   *
   *    @Override
   *    public void close(Throwable errorOrNull) {
   *      // close the connection
   *    }
   *  }).start();
   * }}}
   *
   * @since 2.0.0
   */
  @Experimental
  def foreach(writer: ForeachWriter[T]): DataStreamWriter[T] = {
    this.source = "foreach"
    this.foreachWriter = if (writer != null) {
      ds.sparkSession.sparkContext.clean(writer)
    } else {
      throw new IllegalArgumentException("foreach writer cannot be null")
    }
    this
  }

  private def normalizedParCols: Option[Seq[String]] = partitioningColumns.map { cols =>
    cols.map(normalize(_, "Partition"))
  }

  /**
   * The given column name may not be equal to any of the existing column names if we were in
   * case-insensitive context. Normalize the given column name to the real one so that we don't
   * need to care about case sensitivity afterwards.
   */
  private def normalize(columnName: String, columnType: String): String = {
    val validColumnNames = df.logicalPlan.output.map(_.name)
    validColumnNames.find(df.sparkSession.sessionState.analyzer.resolver(_, columnName))
      .getOrElse(throw new AnalysisException(s"$columnType column $columnName not found in " +
        s"existing columns (${validColumnNames.mkString(", ")})"))
  }

  private def assertNotPartitioned(operation: String): Unit = {
    if (partitioningColumns.isDefined) {
      throw new AnalysisException(s"'$operation' does not support partitioning")
    }
  }

  ///////////////////////////////////////////////////////////////////////////////////////
  // Builder pattern config options
  ///////////////////////////////////////////////////////////////////////////////////////

  private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName

  private var outputMode: OutputMode = OutputMode.Append

  private var trigger: Trigger = ProcessingTime(0L)

  private var extraOptions = new scala.collection.mutable.HashMap[String, String]

  private var foreachWriter: ForeachWriter[T] = null

  private var partitioningColumns: Option[Seq[String]] = None
}