aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19670][SQL][TEST] Enable Bucketed Table Reading and Writing Testing ↵Xiao Li2017-02-214-21/+101
| | | | | | | | | | | | | | Without Hive Support ### What changes were proposed in this pull request? Bucketed table reading and writing does not need Hive support. We can move the test cases from `sql/hive` to `sql/core`. After this PR, we can improve the test case coverage. Bucket table reading and writing can be tested with and without Hive support. ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #17004 from gatorsmile/mvTestCaseForBuckets.
* [SPARK-19652][UI] Do auth checks for REST API access.Marcelo Vanzin2017-02-217-45/+123
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | The REST API has a security filter that performs auth checks based on the UI root's security manager. That works fine when the UI root is the app's UI, but not when it's the history server. In the SHS case, all users would be allowed to see all applications through the REST API, even if the UI itself wouldn't be available to them. This change adds auth checks for each app access through the API too, so that only authorized users can see the app's data. The change also modifies the existing security filter to use `HttpServletRequest.getRemoteUser()`, which is used in other places. That is not necessarily the same as the principal's name; for example, when using Hadoop's SPNEGO auth filter, the remote user strips the realm information, which then matches the user name registered as the owner of the application. I also renamed the UIRootFromServletContext trait to a more generic name since I'm using it to store more context information now. Tested manually with an authentication filter enabled. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16978 from vanzin/SPARK-19652.
* [SPARK-19626][YARN] Using the correct config to set credentials update timeKent Yao2017-02-211-1/+1
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In https://github.com/apache/spark/pull/14065, we introduced a configurable credential manager for Spark running on YARN. Also two configs `spark.yarn.credentials.renewalTime` and `spark.yarn.credentials.updateTime` were added, one is for the credential renewer and the other updater. But now we just query `spark.yarn.credentials.renewalTime` by mistake during CREDENTIALS UPDATING, where should be actually `spark.yarn.credentials.updateTime` . This PR fixes this mistake. ## How was this patch tested? existing test cc jerryshao vanzin Author: Kent Yao <yaooqinn@hotmail.com> Closes #16955 from yaooqinn/cred_update.
* [SPARK-19337][ML][DOC] Documentation and examples for LinearSVCYuhao Yang2017-02-214-0/+196
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Documentation and examples (Java, scala, python, R) for LinearSVC ## How was this patch tested? local doc generation Author: Yuhao Yang <yuhao.yang@intel.com> Closes #16968 from hhbyyh/mlsvmdoc.
* [SPARK-18922][TESTS] Fix new test failures on Windows due to path and ↵hyukjinkwon2017-02-205-28/+35
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | resource not closed ## What changes were proposed in this pull request? This PR proposes to fix new test failures on WIndows as below: **Before** ``` KafkaRelationSuite: - test late binding start offsets *** FAILED *** (7 seconds, 679 milliseconds) Cause: java.nio.file.FileSystemException: C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log -> C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log.deleted: The process cannot access the file because it is being used by another process. KafkaSourceSuite: - deserialization of initial offset with Spark 2.1.0 *** FAILED *** (3 seconds, 542 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-97ef64fc-ae61-4ce3-ac59-287fd38bd824 - deserialization of initial offset written by Spark 2.1.0 *** FAILED *** (60 milliseconds) java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/external/kafka-0-10-sql/target/scala-2.11/test-classes/kafka-source-initial-offset-version-2.1.0.b HiveDDLSuite: - partitioned table should always put partition columns at the end of table schema *** FAILED *** (657 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-f1b83d09-850a-4bba-8e43-a2a28dfaa757; DDLSuite: - create a data source table without schema *** FAILED *** (94 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-a3f3c161-afae-4d6f-9182-e8642f77062b; - SET LOCATION for managed table *** FAILED *** (219 milliseconds) org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: Exchange SinglePartit +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#99367L]) +- *FileScan parquet default.tbl[] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/C:projectsspark arget mpspark-15be2f2f-4ea9-4c47-bfee-1b7b49363033], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<> - insert data to a data source table which has a not existed location should succeed *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-34987671-e8d1-4624-ba5b-db1012e1246b; - insert into a data source table with no existed partition location should succeed *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4c6ccfbf-4091-4032-9fbc-3d40c58267d5; - read data from a data source table which has a not existed location should succeed *** FAILED *** (0 milliseconds) - read data from a data source table with no existed partition location should succeed *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-6af39e37-abd1-44e8-ac68-e2dfcf67a2f3; InputOutputMetricsSuite: - output metrics on records written *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-cd69ee77-88f2-4202-bed6-19c0ee05ef55\InputOutputMetricsSuite, expected: file:/// - output metrics on records written - new Hadoop API *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-b69e8fcb-047b-4de8-9cdf-5f026efb6762\InputOutputMetricsSuite, expected: file:/// ``` **After** ``` KafkaRelationSuite: - test late binding start offsets !!! CANCELED !!! (62 milliseconds) KafkaSourceSuite: - deserialization of initial offset with Spark 2.1.0 (5 seconds, 341 milliseconds) - deserialization of initial offset written by Spark 2.1.0 (910 milliseconds) HiveDDLSuite: - partitioned table should always put partition columns at the end of table schema (2 seconds) DDLSuite: - create a data source table without schema (828 milliseconds) - SET LOCATION for managed table (406 milliseconds) - insert data to a data source table which has a not existed location should succeed (406 milliseconds) - insert into a data source table with no existed partition location should succeed (453 milliseconds) - read data from a data source table which has a not existed location should succeed (94 milliseconds) - read data from a data source table with no existed partition location should succeed (265 milliseconds) InputOutputMetricsSuite: - output metrics on records written (172 milliseconds) - output metrics on records written - new Hadoop API (297 milliseconds) ``` ## How was this patch tested? Fixed tests in `InputOutputMetricsSuite`, `KafkaRelationSuite`, `KafkaSourceSuite`, `DDLSuite.scala` and `HiveDDLSuite`. Manually tested via AppVeyor as below: `InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/ex8nvwa6tsh7rmto `KafkaRelationSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/h8dlcowew52y8ncw `KafkaSourceSuite`: https://ci.appveyor.com/project/spark-test/spark/build/634-20170219-windows-test/job/9ybgjl7yeubxcre4 `DDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/635-20170219-windows-test `HiveDDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/up6o9n47er087ltb Author: hyukjinkwon <gurwls223@gmail.com> Closes #16999 from HyukjinKwon/windows-fix.
* [SPARK-19508][CORE] Improve error message when binding service failsLiang-Chi Hsieh2017-02-201-6/+21
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Utils provides a helper function to bind service on port. This function can bind the service to a random free port. However, if the binding fails on a random free port, the retrying and final exception messages look confusing. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. ... 17/02/06 16:25:43 ERROR SparkContext: Error initializing SparkContext. java.net.BindException: Can't assign requested address: Service 'sparkDriver' failed after 16 retries (starting from 0)! Consider explicitly setting the appropriate port for the service 'sparkDriver' (for example spark.ui.port for SparkUI) to an available port or increasing spark.port.maxRetries. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16851 from viirya/better-log-message.
* [SPARK-19669][HOTFIX][SQL] sessionState access privileges compiled failed in ↵windpiger2017-02-201-1/+1
| | | | | | | | | | | | | | | | | TestSQLContext ## What changes were proposed in this pull request? In [SPARK-19669](https://github.com/apache/spark/commit/0733a54a4517b82291efed9ac7f7407d9044593c) change the sessionState access privileges from private to public, this lead to the compile failed in TestSQLContext this pr is a hotfix for this. ## How was this patch tested? N/A Author: windpiger <songjun@outlook.com> Closes #17008 from windpiger/hotfixcompile.
* [SPARK-19669][SQL] Open up visibility for sharedState, sessionState, and a ↵Reynold Xin2017-02-203-8/+27
| | | | | | | | | | | | | | | | few other functions ## What changes were proposed in this pull request? To ease debugging, most of Spark SQL internals have public level visibility. Two of the most important internal states, sharedState and sessionState, however, are package private. It would make more sense to open these up as well with clear documentation that they are internal. In addition, users currently have way to set active/default SparkSession, but no way to actually get them back. We should open those up as well. ## How was this patch tested? N/A - only visibility change. Author: Reynold Xin <rxin@databricks.com> Closes #17002 from rxin/SPARK-19669.
* [SPARK-15453][SQL][FOLLOW-UP] FileSourceScanExec to extract `outputOrdering` ↵Xiao Li2017-02-201-92/+137
| | | | | | | | | | | | | | information ### What changes were proposed in this pull request? `outputOrdering` is also dependent on whether the bucket has more than one files. The test cases fail when we try to move them to sql/core. This PR is to fix the test cases introduced in https://github.com/apache/spark/pull/14864 and add a test case to verify [the related logics](https://github.com/tejasapatil/spark/blob/070c24994747c0479fb2520774ede27ff1cf8cac/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L197-L206). ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #16994 from gatorsmile/bucketingTS.
* [SPARK-19646][CORE][STREAMING] binaryRecords replicates records in scala APISean Owen2017-02-204-156/+53
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Use `BytesWritable.copyBytes`, not `getBytes`, because `getBytes` returns the underlying array, which may be reused when repeated reads don't need a different size, as is the case with binaryRecords APIs ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16974 from srowen/SPARK-19646.
* [SPARK-19563][SQL] avoid unnecessary sort in FileFormatWriterWenchen Fan2017-02-191-99/+90
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In `FileFormatWriter`, we will sort the input rows by partition columns and bucket id and sort columns, if we want to write data out partitioned or bucketed. However, if the data is already sorted, we will sort it again, which is unnecssary. This PR removes the sorting logic in `FileFormatWriter` and use `SortExec` instead. We will not add `SortExec` if the data is already sorted. ## How was this patch tested? I did a micro benchmark manually ``` val df = spark.range(10000000).select($"id", $"id" % 10 as "part").sort("part") spark.time(df.write.partitionBy("part").parquet("/tmp/test")) ``` The result was about 6.4 seconds before this PR, and is 5.7 seconds afterwards. close https://github.com/apache/spark/pull/16724 Author: Wenchen Fan <wenchen@databricks.com> Closes #16898 from cloud-fan/writer.
* [SPARK-19598][SQL] Remove the alias parameter in UnresolvedRelationwindpiger2017-02-1912-75/+51
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Remove the alias parameter in `UnresolvedRelation`, and use `SubqueryAlias` to replace it. This can simplify some `match case` situations. For example, the broadcast hint pull request can have one fewer case https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala#L57-L61 ## How was this patch tested? add some unit tests Author: windpiger <songjun@outlook.com> Closes #16956 from windpiger/removeUnresolveTableAlias.
* [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 featuresSean Owen2017-02-1945-1574/+662
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code. ## How was this patch tested? Jenkins tests Author: Sean Owen <sowen@cloudera.com> Closes #16964 from srowen/SPARK-19534.
* [SPARK-19533][EXAMPLES] Convert Java tests to use lambdas, Java 8 featuresSean Owen2017-02-1952-1018/+380
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Convert Java tests to use lambdas, Java 8 features. ## How was this patch tested? Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #16961 from srowen/SPARK-19533.
* [SPARK-19450] Replace askWithRetry with askSync.jinxing2017-02-1924-119/+58
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated. As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218): >askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it. Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`. ## How was this patch tested? This PR doesn't change code logic, existing unit test can cover. Author: jinxing <jinxing@meituan.com> Closes #16790 from jinxing64/SPARK-19450.
* [SPARK-19550][BUILD][WIP] Addendum: select Java 1.7 for scalac 2.10, stillSean Owen2017-02-191-1/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Go back to selecting source/target 1.7 for Scala 2.10 builds, because the SBT-based build for 2.10 won't work otherwise. ## How was this patch tested? Existing tests, but, we need to verify this vs what the SBT build would exactly run on Jenkins Author: Sean Owen <sowen@cloudera.com> Closes #16983 from srowen/SPARK-19550.3.
* [SPARK-19447] Make Range operator generate "recordsRead" metricAla Luszczak2017-02-186-155/+125
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The Range was modified to produce "recordsRead" metric instead of "generated rows". The tests were updated and partially moved to SQLMetricsSuite. ## How was this patch tested? Unit tests. Author: Ala Luszczak <ala@databricks.com> Closes #16960 from ala/range-records-read.
* [SPARK-19263] DAGScheduler should avoid sending conflicting task set.jinxing2017-02-183-3/+91
| | | | | | | | | | | | | | | | | | | | | | | | | In current `DAGScheduler handleTaskCompletion` code, when event.reason is `Success`, it will first do `stage.pendingPartitions -= task.partitionId`, which maybe a bug when `FetchFailed` happens. **Think about below** 1. Stage 0 runs and generates shuffle output data. 2. Stage 1 reads the output from stage 0 and generates more shuffle data. It has two tasks: ShuffleMapTask1 and ShuffleMapTask2, and these tasks are launched on executorA. 3. ShuffleMapTask1 fails to fetch blocks locally and sends a FetchFailed to the driver. The driver marks executorA as lost and updates failedEpoch; 4. The driver resubmits stage 0 so the missing output can be re-generated, and then once it completes, resubmits stage 1 with ShuffleMapTask1x and ShuffleMapTask2x. 5. ShuffleMapTask2 (from the original attempt of stage 1) successfully finishes on executorA and sends Success back to driver. This causes DAGScheduler::handleTaskCompletion to remove partition 2 from stage.pendingPartitions (line 1149), but it does not add the partition to the set of output locations (line 1192), because the task’s epoch is less than the failure epoch for the executor (because of the earlier failure on executor A) 6. ShuffleMapTask1x successfully finishes on executorB, causing the driver to remove partition 1 from stage.pendingPartitions. Combined with the previous step, this means that there are no more pending partitions for the stage, so the DAGScheduler marks the stage as finished (line 1196). However, the shuffle stage is not available (line 1215) because the completion for ShuffleMapTask2 was ignored because of its epoch, so the DAGScheduler resubmits the stage. 7. ShuffleMapTask2x is still running, so when TaskSchedulerImpl::submitTasks is called for the re-submitted stage, it throws an error, because there’s an existing active task set **In this fix** If a task completion is from a previous stage attempt and the epoch is too low (i.e., it was from a failed executor), don't remove the corresponding partition from pendingPartitions. Author: jinxing <jinxing@meituan.com> Author: jinxing <jinxing6042@126.com> Closes #16620 from jinxing64/SPARK-19263.
* [MLLIB][TYPO] Replace LeastSquaresAggregator with LogisticAggregatorMoussa Taifi2017-02-181-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Replace LeastSquaresAggregator with LogisticAggregator in the require statement of the merge op. ## How was this patch tested? Simple message fix. Author: Moussa Taifi <moutai10@gmail.com> Closes #16903 from moutai/master.
* [SPARK-19550] Follow-up: fixed a typo that fails the ↵Shuai Lin2017-02-181-1/+1
| | | | | | | | | | | | | | | | dev/make-distribution.sh script. ## What changes were proposed in this pull request? Fixed a typo in `dev/make-distribution.sh` script that sets the MAVEN_OPTS variable, introduced [here](https://github.com/apache/spark/commit/0e24054#diff-ba2c046d92a1d2b5b417788bfb5cb5f8R149). ## How was this patch tested? Run `dev/make-distribution.sh` manually. Author: Shuai Lin <linshuai2012@gmail.com> Closes #16984 from lins05/fix-spark-make-distribution-after-removing-java7.
* [SPARK-19639][SPARKR][EXAMPLE] Add spark.svmLinear example and update vignetteswm624@hotmail.com2017-02-173-0/+65
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We recently add the spark.svmLinear API for SparkR. We need to add an example and update the vignettes. ## How was this patch tested? Manually run example. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16969 from wangmiao1981/example.
* [SPARK-19617][SS] Fix the race condition when starting and stopping a query ↵Shixiong Zhu2017-02-178-107/+64
| | | | | | | | | | | | | | | | | | | | | | | | | | quickly ## What changes were proposed in this pull request? The streaming thread in StreamExecution uses the following ways to check if it should exit: - Catch an InterruptException. - `StreamExecution.state` is TERMINATED. When starting and stopping a query quickly, the above two checks may both fail: - Hit [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) and swallow InterruptException - StreamExecution.stop is called before `state` becomes `ACTIVE`. Then [runBatches](https://github.com/apache/spark/blob/dcc2d540a53f0bd04baead43fdee1c170ef2b9f3/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L252) changes the state from `TERMINATED` to `ACTIVE`. If the above cases both happen, the query will hang forever. This PR changes `state` to `AtomicReference` and uses`compareAndSet` to make sure we only change the state from `INITIALIZING` to `ACTIVE`. It also removes the `runUninterruptibly` hack from ``HDFSMetadata`, because HADOOP-14084 won't cause any problem after we fix the race condition. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16947 from zsxwing/SPARK-19617.
* [SPARKR][EXAMPLES] update examples to stop spark sessionFelix Cheung2017-02-1715-18/+46
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? stop session at end of example ## How was this patch tested? manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16973 from felixcheung/rexamples.
* [SPARK-18285][SPARKR] SparkR approxQuantile supports input multiple columnsYanbo Liang2017-02-173-14/+31
| | | | | | | | | | | | ## What changes were proposed in this pull request? SparkR ```approxQuantile``` supports input multiple columns. ## How was this patch tested? Unit test. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16951 from yanboliang/spark-19619.
* [SPARK-19517][SS] KafkaSource fails to initialize partition offsetsRoberto Agostino Vitillo2017-02-174-7/+131
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes a bug in `KafkaSource` with the (de)serialization of the length of the JSON string that contains the initial partition offsets. ## How was this patch tested? I ran the test suite for spark-sql-kafka-0-10. Author: Roberto Agostino Vitillo <ra.vitillo@gmail.com> Closes #16857 from vitillo/kafka_source_fix.
* [SPARK-18986][CORE] ExternalAppendOnlyMap shouldn't fail when forced to ↵Liang-Chi Hsieh2017-02-172-5/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | spill before calling its iterator ## What changes were proposed in this pull request? `ExternalAppendOnlyMap.forceSpill` now uses an assert to check if an iterator is not null in the map. However, the assertion is only true after the map is asked for iterator. Before it, if another memory consumer asks more memory than currently available, `ExternalAppendOnlyMap.forceSpill` is also be called too. In this case, we will see failure like this: [info] java.lang.AssertionError: assertion failed [info] at scala.Predef$.assert(Predef.scala:156) [info] at org.apache.spark.util.collection.ExternalAppendOnlyMap.forceSpill(ExternalAppendOnlyMap.scala:196) [info] at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:111) [info] at org.apache.spark.util.collection.ExternalAppendOnlyMapSuite$$anonfun$13.apply$mcV$sp(ExternalAppendOnlyMapSuite.scala:294) This fixing is motivated by http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-AssertionError-assertion-failed-tc20277.html. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16387 from viirya/fix-externalappendonlymap.
* [SPARK-19500] [SQL] Fix off-by-one bug in BytesToBytesMapDavies Liu2017-02-172-2/+43
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Radix sort require that half of array as free (as temporary space), so we use 0.5 as the scale factor to make sure that BytesToBytesMap will not have more items than 1/2 of capacity. Turned out this is not true, the current implementation of append() could leave 1 more item than the threshold (1/2 of capacity) in the array, which break the requirement of radix sort (fail the assert in 2.2, or fail to insert into InMemorySorter in 2.1). This PR fix the off-by-one bug in BytesToBytesMap. This PR also fix a bug that the array will never grow if it fail to grow once (stay as initial capacity), introduced by #15722 . ## How was this patch tested? Added regression test. Author: Davies Liu <davies@databricks.com> Closes #16844 from davies/off_by_one.
* [SPARK-19622][WEBUI] Fix a http error in a paged table when using a `Go` ↵Stan Zhai2017-02-171-2/+3
| | | | | | | | | | | | | | | | | | button to search. ## What changes were proposed in this pull request? The search function of paged table is not available because of we don't skip the hash data of the reqeust path. ![](https://issues.apache.org/jira/secure/attachment/12852996/screenshot-1.png) ## How was this patch tested? Tested manually with my browser. Author: Stan Zhai <zhaishidan@haizhi.com> Closes #16953 from stanzhai/fix-webui-paged-table.
* [MINOR][PYTHON] Fix typo docstring: 'top' -> 'topic'Rolando Espinoza2017-02-171-1/+1
| | | | | | | | | | ## What changes were proposed in this pull request? Fix typo in docstring. Author: Rolando Espinoza <rndmax84@gmail.com> Closes #16967 from rolando/pyspark-doc-typo.
* [BUILD] Close stale PRshyukjinkwon2017-02-170-0/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to close stale PRs. What I mean by "stale" here includes that there are some review comments by reviewers but the author looks inactive without any answer to them more than a month. I left some comments roughly a week ago to ping and the author looks still inactive in these PR below These below includes some PR suggested to be closed and a PR against another branch which seems obviously inappropriate. Given the comments in the last three PRs below, they are probably worth being taken over by anyone who is interested in it. Closes #7963 Closes #8374 Closes #11192 Closes #11374 Closes #11692 Closes #12243 Closes #12583 Closes #12620 Closes #12675 Closes #12697 Closes #12800 Closes #13715 Closes #14266 Closes #15053 Closes #15159 Closes #15209 Closes #15264 Closes #15267 Closes #15871 Closes #15861 Closes #16319 Closes #16324 Closes #16890 Closes #12398 Closes #12933 Closes #14517 ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #16937 from HyukjinKwon/stale-prs-close.
* [SPARK-18120][SPARK-19557][SQL] Call QueryExecutionListener callback methods ↵Wenchen Fan2017-02-163-16/+142
| | | | | | | | | | | | | | | | | | for DataFrameWriter methods ## What changes were proposed in this pull request? We only notify `QueryExecutionListener` for several `Dataset` operations, e.g. collect, take, etc. We should also do the notification for `DataFrameWriter` operations. ## How was this patch tested? new regression test close https://github.com/apache/spark/pull/16664 Author: Wenchen Fan <wenchen@databricks.com> Closes #16962 from cloud-fan/insert.
* [SPARK-18352][SQL] Support parsing multiline json filesNathan Howell2017-02-1617-231/+740
| | | | | | | | | | | | | | | | | | | | ## 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-19550][HOTFIX][BUILD] Use JAVA_HOME/bin/java if JAVA_HOME is set in ↵Sean Owen2017-02-161-1/+7
| | | | | | | | | | | | | | | | | dev/mima ## What changes were proposed in this pull request? Use JAVA_HOME/bin/java if JAVA_HOME is set in dev/mima script to run MiMa This follows on https://github.com/apache/spark/pull/16871 -- it's a slightly separate issue, but, is currently causing a build failure. ## How was this patch tested? Manually tested. Author: Sean Owen <sowen@cloudera.com> Closes #16957 from srowen/SPARK-19550.2.
* [SPARK-19436][SQL] Add missing tests for approxQuantileZheng RuiFeng2017-02-163-23/+88
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1, check the behavior with illegal `quantiles` and `relativeError` 2, add tests for `relativeError` > 1 3, update tests for `null` data 4, update some docs for javadoc8 ## How was this patch tested? local test in spark-shell Author: Zheng RuiFeng <ruifengz@foxmail.com> Author: Ruifeng Zheng <ruifengz@foxmail.com> Closes #16776 from zhengruifeng/fix_approxQuantile.
* [MINOR][BUILD] Fix javadoc8 breakhyukjinkwon2017-02-161-5/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? These error below seems caused by unidoc that does not understand double commented block. ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected [error] * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction = [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected [error] * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction = [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected [error] * new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected [error] * new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: illegal character: '#' [error] * &#64;Override [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: class, interface, or enum expected [error] * &#64;Override [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:76: error: class, interface, or enum expected [error] * boolean shouldRemove = ...; // Decide whether to remove the state [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:77: error: class, interface, or enum expected [error] * if (shouldRemove) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:79: error: class, interface, or enum expected [error] * } else { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:81: error: class, interface, or enum expected [error] * state.update(newState); // Set the new state [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:82: error: class, interface, or enum expected [error] * } [error] ^ [error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:85: error: class, interface, or enum expected [error] * state.update(initialState); [error] ^ [error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:86: error: class, interface, or enum expected [error] * } [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:90: error: class, interface, or enum expected [error] * </code></pre> [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:92: error: class, interface, or enum expected [error] * tparam S User-defined type of the state to be stored for each key. Must be encodable into [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:93: error: class, interface, or enum expected [error] * Spark SQL types (see {link Encoder} for more details). [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:94: error: class, interface, or enum expected [error] * since 2.1.1 [error] ^ ``` And another link seems unrecognisable. ``` .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:16: error: reference not found [error] * That is, in every batch of the {link streaming.StreamingQuery StreamingQuery}, [error] ``` Note that this PR does not fix the two breaks as below: ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:43: error: unexpected content [error] * see {link DataFrameStatsFunctions.approxQuantile(col:Str* approxQuantile} for [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:52: error: bad use of '>' [error] * param relativeError The relative target precision to achieve (>= 0). [error] ^ [error] ``` because these seem probably fixed soon in https://github.com/apache/spark/pull/16776 and I intended to avoid potential conflicts. ## How was this patch tested? Manually via `jekyll build` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16926 from HyukjinKwon/javadoc-break.
* [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 supportSean Owen2017-02-16101-1186/+513
| | | | | | | | | | | | | | | | | | | | | | | | - Move external/java8-tests tests into core, streaming, sql and remove - Remove MaxPermGen and related options - Fix some reflection / TODOs around Java 8+ methods - Update doc references to 1.7/1.8 differences - Remove Java 7/8 related build profiles - Update some plugins for better Java 8 compatibility - Fix a few Java-related warnings For the future: - Update Java 8 examples to fully use Java 8 - Update Java tests to use lambdas for simplicity - Update Java internal implementations to use lambdas ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16871 from srowen/SPARK-19493.
* [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 3rd batchKevin Yu2017-02-166-0/+1297
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is 3ird batch of test case for IN/NOT IN subquery. In this PR, it has these test files: `in-having.sql` `in-joins.sql` `in-multiple-columns.sql` These are the queries and results from running on DB2. [in-having DB2 version](https://github.com/apache/spark/files/772668/in-having.sql.db2.txt) [output of in-having](https://github.com/apache/spark/files/772670/in-having.sql.db2.out.txt) [in-joins DB2 version](https://github.com/apache/spark/files/772672/in-joins.sql.db2.txt) [output of in-joins](https://github.com/apache/spark/files/772673/in-joins.sql.db2.out.txt) [in-multiple-columns DB2 version](https://github.com/apache/spark/files/772678/in-multiple-columns.sql.db2.txt) [output of in-multiple-columns](https://github.com/apache/spark/files/772680/in-multiple-columns.sql.db2.out.txt) ## How was this patch tested? This pr is adding new test cases. We compare the result from spark with the result from another RDBMS(We used DB2 LUW). If the results are the same, we assume the result is correct. Author: Kevin Yu <qyu@us.ibm.com> Closes #16841 from kevinyu98/spark-18871-33.
* [SPARK-19618][SQL] Inconsistency wrt max. buckets allowed from Dataframe API ↵Tejas Patil2017-02-154-19/+25
| | | | | | | | | | | | | | | | | | vs SQL ## What changes were proposed in this pull request? Jira: https://issues.apache.org/jira/browse/SPARK-19618 Moved the check for validating number of buckets from `DataFrameWriter` to `BucketSpec` creation ## How was this patch tested? - Added more unit tests Author: Tejas Patil <tejasp@fb.com> Closes #16948 from tejasapatil/SPARK-19618_max_buckets.
* [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 4th batchKevin Yu2017-02-156-0/+2114
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is 4th batch of test case for IN/NOT IN subquery. In this PR, it has these test files: `in-set-operations.sql` `in-with-cte.sql` `not-in-joins.sql` Here are the queries and results from running on DB2. [in-set-operations DB2 version](https://github.com/apache/spark/files/772846/in-set-operations.sql.db2.txt) [Output of in-set-operations](https://github.com/apache/spark/files/772848/in-set-operations.sql.db2.out.txt) [in-with-cte DB2 version](https://github.com/apache/spark/files/772849/in-with-cte.sql.db2.txt) [Output of in-with-cte](https://github.com/apache/spark/files/772856/in-with-cte.sql.db2.out.txt) [not-in-joins DB2 version](https://github.com/apache/spark/files/772851/not-in-joins.sql.db2.txt) [Output of not-in-joins](https://github.com/apache/spark/files/772852/not-in-joins.sql.db2.out.txt) ## How was this patch tested? This pr is adding new test cases. We compare the result from spark with the result from another RDBMS(We used DB2 LUW). If the results are the same, we assume the result is correct. Author: Kevin Yu <qyu@us.ibm.com> Closes #16915 from kevinyu98/spark-18871-44.
* [SPARK-19603][SS] Fix StreamingQuery explain commandShixiong Zhu2017-02-153-11/+52
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `StreamingQuery.explain` doesn't show the correct streaming physical plan right now because `ExplainCommand` receives a runtime batch plan and its `logicalPlan.isStreaming` is always false. This PR adds `streaming` parameter to `ExplainCommand` to allow `StreamExecution` to specify that it's a streaming plan. Examples of the explain outputs: - streaming DataFrame.explain() ``` == Physical Plan == *HashAggregate(keys=[value#518], functions=[count(1)]) +- StateStoreSave [value#518], OperatorStateId(<unknown>,0,0), Append, 0 +- *HashAggregate(keys=[value#518], functions=[merge_count(1)]) +- StateStoreRestore [value#518], OperatorStateId(<unknown>,0,0) +- *HashAggregate(keys=[value#518], functions=[merge_count(1)]) +- Exchange hashpartitioning(value#518, 5) +- *HashAggregate(keys=[value#518], functions=[partial_count(1)]) +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518] +- *MapElements <function1>, obj#517: java.lang.String +- *DeserializeToObject value#513.toString, obj#516: java.lang.String +- StreamingRelation MemoryStream[value#513], [value#513] ``` - StreamingQuery.explain(extended = false) ``` == Physical Plan == *HashAggregate(keys=[value#518], functions=[count(1)]) +- StateStoreSave [value#518], OperatorStateId(...,0,0), Complete, 0 +- *HashAggregate(keys=[value#518], functions=[merge_count(1)]) +- StateStoreRestore [value#518], OperatorStateId(...,0,0) +- *HashAggregate(keys=[value#518], functions=[merge_count(1)]) +- Exchange hashpartitioning(value#518, 5) +- *HashAggregate(keys=[value#518], functions=[partial_count(1)]) +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518] +- *MapElements <function1>, obj#517: java.lang.String +- *DeserializeToObject value#543.toString, obj#516: java.lang.String +- LocalTableScan [value#543] ``` - StreamingQuery.explain(extended = true) ``` == Parsed Logical Plan == Aggregate [value#518], [value#518, count(1) AS count(1)#524L] +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518] +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String +- DeserializeToObject cast(value#543 as string).toString, obj#516: java.lang.String +- LocalRelation [value#543] == Analyzed Logical Plan == value: string, count(1): bigint Aggregate [value#518], [value#518, count(1) AS count(1)#524L] +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518] +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String +- DeserializeToObject cast(value#543 as string).toString, obj#516: java.lang.String +- LocalRelation [value#543] == Optimized Logical Plan == Aggregate [value#518], [value#518, count(1) AS count(1)#524L] +- SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518] +- MapElements <function1>, class java.lang.String, [StructField(value,StringType,true)], obj#517: java.lang.String +- DeserializeToObject value#543.toString, obj#516: java.lang.String +- LocalRelation [value#543] == Physical Plan == *HashAggregate(keys=[value#518], functions=[count(1)], output=[value#518, count(1)#524L]) +- StateStoreSave [value#518], OperatorStateId(...,0,0), Complete, 0 +- *HashAggregate(keys=[value#518], functions=[merge_count(1)], output=[value#518, count#530L]) +- StateStoreRestore [value#518], OperatorStateId(...,0,0) +- *HashAggregate(keys=[value#518], functions=[merge_count(1)], output=[value#518, count#530L]) +- Exchange hashpartitioning(value#518, 5) +- *HashAggregate(keys=[value#518], functions=[partial_count(1)], output=[value#518, count#530L]) +- *SerializeFromObject [staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, input[0, java.lang.String, true], true) AS value#518] +- *MapElements <function1>, obj#517: java.lang.String +- *DeserializeToObject value#543.toString, obj#516: java.lang.String +- LocalTableScan [value#543] ``` ## How was this patch tested? The updated unit test. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16934 from zsxwing/SPARK-19603.
* [SPARK-18080][ML][PYTHON] Python API & Examples for Locality Sensitive HashingYun Ni2017-02-159-53/+601
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request includes python API and examples for LSH. The API changes was based on yanboliang 's PR #15768 and resolved conflicts and API changes on the Scala API. The examples are consistent with Scala examples of MinHashLSH and BucketedRandomProjectionLSH. ## How was this patch tested? API and examples are tested using spark-submit: `bin/spark-submit examples/src/main/python/ml/min_hash_lsh.py` `bin/spark-submit examples/src/main/python/ml/bucketed_random_projection_lsh.py` User guide changes are generated and manually inspected: `SKIP_API=1 jekyll build` Author: Yun Ni <yunn@uber.com> Author: Yanbo Liang <ybliang8@gmail.com> Author: Yunni <Euler57721@gmail.com> Closes #16715 from Yunni/spark-18080.
* [SPARK-19599][SS] Clean up HDFSMetadataLogShixiong Zhu2017-02-152-24/+19
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-19464 removed support for Hadoop 2.5 and earlier, so we can do some cleanup for HDFSMetadataLog. This PR includes the following changes: - ~~Remove the workaround codes for HADOOP-10622.~~ Unfortunately, there is another issue [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) that prevents us from removing the workaround codes. - Remove unnecessary `writer: (T, OutputStream) => Unit` and just call `serialize` directly. - Remove catching FileNotFoundException. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16932 from zsxwing/metadata-cleanup.
* [SPARK-19604][TESTS] Log the start of every Python testYin Huai2017-02-151-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? Right now, we only have info level log after we finish the tests of a Python test file. We should also log the start of a test. So, if a test is hanging, we can tell which test file is running. ## How was this patch tested? This is a change for python tests. Author: Yin Huai <yhuai@databricks.com> Closes #16935 from yhuai/SPARK-19604.
* [SPARK-18937][SQL] Timezone support in CSV/JSON parsingTakuya UESHIN2017-02-1520-123/+351
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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-19329][SQL] Reading from or writing to a datasource table with a non ↵windpiger2017-02-152-1/+121
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | pre-existing location should succeed ## What changes were proposed in this pull request? when we insert data into a datasource table use `sqlText`, and the table has an not exists location, this will throw an Exception. example: ``` spark.sql("create table t(a string, b int) using parquet") spark.sql("alter table t set location '/xx'") spark.sql("insert into table t select 'c', 1") ``` Exception: ``` com.google.common.util.concurrent.UncheckedExecutionException: org.apache.spark.sql.AnalysisException: Path does not exist: /xx; at com.google.common.cache.LocalCache$LocalLoadingCache.getUnchecked(LocalCache.java:4814) at com.google.common.cache.LocalCache$LocalLoadingCache.apply(LocalCache.java:4830) at org.apache.spark.sql.hive.HiveMetastoreCatalog.lookupRelation(HiveMetastoreCatalog.scala:122) at org.apache.spark.sql.hive.HiveSessionCatalog.lookupRelation(HiveSessionCatalog.scala:69) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveRelations$$lookupTableFromCatalog(Analyzer.scala:456) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:465) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$$anonfun$apply$8.applyOrElse(Analyzer.scala:463) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveOperators$1.apply(LogicalPlan.scala:61) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolveOperators(LogicalPlan.scala:60) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:463) at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveRelations$.apply(Analyzer.scala:453) ``` As discussed following comments, we should unify the action when we reading from or writing to a datasource table with a non pre-existing locaiton: 1. reading from a datasource table: return 0 rows 2. writing to a datasource table: write data successfully ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16672 from windpiger/insertNotExistLocation.
* [SPARK-19607][HOTFIX] Finding QueryExecution that matches provided executionIdDongjoon Hyun2017-02-151-0/+2
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? #16940 adds a test case which does not stop the spark job. It causes many failures of other test cases. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/2403/consoleFull - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/2600/consoleFull ``` [info] org.apache.spark.SparkException: Only one SparkContext may be running in this JVM (see SPARK-2243). To ignore this error, set spark.driver.allowMultipleContexts = true. The currently running SparkContext was created at: ``` ## How was this patch tested? Pass the Jenkins test. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16943 from dongjoon-hyun/SPARK-19607-2.
* [SPARK-19331][SQL][TESTS] Improve the test coverage of SQLViewSuitejiangxingbo2017-02-155-302/+302
| | | | | | | | | | | | | | | | | Move `SQLViewSuite` from `sql/hive` to `sql/core`, so we can test the view supports without hive metastore. Also moved the test cases that specified to hive to `HiveSQLViewSuite`. Improve the test coverage of SQLViewSuite, cover the following cases: 1. view resolution(possibly a referenced table/view have changed after the view creation); 2. handle a view with user specified column names; 3. improve the test cases for a nested view. Also added a test case for cyclic view reference, which is a known issue that is not fixed yet. N/A Author: jiangxingbo <jiangxb1987@gmail.com> Closes #16674 from jiangxb1987/view-test.
* [SPARK-19399][SPARKR] Add R coalesce API for DataFrame and ColumnFelix Cheung2017-02-1511-18/+135
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add coalesce on DataFrame for down partitioning without shuffle and coalesce on Column ## How was this patch tested? manual, unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16739 from felixcheung/rcoalesce.
* [SPARK-19160][PYTHON][SQL] Add udf decoratorzero3232017-02-152-7/+91
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds `udf` decorator syntax as proposed in [SPARK-19160](https://issues.apache.org/jira/browse/SPARK-19160). This allows users to define UDF using simplified syntax: ```python from pyspark.sql.decorators import udf udf(IntegerType()) def add_one(x): """Adds one""" if x is not None: return x + 1 ``` without need to define a separate function and udf. ## How was this patch tested? Existing unit tests to ensure backward compatibility and additional unit tests covering new functionality. Author: zero323 <zero323@users.noreply.github.com> Closes #16533 from zero323/SPARK-19160.
* [SPARK-19590][PYSPARK][ML] Update the document for QuantileDiscretizer in ↵VinceShieh2017-02-151-1/+11
| | | | | | | | | | | | | | | | | pyspark ## What changes were proposed in this pull request? This PR is to document the changes on QuantileDiscretizer in pyspark for PR: https://github.com/apache/spark/pull/15428 ## How was this patch tested? No test needed Signed-off-by: VinceShieh <vincent.xieintel.com> Author: VinceShieh <vincent.xie@intel.com> Closes #16922 from VinceShieh/spark-19590.