|
18 | 18 | package org.apache.spark.sql.streaming
|
19 | 19 |
|
20 | 20 | import java.util.UUID
|
21 |
| -import java.util.concurrent.TimeoutException |
22 | 21 |
|
23 | 22 | import scala.jdk.CollectionConverters._
|
24 | 23 |
|
25 |
| -import org.apache.spark.annotation.Evolving |
26 | 24 | import org.apache.spark.connect.proto.Command
|
27 | 25 | import org.apache.spark.connect.proto.ExecutePlanResponse
|
28 | 26 | import org.apache.spark.connect.proto.StreamingQueryCommand
|
29 | 27 | import org.apache.spark.connect.proto.StreamingQueryCommandResult
|
30 | 28 | import org.apache.spark.connect.proto.StreamingQueryManagerCommandResult.StreamingQueryInstance
|
31 |
| -import org.apache.spark.sql.SparkSession |
| 29 | +import org.apache.spark.sql.{api, Dataset, SparkSession} |
32 | 30 |
|
33 |
| -/** |
34 |
| - * A handle to a query that is executing continuously in the background as new data arrives. All |
35 |
| - * these methods are thread-safe. |
36 |
| - * @since 3.5.0 |
37 |
| - */ |
38 |
| -@Evolving |
39 |
| -trait StreamingQuery { |
40 |
| - // This is a copy of StreamingQuery in sql/core/.../streaming/StreamingQuery.scala |
41 |
| - |
42 |
| - /** |
43 |
| - * Returns the user-specified name of the query, or null if not specified. This name can be |
44 |
| - * specified in the `org.apache.spark.sql.streaming.DataStreamWriter` as |
45 |
| - * `dataframe.writeStream.queryName("query").start()`. This name, if set, must be unique across |
46 |
| - * all active queries. |
47 |
| - * |
48 |
| - * @since 3.5.0 |
49 |
| - */ |
50 |
| - def name: String |
51 |
| - |
52 |
| - /** |
53 |
| - * Returns the unique id of this query that persists across restarts from checkpoint data. That |
54 |
| - * is, this id is generated when a query is started for the first time, and will be the same |
55 |
| - * every time it is restarted from checkpoint data. Also see [[runId]]. |
56 |
| - * |
57 |
| - * @since 3.5.0 |
58 |
| - */ |
59 |
| - def id: UUID |
60 |
| - |
61 |
| - /** |
62 |
| - * Returns the unique id of this run of the query. That is, every start/restart of a query will |
63 |
| - * generate a unique runId. Therefore, every time a query is restarted from checkpoint, it will |
64 |
| - * have the same [[id]] but different [[runId]]s. |
65 |
| - */ |
66 |
| - def runId: UUID |
67 |
| - |
68 |
| - /** |
69 |
| - * Returns the `SparkSession` associated with `this`. |
70 |
| - * |
71 |
| - * @since 3.5.0 |
72 |
| - */ |
73 |
| - def sparkSession: SparkSession |
74 |
| - |
75 |
| - /** |
76 |
| - * Returns `true` if this query is actively running. |
77 |
| - * |
78 |
| - * @since 3.5.0 |
79 |
| - */ |
80 |
| - def isActive: Boolean |
81 |
| - |
82 |
| - /** |
83 |
| - * Returns the [[StreamingQueryException]] if the query was terminated by an exception. |
84 |
| - * @since 3.5.0 |
85 |
| - */ |
86 |
| - def exception: Option[StreamingQueryException] |
87 |
| - |
88 |
| - /** |
89 |
| - * Returns the current status of the query. |
90 |
| - * |
91 |
| - * @since 3.5.0 |
92 |
| - */ |
93 |
| - def status: StreamingQueryStatus |
94 |
| - |
95 |
| - /** |
96 |
| - * Returns an array of the most recent [[StreamingQueryProgress]] updates for this query. The |
97 |
| - * number of progress updates retained for each stream is configured by Spark session |
98 |
| - * configuration `spark.sql.streaming.numRecentProgressUpdates`. |
99 |
| - * |
100 |
| - * @since 3.5.0 |
101 |
| - */ |
102 |
| - def recentProgress: Array[StreamingQueryProgress] |
103 |
| - |
104 |
| - /** |
105 |
| - * Returns the most recent [[StreamingQueryProgress]] update of this streaming query. |
106 |
| - * |
107 |
| - * @since 3.5.0 |
108 |
| - */ |
109 |
| - def lastProgress: StreamingQueryProgress |
110 |
| - |
111 |
| - /** |
112 |
| - * Waits for the termination of `this` query, either by `query.stop()` or by an exception. If |
113 |
| - * the query has terminated with an exception, then the exception will be thrown. |
114 |
| - * |
115 |
| - * If the query has terminated, then all subsequent calls to this method will either return |
116 |
| - * immediately (if the query was terminated by `stop()`), or throw the exception immediately (if |
117 |
| - * the query has terminated with exception). |
118 |
| - * |
119 |
| - * @throws StreamingQueryException |
120 |
| - * if the query has terminated with an exception. |
121 |
| - * @since 3.5.0 |
122 |
| - */ |
123 |
| - @throws[StreamingQueryException] |
124 |
| - def awaitTermination(): Unit |
125 |
| - |
126 |
| - /** |
127 |
| - * Waits for the termination of `this` query, either by `query.stop()` or by an exception. If |
128 |
| - * the query has terminated with an exception, then the exception will be thrown. Otherwise, it |
129 |
| - * returns whether the query has terminated or not within the `timeoutMs` milliseconds. |
130 |
| - * |
131 |
| - * If the query has terminated, then all subsequent calls to this method will either return |
132 |
| - * `true` immediately (if the query was terminated by `stop()`), or throw the exception |
133 |
| - * immediately (if the query has terminated with exception). |
134 |
| - * |
135 |
| - * @throws StreamingQueryException |
136 |
| - * if the query has terminated with an exception |
137 |
| - * @since 3.5.0 |
138 |
| - */ |
139 |
| - @throws[StreamingQueryException] |
140 |
| - def awaitTermination(timeoutMs: Long): Boolean |
141 |
| - |
142 |
| - /** |
143 |
| - * Blocks until all available data in the source has been processed and committed to the sink. |
144 |
| - * This method is intended for testing. Note that in the case of continually arriving data, this |
145 |
| - * method may block forever. Additionally, this method is only guaranteed to block until data |
146 |
| - * that has been synchronously appended data to a |
147 |
| - * `org.apache.spark.sql.execution.streaming.Source` prior to invocation. (i.e. `getOffset` must |
148 |
| - * immediately reflect the addition). |
149 |
| - * @since 3.5.0 |
150 |
| - */ |
151 |
| - def processAllAvailable(): Unit |
152 |
| - |
153 |
| - /** |
154 |
| - * Stops the execution of this query if it is running. This waits until the termination of the |
155 |
| - * query execution threads or until a timeout is hit. |
156 |
| - * |
157 |
| - * By default stop will block indefinitely. You can configure a timeout by the configuration |
158 |
| - * `spark.sql.streaming.stopTimeout`. A timeout of 0 (or negative) milliseconds will block |
159 |
| - * indefinitely. If a `TimeoutException` is thrown, users can retry stopping the stream. If the |
160 |
| - * issue persists, it is advisable to kill the Spark application. |
161 |
| - * |
162 |
| - * @since 3.5.0 |
163 |
| - */ |
164 |
| - @throws[TimeoutException] |
165 |
| - def stop(): Unit |
166 |
| - |
167 |
| - /** |
168 |
| - * Prints the physical plan to the console for debugging purposes. |
169 |
| - * @since 3.5.0 |
170 |
| - */ |
171 |
| - def explain(): Unit |
172 |
| - |
173 |
| - /** |
174 |
| - * Prints the physical plan to the console for debugging purposes. |
175 |
| - * |
176 |
| - * @param extended |
177 |
| - * whether to do extended explain or not |
178 |
| - * @since 3.5.0 |
179 |
| - */ |
180 |
| - def explain(extended: Boolean): Unit |
| 31 | +/** @inheritdoc */ |
| 32 | +trait StreamingQuery extends api.StreamingQuery[Dataset] { |
| 33 | + |
| 34 | + /** @inheritdoc */ |
| 35 | + override def sparkSession: SparkSession |
181 | 36 | }
|
182 | 37 |
|
183 | 38 | class RemoteStreamingQuery(
|
|
0 commit comments