|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.streaming |
| 19 | + |
| 20 | +import java.util.concurrent.TimeoutException |
| 21 | + |
| 22 | +import scala.collection.JavaConverters._ |
| 23 | +import scala.collection.mutable |
| 24 | + |
| 25 | +import org.apache.spark.annotation.Experimental |
| 26 | +import org.apache.spark.sql.{DataFrame, Dataset} |
| 27 | +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException |
| 28 | +import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table} |
| 29 | +import org.apache.spark.sql.connector.catalog.TableCapability.{STREAMING_WRITE, TRUNCATE} |
| 30 | + |
| 31 | +@Experimental |
| 32 | +final class DataStreamWriterV2[T] private[sql](table: String, ds: Dataset[T]) { |
| 33 | + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ |
| 34 | + import org.apache.spark.sql.connector.catalog.CatalogV2Util._ |
| 35 | + import df.sparkSession.sessionState.analyzer.CatalogAndIdentifier |
| 36 | + |
| 37 | + private val df: DataFrame = ds.toDF() |
| 38 | + |
| 39 | + private val sparkSession = ds.sparkSession |
| 40 | + |
| 41 | + private var trigger: Trigger = Trigger.ProcessingTime(0L) |
| 42 | + |
| 43 | + private var extraOptions = new mutable.HashMap[String, String]() |
| 44 | + |
| 45 | + private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table) |
| 46 | + |
| 47 | + private val (catalog, identifier) = { |
| 48 | + val CatalogAndIdentifier(catalog, identifier) = tableName |
| 49 | + (catalog.asTableCatalog, identifier) |
| 50 | + } |
| 51 | + |
| 52 | + def trigger(trigger: Trigger): DataStreamWriterV2[T] = { |
| 53 | + this.trigger = trigger |
| 54 | + this |
| 55 | + } |
| 56 | + |
| 57 | + def queryName(queryName: String): DataStreamWriterV2[T] = { |
| 58 | + this.extraOptions += ("queryName" -> queryName) |
| 59 | + this |
| 60 | + } |
| 61 | + |
| 62 | + def option(key: String, value: String): DataStreamWriterV2[T] = { |
| 63 | + this.extraOptions += (key -> value) |
| 64 | + this |
| 65 | + } |
| 66 | + |
| 67 | + def option(key: String, value: Boolean): DataStreamWriterV2[T] = option(key, value.toString) |
| 68 | + |
| 69 | + def option(key: String, value: Long): DataStreamWriterV2[T] = option(key, value.toString) |
| 70 | + |
| 71 | + def option(key: String, value: Double): DataStreamWriterV2[T] = option(key, value.toString) |
| 72 | + |
| 73 | + def options(options: scala.collection.Map[String, String]): DataStreamWriterV2[T] = { |
| 74 | + this.extraOptions ++= options |
| 75 | + this |
| 76 | + } |
| 77 | + |
| 78 | + def options(options: java.util.Map[String, String]): DataStreamWriterV2[T] = { |
| 79 | + this.options(options.asScala) |
| 80 | + this |
| 81 | + } |
| 82 | + |
| 83 | + def checkpointLocation(location: String): DataStreamWriterV2[T] = { |
| 84 | + this.extraOptions += "checkpointLocation" -> location |
| 85 | + this |
| 86 | + } |
| 87 | + |
| 88 | + @throws[NoSuchTableException] |
| 89 | + @throws[TimeoutException] |
| 90 | + def append(): StreamingQuery = { |
| 91 | + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ |
| 92 | + loadTable(catalog, identifier) match { |
| 93 | + case Some(t: SupportsWrite) if t.supports(STREAMING_WRITE) => |
| 94 | + start(t, OutputMode.Append()) |
| 95 | + |
| 96 | + case Some(t) => |
| 97 | + throw new IllegalArgumentException(s"Table ${t.name()} doesn't support streaming" + |
| 98 | + " write!") |
| 99 | + |
| 100 | + case _ => |
| 101 | + throw new NoSuchTableException(identifier) |
| 102 | + } |
| 103 | + } |
| 104 | + |
| 105 | + @throws[NoSuchTableException] |
| 106 | + @throws[TimeoutException] |
| 107 | + def truncateAndAppend(): StreamingQuery = { |
| 108 | + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ |
| 109 | + loadTable(catalog, identifier) match { |
| 110 | + case Some(t: SupportsWrite) if t.supports(STREAMING_WRITE) && t.supports(TRUNCATE) => |
| 111 | + start(t, OutputMode.Complete()) |
| 112 | + |
| 113 | + case Some(t) => |
| 114 | + throw new IllegalArgumentException(s"Table ${t.name()} doesn't support streaming" + |
| 115 | + " write with truncate!") |
| 116 | + |
| 117 | + case _ => |
| 118 | + throw new NoSuchTableException(identifier) |
| 119 | + } |
| 120 | + } |
| 121 | + |
| 122 | + private def start(table: Table, outputMode: OutputMode): StreamingQuery = { |
| 123 | + df.sparkSession.sessionState.streamingQueryManager.startQuery( |
| 124 | + extraOptions.get("queryName"), |
| 125 | + extraOptions.get("checkpointLocation"), |
| 126 | + df, |
| 127 | + extraOptions.toMap, |
| 128 | + table, |
| 129 | + outputMode, |
| 130 | + // Here we simply use default values of `useTempCheckpointLocation` and |
| 131 | + // `recoverFromCheckpointLocation`, which is required to be changed for some special built-in |
| 132 | + // data sources. They're not available in catalog, hence it's safe as of now, but once the |
| 133 | + // condition is broken we should take care of that. |
| 134 | + trigger = trigger) |
| 135 | + } |
| 136 | +} |
0 commit comments