aboutsummaryrefslogtreecommitdiff
path: root/python/pyspark/sql/streaming.py
Commit message (Collapse)AuthorAgeFilesLines
* [MINOR][DOCS] JSON APIs related documentation fixeshyukjinkwon2017-04-121-2/+2
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes corrections related to JSON APIs as below: - Rendering links in Python documentation - Replacing `RDD` to `Dataset` in programing guide - Adding missing description about JSON Lines consistently in `DataFrameReader.json` in Python API - De-duplicating little bit of `DataFrameReader.json` in Scala/Java API ## How was this patch tested? Manually build the documentation via `jekyll build`. Corresponding snapstops will be left on the codes. Note that currently there are Javadoc8 breaks in several places. These are proposed to be handled in https://github.com/apache/spark/pull/17477. So, this PR does not fix those. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17602 from HyukjinKwon/minor-json-documentation.
* [SPARK-20166][SQL] Use XXX for ISO 8601 timezone instead of ZZ ↵hyukjinkwon2017-04-031-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | (FastDateFormat specific) in CSV/JSON timeformat options ## What changes were proposed in this pull request? This PR proposes to use `XXX` format instead of `ZZ`. `ZZ` seems a `FastDateFormat` specific. `ZZ` supports "ISO 8601 extended format time zones" but it seems `FastDateFormat` specific option. I misunderstood this is compatible format with `SimpleDateFormat` when this change is introduced. Please see [SimpleDateFormat documentation]( https://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html#iso8601timezone) and [FastDateFormat documentation](https://commons.apache.org/proper/commons-lang/apidocs/org/apache/commons/lang3/time/FastDateFormat.html). It seems we better replace `ZZ` to `XXX` because they look using the same strategy - [FastDateParser.java#L930](https://github.com/apache/commons-lang/blob/8767cd4f1a6af07093c1e6c422dae8e574be7e5e/src/main/java/org/apache/commons/lang3/time/FastDateParser.java#L930), [FastDateParser.java#L932-L951 ](https://github.com/apache/commons-lang/blob/8767cd4f1a6af07093c1e6c422dae8e574be7e5e/src/main/java/org/apache/commons/lang3/time/FastDateParser.java#L932-L951) and [FastDateParser.java#L596-L601](https://github.com/apache/commons-lang/blob/8767cd4f1a6af07093c1e6c422dae8e574be7e5e/src/main/java/org/apache/commons/lang3/time/FastDateParser.java#L596-L601). I also checked the codes and manually debugged it for sure. It seems both cases use the same pattern `( Z|(?:[+-]\\d{2}(?::)\\d{2}))`. _Note that this should be rather a fix about documentation and not the behaviour change because `ZZ` seems invalid date format in `SimpleDateFormat` as documented in `DataFrameReader` and etc, and both `ZZ` and `XXX` look identically working with `FastDateFormat`_ Current documentation is as below: ``` * <li>`timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.</li> ``` ## How was this patch tested? Existing tests should cover this. Also, manually tested as below (BTW, I don't think these are worth being added as tests within Spark): **Parse** ```scala scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00") res4: java.util.Date = Tue Mar 21 20:00:00 KST 2017 scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z") res10: java.util.Date = Tue Mar 21 09:00:00 KST 2017 scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00") java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000-11:00" at java.text.DateFormat.parse(DateFormat.java:366) ... 48 elided scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z") java.text.ParseException: Unparseable date: "2017-03-21T00:00:00.000Z" at java.text.DateFormat.parse(DateFormat.java:366) ... 48 elided ``` ```scala scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00") res7: java.util.Date = Tue Mar 21 20:00:00 KST 2017 scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000Z") res1: java.util.Date = Tue Mar 21 09:00:00 KST 2017 scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000-11:00") res8: java.util.Date = Tue Mar 21 20:00:00 KST 2017 scala> org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ").parse("2017-03-21T00:00:00.000Z") res2: java.util.Date = Tue Mar 21 09:00:00 KST 2017 ``` **Format** ```scala scala> new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").format(new java.text.SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSXXX").parse("2017-03-21T00:00:00.000-11:00")) res6: String = 2017-03-21T20:00:00.000+09:00 ``` ```scala scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSZZ") fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSZZ,ko_KR,Asia/Seoul] scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00")) res1: String = 2017-03-21T20:00:00.000+09:00 scala> val fd = org.apache.commons.lang3.time.FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSSXXX") fd: org.apache.commons.lang3.time.FastDateFormat = FastDateFormat[yyyy-MM-dd'T'HH:mm:ss.SSSXXX,ko_KR,Asia/Seoul] scala> fd.format(fd.parse("2017-03-21T00:00:00.000-11:00")) res2: String = 2017-03-21T20:00:00.000+09:00 ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #17489 from HyukjinKwon/SPARK-20166.
* [SPARK-19876][SS][WIP] OneTime Trigger ExecutorTyson Condie2017-03-231-46/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-18579][SQL] Use ignoreLeadingWhiteSpace and ignoreTrailingWhiteSpace ↵hyukjinkwon2017-03-231-6/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | options in CSV writing ## What changes were proposed in this pull request? This PR proposes to support _not_ trimming the white spaces when writing out. These are `false` by default in CSV reading path but these are `true` by default in CSV writing in univocity parser. Both `ignoreLeadingWhiteSpace` and `ignoreTrailingWhiteSpace` options are not being used for writing and therefore, we are always trimming the white spaces. It seems we should provide a way to keep this white spaces easily. WIth the data below: ```scala val df = spark.read.csv(Seq("a , b , c").toDS) df.show() ``` ``` +---+----+---+ |_c0| _c1|_c2| +---+----+---+ | a | b | c| +---+----+---+ ``` **Before** ```scala df.write.csv("/tmp/text.csv") spark.read.text("/tmp/text.csv").show() ``` ``` +-----+ |value| +-----+ |a,b,c| +-----+ ``` It seems this can't be worked around via `quoteAll` too. ```scala df.write.option("quoteAll", true).csv("/tmp/text.csv") spark.read.text("/tmp/text.csv").show() ``` ``` +-----------+ | value| +-----------+ |"a","b","c"| +-----------+ ``` **After** ```scala df.write.option("ignoreLeadingWhiteSpace", false).option("ignoreTrailingWhiteSpace", false).csv("/tmp/text.csv") spark.read.text("/tmp/text.csv").show() ``` ``` +----------+ | value| +----------+ |a , b , c| +----------+ ``` Note that this case is possible in R ```r > system("cat text.csv") f1,f2,f3 a , b , c > df <- read.csv(file="text.csv") > df f1 f2 f3 1 a b c > write.csv(df, file="text1.csv", quote=F, row.names=F) > system("cat text1.csv") f1,f2,f3 a , b , c ``` ## How was this patch tested? Unit tests in `CSVSuite` and manual tests for Python. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17310 from HyukjinKwon/SPARK-18579.
* [SPARK-19949][SQL][FOLLOW-UP] Clean up parse modes and update related commentshyukjinkwon2017-03-221-0/+2
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to make `mode` options in both CSV and JSON to use `cass object` and fix some related comments related previous fix. Also, this PR modifies some tests related parse modes. ## How was this patch tested? Modified unit tests in both `CSVSuite.scala` and `JsonSuite.scala`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17377 from HyukjinKwon/SPARK-19949.
* [SPARK-19817][SS] Make it clear that `timeZone` is a general option in ↵Liwei Lin2017-03-141-8/+24
| | | | | | | | | | | | | | | | DataStreamReader/Writer ## What changes were proposed in this pull request? As timezone setting can also affect partition values, it works for all formats, we should make it clear. ## How was this patch tested? N/A Author: Liwei Lin <lwlin7@gmail.com> Closes #17299 from lw-lin/timezone.
* [SPARK-18352][DOCS] wholeFile JSON update doc and programming guideFelix Cheung2017-03-021-2/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Update doc for R, programming guide. Clarify default behavior for all languages. ## How was this patch tested? manually Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #17128 from felixcheung/jsonwholefiledoc.
* [SPARK-19610][SQL] Support parsing multiline CSV fileshyukjinkwon2017-02-281-2/+4
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes the support for multiple lines for CSV by resembling the multiline supports in JSON datasource (in case of JSON, per file). So, this PR introduces `wholeFile` option which makes the format not splittable and reads each whole file. Since Univocity parser can produces each row from a stream, it should be capable of parsing very large documents when the internal rows are fix in the memory. ## How was this patch tested? Unit tests in `CSVSuite` and `tests.py` Manual tests with a single 9GB CSV file in local file system, for example, ```scala spark.read.option("wholeFile", true).option("inferSchema", true).csv("tmp.csv").count() ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16976 from HyukjinKwon/SPARK-19610.
* [SPARK-18699][SQL] Put malformed tokens into a new field when parsing CSV dataTakeshi Yamamuro2017-02-231-8/+24
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pr added a logic to put malformed tokens into a new field when parsing CSV data in case of permissive modes. In the current master, if the CSV parser hits these malformed ones, it throws an exception below (and then a job fails); ``` Caused by: java.lang.IllegalArgumentException at java.sql.Date.valueOf(Date.java:143) at org.apache.spark.sql.catalyst.util.DateTimeUtils$.stringToTime(DateTimeUtils.scala:137) at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply$mcJ$sp(CSVInferSchema.scala:272) at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272) at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$$anonfun$castTo$6.apply(CSVInferSchema.scala:272) at scala.util.Try.getOrElse(Try.scala:79) at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:269) at ``` In case that users load large CSV-formatted data, the job failure makes users get some confused. So, this fix set NULL for original columns and put malformed tokens in a new field. ## How was this patch tested? Added tests in `CSVSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #16928 from maropu/SPARK-18699-2.
* [SPARK-18352][SQL] Support parsing multiline json filesNathan Howell2017-02-161-5/+9
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory. Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired. These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing. I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one. ## How was this patch tested? New and existing unit tests. No performance or load tests have been run. Author: Nathan Howell <nhowell@godaddy.com> Closes #16386 from NathanHowell/SPARK-18352.
* [SPARK-18937][SQL] Timezone support in CSV/JSON parsingTakuya UESHIN2017-02-151-8/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a follow-up pr of #16308. This pr enables timezone support in CSV/JSON parsing. We should introduce `timeZone` option for CSV/JSON datasources (the default value of the option is session local timezone). The datasources should use the `timeZone` option to format/parse to write/read timestamp values. Notice that while reading, if the timestampFormat has the timezone info, the timezone will not be used because we should respect the timezone in the values. For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values written with the default timezone option, which is `"GMT"` because session local timezone is `"GMT"` here, are: ```scala scala> spark.conf.set("spark.sql.session.timeZone", "GMT") scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts") df: org.apache.spark.sql.DataFrame = [ts: timestamp] scala> df.show() +-------------------+ |ts | +-------------------+ |2016-01-01 00:00:00| +-------------------+ scala> df.write.json("/path/to/gmtjson") ``` ```sh $ cat /path/to/gmtjson/part-* {"ts":"2016-01-01T00:00:00.000Z"} ``` whereas setting the option to `"PST"`, they are: ```scala scala> df.write.option("timeZone", "PST").json("/path/to/pstjson") ``` ```sh $ cat /path/to/pstjson/part-* {"ts":"2015-12-31T16:00:00.000-08:00"} ``` We can properly read these files even if the timezone option is wrong because the timestamp values have timezone info: ```scala scala> val schema = new StructType().add("ts", TimestampType) schema: org.apache.spark.sql.types.StructType = StructType(StructField(ts,TimestampType,true)) scala> spark.read.schema(schema).json("/path/to/gmtjson").show() +-------------------+ |ts | +-------------------+ |2016-01-01 00:00:00| +-------------------+ scala> spark.read.schema(schema).option("timeZone", "PST").json("/path/to/gmtjson").show() +-------------------+ |ts | +-------------------+ |2016-01-01 00:00:00| +-------------------+ ``` And even if `timezoneFormat` doesn't contain timezone info, we can properly read the values with setting correct timezone option: ```scala scala> df.write.option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson") ``` ```sh $ cat /path/to/jstjson/part-* {"ts":"2016-01-01T09:00:00"} ``` ```scala // wrong result scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").json("/path/to/jstjson").show() +-------------------+ |ts | +-------------------+ |2016-01-01 09:00:00| +-------------------+ // correct result scala> spark.read.schema(schema).option("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss").option("timeZone", "JST").json("/path/to/jstjson").show() +-------------------+ |ts | +-------------------+ |2016-01-01 00:00:00| +-------------------+ ``` This pr also makes `JsonToStruct` and `StructToJson` `TimeZoneAwareExpression` to be able to evaluate values with timezone option. ## How was this patch tested? Existing tests and added some tests. Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #16750 from ueshin/issues/SPARK-18937.
* [SPARK-19140][SS] Allow update mode for non-aggregation streaming queriesShixiong Zhu2017-01-101-8/+19
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR allow update mode for non-aggregation streaming queries. It will be same as the append mode if a query has no aggregations. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16520 from zsxwing/update-without-agg.
* [SPARK-18888] partitionBy in DataStreamWriter in Python throws _to_seq not ↵Burak Yavuz2016-12-151-0/+1
| | | | | | | | | | | | | | | | defined ## What changes were proposed in this pull request? `_to_seq` wasn't imported. ## How was this patch tested? Added partitionBy to existing write path unit test Author: Burak Yavuz <brkyvz@gmail.com> Closes #16297 from brkyvz/SPARK-18888.
* [SPARK-18852][SS] StreamingQuery.lastProgress should be null when ↵Shixiong Zhu2016-12-141-2/+7
| | | | | | | | | | | | | | | | | | recentProgress is empty ## What changes were proposed in this pull request? Right now `StreamingQuery.lastProgress` throws NoSuchElementException and it's hard to be used in Python since Python user will just see Py4jError. This PR just makes it return null instead. ## How was this patch tested? `test("lastProgress should be null when recentProgress is empty")` Author: Shixiong Zhu <shixiong@databricks.com> Closes #16273 from zsxwing/SPARK-18852.
* [SPARK-18754][SS] Rename recentProgresses to recentProgressMichael Armbrust2016-12-071-3/+3
| | | | | | | | Based on an informal survey, users find this option easier to understand / remember. Author: Michael Armbrust <michael@databricks.com> Closes #16182 from marmbrus/renameRecentProgress.
* [SPARK-18657][SPARK-18668] Make StreamingQuery.id persists across restart ↵Tathagata Das2016-12-051-2/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | and not auto-generate StreamingQuery.name ## What changes were proposed in this pull request? Here are the major changes in this PR. - Added the ability to recover `StreamingQuery.id` from checkpoint location, by writing the id to `checkpointLoc/metadata`. - Added `StreamingQuery.runId` which is unique for every query started and does not persist across restarts. This is to identify each restart of a query separately (same as earlier behavior of `id`). - Removed auto-generation of `StreamingQuery.name`. The purpose of name was to have the ability to define an identifier across restarts, but since id is precisely that, there is no need for a auto-generated name. This means name becomes purely cosmetic, and is null by default. - Added `runId` to `StreamingQueryListener` events and `StreamingQueryProgress`. Implementation details - Renamed existing `StreamExecutionMetadata` to `OffsetSeqMetadata`, and moved it to the file `OffsetSeq.scala`, because that is what this metadata is tied to. Also did some refactoring to make the code cleaner (got rid of a lot of `.json` and `.getOrElse("{}")`). - Added the `id` as the new `StreamMetadata`. - When a StreamingQuery is created it gets or writes the `StreamMetadata` from `checkpointLoc/metadata`. - All internal logging in `StreamExecution` uses `(name, id, runId)` instead of just `name` TODO - [x] Test handling of name=null in json generation of StreamingQueryProgress - [x] Test handling of name=null in json generation of StreamingQueryListener events - [x] Test python API of runId ## How was this patch tested? Updated unit tests and new unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16113 from tdas/SPARK-18657.
* [SPARK-18694][SS] Add StreamingQuery.explain and exception to Python and fix ↵Shixiong Zhu2016-12-051-0/+40
| | | | | | | | | | | | | | | | | StreamingQueryException ## What changes were proposed in this pull request? - Add StreamingQuery.explain and exception to Python. - Fix StreamingQueryException to not expose `OffsetSeq`. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16125 from zsxwing/py-streaming-explain.
* [SPARK-18516][STRUCTURED STREAMING] Follow up PR to add ↵Tathagata Das2016-11-291-0/+8
| | | | | | | | | | | | | | | | | StreamingQuery.status to Python ## What changes were proposed in this pull request? - Add StreamingQueryStatus.json - Make it not case class (to avoid unnecessarily exposing implicit object StreamingQueryStatus, consistent with StreamingQueryProgress) - Add StreamingQuery.status to Python - Fix post-termination status ## How was this patch tested? New unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16075 from tdas/SPARK-18516-1.
* [SPARK-18516][SQL] Split state and progress in streamingTathagata Das2016-11-291-304/+22
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-18447][DOCS] Fix the markdown for `Note:`/`NOTE:`/`Note that` across ↵hyukjinkwon2016-11-221-4/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Python API documentation ## What changes were proposed in this pull request? It seems in Python, there are - `Note:` - `NOTE:` - `Note that` - `.. note::` This PR proposes to fix those to `.. note::` to be consistent. **Before** <img width="567" alt="2016-11-21 1 18 49" src="https://cloud.githubusercontent.com/assets/6477701/20464305/85144c86-af88-11e6-8ee9-90f584dd856c.png"> <img width="617" alt="2016-11-21 12 42 43" src="https://cloud.githubusercontent.com/assets/6477701/20464263/27be5022-af88-11e6-8577-4bbca7cdf36c.png"> **After** <img width="554" alt="2016-11-21 1 18 42" src="https://cloud.githubusercontent.com/assets/6477701/20464306/8fe48932-af88-11e6-83e1-fc3cbf74407d.png"> <img width="628" alt="2016-11-21 12 42 51" src="https://cloud.githubusercontent.com/assets/6477701/20464264/2d3e156e-af88-11e6-93f3-cab8d8d02983.png"> ## How was this patch tested? The notes were found via ```bash grep -r "Note: " . grep -r "NOTE: " . grep -r "Note that " . ``` And then fixed one by one comparing with API documentation. After that, manually tested via `make html` under `./python/docs`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15947 from HyukjinKwon/SPARK-18447.
* [SPARK-18459][SPARK-18460][STRUCTUREDSTREAMING] Rename triggerId to batchId ↵Tathagata Das2016-11-161-3/+3
| | | | | | | | | | | | | | | | | and add triggerDetails to json in StreamingQueryStatus ## What changes were proposed in this pull request? SPARK-18459: triggerId seems like a number that should be increasing with each trigger, whether or not there is data in it. However, actually, triggerId increases only where there is a batch of data in a trigger. So its better to rename it to batchId. SPARK-18460: triggerDetails was missing from json representation. Fixed it. ## How was this patch tested? Updated existing unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15895 from tdas/SPARK-18459.
* [SPARK-17829][SQL] Stable format for offset logTyson Condie2016-11-091-6/+6
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we use java serialization for the WAL that stores the offsets contained in each batch. This has two main issues: It can break across spark releases (though this is not the only thing preventing us from upgrading a running query) It is unnecessarily opaque to the user. I'd propose we require offsets to provide a user readable serialization and use that instead. JSON is probably a good option. ## How was this patch tested? Tests were added for KafkaSourceOffset in [KafkaSourceOffsetSuite](external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceOffsetSuite.scala) and for LongOffset in [OffsetSuite](sql/core/src/test/scala/org/apache/spark/sql/streaming/OffsetSuite.scala) Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request. zsxwing marmbrus Author: Tyson Condie <tcondie@gmail.com> Author: Tyson Condie <tcondie@clash.local> Closes #15626 from tcondie/spark-8360.
* [SPARK-17764][SQL] Add `to_json` supporting to convert nested struct column ↵hyukjinkwon2016-11-011-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | to JSON string ## What changes were proposed in this pull request? This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python. It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function. The usage is as below: ``` scala val df = Seq(Tuple1(Tuple1(1))).toDF("a") df.select(to_json($"a").as("json")).show() ``` ``` bash +--------+ | json| +--------+ |{"_1":1}| +--------+ ``` ## How was this patch tested? Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15354 from HyukjinKwon/SPARK-17764.
* [SQL][DOC] updating doc for JSON source to link to jsonlines.orgFelix Cheung2016-10-261-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? API and programming guide doc changes for Scala, Python and R. ## How was this patch tested? manual test Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #15629 from felixcheung/jsondoc.
* [SPARK-17926][SQL][STREAMING] Added json for statusesTathagata Das2016-10-211-6/+5
| | | | | | | | | | | | | ## What changes were proposed in this pull request? StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`. ## How was this patch tested? New unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15476 from tdas/SPARK-17926.
* [SPARK-17731][SQL][STREAMING] Metrics for structured streamingTathagata Das2016-10-131-0/+301
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [MINOR][PYSPARK][DOCS] Fix examples in PySpark documentationhyukjinkwon2016-09-281-3/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [SPARK-17583][SQL] Remove uesless rowSeparator variable and set ↵hyukjinkwon2016-09-211-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | auto-expanding buffer as default for maxCharsPerColumn option in CSV ## What changes were proposed in this pull request? This PR includes the changes below: 1. Upgrade Univocity library from 2.1.1 to 2.2.1 This includes some performance improvement and also enabling auto-extending buffer in `maxCharsPerColumn` option in CSV. Please refer the [release notes](https://github.com/uniVocity/univocity-parsers/releases). 2. Remove useless `rowSeparator` variable existing in `CSVOptions` We have this unused variable in [CSVOptions.scala#L127](https://github.com/apache/spark/blob/29952ed096fd2a0a19079933ff691671d6f00835/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala#L127) but it seems possibly causing confusion that it actually does not care of `\r\n`. For example, we have an issue open about this, [SPARK-17227](https://issues.apache.org/jira/browse/SPARK-17227), describing this variable. This variable is virtually not being used because we rely on `LineRecordReader` in Hadoop which deals with only both `\n` and `\r\n`. 3. Set the default value of `maxCharsPerColumn` to auto-expending. We are setting 1000000 for the length of each column. It'd be more sensible we allow auto-expending rather than fixed length by default. To make sure, using `-1` is being described in the release note, [2.2.0](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.2.0). ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #15138 from HyukjinKwon/SPARK-17583.
* [SPARK-16462][SPARK-16460][SPARK-15144][SQL] Make CSV cast null values properlyLiwei Lin2016-09-181-1/+2
| | | | | | | | | | | | | | | | | | | | ## Problem CSV in Spark 2.0.0: - does not read null values back correctly for certain data types such as `Boolean`, `TimestampType`, `DateType` -- this is a regression comparing to 1.6; - does not read empty values (specified by `options.nullValue`) as `null`s for `StringType` -- this is compatible with 1.6 but leads to problems like SPARK-16903. ## What changes were proposed in this pull request? This patch makes changes to read all empty values back as `null`s. ## How was this patch tested? New test cases. Author: Liwei Lin <lwlin7@gmail.com> Closes #14118 from lw-lin/csv-cast-null.
* [SPARK-17264][SQL] DataStreamWriter should document that it only supports ↵Sean Owen2016-08-301-1/+1
| | | | | | | | | | | | | | | | Parquet for now ## What changes were proposed in this pull request? Clarify that only parquet files are supported by DataStreamWriter now ## How was this patch tested? (Doc build -- no functional changes to test) Author: Sean Owen <sowen@cloudera.com> Closes #14860 from srowen/SPARK-17264.
* [SPARK-17215][SQL] Method `SQLContext.parseDataType(dataTypeString: String)` ↵jiangxingbo2016-08-241-1/+3
| | | | | | | | | | | | | | | | | could be removed. ## What changes were proposed in this pull request? Method `SQLContext.parseDataType(dataTypeString: String)` could be removed, we should use `SparkSession.parseDataType(dataTypeString: String)` instead. This require updating PySpark. ## How was this patch tested? Existing test cases. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #14790 from jiangxb1987/parseDataType.
* [SPARK-16216][SQL] Read/write timestamps and dates in ISO 8601 and ↵hyukjinkwon2016-08-241-8/+22
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | dateFormat/timestampFormat option for CSV and JSON ## What changes were proposed in this pull request? ### Default - ISO 8601 Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below: - CSV ``` // TimestampType 1414459800000000 // DateType 16673 ``` - Json ``` // TimestampType 1970-01-01 11:46:40.0 // DateType 1970-01-01 ``` So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed. So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)). - For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`) ``` 1970-01-01T02:00:01.000-01:00 ``` - For `Date` it becomes as below (`yyyy-MM-dd`) ``` 1970-01-01 ``` ### Custom date format option - `dateFormat` This PR also adds the support to write and read dates and timestamps in a formatted string as below: - **DateType** - With `dateFormat` option (e.g. `yyyy/MM/dd`) ``` +----------+ | date| +----------+ |2015/08/26| |2014/10/27| |2016/01/28| +----------+ ``` ### Custom date format option - `timestampFormat` - **TimestampType** - With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`) ``` +----------------+ | date| +----------------+ |2015/08/26 18:00| |2014/10/27 18:30| |2016/01/28 20:00| +----------------+ ``` ## How was this patch tested? Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14279 from HyukjinKwon/SPARK-16216-json-csv.
* [SPARK-16772] Correct API doc references to PySpark classes + formatting fixesNicholas Chammas2016-07-281-4/+4
| | | | | | | | | | | | | | | | | | ## What's Been Changed The PR corrects several broken or missing class references in the Python API docs. It also correct formatting problems. For example, you can see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.registerFunction) how Sphinx is not picking up the reference to `DataType`. That's because the reference is relative to the current module, whereas `DataType` is in a different module. You can also see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.createDataFrame) how the formatting for byte, tinyint, and so on is italic instead of monospace. That's because in ReST single backticks just make things italic, unlike in Markdown. ## Testing I tested this PR by [building the Python docs](https://github.com/apache/spark/tree/master/docs#generating-the-documentation-html) and reviewing the results locally in my browser. I confirmed that the broken or missing class references were resolved, and that the formatting was corrected. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #14393 from nchammas/python-docstring-fixes.
* [SPARK-16335][SQL] Structured streaming should fail if source directory does ↵Reynold Xin2016-07-011-7/+4
| | | | | | | | | | | | | | not exist ## What changes were proposed in this pull request? In structured streaming, Spark does not report errors when the specified directory does not exist. This is a behavior different from the batch mode. This patch changes the behavior to fail if the directory does not exist (when the path is not a glob pattern). ## How was this patch tested? Updated unit tests to reflect the new behavior. Author: Reynold Xin <rxin@databricks.com> Closes #14002 from rxin/SPARK-16335.
* [SPARK-16313][SQL] Spark should not silently drop exceptions in file listingReynold Xin2016-06-301-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? Spark silently drops exceptions during file listing. This is a very bad behavior because it can mask legitimate errors and the resulting plan will silently have 0 rows. This patch changes it to not silently drop the errors. ## How was this patch tested? Manually verified. Author: Reynold Xin <rxin@databricks.com> Closes #13987 from rxin/SPARK-16313.
* [SPARK-16266][SQL][STREAING] Moved DataStreamReader/Writer from pyspark.sql ↵Tathagata Das2016-06-281-4/+498
| | | | | | | | | | | | | | | | | | to pyspark.sql.streaming ## What changes were proposed in this pull request? - Moved DataStreamReader/Writer from pyspark.sql to pyspark.sql.streaming to make them consistent with scala packaging - Exposed the necessary classes in sql.streaming package so that they appear in the docs - Added pyspark.sql.streaming module to the docs ## How was this patch tested? - updated unit tests. - generated docs for testing visibility of pyspark.sql.streaming classes. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13955 from tdas/SPARK-16266.
* [SPARK-15953][WIP][STREAMING] Renamed ContinuousQuery to StreamingQueryTathagata Das2016-06-151-40/+39
| | | | | | | | | | Renamed for simplicity, so that its obvious that its related to streaming. Existing unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13673 from tdas/SPARK-15953.
* [SPARK-15935][PYSPARK] Fix a wrong format tag in the error messageShixiong Zhu2016-06-141-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? A follow up PR for #13655 to fix a wrong format tag. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13665 from zsxwing/fix.
* [SPARK-15933][SQL][STREAMING] Refactored DF reader-writer to use readStream ↵Tathagata Das2016-06-141-4/+4
| | | | | | | | | | | | | | | | and writeStream for streaming DFs ## What changes were proposed in this pull request? Currently, the DataFrameReader/Writer has method that are needed for streaming and non-streaming DFs. This is quite awkward because each method in them through runtime exception for one case or the other. So rather having half the methods throw runtime exceptions, its just better to have a different reader/writer API for streams. - [x] Python API!! ## How was this patch tested? Existing unit tests + two sets of unit tests for DataFrameReader/Writer and DataStreamReader/Writer. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #13653 from tdas/SPARK-15933.
* [SPARK-15935][PYSPARK] Enable test for sql/streaming.py and fix these testsShixiong Zhu2016-06-141-20/+41
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR just enables tests for sql/streaming.py and also fixes the failures. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #13655 from zsxwing/python-streaming-test.
* [MINOR] Fix Typos 'an -> a'Zheng RuiFeng2016-06-061-1/+1
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `an -> a` Use cmds like `find . -name '*.R' | xargs -i sh -c "grep -in ' an [^aeiou]' {} && echo {}"` to generate candidates, and review them one by one. ## How was this patch tested? manual tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #13515 from zhengruifeng/an_a.
* [SPARK-15686][SQL] Move user-facing streaming classes into sql.streamingReynold Xin2016-06-011-1/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch moves all user-facing structured streaming classes into sql.streaming. As part of this, I also added some since version annotation to methods and classes that don't have them. ## How was this patch tested? Updated tests to reflect the moves. Author: Reynold Xin <rxin@databricks.com> Closes #13429 from rxin/SPARK-15686.
* [SPARK-14896][SQL] Deprecate HiveContext in pythonAndrew Or2016-05-041-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? See title. ## How was this patch tested? PySpark tests. Author: Andrew Or <andrew@databricks.com> Closes #12917 from andrewor14/deprecate-hive-context-python.
* [SPARK-14555] Second cut of Python API for Structured StreamingBurak Yavuz2016-04-281-11/+124
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds Python APIs for: - `ContinuousQueryManager` - `ContinuousQueryException` The `ContinuousQueryException` is a very basic wrapper, it doesn't provide the functionality that the Scala side provides, but it follows the same pattern for `AnalysisException`. For `ContinuousQueryManager`, all APIs are provided except for registering listeners. This PR also attempts to fix test flakiness by stopping all active streams just before tests. ## How was this patch tested? Python Doc tests and unit tests Author: Burak Yavuz <brkyvz@gmail.com> Closes #12673 from brkyvz/pyspark-cqm.
* [SPARK-14555] First cut of Python API for Structured StreamingBurak Yavuz2016-04-201-0/+124
## What changes were proposed in this pull request? This patch provides a first cut of python APIs for structured streaming. This PR provides the new classes: - ContinuousQuery - Trigger - ProcessingTime in pyspark under `pyspark.sql.streaming`. In addition, it contains the new methods added under: - `DataFrameWriter` a) `startStream` b) `trigger` c) `queryName` - `DataFrameReader` a) `stream` - `DataFrame` a) `isStreaming` This PR doesn't contain all methods exposed for `ContinuousQuery`, for example: - `exception` - `sourceStatuses` - `sinkStatus` They may be added in a follow up. This PR also contains some very minor doc fixes in the Scala side. ## How was this patch tested? Python doc tests TODO: - [ ] verify Python docs look good Author: Burak Yavuz <brkyvz@gmail.com> Author: Burak Yavuz <burak@databricks.com> Closes #12320 from brkyvz/stream-python.