2016-04-20 13:32:01 -04:00
|
|
|
#
|
|
|
|
# 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.
|
|
|
|
#
|
|
|
|
|
2016-06-14 05:12:29 -04:00
|
|
|
import sys
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
import json
|
|
|
|
|
2018-06-19 16:56:51 -04:00
|
|
|
from py4j.java_gateway import java_import
|
|
|
|
|
2016-06-29 01:07:11 -04:00
|
|
|
from pyspark import since, keyword_only
|
2016-12-15 17:26:54 -05:00
|
|
|
from pyspark.sql.column import _to_seq
|
2016-06-29 01:07:11 -04:00
|
|
|
from pyspark.sql.readwriter import OptionUtils, to_str
|
2020-08-30 22:23:31 -04:00
|
|
|
from pyspark.sql.types import StructType, StructField, StringType
|
2018-06-19 16:56:51 -04:00
|
|
|
from pyspark.sql.utils import ForeachBatchFunction, StreamingQueryException
|
2016-04-20 13:32:01 -04:00
|
|
|
|
2016-06-29 01:07:11 -04:00
|
|
|
__all__ = ["StreamingQuery", "StreamingQueryManager", "DataStreamReader", "DataStreamWriter"]
|
2016-04-20 13:32:01 -04:00
|
|
|
|
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
class StreamingQuery(object):
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
|
|
|
A handle to a query that is executing continuously in the background as new data arrives.
|
|
|
|
All these methods are thread-safe.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-04-20 13:32:01 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
def __init__(self, jsq):
|
|
|
|
self._jsq = jsq
|
2016-04-20 13:32:01 -04:00
|
|
|
|
2016-06-14 05:12:29 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def id(self):
|
2016-12-05 21:17:38 -05:00
|
|
|
"""Returns the unique id of this query that persists across restarts from checkpoint data.
|
|
|
|
That is, this id is generated when a query is started for the first time, and
|
|
|
|
will be the same every time it is restarted from checkpoint data.
|
|
|
|
There can only be one query with the same id active in a Spark cluster.
|
|
|
|
Also see, `runId`.
|
2016-06-14 05:12:29 -04:00
|
|
|
"""
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
return self._jsq.id().toString()
|
2016-06-14 05:12:29 -04:00
|
|
|
|
2016-12-05 21:17:38 -05:00
|
|
|
@property
|
|
|
|
@since(2.1)
|
|
|
|
def runId(self):
|
|
|
|
"""Returns the unique id of this query that does not persist across restarts. That is, every
|
|
|
|
query that is started (or restarted from checkpoint) will have a different runId.
|
|
|
|
"""
|
|
|
|
return self._jsq.runId().toString()
|
|
|
|
|
2016-04-20 13:32:01 -04:00
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def name(self):
|
2016-12-05 21:17:38 -05:00
|
|
|
"""Returns the user-specified name of the query, or null if not specified.
|
|
|
|
This name can be specified in the `org.apache.spark.sql.streaming.DataStreamWriter`
|
|
|
|
as `dataframe.writeStream.queryName("query").start()`.
|
|
|
|
This name, if set, must be unique across all active queries.
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
return self._jsq.name()
|
2016-04-20 13:32:01 -04:00
|
|
|
|
|
|
|
@property
|
|
|
|
@since(2.0)
|
|
|
|
def isActive(self):
|
2016-06-15 13:46:02 -04:00
|
|
|
"""Whether this streaming query is currently active or not.
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
return self._jsq.isActive()
|
2016-04-20 13:32:01 -04:00
|
|
|
|
|
|
|
@since(2.0)
|
2016-04-28 18:22:28 -04:00
|
|
|
def awaitTermination(self, timeout=None):
|
2016-04-20 13:32:01 -04:00
|
|
|
"""Waits for the termination of `this` query, either by :func:`query.stop()` or by an
|
|
|
|
exception. If the query has terminated with an exception, then the exception will be thrown.
|
2016-04-28 18:22:28 -04:00
|
|
|
If `timeout` is set, it returns whether the query has terminated or not within the
|
|
|
|
`timeout` seconds.
|
2016-04-20 13:32:01 -04:00
|
|
|
|
|
|
|
If the query has terminated, then all subsequent calls to this method will either return
|
|
|
|
immediately (if the query was terminated by :func:`stop()`), or throw the exception
|
|
|
|
immediately (if the query has terminated with exception).
|
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
throws :class:`StreamingQueryException`, if `this` query has terminated with an exception
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
2016-04-28 18:22:28 -04:00
|
|
|
if timeout is not None:
|
|
|
|
if not isinstance(timeout, (int, float)) or timeout < 0:
|
|
|
|
raise ValueError("timeout must be a positive integer or float. Got %s" % timeout)
|
2016-06-15 13:46:02 -04:00
|
|
|
return self._jsq.awaitTermination(int(timeout * 1000))
|
2016-04-20 13:32:01 -04:00
|
|
|
else:
|
2016-06-15 13:46:02 -04:00
|
|
|
return self._jsq.awaitTermination()
|
2016-04-20 13:32:01 -04:00
|
|
|
|
2016-11-30 02:08:56 -05:00
|
|
|
@property
|
|
|
|
@since(2.1)
|
|
|
|
def status(self):
|
|
|
|
"""
|
|
|
|
Returns the current status of the query.
|
|
|
|
"""
|
|
|
|
return json.loads(self._jsq.status().json())
|
|
|
|
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
@property
|
|
|
|
@since(2.1)
|
2016-12-07 18:36:29 -05:00
|
|
|
def recentProgress(self):
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
"""Returns an array of the most recent [[StreamingQueryProgress]] updates for this query.
|
|
|
|
The number of progress updates retained for each stream is configured by Spark session
|
2016-12-07 18:36:29 -05:00
|
|
|
configuration `spark.sql.streaming.numRecentProgressUpdates`.
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
"""
|
2016-12-07 18:36:29 -05:00
|
|
|
return [json.loads(p.json()) for p in self._jsq.recentProgress()]
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
|
|
|
|
@property
|
|
|
|
def lastProgress(self):
|
|
|
|
"""
|
2016-12-14 16:36:41 -05:00
|
|
|
Returns the most recent :class:`StreamingQueryProgress` update of this streaming query or
|
|
|
|
None if there were no progress updates
|
2020-02-18 02:46:45 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
dict
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
"""
|
2016-12-14 16:36:41 -05:00
|
|
|
lastProgress = self._jsq.lastProgress()
|
|
|
|
if lastProgress:
|
|
|
|
return json.loads(lastProgress.json())
|
|
|
|
else:
|
|
|
|
return None
|
[SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
- `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
- `recentProgress` - an array of statistics about the most recent microbatches that have executed.
A recent progress contains the following information:
```
{
"id" : "2be8670a-fce1-4859-a530-748f29553bb6",
"name" : "query-29",
"timestamp" : 1479705392724,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303,
"durationMs" : {
"triggerExecution" : 276,
"queryPlanning" : 3,
"getBatch" : 5,
"getOffset" : 3,
"addBatch" : 234,
"walCommit" : 30
},
"currentWatermark" : 0,
"stateOperators" : [ ],
"sources" : [ {
"description" : "KafkaSource[Subscribe[topic-14]]",
"startOffset" : {
"topic-14" : {
"2" : 0,
"4" : 1,
"1" : 0,
"3" : 0,
"0" : 0
}
},
"endOffset" : {
"topic-14" : {
"2" : 1,
"4" : 2,
"1" : 0,
"3" : 0,
"0" : 1
}
},
"numRecords" : 3,
"inputRowsPerSecond" : 230.76923076923077,
"processedRowsPerSecond" : 10.869565217391303
} ]
}
```
Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>
Closes #15954 from marmbrus/queryProgress.
2016-11-29 20:24:17 -05:00
|
|
|
|
2016-04-20 13:32:01 -04:00
|
|
|
def processAllAvailable(self):
|
2016-06-06 04:35:47 -04:00
|
|
|
"""Blocks until all available data in the source has been processed and committed to the
|
2016-11-22 06:40:18 -05:00
|
|
|
sink. This method is intended for testing.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
In the case of continually arriving data, this method may block forever.
|
|
|
|
Additionally, this method is only guaranteed to block until data that has been
|
|
|
|
synchronously appended data to a stream source prior to invocation.
|
|
|
|
(i.e. `getOffset` must immediately reflect the addition).
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
return self._jsq.processAllAvailable()
|
2016-04-20 13:32:01 -04:00
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def stop(self):
|
2016-06-15 13:46:02 -04:00
|
|
|
"""Stop this streaming query.
|
2016-04-20 13:32:01 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
self._jsq.stop()
|
2016-04-20 13:32:01 -04:00
|
|
|
|
2016-12-05 14:36:11 -05:00
|
|
|
def explain(self, extended=False):
|
|
|
|
"""Prints the (logical and physical) plans to the console for debugging purpose.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
extended : bool, optional
|
|
|
|
default ``False``. If ``False``, prints only the physical plan.
|
2016-12-05 14:36:11 -05:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-12-05 14:36:11 -05:00
|
|
|
>>> sq = sdf.writeStream.format('memory').queryName('query_explain').start()
|
|
|
|
>>> sq.processAllAvailable() # Wait a bit to generate the runtime plans.
|
|
|
|
>>> sq.explain()
|
|
|
|
== Physical Plan ==
|
|
|
|
...
|
|
|
|
>>> sq.explain(True)
|
|
|
|
== Parsed Logical Plan ==
|
|
|
|
...
|
|
|
|
== Analyzed Logical Plan ==
|
|
|
|
...
|
|
|
|
== Optimized Logical Plan ==
|
|
|
|
...
|
|
|
|
== Physical Plan ==
|
|
|
|
...
|
|
|
|
>>> sq.stop()
|
|
|
|
"""
|
|
|
|
# Cannot call `_jsq.explain(...)` because it will print in the JVM process.
|
|
|
|
# We should print it in the Python process.
|
|
|
|
print(self._jsq.explainInternal(extended))
|
|
|
|
|
|
|
|
def exception(self):
|
|
|
|
"""
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.1.0
|
|
|
|
|
|
|
|
Returns
|
|
|
|
-------
|
|
|
|
:class:`StreamingQueryException`
|
|
|
|
the StreamingQueryException if the query was terminated by an exception, or None.
|
2016-12-05 14:36:11 -05:00
|
|
|
"""
|
|
|
|
if self._jsq.exception().isDefined():
|
|
|
|
je = self._jsq.exception().get()
|
|
|
|
msg = je.toString().split(': ', 1)[1] # Drop the Java StreamingQueryException type info
|
|
|
|
stackTrace = '\n\t at '.join(map(lambda x: x.toString(), je.getStackTrace()))
|
2019-04-29 12:44:23 -04:00
|
|
|
return StreamingQueryException(msg, stackTrace, je.getCause())
|
2016-12-05 14:36:11 -05:00
|
|
|
else:
|
|
|
|
return None
|
|
|
|
|
2016-04-20 13:32:01 -04:00
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
class StreamingQueryManager(object):
|
|
|
|
"""A class to manage all the :class:`StreamingQuery` StreamingQueries active.
|
2016-04-28 18:22:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-04-28 18:22:28 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-04-28 18:22:28 -04:00
|
|
|
"""
|
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
def __init__(self, jsqm):
|
|
|
|
self._jsqm = jsqm
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
@property
|
|
|
|
def active(self):
|
|
|
|
"""Returns a list of active queries associated with this SQLContext
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> sq = sdf.writeStream.format('memory').queryName('this_query').start()
|
2016-06-15 13:46:02 -04:00
|
|
|
>>> sqm = spark.streams
|
|
|
|
>>> # get the list of active streaming queries
|
|
|
|
>>> [q.name for q in sqm.active]
|
2020-07-13 22:22:44 -04:00
|
|
|
['this_query']
|
2016-06-15 13:46:02 -04:00
|
|
|
>>> sq.stop()
|
2016-04-28 18:22:28 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
return [StreamingQuery(jsq) for jsq in self._jsqm.active()]
|
2016-04-28 18:22:28 -04:00
|
|
|
|
2016-06-14 05:12:29 -04:00
|
|
|
def get(self, id):
|
2016-04-28 18:22:28 -04:00
|
|
|
"""Returns an active query from this SQLContext or throws exception if an active query
|
|
|
|
with this name doesn't exist.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> sq = sdf.writeStream.format('memory').queryName('this_query').start()
|
2016-06-15 13:46:02 -04:00
|
|
|
>>> sq.name
|
2020-07-13 22:22:44 -04:00
|
|
|
'this_query'
|
2016-06-15 13:46:02 -04:00
|
|
|
>>> sq = spark.streams.get(sq.id)
|
|
|
|
>>> sq.isActive
|
2016-06-14 05:12:29 -04:00
|
|
|
True
|
2016-06-15 13:46:02 -04:00
|
|
|
>>> sq = sqlContext.streams.get(sq.id)
|
|
|
|
>>> sq.isActive
|
2016-04-28 18:22:28 -04:00
|
|
|
True
|
2016-06-15 13:46:02 -04:00
|
|
|
>>> sq.stop()
|
2016-04-28 18:22:28 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
return StreamingQuery(self._jsqm.get(id))
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
@since(2.0)
|
|
|
|
def awaitAnyTermination(self, timeout=None):
|
|
|
|
"""Wait until any of the queries on the associated SQLContext has terminated since the
|
|
|
|
creation of the context, or since :func:`resetTerminated()` was called. If any query was
|
|
|
|
terminated with an exception, then the exception will be thrown.
|
|
|
|
If `timeout` is set, it returns whether the query has terminated or not within the
|
|
|
|
`timeout` seconds.
|
|
|
|
|
|
|
|
If a query has terminated, then subsequent calls to :func:`awaitAnyTermination()` will
|
|
|
|
either return immediately (if the query was terminated by :func:`query.stop()`),
|
|
|
|
or throw the exception immediately (if the query was terminated with exception). Use
|
|
|
|
:func:`resetTerminated()` to clear past terminations and wait for new terminations.
|
|
|
|
|
|
|
|
In the case where multiple queries have terminated since :func:`resetTermination()`
|
|
|
|
was called, if any query has terminated with exception, then :func:`awaitAnyTermination()`
|
|
|
|
will throw any of the exception. For correctly documenting exceptions across multiple
|
|
|
|
queries, users need to stop all of them after any of them terminates with exception, and
|
|
|
|
then check the `query.exception()` for each query.
|
|
|
|
|
2016-06-15 13:46:02 -04:00
|
|
|
throws :class:`StreamingQueryException`, if `this` query has terminated with an exception
|
2016-04-28 18:22:28 -04:00
|
|
|
"""
|
|
|
|
if timeout is not None:
|
|
|
|
if not isinstance(timeout, (int, float)) or timeout < 0:
|
|
|
|
raise ValueError("timeout must be a positive integer or float. Got %s" % timeout)
|
2016-06-15 13:46:02 -04:00
|
|
|
return self._jsqm.awaitAnyTermination(int(timeout * 1000))
|
2016-04-28 18:22:28 -04:00
|
|
|
else:
|
2016-06-15 13:46:02 -04:00
|
|
|
return self._jsqm.awaitAnyTermination()
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
def resetTerminated(self):
|
|
|
|
"""Forget about past terminated queries so that :func:`awaitAnyTermination()` can be used
|
|
|
|
again to wait for new terminations.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-14 05:12:29 -04:00
|
|
|
>>> spark.streams.resetTerminated()
|
2016-04-28 18:22:28 -04:00
|
|
|
"""
|
2016-06-15 13:46:02 -04:00
|
|
|
self._jsqm.resetTerminated()
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
|
2016-06-29 01:07:11 -04:00
|
|
|
class DataStreamReader(OptionUtils):
|
|
|
|
"""
|
2020-02-13 21:00:35 -05:00
|
|
|
Interface used to load a streaming :class:`DataFrame <pyspark.sql.DataFrame>` from external
|
|
|
|
storage systems (e.g. file systems, key-value stores, etc).
|
|
|
|
Use :attr:`SparkSession.readStream <pyspark.sql.SparkSession.readStream>` to access this.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, spark):
|
|
|
|
self._jreader = spark._ssql_ctx.readStream()
|
|
|
|
self._spark = spark
|
|
|
|
|
|
|
|
def _df(self, jdf):
|
|
|
|
from pyspark.sql.dataframe import DataFrame
|
|
|
|
return DataFrame(jdf, self._spark)
|
|
|
|
|
|
|
|
def format(self, source):
|
|
|
|
"""Specifies the input data source format.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
source : str
|
|
|
|
name of the data source, e.g. 'json', 'parquet'.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> s = spark.readStream.format("text")
|
|
|
|
"""
|
|
|
|
self._jreader = self._jreader.format(source)
|
|
|
|
return self
|
|
|
|
|
|
|
|
def schema(self, schema):
|
|
|
|
"""Specifies the input schema.
|
|
|
|
|
|
|
|
Some data sources (e.g. JSON) can infer the input schema automatically from data.
|
|
|
|
By specifying the schema here, the underlying data source can skip the schema
|
|
|
|
inference step, and thus speed up data loading.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
schema : :class:`pyspark.sql.types.StructType` or str
|
|
|
|
a :class:`pyspark.sql.types.StructType` object or a DDL-formatted string
|
|
|
|
(For example ``col0 INT, col1 DOUBLE``).
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> s = spark.readStream.schema(sdf_schema)
|
2017-06-23 23:39:41 -04:00
|
|
|
>>> s = spark.readStream.schema("col0 INT, col1 DOUBLE")
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
2016-08-25 02:36:04 -04:00
|
|
|
from pyspark.sql import SparkSession
|
|
|
|
spark = SparkSession.builder.getOrCreate()
|
2017-06-23 23:39:41 -04:00
|
|
|
if isinstance(schema, StructType):
|
|
|
|
jschema = spark._jsparkSession.parseDataType(schema.json())
|
|
|
|
self._jreader = self._jreader.schema(jschema)
|
2020-07-13 22:22:44 -04:00
|
|
|
elif isinstance(schema, str):
|
2017-06-23 23:39:41 -04:00
|
|
|
self._jreader = self._jreader.schema(schema)
|
|
|
|
else:
|
|
|
|
raise TypeError("schema should be StructType or string")
|
2016-06-29 01:07:11 -04:00
|
|
|
return self
|
|
|
|
|
|
|
|
def option(self, key, value):
|
|
|
|
"""Adds an input option for the underlying data source.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> s = spark.readStream.option("x", 1)
|
|
|
|
"""
|
|
|
|
self._jreader = self._jreader.option(key, to_str(value))
|
|
|
|
return self
|
|
|
|
|
|
|
|
def options(self, **options):
|
|
|
|
"""Adds input options for the underlying data source.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> s = spark.readStream.options(x="1", y=2)
|
|
|
|
"""
|
|
|
|
for k in options:
|
|
|
|
self._jreader = self._jreader.option(k, to_str(options[k]))
|
|
|
|
return self
|
|
|
|
|
|
|
|
def load(self, path=None, format=None, schema=None, **options):
|
2020-02-18 02:46:45 -05:00
|
|
|
"""Loads a data stream from a data source and returns it as a
|
|
|
|
:class:`DataFrame <pyspark.sql.DataFrame>`.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str, optional
|
|
|
|
optional string for file-system backed data sources.
|
|
|
|
format : str, optional
|
|
|
|
optional string for format of the data source. Default to 'parquet'.
|
|
|
|
schema : :class:`pyspark.sql.types.StructType` or str, optional
|
|
|
|
optional :class:`pyspark.sql.types.StructType` for the input schema
|
|
|
|
or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
|
|
|
|
**options : dict
|
|
|
|
all other string options
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
[MINOR][PYSPARK][DOCS] Fix examples in PySpark documentation
## What changes were proposed in this pull request?
This PR proposes to fix wrongly indented examples in PySpark documentation
```
- >>> json_sdf = spark.readStream.format("json")\
- .schema(sdf_schema)\
- .load(tempfile.mkdtemp())
+ >>> json_sdf = spark.readStream.format("json") \\
+ ... .schema(sdf_schema) \\
+ ... .load(tempfile.mkdtemp())
```
```
- people.filter(people.age > 30).join(department, people.deptId == department.id)\
+ people.filter(people.age > 30).join(department, people.deptId == department.id) \\
```
```
- >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])), \
- LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+ >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, 1.23), (2, 4.56)])),
+ ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```
```
- >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])), \
- LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
+ >>> examples = [LabeledPoint(1.1, Vectors.sparse(3, [(0, -1.23), (2, 4.56e-7)])),
+ ... LabeledPoint(0.0, Vectors.dense([1.01, 2.02, 3.03]))]
```
```
- ... for x in iterator:
- ... print(x)
+ ... for x in iterator:
+ ... print(x)
```
## How was this patch tested?
Manually tested.
**Before**
![2016-09-26 8 36 02](https://cloud.githubusercontent.com/assets/6477701/18834471/05c7a478-8431-11e6-94bb-09aa37b12ddb.png)
![2016-09-26 9 22 16](https://cloud.githubusercontent.com/assets/6477701/18834472/06c8735c-8431-11e6-8775-78631eab0411.png)
<img width="601" alt="2016-09-27 2 29 27" src="https://cloud.githubusercontent.com/assets/6477701/18861294/29c0d5b4-84bf-11e6-99c5-3c9d913c125d.png">
<img width="1056" alt="2016-09-27 2 29 58" src="https://cloud.githubusercontent.com/assets/6477701/18861298/31694cd8-84bf-11e6-9e61-9888cb8c2089.png">
<img width="1079" alt="2016-09-27 2 30 05" src="https://cloud.githubusercontent.com/assets/6477701/18861301/359722da-84bf-11e6-97f9-5f5365582d14.png">
**After**
![2016-09-26 9 29 47](https://cloud.githubusercontent.com/assets/6477701/18834467/0367f9da-8431-11e6-86d9-a490d3297339.png)
![2016-09-26 9 30 24](https://cloud.githubusercontent.com/assets/6477701/18834463/f870fae0-8430-11e6-9482-01fc47898492.png)
<img width="515" alt="2016-09-27 2 28 19" src="https://cloud.githubusercontent.com/assets/6477701/18861305/3ff88b88-84bf-11e6-902c-9f725e8a8b10.png">
<img width="652" alt="2016-09-27 3 50 59" src="https://cloud.githubusercontent.com/assets/6477701/18863053/592fbc74-84ca-11e6-8dbf-99cf57947de8.png">
<img width="709" alt="2016-09-27 3 51 03" src="https://cloud.githubusercontent.com/assets/6477701/18863060/601607be-84ca-11e6-80aa-a401df41c321.png">
Author: hyukjinkwon <gurwls223@gmail.com>
Closes #15242 from HyukjinKwon/minor-example-pyspark.
2016-09-28 06:19:04 -04:00
|
|
|
>>> json_sdf = spark.readStream.format("json") \\
|
|
|
|
... .schema(sdf_schema) \\
|
|
|
|
... .load(tempfile.mkdtemp())
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> json_sdf.isStreaming
|
|
|
|
True
|
|
|
|
>>> json_sdf.schema == sdf_schema
|
|
|
|
True
|
|
|
|
"""
|
|
|
|
if format is not None:
|
|
|
|
self.format(format)
|
|
|
|
if schema is not None:
|
|
|
|
self.schema(schema)
|
|
|
|
self.options(**options)
|
|
|
|
if path is not None:
|
|
|
|
if type(path) != str or len(path.strip()) == 0:
|
|
|
|
raise ValueError("If the path is provided for stream, it needs to be a " +
|
|
|
|
"non-empty string. List of paths are not supported.")
|
|
|
|
return self._df(self._jreader.load(path))
|
|
|
|
else:
|
|
|
|
return self._df(self._jreader.load())
|
|
|
|
|
|
|
|
def json(self, path, schema=None, primitivesAsString=None, prefersDecimal=None,
|
|
|
|
allowComments=None, allowUnquotedFieldNames=None, allowSingleQuotes=None,
|
|
|
|
allowNumericLeadingZero=None, allowBackslashEscapingAnyCharacter=None,
|
2017-02-16 23:51:19 -05:00
|
|
|
mode=None, columnNameOfCorruptRecord=None, dateFormat=None, timestampFormat=None,
|
2018-11-11 08:01:29 -05:00
|
|
|
multiLine=None, allowUnquotedControlChars=None, lineSep=None, locale=None,
|
2019-12-22 19:57:42 -05:00
|
|
|
dropFieldIfAllNull=None, encoding=None, pathGlobFilter=None,
|
2020-07-28 23:14:13 -04:00
|
|
|
recursiveFileLookup=None, allowNonNumericNumbers=None):
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
2017-02-16 23:51:19 -05:00
|
|
|
Loads a JSON file stream and returns the results as a :class:`DataFrame`.
|
|
|
|
|
2017-04-12 04:16:39 -04:00
|
|
|
`JSON Lines <http://jsonlines.org/>`_ (newline-delimited JSON) is supported by default.
|
2017-06-15 01:18:19 -04:00
|
|
|
For JSON (one record per file), set the ``multiLine`` parameter to ``true``.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
|
|
|
If the ``schema`` parameter is not specified, this function goes
|
|
|
|
through the input once to determine the input schema.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str
|
|
|
|
string represents path to the JSON dataset,
|
|
|
|
or RDD of Strings storing JSON objects.
|
|
|
|
schema : :class:`pyspark.sql.types.StructType` or str, optional
|
|
|
|
an optional :class:`pyspark.sql.types.StructType` for the input schema
|
|
|
|
or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
|
|
|
|
|
2021-05-21 05:05:13 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-json.html#data-source-option>`_
|
2021-05-21 05:05:13 -04:00
|
|
|
in the version you use.
|
2019-12-03 20:10:30 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-07-01 18:16:04 -04:00
|
|
|
>>> json_sdf = spark.readStream.json(tempfile.mkdtemp(), schema = sdf_schema)
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> json_sdf.isStreaming
|
|
|
|
True
|
|
|
|
>>> json_sdf.schema == sdf_schema
|
|
|
|
True
|
|
|
|
"""
|
|
|
|
self._set_opts(
|
|
|
|
schema=schema, primitivesAsString=primitivesAsString, prefersDecimal=prefersDecimal,
|
|
|
|
allowComments=allowComments, allowUnquotedFieldNames=allowUnquotedFieldNames,
|
|
|
|
allowSingleQuotes=allowSingleQuotes, allowNumericLeadingZero=allowNumericLeadingZero,
|
|
|
|
allowBackslashEscapingAnyCharacter=allowBackslashEscapingAnyCharacter,
|
2016-08-24 16:16:20 -04:00
|
|
|
mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, dateFormat=dateFormat,
|
2017-08-25 13:18:03 -04:00
|
|
|
timestampFormat=timestampFormat, multiLine=multiLine,
|
2018-11-11 08:01:29 -05:00
|
|
|
allowUnquotedControlChars=allowUnquotedControlChars, lineSep=lineSep, locale=locale,
|
2019-12-03 20:10:30 -05:00
|
|
|
dropFieldIfAllNull=dropFieldIfAllNull, encoding=encoding,
|
2020-07-28 23:14:13 -04:00
|
|
|
pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup,
|
|
|
|
allowNonNumericNumbers=allowNonNumericNumbers)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2016-06-29 01:07:11 -04:00
|
|
|
return self._df(self._jreader.json(path))
|
|
|
|
else:
|
|
|
|
raise TypeError("path can be only a single string")
|
|
|
|
|
2019-12-22 19:57:42 -05:00
|
|
|
def orc(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None):
|
2017-12-20 02:50:06 -05:00
|
|
|
"""Loads a ORC file stream, returning the result as a :class:`DataFrame`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.3.0
|
|
|
|
|
2021-05-21 05:03:57 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-orc.html#data-source-option>`_
|
2021-05-21 05:03:57 -04:00
|
|
|
in the version you use.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2017-12-20 02:50:06 -05:00
|
|
|
>>> orc_sdf = spark.readStream.schema(sdf_schema).orc(tempfile.mkdtemp())
|
|
|
|
>>> orc_sdf.isStreaming
|
|
|
|
True
|
|
|
|
>>> orc_sdf.schema == sdf_schema
|
|
|
|
True
|
|
|
|
"""
|
2019-12-22 19:57:42 -05:00
|
|
|
self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter,
|
|
|
|
recursiveFileLookup=recursiveFileLookup)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2017-12-20 02:50:06 -05:00
|
|
|
return self._df(self._jreader.orc(path))
|
|
|
|
else:
|
|
|
|
raise TypeError("path can be only a single string")
|
|
|
|
|
[SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read
### What changes were proposed in this pull request?
In the PR, I propose new options for the Parquet datasource:
1. `datetimeRebaseMode`
2. `int96RebaseMode`
Both options influence on loading ancient dates and timestamps column values from parquet files. The `datetimeRebaseMode` option impacts on loading values of the `DATE`, `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` types, `int96RebaseMode` impacts on loading of `INT96` timestamps.
The options support the same values as the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInRead` namely;
- `"LEGACY"`, when an option is set to this value, Spark rebases dates/timestamps from the legacy hybrid calendar (Julian + Gregorian) to the Proleptic Gregorian calendar.
- `"CORRECTED"`, dates/timestamps are read AS IS from parquet files.
- `"EXCEPTION"`, when it is set as an option value, Spark will fail the reading if it sees ancient dates/timestamps that are ambiguous between the two calendars.
### Why are the changes needed?
1. New options will allow to load parquet files from at least two sources in different rebasing modes in the same query. For instance:
```scala
val df1 = spark.read.option("datetimeRebaseMode", "legacy").parquet(folder1)
val df2 = spark.read.option("datetimeRebaseMode", "corrected").parquet(folder2)
df1.join(df2, ...)
```
Before the changes, it is impossible because the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` influences on both reads.
2. Mixing of Dataset/DataFrame and RDD APIs should become possible. Since SQL configs are not propagated through RDDs, the following code fails on ancient timestamps:
```scala
spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "legacy")
spark.read.parquet(folder).distinct.rdd.collect()
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
By running the modified test suites:
```
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite"
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite"
```
Closes #31489 from MaxGekk/parquet-rebase-options.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 08:28:40 -05:00
|
|
|
def parquet(self, path, mergeSchema=None, pathGlobFilter=None, recursiveFileLookup=None,
|
|
|
|
datetimeRebaseMode=None, int96RebaseMode=None):
|
[SPARK-30091][SQL][PYTHON] Document mergeSchema option directly in the PySpark Parquet APIs
### What changes were proposed in this pull request?
This change properly documents the `mergeSchema` option directly in the Python APIs for reading Parquet data.
### Why are the changes needed?
The docstring for `DataFrameReader.parquet()` mentions `mergeSchema` but doesn't show it in the API. It seems like a simple oversight.
Before this PR, you'd have to do this to use `mergeSchema`:
```python
spark.read.option('mergeSchema', True).parquet('test-parquet').show()
```
After this PR, you can use the option as (I believe) it was intended to be used:
```python
spark.read.parquet('test-parquet', mergeSchema=True).show()
```
### Does this PR introduce any user-facing change?
Yes, this PR changes the signatures of `DataFrameReader.parquet()` and `DataStreamReader.parquet()` to match their docstrings.
### How was this patch tested?
Testing the `mergeSchema` option directly seems to be left to the Scala side of the codebase. I tested my change manually to confirm the API works.
I also confirmed that setting `spark.sql.parquet.mergeSchema` at the session does not get overridden by leaving `mergeSchema` at its default when calling `parquet()`:
```
>>> spark.conf.set('spark.sql.parquet.mergeSchema', True)
>>> spark.range(3).write.parquet('test-parquet/id')
>>> spark.range(3).withColumnRenamed('id', 'name').write.parquet('test-parquet/name')
>>> spark.read.option('recursiveFileLookup', True).parquet('test-parquet').show()
+----+----+
| id|name|
+----+----+
|null| 1|
|null| 2|
|null| 0|
| 1|null|
| 2|null|
| 0|null|
+----+----+
>>> spark.read.option('recursiveFileLookup', True).parquet('test-parquet', mergeSchema=False).show()
+----+
| id|
+----+
|null|
|null|
|null|
| 1|
| 2|
| 0|
+----+
```
Closes #26730 from nchammas/parquet-merge-schema.
Authored-by: Nicholas Chammas <nicholas.chammas@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-03 21:31:57 -05:00
|
|
|
"""
|
|
|
|
Loads a Parquet file stream, returning the result as a :class:`DataFrame`.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
2021-05-21 05:05:49 -04:00
|
|
|
path : str
|
|
|
|
the path in any Hadoop supported file system
|
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-parquet.html#data-source-option>`_.
|
2021-05-21 05:05:49 -04:00
|
|
|
in the version you use.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-07-01 18:16:04 -04:00
|
|
|
>>> parquet_sdf = spark.readStream.schema(sdf_schema).parquet(tempfile.mkdtemp())
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> parquet_sdf.isStreaming
|
|
|
|
True
|
|
|
|
>>> parquet_sdf.schema == sdf_schema
|
|
|
|
True
|
|
|
|
"""
|
2019-12-22 19:57:42 -05:00
|
|
|
self._set_opts(mergeSchema=mergeSchema, pathGlobFilter=pathGlobFilter,
|
[SPARK-34377][SQL] Add new parquet datasource options to control datetime rebasing in read
### What changes were proposed in this pull request?
In the PR, I propose new options for the Parquet datasource:
1. `datetimeRebaseMode`
2. `int96RebaseMode`
Both options influence on loading ancient dates and timestamps column values from parquet files. The `datetimeRebaseMode` option impacts on loading values of the `DATE`, `TIMESTAMP_MICROS` and `TIMESTAMP_MILLIS` types, `int96RebaseMode` impacts on loading of `INT96` timestamps.
The options support the same values as the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.int96RebaseModeInRead` namely;
- `"LEGACY"`, when an option is set to this value, Spark rebases dates/timestamps from the legacy hybrid calendar (Julian + Gregorian) to the Proleptic Gregorian calendar.
- `"CORRECTED"`, dates/timestamps are read AS IS from parquet files.
- `"EXCEPTION"`, when it is set as an option value, Spark will fail the reading if it sees ancient dates/timestamps that are ambiguous between the two calendars.
### Why are the changes needed?
1. New options will allow to load parquet files from at least two sources in different rebasing modes in the same query. For instance:
```scala
val df1 = spark.read.option("datetimeRebaseMode", "legacy").parquet(folder1)
val df2 = spark.read.option("datetimeRebaseMode", "corrected").parquet(folder2)
df1.join(df2, ...)
```
Before the changes, it is impossible because the SQL config `spark.sql.legacy.parquet.datetimeRebaseModeInRead` influences on both reads.
2. Mixing of Dataset/DataFrame and RDD APIs should become possible. Since SQL configs are not propagated through RDDs, the following code fails on ancient timestamps:
```scala
spark.conf.set("spark.sql.legacy.parquet.datetimeRebaseModeInRead", "legacy")
spark.read.parquet(folder).distinct.rdd.collect()
```
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
By running the modified test suites:
```
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV1Suite"
$ build/sbt "sql/test:testOnly *ParquetRebaseDatetimeV2Suite"
```
Closes #31489 from MaxGekk/parquet-rebase-options.
Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2021-02-08 08:28:40 -05:00
|
|
|
recursiveFileLookup=recursiveFileLookup,
|
|
|
|
datetimeRebaseMode=datetimeRebaseMode, int96RebaseMode=int96RebaseMode)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2016-06-29 01:07:11 -04:00
|
|
|
return self._df(self._jreader.parquet(path))
|
|
|
|
else:
|
|
|
|
raise TypeError("path can be only a single string")
|
|
|
|
|
2019-12-22 19:57:42 -05:00
|
|
|
def text(self, path, wholetext=False, lineSep=None, pathGlobFilter=None,
|
|
|
|
recursiveFileLookup=None):
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
|
|
|
Loads a text file stream and returns a :class:`DataFrame` whose schema starts with a
|
|
|
|
string column named "value", and followed by partitioned columns if there
|
|
|
|
are any.
|
2019-03-04 18:03:39 -05:00
|
|
|
The text files must be encoded as UTF-8.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2018-03-21 12:46:47 -04:00
|
|
|
By default, each line in the text file is a new row in the resulting DataFrame.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
paths : str or list
|
|
|
|
string, or list of strings, for input path(s).
|
2021-05-26 04:12:49 -04:00
|
|
|
|
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
2021-05-31 21:58:49 -04:00
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-text.html#data-source-option>`_
|
2021-05-26 04:12:49 -04:00
|
|
|
in the version you use.
|
2020-11-02 20:00:49 -05:00
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-30 19:51:11 -04:00
|
|
|
>>> text_sdf = spark.readStream.text(tempfile.mkdtemp())
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> text_sdf.isStreaming
|
|
|
|
True
|
|
|
|
>>> "value" in str(text_sdf.schema)
|
|
|
|
True
|
|
|
|
"""
|
2019-12-03 20:10:30 -05:00
|
|
|
self._set_opts(
|
2019-12-22 19:57:42 -05:00
|
|
|
wholetext=wholetext, lineSep=lineSep, pathGlobFilter=pathGlobFilter,
|
|
|
|
recursiveFileLookup=recursiveFileLookup)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2016-06-29 01:07:11 -04:00
|
|
|
return self._df(self._jreader.text(path))
|
|
|
|
else:
|
|
|
|
raise TypeError("path can be only a single string")
|
|
|
|
|
|
|
|
def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=None,
|
|
|
|
comment=None, header=None, inferSchema=None, ignoreLeadingWhiteSpace=None,
|
|
|
|
ignoreTrailingWhiteSpace=None, nullValue=None, nanValue=None, positiveInf=None,
|
2016-08-24 16:16:20 -04:00
|
|
|
negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None,
|
2017-03-15 01:30:16 -04:00
|
|
|
maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None,
|
2018-06-04 01:02:21 -04:00
|
|
|
columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None,
|
2019-12-03 20:10:30 -05:00
|
|
|
enforceSchema=None, emptyValue=None, locale=None, lineSep=None,
|
2020-11-27 01:47:39 -05:00
|
|
|
pathGlobFilter=None, recursiveFileLookup=None, unescapedQuoteHandling=None):
|
2018-09-12 23:19:43 -04:00
|
|
|
r"""Loads a CSV file stream and returns the result as a :class:`DataFrame`.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
|
|
|
This function will go through the input once to determine the input schema if
|
|
|
|
``inferSchema`` is enabled. To avoid going through the entire data once, disable
|
|
|
|
``inferSchema`` option or specify the schema explicitly using ``schema``.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str or list
|
|
|
|
string, or list of strings, for input path(s).
|
|
|
|
schema : :class:`pyspark.sql.types.StructType` or str, optional
|
|
|
|
an optional :class:`pyspark.sql.types.StructType` for the input schema
|
|
|
|
or a DDL-formatted string (For example ``col0 INT, col1 DOUBLE``).
|
|
|
|
|
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
2021-05-31 21:58:49 -04:00
|
|
|
Other Parameters
|
|
|
|
----------------
|
|
|
|
Extra options
|
|
|
|
For the extra options, refer to
|
|
|
|
`Data Source Option <https://spark.apache.org/docs/latest/sql-data-sources-csv.html#data-source-option>`_
|
|
|
|
in the version you use.
|
|
|
|
|
|
|
|
.. # noqa
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-07-01 18:16:04 -04:00
|
|
|
>>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema)
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> csv_sdf.isStreaming
|
|
|
|
True
|
|
|
|
>>> csv_sdf.schema == sdf_schema
|
|
|
|
True
|
|
|
|
"""
|
|
|
|
self._set_opts(
|
|
|
|
schema=schema, sep=sep, encoding=encoding, quote=quote, escape=escape, comment=comment,
|
|
|
|
header=header, inferSchema=inferSchema, ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace,
|
|
|
|
ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, nullValue=nullValue,
|
|
|
|
nanValue=nanValue, positiveInf=positiveInf, negativeInf=negativeInf,
|
2016-08-24 16:16:20 -04:00
|
|
|
dateFormat=dateFormat, timestampFormat=timestampFormat, maxColumns=maxColumns,
|
|
|
|
maxCharsPerColumn=maxCharsPerColumn,
|
2017-03-15 01:30:16 -04:00
|
|
|
maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode,
|
2017-12-28 18:30:06 -05:00
|
|
|
columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine,
|
2018-09-11 08:47:14 -04:00
|
|
|
charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema,
|
2019-12-03 20:10:30 -05:00
|
|
|
emptyValue=emptyValue, locale=locale, lineSep=lineSep,
|
2020-11-27 01:47:39 -05:00
|
|
|
pathGlobFilter=pathGlobFilter, recursiveFileLookup=recursiveFileLookup,
|
|
|
|
unescapedQuoteHandling=unescapedQuoteHandling)
|
2020-07-13 22:22:44 -04:00
|
|
|
if isinstance(path, str):
|
2016-06-29 01:07:11 -04:00
|
|
|
return self._df(self._jreader.csv(path))
|
|
|
|
else:
|
|
|
|
raise TypeError("path can be only a single string")
|
|
|
|
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
def table(self, tableName):
|
|
|
|
"""Define a Streaming DataFrame on a Table. The DataSource corresponding to the table should
|
|
|
|
support streaming mode.
|
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
tableName : str
|
|
|
|
string, for the name of the table.
|
|
|
|
|
|
|
|
Returns
|
|
|
|
--------
|
|
|
|
:class:`DataFrame`
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2020-12-21 16:27:27 -05:00
|
|
|
>>> spark.readStream.table('input_table') # doctest: +SKIP
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
"""
|
|
|
|
if isinstance(tableName, str):
|
|
|
|
return self._df(self._jreader.table(tableName))
|
|
|
|
else:
|
|
|
|
raise TypeError("tableName can be only a single string")
|
|
|
|
|
2016-06-29 01:07:11 -04:00
|
|
|
|
|
|
|
class DataStreamWriter(object):
|
|
|
|
"""
|
2020-02-13 21:00:35 -05:00
|
|
|
Interface used to write a streaming :class:`DataFrame <pyspark.sql.DataFrame>` to external
|
|
|
|
storage systems (e.g. file systems, key-value stores, etc).
|
|
|
|
Use :attr:`DataFrame.writeStream <pyspark.sql.DataFrame.writeStream>`
|
2016-06-29 01:07:11 -04:00
|
|
|
to access this.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
def __init__(self, df):
|
|
|
|
self._df = df
|
|
|
|
self._spark = df.sql_ctx
|
|
|
|
self._jwrite = df._jdf.writeStream()
|
|
|
|
|
|
|
|
def _sq(self, jsq):
|
|
|
|
from pyspark.sql.streaming import StreamingQuery
|
|
|
|
return StreamingQuery(jsq)
|
|
|
|
|
|
|
|
def outputMode(self, outputMode):
|
|
|
|
"""Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
2016-06-29 01:07:11 -04:00
|
|
|
Options include:
|
|
|
|
|
2020-05-18 07:25:02 -04:00
|
|
|
* `append`: Only the new rows in the streaming DataFrame/Dataset will be written to
|
2016-06-29 01:07:11 -04:00
|
|
|
the sink
|
2020-05-18 07:25:02 -04:00
|
|
|
* `complete`: All the rows in the streaming DataFrame/Dataset will be written to the sink
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
every time these are some updates
|
2020-05-18 07:25:02 -04:00
|
|
|
* `update`: only the rows that were updated in the streaming DataFrame/Dataset will be
|
2017-01-10 20:58:11 -05:00
|
|
|
written to the sink every time there are some updates. If the query doesn't contain
|
|
|
|
aggregations, it will be equivalent to `append` mode.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> writer = sdf.writeStream.outputMode('append')
|
|
|
|
"""
|
|
|
|
if not outputMode or type(outputMode) != str or len(outputMode.strip()) == 0:
|
|
|
|
raise ValueError('The output mode must be a non-empty string. Got: %s' % outputMode)
|
|
|
|
self._jwrite = self._jwrite.outputMode(outputMode)
|
|
|
|
return self
|
|
|
|
|
|
|
|
def format(self, source):
|
|
|
|
"""Specifies the underlying output data source.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
source : str
|
|
|
|
string, name of the data source, which for now can be 'parquet'.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> writer = sdf.writeStream.format('json')
|
|
|
|
"""
|
|
|
|
self._jwrite = self._jwrite.format(source)
|
|
|
|
return self
|
|
|
|
|
|
|
|
def option(self, key, value):
|
|
|
|
"""Adds an output option for the underlying data source.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
|
|
|
self._jwrite = self._jwrite.option(key, to_str(value))
|
|
|
|
return self
|
|
|
|
|
|
|
|
def options(self, **options):
|
|
|
|
"""Adds output options for the underlying data source.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
|
|
|
for k in options:
|
|
|
|
self._jwrite = self._jwrite.option(k, to_str(options[k]))
|
|
|
|
return self
|
|
|
|
|
|
|
|
def partitionBy(self, *cols):
|
|
|
|
"""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.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
cols : str or list
|
|
|
|
name of columns
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
|
|
|
if len(cols) == 1 and isinstance(cols[0], (list, tuple)):
|
|
|
|
cols = cols[0]
|
|
|
|
self._jwrite = self._jwrite.partitionBy(_to_seq(self._spark._sc, cols))
|
|
|
|
return self
|
|
|
|
|
|
|
|
def queryName(self, queryName):
|
|
|
|
"""Specifies the name of the :class:`StreamingQuery` that can be started with
|
|
|
|
:func:`start`. This name must be unique among all the currently active queries
|
|
|
|
in the associated SparkSession.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
queryName : str
|
|
|
|
unique name for the query
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> writer = sdf.writeStream.queryName('streaming_query')
|
|
|
|
"""
|
|
|
|
if not queryName or type(queryName) != str or len(queryName.strip()) == 0:
|
|
|
|
raise ValueError('The queryName must be a non-empty string. Got: %s' % queryName)
|
|
|
|
self._jwrite = self._jwrite.queryName(queryName)
|
|
|
|
return self
|
|
|
|
|
|
|
|
@keyword_only
|
[SPARK-32933][PYTHON] Use keyword-only syntax for keyword_only methods
### What changes were proposed in this pull request?
This PR adjusts signatures of methods decorated with `keyword_only` to indicate using [Python 3 keyword-only syntax](https://www.python.org/dev/peps/pep-3102/).
__Note__:
For the moment the goal is not to replace `keyword_only`. For justification see https://github.com/apache/spark/pull/29591#discussion_r489402579
### Why are the changes needed?
Right now it is not clear that `keyword_only` methods are indeed keyword only. This proposal addresses that.
In practice we could probably capture `locals` and drop `keyword_only` completel, i.e:
```python
keyword_only
def __init__(self, *, featuresCol="features"):
...
kwargs = self._input_kwargs
self.setParams(**kwargs)
```
could be replaced with
```python
def __init__(self, *, featuresCol="features"):
kwargs = locals()
del kwargs["self"]
...
self.setParams(**kwargs)
```
### Does this PR introduce _any_ user-facing change?
Docstrings and inspect tools will now indicate that `keyword_only` methods expect only keyword arguments.
For example with ` LinearSVC` will change from
```
>>> from pyspark.ml.classification import LinearSVC
>>> ?LinearSVC.__init__
Signature:
LinearSVC.__init__(
self,
featuresCol='features',
labelCol='label',
predictionCol='prediction',
maxIter=100,
regParam=0.0,
tol=1e-06,
rawPredictionCol='rawPrediction',
fitIntercept=True,
standardization=True,
threshold=0.0,
weightCol=None,
aggregationDepth=2,
)
Docstring: __init__(self, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, aggregationDepth=2):
File: /path/to/python/pyspark/ml/classification.py
Type: function
```
to
```
>>> from pyspark.ml.classification import LinearSVC
>>> ?LinearSVC.__init__
Signature:
LinearSVC.__init__ (
self,
*,
featuresCol='features',
labelCol='label',
predictionCol='prediction',
maxIter=100,
regParam=0.0,
tol=1e-06,
rawPredictionCol='rawPrediction',
fitIntercept=True,
standardization=True,
threshold=0.0,
weightCol=None,
aggregationDepth=2,
blockSize=1,
)
Docstring: __init__(self, \*, featuresCol="features", labelCol="label", predictionCol="prediction", maxIter=100, regParam=0.0, tol=1e-6, rawPredictionCol="rawPrediction", fitIntercept=True, standardization=True, threshold=0.0, weightCol=None, aggregationDepth=2, blockSize=1):
File: ~/Workspace/spark/python/pyspark/ml/classification.py
Type: function
```
### How was this patch tested?
Existing tests.
Closes #29799 from zero323/SPARK-32933.
Authored-by: zero323 <mszymkiewicz@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-09-22 20:28:33 -04:00
|
|
|
def trigger(self, *, processingTime=None, once=None, continuous=None):
|
2016-06-29 01:07:11 -04:00
|
|
|
"""Set the trigger for the stream query. If this is not set it will run the query as fast
|
|
|
|
as possible, which is equivalent to setting the trigger to ``processingTime='0 seconds'``.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
processingTime : str, optional
|
|
|
|
a processing time interval as a string, e.g. '5 seconds', '1 minute'.
|
|
|
|
Set a trigger that runs a microbatch query periodically based on the
|
|
|
|
processing time. Only one trigger can be set.
|
|
|
|
once : bool, optional
|
|
|
|
if set to True, set a trigger that processes only one batch of data in a
|
|
|
|
streaming query then terminates the query. Only one trigger can be set.
|
|
|
|
continuous : str, optional
|
|
|
|
a time interval as a string, e.g. '5 seconds', '1 minute'.
|
|
|
|
Set a trigger that runs a continuous query with a given checkpoint
|
|
|
|
interval. Only one trigger can be set.
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> # trigger the query for execution every 5 seconds
|
|
|
|
>>> writer = sdf.writeStream.trigger(processingTime='5 seconds')
|
[SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?
An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.
In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.
## How was this patch tested?
A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.
In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.
marmbrus tdas zsxwing
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
|
|
|
>>> # trigger the query for just once batch of data
|
|
|
|
>>> writer = sdf.writeStream.trigger(once=True)
|
2018-01-18 15:25:52 -05:00
|
|
|
>>> # trigger the query for execution every 5 seconds
|
|
|
|
>>> writer = sdf.writeStream.trigger(continuous='5 seconds')
|
2016-06-29 01:07:11 -04:00
|
|
|
"""
|
2018-01-18 15:25:52 -05:00
|
|
|
params = [processingTime, once, continuous]
|
|
|
|
|
|
|
|
if params.count(None) == 3:
|
|
|
|
raise ValueError('No trigger provided')
|
|
|
|
elif params.count(None) < 2:
|
|
|
|
raise ValueError('Multiple triggers not allowed.')
|
|
|
|
|
[SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?
An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.
In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.
## How was this patch tested?
A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.
In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.
marmbrus tdas zsxwing
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
|
|
|
jTrigger = None
|
2016-06-29 01:07:11 -04:00
|
|
|
if processingTime is not None:
|
|
|
|
if type(processingTime) != str or len(processingTime.strip()) == 0:
|
[SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?
An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.
In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.
## How was this patch tested?
A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.
In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.
marmbrus tdas zsxwing
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
|
|
|
raise ValueError('Value for processingTime must be a non empty string. Got: %s' %
|
2016-06-29 01:07:11 -04:00
|
|
|
processingTime)
|
[SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?
An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.
In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.
## How was this patch tested?
A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.
In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.
marmbrus tdas zsxwing
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
|
|
|
interval = processingTime.strip()
|
|
|
|
jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.ProcessingTime(
|
|
|
|
interval)
|
2018-01-18 15:25:52 -05:00
|
|
|
|
[SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?
An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.
In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.
## How was this patch tested?
A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.
In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.
marmbrus tdas zsxwing
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
|
|
|
elif once is not None:
|
|
|
|
if once is not True:
|
|
|
|
raise ValueError('Value for once must be True. Got: %s' % once)
|
|
|
|
jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Once()
|
2018-01-18 15:25:52 -05:00
|
|
|
|
[SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?
An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.
In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.
## How was this patch tested?
A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.
In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.
marmbrus tdas zsxwing
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
|
|
|
else:
|
2018-01-18 15:25:52 -05:00
|
|
|
if type(continuous) != str or len(continuous.strip()) == 0:
|
|
|
|
raise ValueError('Value for continuous must be a non empty string. Got: %s' %
|
|
|
|
continuous)
|
|
|
|
interval = continuous.strip()
|
|
|
|
jTrigger = self._spark._sc._jvm.org.apache.spark.sql.streaming.Trigger.Continuous(
|
|
|
|
interval)
|
|
|
|
|
[SPARK-19876][SS][WIP] OneTime Trigger Executor
## What changes were proposed in this pull request?
An additional trigger and trigger executor that will execute a single trigger only. One can use this OneTime trigger to have more control over the scheduling of triggers.
In addition, this patch requires an optimization to StreamExecution that logs a commit record at the end of successfully processing a batch. This new commit log will be used to determine the next batch (offsets) to process after a restart, instead of using the offset log itself to determine what batch to process next after restart; using the offset log to determine this would process the previously logged batch, always, thus not permitting a OneTime trigger feature.
## How was this patch tested?
A number of existing tests have been revised. These tests all assumed that when restarting a stream, the last batch in the offset log is to be re-processed. Given that we now have a commit log that will tell us if that last batch was processed successfully, the results/assumptions of those tests needed to be revised accordingly.
In addition, a OneTime trigger test was added to StreamingQuerySuite, which tests:
- The semantics of OneTime trigger (i.e., on start, execute a single batch, then stop).
- The case when the commit log was not able to successfully log the completion of a batch before restart, which would mean that we should fall back to what's in the offset log.
- A OneTime trigger execution that results in an exception being thrown.
marmbrus tdas zsxwing
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Tyson Condie <tcondie@gmail.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #17219 from tcondie/stream-commit.
2017-03-23 17:32:05 -04:00
|
|
|
self._jwrite = self._jwrite.trigger(jTrigger)
|
2016-06-29 01:07:11 -04:00
|
|
|
return self
|
|
|
|
|
2018-06-15 15:56:39 -04:00
|
|
|
def foreach(self, f):
|
|
|
|
"""
|
|
|
|
Sets the output of the streaming query to be processed using the provided writer ``f``.
|
|
|
|
This is often used to write the output of a streaming query to arbitrary storage systems.
|
|
|
|
The processing logic can be specified in two ways.
|
|
|
|
|
|
|
|
#. A **function** that takes a row as input.
|
|
|
|
This is a simple way to express your processing logic. Note that this does
|
|
|
|
not allow you to deduplicate generated data when failures cause reprocessing of
|
|
|
|
some input data. That would require you to specify the processing logic in the next
|
|
|
|
way.
|
|
|
|
|
|
|
|
#. An **object** with a ``process`` method and optional ``open`` and ``close`` methods.
|
|
|
|
The object can have the following methods.
|
|
|
|
|
|
|
|
* ``open(partition_id, epoch_id)``: *Optional* method that initializes the processing
|
|
|
|
(for example, open a connection, start a transaction, etc). Additionally, you can
|
|
|
|
use the `partition_id` and `epoch_id` to deduplicate regenerated data
|
|
|
|
(discussed later).
|
|
|
|
|
|
|
|
* ``process(row)``: *Non-optional* method that processes each :class:`Row`.
|
|
|
|
|
|
|
|
* ``close(error)``: *Optional* method that finalizes and cleans up (for example,
|
|
|
|
close connection, commit transaction, etc.) after all rows have been processed.
|
|
|
|
|
|
|
|
The object will be used by Spark in the following way.
|
|
|
|
|
|
|
|
* A single copy of this object is responsible of all the data generated by a
|
|
|
|
single task in a query. In other words, one instance is responsible for
|
|
|
|
processing one partition of the data generated in a distributed manner.
|
|
|
|
|
|
|
|
* This object must be serializable because each task will get a fresh
|
|
|
|
serialized-deserialized copy of the provided object. Hence, it is strongly
|
|
|
|
recommended that any initialization for writing data (e.g. opening a
|
|
|
|
connection or starting a transaction) is done after the `open(...)`
|
|
|
|
method has been called, which signifies that the task is ready to generate data.
|
|
|
|
|
|
|
|
* The lifecycle of the methods are as follows.
|
|
|
|
|
|
|
|
For each partition with ``partition_id``:
|
|
|
|
|
|
|
|
... For each batch/epoch of streaming data with ``epoch_id``:
|
|
|
|
|
|
|
|
....... Method ``open(partitionId, epochId)`` is called.
|
|
|
|
|
|
|
|
....... If ``open(...)`` returns true, for each row in the partition and
|
|
|
|
batch/epoch, method ``process(row)`` is called.
|
|
|
|
|
|
|
|
....... Method ``close(errorOrNull)`` is called with error (if any) seen while
|
|
|
|
processing rows.
|
|
|
|
|
|
|
|
Important points to note:
|
|
|
|
|
|
|
|
* The `partitionId` and `epochId` can be used to deduplicate generated data when
|
|
|
|
failures cause reprocessing of some input data. This depends on the execution
|
|
|
|
mode of the query. If the streaming query is being executed in the micro-batch
|
|
|
|
mode, then every partition represented by a unique tuple (partition_id, epoch_id)
|
|
|
|
is guaranteed to have the same data. Hence, (partition_id, epoch_id) can be used
|
|
|
|
to deduplicate and/or transactionally commit data and achieve exactly-once
|
|
|
|
guarantees. However, if the streaming query is being executed in the continuous
|
|
|
|
mode, then this guarantee does not hold and therefore should not be used for
|
|
|
|
deduplication.
|
|
|
|
|
|
|
|
* The ``close()`` method (if exists) will be called if `open()` method exists and
|
|
|
|
returns successfully (irrespective of the return value), except if the Python
|
|
|
|
crashes in the middle.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
2018-06-15 15:56:39 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2018-06-15 15:56:39 -04:00
|
|
|
>>> # Print every row using a function
|
|
|
|
>>> def print_row(row):
|
|
|
|
... print(row)
|
|
|
|
...
|
|
|
|
>>> writer = sdf.writeStream.foreach(print_row)
|
|
|
|
>>> # Print every row using a object with process() method
|
|
|
|
>>> class RowPrinter:
|
|
|
|
... def open(self, partition_id, epoch_id):
|
|
|
|
... print("Opened %d, %d" % (partition_id, epoch_id))
|
|
|
|
... return True
|
|
|
|
... def process(self, row):
|
|
|
|
... print(row)
|
|
|
|
... def close(self, error):
|
|
|
|
... print("Closed with error: %s" % str(error))
|
|
|
|
...
|
|
|
|
>>> writer = sdf.writeStream.foreach(RowPrinter())
|
|
|
|
"""
|
|
|
|
|
|
|
|
from pyspark.rdd import _wrap_function
|
|
|
|
from pyspark.serializers import PickleSerializer, AutoBatchedSerializer
|
|
|
|
from pyspark.taskcontext import TaskContext
|
|
|
|
|
|
|
|
if callable(f):
|
|
|
|
# The provided object is a callable function that is supposed to be called on each row.
|
|
|
|
# Construct a function that takes an iterator and calls the provided function on each
|
|
|
|
# row.
|
|
|
|
def func_without_process(_, iterator):
|
|
|
|
for x in iterator:
|
|
|
|
f(x)
|
|
|
|
return iter([])
|
|
|
|
|
|
|
|
func = func_without_process
|
|
|
|
|
|
|
|
else:
|
|
|
|
# The provided object is not a callable function. Then it is expected to have a
|
|
|
|
# 'process(row)' method, and optional 'open(partition_id, epoch_id)' and
|
|
|
|
# 'close(error)' methods.
|
|
|
|
|
|
|
|
if not hasattr(f, 'process'):
|
2021-05-25 22:54:40 -04:00
|
|
|
raise AttributeError("Provided object does not have a 'process' method")
|
2018-06-15 15:56:39 -04:00
|
|
|
|
|
|
|
if not callable(getattr(f, 'process')):
|
2021-05-25 22:54:40 -04:00
|
|
|
raise TypeError("Attribute 'process' in provided object is not callable")
|
2018-06-15 15:56:39 -04:00
|
|
|
|
|
|
|
def doesMethodExist(method_name):
|
|
|
|
exists = hasattr(f, method_name)
|
|
|
|
if exists and not callable(getattr(f, method_name)):
|
2021-05-25 22:54:40 -04:00
|
|
|
raise TypeError(
|
2018-06-15 15:56:39 -04:00
|
|
|
"Attribute '%s' in provided object is not callable" % method_name)
|
|
|
|
return exists
|
|
|
|
|
|
|
|
open_exists = doesMethodExist('open')
|
|
|
|
close_exists = doesMethodExist('close')
|
|
|
|
|
|
|
|
def func_with_open_process_close(partition_id, iterator):
|
|
|
|
epoch_id = TaskContext.get().getLocalProperty('streaming.sql.batchId')
|
|
|
|
if epoch_id:
|
|
|
|
epoch_id = int(epoch_id)
|
|
|
|
else:
|
2021-05-25 22:54:40 -04:00
|
|
|
raise RuntimeError("Could not get batch id from TaskContext")
|
2018-06-15 15:56:39 -04:00
|
|
|
|
|
|
|
# Check if the data should be processed
|
|
|
|
should_process = True
|
|
|
|
if open_exists:
|
|
|
|
should_process = f.open(partition_id, epoch_id)
|
|
|
|
|
|
|
|
error = None
|
|
|
|
|
|
|
|
try:
|
|
|
|
if should_process:
|
|
|
|
for x in iterator:
|
|
|
|
f.process(x)
|
|
|
|
except Exception as ex:
|
|
|
|
error = ex
|
|
|
|
finally:
|
|
|
|
if close_exists:
|
|
|
|
f.close(error)
|
|
|
|
if error:
|
|
|
|
raise error
|
|
|
|
|
|
|
|
return iter([])
|
|
|
|
|
|
|
|
func = func_with_open_process_close
|
|
|
|
|
|
|
|
serializer = AutoBatchedSerializer(PickleSerializer())
|
|
|
|
wrapped_func = _wrap_function(self._spark._sc, func, serializer, serializer)
|
|
|
|
jForeachWriter = \
|
|
|
|
self._spark._sc._jvm.org.apache.spark.sql.execution.python.PythonForeachWriter(
|
|
|
|
wrapped_func, self._df._jdf.schema())
|
|
|
|
self._jwrite.foreach(jForeachWriter)
|
|
|
|
return self
|
|
|
|
|
2018-06-19 16:56:51 -04:00
|
|
|
def foreachBatch(self, func):
|
|
|
|
"""
|
|
|
|
Sets the output of the streaming query to be processed using the provided
|
|
|
|
function. This is supported only the in the micro-batch execution modes (that is, when the
|
|
|
|
trigger is not continuous). In every micro-batch, the provided function will be called in
|
|
|
|
every micro-batch with (i) the output rows as a DataFrame and (ii) the batch identifier.
|
|
|
|
The batchId can be used deduplicate and transactionally write the output
|
|
|
|
(that is, the provided Dataset) to external systems. The output DataFrame is guaranteed
|
|
|
|
to exactly same for the same batchId (assuming all operations are deterministic in the
|
|
|
|
query).
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.4.0
|
2018-06-19 16:56:51 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2018-06-19 16:56:51 -04:00
|
|
|
>>> def func(batch_df, batch_id):
|
|
|
|
... batch_df.collect()
|
|
|
|
...
|
2019-11-01 14:55:29 -04:00
|
|
|
>>> writer = sdf.writeStream.foreachBatch(func)
|
2018-06-19 16:56:51 -04:00
|
|
|
"""
|
|
|
|
|
|
|
|
from pyspark.java_gateway import ensure_callback_server_started
|
|
|
|
gw = self._spark._sc._gateway
|
|
|
|
java_import(gw.jvm, "org.apache.spark.sql.execution.streaming.sources.*")
|
|
|
|
|
|
|
|
wrapped_func = ForeachBatchFunction(self._spark, func)
|
|
|
|
gw.jvm.PythonForeachBatchHelper.callForeachBatch(self._jwrite, wrapped_func)
|
|
|
|
ensure_callback_server_started(gw)
|
|
|
|
return self
|
|
|
|
|
2017-01-10 20:58:11 -05:00
|
|
|
def start(self, path=None, format=None, outputMode=None, partitionBy=None, queryName=None,
|
|
|
|
**options):
|
2016-06-29 01:07:11 -04:00
|
|
|
"""Streams the contents of the :class:`DataFrame` to a data source.
|
|
|
|
|
|
|
|
The data source is specified by the ``format`` and a set of ``options``.
|
|
|
|
If ``format`` is not specified, the default data source configured by
|
|
|
|
``spark.sql.sources.default`` will be used.
|
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
.. versionadded:: 2.0.0
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2020-11-02 20:00:49 -05:00
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
path : str, optional
|
|
|
|
the path in a Hadoop supported file system
|
|
|
|
format : str, optional
|
|
|
|
the format used to save
|
|
|
|
outputMode : str, optional
|
|
|
|
specifies how data of a streaming DataFrame/Dataset is written to a
|
|
|
|
streaming sink.
|
2017-01-10 20:58:11 -05:00
|
|
|
|
2020-05-18 07:25:02 -04:00
|
|
|
* `append`: Only the new rows in the streaming DataFrame/Dataset will be written to the
|
2017-01-10 20:58:11 -05:00
|
|
|
sink
|
2020-05-18 07:25:02 -04:00
|
|
|
* `complete`: All the rows in the streaming DataFrame/Dataset will be written to the
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
sink every time these are some updates
|
2020-05-18 07:25:02 -04:00
|
|
|
* `update`: only the rows that were updated in the streaming DataFrame/Dataset will be
|
2017-01-10 20:58:11 -05:00
|
|
|
written to the sink every time there are some updates. If the query doesn't contain
|
|
|
|
aggregations, it will be equivalent to `append` mode.
|
2020-11-02 20:00:49 -05:00
|
|
|
partitionBy : str or list, optional
|
|
|
|
names of partitioning columns
|
|
|
|
queryName : str, optional
|
|
|
|
unique name for the query
|
|
|
|
**options : dict
|
|
|
|
All other string options. You may want to provide a `checkpointLocation`
|
|
|
|
for most streams, however it is not required for a `memory` stream.
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
|
|
|
Examples
|
|
|
|
--------
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> sq = sdf.writeStream.format('memory').queryName('this_query').start()
|
|
|
|
>>> sq.isActive
|
|
|
|
True
|
|
|
|
>>> sq.name
|
2020-07-13 22:22:44 -04:00
|
|
|
'this_query'
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> sq.stop()
|
|
|
|
>>> sq.isActive
|
|
|
|
False
|
|
|
|
>>> sq = sdf.writeStream.trigger(processingTime='5 seconds').start(
|
2017-01-10 20:58:11 -05:00
|
|
|
... queryName='that_query', outputMode="append", format='memory')
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> sq.name
|
2020-07-13 22:22:44 -04:00
|
|
|
'that_query'
|
2016-06-29 01:07:11 -04:00
|
|
|
>>> sq.isActive
|
|
|
|
True
|
|
|
|
>>> sq.stop()
|
|
|
|
"""
|
|
|
|
self.options(**options)
|
2017-01-10 20:58:11 -05:00
|
|
|
if outputMode is not None:
|
|
|
|
self.outputMode(outputMode)
|
2016-06-29 01:07:11 -04:00
|
|
|
if partitionBy is not None:
|
|
|
|
self.partitionBy(partitionBy)
|
|
|
|
if format is not None:
|
|
|
|
self.format(format)
|
|
|
|
if queryName is not None:
|
|
|
|
self.queryName(queryName)
|
|
|
|
if path is None:
|
|
|
|
return self._sq(self._jwrite.start())
|
|
|
|
else:
|
|
|
|
return self._sq(self._jwrite.start(path))
|
|
|
|
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
def toTable(self, tableName, format=None, outputMode=None, partitionBy=None, queryName=None,
|
|
|
|
**options):
|
|
|
|
"""
|
|
|
|
Starts the execution of the streaming query, which will continually output results to the
|
|
|
|
given table as new data arrives.
|
|
|
|
|
2020-12-23 22:44:37 -05:00
|
|
|
The returned :class:`StreamingQuery` object can be used to interact with the stream.
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
|
|
|
|
.. versionadded:: 3.1.0
|
|
|
|
|
|
|
|
Parameters
|
|
|
|
----------
|
|
|
|
tableName : str
|
|
|
|
string, for the name of the table.
|
|
|
|
format : str, optional
|
|
|
|
the format used to save.
|
|
|
|
outputMode : str, optional
|
|
|
|
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 are some updates
|
|
|
|
* `update`: only the rows that were updated in the streaming DataFrame/Dataset will be
|
|
|
|
written to the sink every time there are some updates. If the query doesn't contain
|
|
|
|
aggregations, it will be equivalent to `append` mode.
|
|
|
|
partitionBy : str or list, optional
|
|
|
|
names of partitioning columns
|
|
|
|
queryName : str, optional
|
|
|
|
unique name for the query
|
|
|
|
**options : dict
|
|
|
|
All other string options. You may want to provide a `checkpointLocation`.
|
|
|
|
|
|
|
|
Notes
|
|
|
|
-----
|
|
|
|
This API is evolving.
|
|
|
|
|
2020-12-23 22:44:37 -05:00
|
|
|
For v1 table, partitioning columns provided by `partitionBy` will be respected no matter
|
|
|
|
the table exists or not. A new table will be created if the table not exists.
|
|
|
|
|
|
|
|
For v2 table, `partitionBy` will be ignored if the table already exists. `partitionBy` will
|
|
|
|
be respected only if the v2 table does not exist. Besides, the v2 table created by this API
|
|
|
|
lacks some functionalities (e.g., customized properties, options, and serde info). If you
|
|
|
|
need them, please create the v2 table manually before the execution to avoid creating a
|
|
|
|
table with incomplete information.
|
|
|
|
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
Examples
|
|
|
|
--------
|
2020-12-21 16:27:27 -05:00
|
|
|
>>> sdf.writeStream.format('parquet').queryName('query').toTable('output_table')
|
|
|
|
... # doctest: +SKIP
|
|
|
|
|
|
|
|
>>> sdf.writeStream.trigger(processingTime='5 seconds').toTable(
|
|
|
|
... 'output_table',
|
|
|
|
... queryName='that_query',
|
|
|
|
... outputMode="append",
|
|
|
|
... format='parquet',
|
[SPARK-33836][SS][PYTHON] Expose DataStreamReader.table and DataStreamWriter.toTable
### What changes were proposed in this pull request?
This PR proposes to expose `DataStreamReader.table` (SPARK-32885) and `DataStreamWriter.toTable` (SPARK-32896) to PySpark, which are the only way to read and write with table in Structured Streaming.
### Why are the changes needed?
Please refer SPARK-32885 and SPARK-32896 for rationalizations of these public APIs. This PR only exposes them to PySpark.
### Does this PR introduce _any_ user-facing change?
Yes, PySpark users will be able to read and write with table in Structured Streaming query.
### How was this patch tested?
Manually tested.
> v1 table
>> create table A and ingest to the table A
```
spark.sql("""
create table table_pyspark_parquet (
value long,
`timestamp` timestamp
) USING parquet
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.writeStream.toTable('table_pyspark_parquet', checkpointLocation='/tmp/checkpoint5')
query.lastProgress
query.stop()
```
>> read table A and ingest to the table B which doesn't exist
```
df2 = spark.readStream.table('table_pyspark_parquet')
query2 = df2.writeStream.toTable('table_pyspark_parquet_nonexist', format='parquet', checkpointLocation='/tmp/checkpoint2')
query2.lastProgress
query2.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE table_pyspark_parquet").show()
spark.sql("SELECT * FROM table_pyspark_parquet").show()
spark.sql("DESCRIBE TABLE table_pyspark_parquet_nonexist").show()
spark.sql("SELECT * FROM table_pyspark_parquet_nonexist").show()
```
> v2 table (leveraging Apache Iceberg as it provides V2 table and custom catalog as well)
>> create table A and ingest to the table A
```
spark.sql("""
create table iceberg_catalog.default.table_pyspark_v2table (
value long,
`timestamp` timestamp
) USING iceberg
""")
df = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query = df.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table', checkpointLocation='/tmp/checkpoint_v2table_1')
query.lastProgress
query.stop()
```
>> ingest to the non-exist table B
```
df2 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
query2 = df2.select('value', 'timestamp').writeStream.toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist', checkpointLocation='/tmp/checkpoint_v2table_2')
query2.lastProgress
query2.stop()
```
>> ingest to the non-exist table C partitioned by `value % 10`
```
df3 = spark.readStream.format('rate').option('rowsPerSecond', 100).load()
df3a = df3.selectExpr('value', 'timestamp', 'value % 10 AS partition').repartition('partition')
query3 = df3a.writeStream.partitionBy('partition').toTable('iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned', checkpointLocation='/tmp/checkpoint_v2table_3')
query3.lastProgress
query3.stop()
```
>> select tables
```
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist").show()
spark.sql("DESCRIBE TABLE iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
spark.sql("SELECT * FROM iceberg_catalog.default.table_pyspark_v2table_nonexist_partitioned").show()
```
Closes #30835 from HeartSaVioR/SPARK-33836.
Lead-authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-12-21 05:42:59 -05:00
|
|
|
... checkpointLocation='/tmp/checkpoint') # doctest: +SKIP
|
|
|
|
"""
|
|
|
|
self.options(**options)
|
|
|
|
if outputMode is not None:
|
|
|
|
self.outputMode(outputMode)
|
|
|
|
if partitionBy is not None:
|
|
|
|
self.partitionBy(partitionBy)
|
|
|
|
if format is not None:
|
|
|
|
self.format(format)
|
|
|
|
if queryName is not None:
|
|
|
|
self.queryName(queryName)
|
|
|
|
return self._sq(self._jwrite.toTable(tableName))
|
|
|
|
|
2016-06-29 01:07:11 -04:00
|
|
|
|
2016-04-28 18:22:28 -04:00
|
|
|
def _test():
|
|
|
|
import doctest
|
|
|
|
import os
|
|
|
|
import tempfile
|
2020-08-08 11:51:57 -04:00
|
|
|
from pyspark.sql import SparkSession, SQLContext
|
2016-06-14 05:12:29 -04:00
|
|
|
import pyspark.sql.streaming
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
os.chdir(os.environ["SPARK_HOME"])
|
|
|
|
|
2016-06-14 05:12:29 -04:00
|
|
|
globs = pyspark.sql.streaming.__dict__.copy()
|
|
|
|
try:
|
2016-06-14 20:58:45 -04:00
|
|
|
spark = SparkSession.builder.getOrCreate()
|
2020-08-30 22:23:31 -04:00
|
|
|
except py4j.protocol.Py4JError: # noqa: F821
|
|
|
|
spark = SparkSession(sc) # noqa: F821
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
globs['tempfile'] = tempfile
|
|
|
|
globs['os'] = os
|
2016-06-14 05:12:29 -04:00
|
|
|
globs['spark'] = spark
|
|
|
|
globs['sqlContext'] = SQLContext.getOrCreate(spark.sparkContext)
|
2016-06-29 01:07:11 -04:00
|
|
|
globs['sdf'] = \
|
|
|
|
spark.readStream.format('text').load('python/test_support/sql/streaming')
|
2019-08-09 05:54:55 -04:00
|
|
|
globs['sdf_schema'] = StructType([StructField("data", StringType(), True)])
|
2016-04-28 18:22:28 -04:00
|
|
|
globs['df'] = \
|
2016-06-14 20:58:45 -04:00
|
|
|
globs['spark'].readStream.format('text').load('python/test_support/sql/streaming')
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
(failure_count, test_count) = doctest.testmod(
|
2016-06-14 05:12:29 -04:00
|
|
|
pyspark.sql.streaming, globs=globs,
|
2016-04-28 18:22:28 -04:00
|
|
|
optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF)
|
2016-06-14 05:12:29 -04:00
|
|
|
globs['spark'].stop()
|
[SPARK-17731][SQL][STREAMING] Metrics for structured streaming
## What changes were proposed in this pull request?
Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics.
https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing
Specifically, this PR adds the following public APIs changes.
### New APIs
- `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later)
- `StreamingQueryStatus` has the following important fields
- inputRate - Current rate (rows/sec) at which data is being generated by all the sources
- processingRate - Current rate (rows/sec) at which the query is processing data from
all the sources
- ~~outputRate~~ - *Does not work with wholestage codegen*
- latency - Current average latency between the data being available in source and the sink writing the corresponding output
- sourceStatuses: Array[SourceStatus] - Current statuses of the sources
- sinkStatus: SinkStatus - Current status of the sink
- triggerStatus - Low-level detailed status of the last completed/currently active trigger
- latencies - getOffset, getBatch, full trigger, wal writes
- timestamps - trigger start, finish, after getOffset, after getBatch
- numRows - input, output, state total/updated rows for aggregations
- `SourceStatus` has the following important fields
- inputRate - Current rate (rows/sec) at which data is being generated by the source
- processingRate - Current rate (rows/sec) at which the query is processing data from the source
- triggerStatus - Low-level detailed status of the last completed/currently active trigger
- Python API for `StreamingQuery.status()`
### Breaking changes to existing APIs
**Existing direct public facing APIs**
- Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`.
- Branch 2.0 should have it deprecated, master should have it removed.
**Existing advanced listener APIs**
- `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus`
- Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status)
- Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`.
- Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`.
- For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java.
## How was this patch tested?
Old and new unit tests.
- Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite.
- New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite.
- New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite.
- Source-specific tests for making sure input rows are counted are is source-specific test suites.
- Additional tests to test minor additions in LocalTableScanExec, StateStore, etc.
Metrics also manually tested using Ganglia sink
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes #15307 from tdas/SPARK-17731.
2016-10-13 16:36:26 -04:00
|
|
|
|
2016-04-28 18:22:28 -04:00
|
|
|
if failure_count:
|
2018-03-08 06:38:34 -05:00
|
|
|
sys.exit(-1)
|
2016-04-28 18:22:28 -04:00
|
|
|
|
|
|
|
|
|
|
|
if __name__ == "__main__":
|
|
|
|
_test()
|