aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-16552][SQL] Store the Inferred Schemas into External Catalog Tables ↵gatorsmile2016-07-287-79/+291
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | when Creating Tables #### What changes were proposed in this pull request? Currently, in Spark SQL, the initial creation of schema can be classified into two groups. It is applicable to both Hive tables and Data Source tables: **Group A. Users specify the schema.** _Case 1 CREATE TABLE AS SELECT_: the schema is determined by the result schema of the SELECT clause. For example, ```SQL CREATE TABLE tab STORED AS TEXTFILE AS SELECT * from input ``` _Case 2 CREATE TABLE_: users explicitly specify the schema. For example, ```SQL CREATE TABLE jsonTable (_1 string, _2 string) USING org.apache.spark.sql.json ``` **Group B. Spark SQL infers the schema at runtime.** _Case 3 CREATE TABLE_. Users do not specify the schema but the path to the file location. For example, ```SQL CREATE TABLE jsonTable USING org.apache.spark.sql.json OPTIONS (path '${tempDir.getCanonicalPath}') ``` Before this PR, Spark SQL does not store the inferred schema in the external catalog for the cases in Group B. When users refreshing the metadata cache, accessing the table at the first time after (re-)starting Spark, Spark SQL will infer the schema and store the info in the metadata cache for improving the performance of subsequent metadata requests. However, the runtime schema inference could cause undesirable schema changes after each reboot of Spark. This PR is to store the inferred schema in the external catalog when creating the table. When users intend to refresh the schema after possible changes on external files (table location), they issue `REFRESH TABLE`. Spark SQL will infer the schema again based on the previously specified table location and update/refresh the schema in the external catalog and metadata cache. In this PR, we do not use the inferred schema to replace the user specified schema for avoiding external behavior changes . Based on the design, user-specified schemas (as described in Group A) can be changed by ALTER TABLE commands, although we do not support them now. #### How was this patch tested? TODO: add more cases to cover the changes. Author: gatorsmile <gatorsmile@gmail.com> Closes #14207 from gatorsmile/userSpecifiedSchema.
* [SPARK-15232][SQL] Add subquery SQL building tests to LogicalPlanToSQLSuiteDongjoon Hyun2016-07-2715-0/+282
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We currently test subquery SQL building using the `HiveCompatibilitySuite`. The is not desired since SQL building is actually a part of `sql/core` and because we are slowly reducing our dependency on Hive. This PR adds the same tests from the whitelist of `HiveCompatibilitySuite` into `LogicalPlanToSQLSuite`. ## How was this patch tested? This adds more testcases. Pass the Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14383 from dongjoon-hyun/SPARK-15232.
* [SPARK-16730][SQL] Implement function aliases for type castspetermaxlee2016-07-283-7/+73
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark 1.x supports using the Hive type name as function names for doing casts, e.g. ```sql SELECT int(1.0); SELECT string(2.0); ``` The above query would work in Spark 1.x because Spark 1.x fail back to Hive for unimplemented functions, and break in Spark 2.0 because the fall back was removed. This patch implements function aliases using an analyzer rule for the following cast functions: - boolean - tinyint - smallint - int - bigint - float - double - decimal - date - timestamp - binary - string ## How was this patch tested? Added end-to-end tests in SQLCompatibilityFunctionSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14364 from petermaxlee/SPARK-16730-2.
* [SPARK-16110][YARN][PYSPARK] Fix allowing python version to be specified per ↵KevinGrealish2016-07-272-6/+27
| | | | | | | | | | | | | | | | | submit for cluster mode. ## What changes were proposed in this pull request? This fix allows submit of pyspark jobs to specify python 2 or 3. Change ordering in setup for application master environment so env vars PYSPARK_PYTHON and PYSPARK_DRIVER_PYTHON can be overridden by spark.yarn.appMasterEnv.* conf settings. This applies to YARN in cluster mode. This allows them to be set per submission without needing the unset the env vars (which is not always possible - e.g. batch submit with LIVY only exposes the arguments to spark-submit) ## How was this patch tested? Manual and existing unit tests. Author: KevinGrealish <KevinGre@microsoft.com> Closes #13824 from KevinGrealish/SPARK-16110.
* [MINOR][DOC] missing keyword newBartek Wiśniewski2016-07-271-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? added missing keyword for java example ## How was this patch tested? wasn't Author: Bartek Wiśniewski <wedi@Ava.local> Closes #14381 from wedi-dev/quickfix/missing_keyword.
* [SPARK-5847][CORE] Allow for configuring MetricsSystem's use of app ID to ↵Mark Grover2016-07-276-26/+170
| | | | | | | | | | | | | | | | | | | | | | | | | | namespace all metrics ## What changes were proposed in this pull request? Adding a new property to SparkConf called spark.metrics.namespace that allows users to set a custom namespace for executor and driver metrics in the metrics systems. By default, the root namespace used for driver or executor metrics is the value of `spark.app.id`. However, often times, users want to be able to track the metrics across apps for driver and executor metrics, which is hard to do with application ID (i.e. `spark.app.id`) since it changes with every invocation of the app. For such use cases, users can set the `spark.metrics.namespace` property to another spark configuration key like `spark.app.name` which is then used to populate the root namespace of the metrics system (with the app name in our example). `spark.metrics.namespace` property can be set to any arbitrary spark property key, whose value would be used to set the root namespace of the metrics system. Non driver and executor metrics are never prefixed with `spark.app.id`, nor does the `spark.metrics.namespace` property have any such affect on such metrics. ## How was this patch tested? Added new unit tests, modified existing unit tests. Author: Mark Grover <mark@apache.org> Closes #14270 from markgrover/spark-5847.
* [SPARK-15254][DOC] Improve ML pipeline Cross Validation Scaladoc & PyDockrishnakalyan32016-07-272-4/+19
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Updated ML pipeline Cross Validation Scaladoc & PyDoc. ## How was this patch tested? Documentation update (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: krishnakalyan3 <krishnakalyan3@gmail.com> Closes #13894 from krishnakalyan3/kfold-cv.
* [MINOR][DOC][SQL] Fix two documents regarding size in bytesLiang-Chi Hsieh2016-07-271-5/+7
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix two places in SQLConf documents regarding size in bytes and statistics. ## How was this patch tested? No. Just change document. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14341 from viirya/fix-doc-size-in-bytes.
* [MINOR][ML] Fix some mistake in LinearRegression formula.Yanbo Liang2016-07-271-3/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? Fix some mistake in ```LinearRegression``` formula. ## How was this patch tested? Documents change, no tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14369 from yanboliang/LiR-formula.
* [SPARK-16729][SQL] Throw analysis exception for invalid date castspetermaxlee2016-07-272-9/+16
| | | | | | | | | | | | ## What changes were proposed in this pull request? Spark currently throws exceptions for invalid casts for all other data types except date type. Somehow date type returns null. It should be consistent and throws analysis exception as well. ## How was this patch tested? Added a unit test case in CastSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14358 from petermaxlee/SPARK-16729.
* [SPARK-16621][SQL] Generate stable SQLs in SQLBuilderDongjoon Hyun2016-07-27105-114/+124
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, the generated SQLs have not-stable IDs for generated attributes. The stable generated SQL will give more benefit for understanding or testing the queries. This PR provides stable SQL generation by the followings. - Provide unique ids for generated subqueries, `gen_subquery_xxx`. - Provide unique and stable ids for generated attributes, `gen_attr_xxx`. **Before** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res0: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_4` AS `1` FROM (SELECT 1 AS `gen_attr_4`) AS gen_subquery_0 ``` **After** ```scala scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res1: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 scala> new org.apache.spark.sql.catalyst.SQLBuilder(sql("select 1")).toSQL res2: String = SELECT `gen_attr_0` AS `1` FROM (SELECT 1 AS `gen_attr_0`) AS gen_subquery_0 ``` ## How was this patch tested? Pass the existing Jenkins tests. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14257 from dongjoon-hyun/SPARK-16621.
* [SPARK-16524][SQL] Add RowBatch and RowBasedHashMapGeneratorQifan Pu2016-07-267-126/+1356
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is the first step for the following feature: For hash aggregation in Spark SQL, we use a fast aggregation hashmap to act as a "cache" in order to boost aggregation performance. Previously, the hashmap is backed by a `ColumnarBatch`. This has performance issues when we have wide schema for the aggregation table (large number of key fields or value fields). In this JIRA, we support another implementation of fast hashmap, which is backed by a `RowBasedKeyValueBatch`. We then automatically pick between the two implementations based on certain knobs. In this first-step PR, implementations for `RowBasedKeyValueBatch` and `RowBasedHashMapGenerator` are added. ## How was this patch tested? Unit tests: `RowBasedKeyValueBatchSuite` Author: Qifan Pu <qifan.pu@gmail.com> Closes #14349 from ooq/SPARK-16524.
* [SPARK-15703][SCHEDULER][CORE][WEBUI] Make ListenerBus event queue size ↵Dhruve Ashar2016-07-269-37/+60
| | | | | | | | | | | | | | | | | configurable ## What changes were proposed in this pull request? This change adds a new configuration entry to specify the size of the spark listener bus event queue. The value for this config ("spark.scheduler.listenerbus.eventqueue.size") is set to a default to 10000. Note: I haven't currently documented the configuration entry. We can decide whether it would be appropriate to make it a public configuration or keep it as an undocumented one. Refer JIRA for more details. ## How was this patch tested? Ran existing jobs and verified the event queue size with debug logs and from the Spark WebUI Environment tab. Author: Dhruve Ashar <dhruveashar@gmail.com> Closes #14269 from dhruve/bug/SPARK-15703.
* [SPARK-15271][MESOS] Allow force pulling executor docker imagesPhilipp Hoffmann2016-07-2614-29/+110
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Mesos agents by default will not pull docker images which are cached locally already. In order to run Spark executors from mutable tags like `:latest` this commit introduces a Spark setting (`spark.mesos.executor.docker.forcePullImage`). Setting this flag to true will tell the Mesos agent to force pull the docker image (default is `false` which is consistent with the previous implementation and Mesos' default behaviour). Author: Philipp Hoffmann <mail@philipphoffmann.de> Closes #14348 from philipphoffmann/force-pull-image.
* [SPARK-16663][SQL] desc table should be consistent between data source and ↵Wenchen Fan2016-07-265-31/+47
| | | | | | | | | | | | | | | | | | | hive serde tables ## What changes were proposed in this pull request? Currently there are 2 inconsistence: 1. for data source table, we only print partition names, for hive table, we also print partition schema. After this PR, we will always print schema 2. if column doesn't have comment, data source table will print empty string, hive table will print null. After this PR, we will always print null ## How was this patch tested? new test in `HiveDDLSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14302 from cloud-fan/minor3.
* [SPARK-16697][ML][MLLIB] improve LDA submitMiniBatch method to avoid ↵WeichenXu2016-07-261-2/+4
| | | | | | | | | | | | | | | | | | | redundant RDD computation ## What changes were proposed in this pull request? In `LDAOptimizer.submitMiniBatch`, do persist on `stats: RDD[(BDM[Double], List[BDV[Double]])]` and also move the place of unpersisting `expElogbetaBc` broadcast variable, to avoid the `expElogbetaBc` broadcast variable to be unpersisted too early, and update previous `expElogbetaBc.unpersist()` into `expElogbetaBc.destroy(false)` ## How was this patch tested? Existing test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14335 from WeichenXu123/improve_LDA.
* [SPARK-16675][SQL] Avoid per-record type dispatch in JDBC when writinghyukjinkwon2016-07-262-36/+88
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `JdbcUtils.savePartition` is doing type-based dispatch for each row to write appropriate values. So, appropriate setters for `PreparedStatement` can be created first according to the schema, and then apply them to each row. This approach is similar with `CatalystWriteSupport`. This PR simply make the setters to avoid this. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14323 from HyukjinKwon/SPARK-16675.
* [TEST][STREAMING] Fix flaky Kafka rate controlling testTathagata Das2016-07-261-5/+4
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current test is incorrect, because - The expected number of messages does not take into account that the topic has 2 partitions, and rate is set per partition. - Also in some cases, the test ran out of data in Kafka while waiting for the right amount of data per batch. The PR - Reduces the number of partitions to 1 - Adds more data to Kafka - Runs with 0.5 second so that batches are created slowly ## How was this patch tested? Ran many times locally, going to run it many times in Jenkins (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14361 from tdas/kafka-rate-test-fix.
* [SPARK-16706][SQL] support java map in encoderWenchen Fan2016-07-265-32/+236
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? finish the TODO, create a new expression `ExternalMapToCatalyst` to iterate the map directly. ## How was this patch tested? new test in `JavaDatasetSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14344 from cloud-fan/java-map.
* [SPARK-16686][SQL] Remove PushProjectThroughSample since it is handled by ↵Liang-Chi Hsieh2016-07-264-29/+40
| | | | | | | | | | | | | | | | | | ColumnPruning ## What changes were proposed in this pull request? We push down `Project` through `Sample` in `Optimizer` by the rule `PushProjectThroughSample`. However, if the projected columns produce new output, they will encounter whole data instead of sampled data. It will bring some inconsistency between original plan (Sample then Project) and optimized plan (Project then Sample). In the extreme case such as attached in the JIRA, if the projected column is an UDF which is supposed to not see the sampled out data, the result of UDF will be incorrect. Since the rule `ColumnPruning` already handles general `Project` pushdown. We don't need `PushProjectThroughSample` anymore. The rule `ColumnPruning` also avoids the described issue. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14327 from viirya/fix-sample-pushdown.
* [SPARK-16633][SPARK-16642][SPARK-16721][SQL] Fixes three issues related to ↵Yin Huai2016-07-254-40/+108
| | | | | | | | | | | | | | | | | | | | | | | lead and lag functions ## What changes were proposed in this pull request? This PR contains three changes. First, this PR changes the behavior of lead/lag back to Spark 1.6's behavior, which is described as below: 1. lead/lag respect null input values, which means that if the offset row exists and the input value is null, the result will be null instead of the default value. 2. If the offset row does not exist, the default value will be used. 3. OffsetWindowFunction's nullable setting also considers the nullability of its input (because of the first change). Second, this PR fixes the evaluation of lead/lag when the input expression is a literal. This fix is a result of the first change. In current master, if a literal is used as the input expression of a lead or lag function, the result will be this literal even if the offset row does not exist. Third, this PR makes ResolveWindowFrame not fire if a window function is not resolved. ## How was this patch tested? New tests in SQLWindowFunctionSuite Author: Yin Huai <yhuai@databricks.com> Closes #14284 from yhuai/lead-lag.
* [SPARK-16724] Expose DefinedByConstructorParamsMichael Armbrust2016-07-251-1/+1
| | | | | | | | We don't generally make things in catalyst/execution private. Instead they are just undocumented due to their lack of stability guarantees. Author: Michael Armbrust <michael@databricks.com> Closes #14356 from marmbrus/patch-1.
* [SPARK-16672][SQL] SQLBuilder should not raise exceptions on EXISTS queriesDongjoon Hyun2016-07-253-2/+21
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `SQLBuilder` raises `empty.reduceLeft` exceptions on *unoptimized* `EXISTS` queries. We had better prevent this. ```scala scala> sql("CREATE TABLE t1(a int)") scala> val df = sql("select * from t1 b where exists (select * from t1 a)") scala> new org.apache.spark.sql.catalyst.SQLBuilder(df).toSQL java.lang.UnsupportedOperationException: empty.reduceLeft ``` ## How was this patch tested? Pass the Jenkins tests with a new test suite. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14307 from dongjoon-hyun/SPARK-16672.
* Fix description of spark.speculation.quantileNicholas Brown2016-07-251-1/+1
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Minor doc fix regarding the spark.speculation.quantile configuration parameter. It incorrectly states it should be a percentage, when it should be a fraction. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) I tried building the documentation but got some unidoc errors. I also got them when building off origin/master, so I don't think I caused that problem. I did run the web app and saw the changes reflected as expected. Author: Nicholas Brown <nbrown@adroitdigital.com> Closes #14352 from nwbvt/master.
* [SPARK-16678][SPARK-16677][SQL] Fix two View-related bugsgatorsmile2016-07-264-19/+96
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? **Issue 1: Disallow Creating/Altering a View when the same-name Table Exists (without IF NOT EXISTS)** When we create OR alter a view, we check whether the view already exists. In the current implementation, if a table with the same name exists, we treat it as a view. However, this is not the right behavior. We should follow what Hive does. For example, ``` hive> CREATE TABLE tab1 (id int); OK Time taken: 0.196 seconds hive> CREATE OR REPLACE VIEW tab1 AS SELECT * FROM t1; FAILED: SemanticException [Error 10218]: Existing table is not a view The following is an existing table, not a view: default.tab1 hive> ALTER VIEW tab1 AS SELECT * FROM t1; FAILED: SemanticException [Error 10218]: Existing table is not a view The following is an existing table, not a view: default.tab1 hive> CREATE VIEW IF NOT EXISTS tab1 AS SELECT * FROM t1; OK Time taken: 0.678 seconds ``` **Issue 2: Strange Error when Issuing Load Table Against A View** Users should not be allowed to issue LOAD DATA against a view. Currently, when users doing it, we got a very strange runtime error. For example, ```SQL LOAD DATA LOCAL INPATH "$testData" INTO TABLE $viewName ``` ``` java.lang.reflect.InvocationTargetException was thrown. java.lang.reflect.InvocationTargetException at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) at org.apache.spark.sql.hive.client.Shim_v0_14.loadTable(HiveShim.scala:680) ``` ## How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14314 from gatorsmile/tableDDLAgainstView.
* [SPARK-16722][TESTS] Fix a StreamingContext leak in StreamingContextSuite ↵Shixiong Zhu2016-07-251-3/+6
| | | | | | | | | | | | | | | | when eventually fails ## What changes were proposed in this pull request? This PR moves `ssc.stop()` into `finally` for `StreamingContextSuite.createValidCheckpoint` to avoid leaking a StreamingContext since leaking a StreamingContext will fail a lot of tests and make us hard to find the real failure one. ## How was this patch tested? Jenkins unit tests Author: Shixiong Zhu <shixiong@databricks.com> Closes #14354 from zsxwing/ssc-leak.
* [SPARK-15590][WEBUI] Paginate Job Table in Jobs tabTao Lin2016-07-252-62/+312
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds pagination support for the Job Tables in the Jobs tab. Pagination is provided for all of the three Job Tables (active, completed, and failed). Interactions (jumping, sorting, and setting page size) for paged tables are also included. The diff didn't keep track of some lines based on the original ones. The function `makeRow`of the original `AllJobsPage.scala` is reused. They are separated at the beginning of the function `jobRow` (L427-439) and the function `row`(L594-618) in the new `AllJobsPage.scala`. ## How was this patch tested? Tested manually by using checking the Web UI after completing and failing hundreds of jobs. Generate completed jobs by: ```scala val d = sc.parallelize(Array(1,2,3,4,5)) for(i <- 1 to 255){ var b = d.collect() } ``` Generate failed jobs by calling the following code multiple times: ```scala var b = d.map(_/0).collect() ``` Interactions like jumping, sorting, and setting page size are all tested. This shows the pagination for completed jobs: ![paginate success jobs](https://cloud.githubusercontent.com/assets/5558370/15986498/efa12ef6-303b-11e6-8b1d-c3382aeb9ad0.png) This shows the sorting works in job tables: ![sorting](https://cloud.githubusercontent.com/assets/5558370/15986539/98c8a81a-303c-11e6-86f2-8d2bc7924ee9.png) This shows the pagination for failed jobs and the effect of jumping and setting page size: ![paginate failed jobs](https://cloud.githubusercontent.com/assets/5558370/15986556/d8c1323e-303c-11e6-8e4b-7bdb030ea42b.png) Author: Tao Lin <nblintao@gmail.com> Closes #13620 from nblintao/dev.
* [SPARK-14131][STREAMING] SQL Improved fix for avoiding potential deadlocks ↵Tathagata Das2016-07-255-36/+80
| | | | | | | | | | | | | | in HDFSMetadataLog ## What changes were proposed in this pull request? Current fix for deadlock disables interrupts in the StreamExecution which getting offsets for all sources, and when writing to any metadata log, to avoid potential deadlocks in HDFSMetadataLog(see JIRA for more details). However, disabling interrupts can have unintended consequences in other sources. So I am making the fix more narrow, by disabling interrupt it only in the HDFSMetadataLog. This is a narrower fix for something risky like disabling interrupt. ## How was this patch tested? Existing tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14292 from tdas/SPARK-14131.
* [SPARK-16715][TESTS] Fix a potential ExprId conflict for ↵Shixiong Zhu2016-07-251-1/+5
| | | | | | | | | | | | | | | | | | SubexpressionEliminationSuite."Semantic equals and hash" ## What changes were proposed in this pull request? SubexpressionEliminationSuite."Semantic equals and hash" assumes the default AttributeReference's exprId wont' be "ExprId(1)". However, that depends on when this test runs. It may happen to use "ExprId(1)". This PR detects the conflict and makes sure we create a different ExprId when the conflict happens. ## How was this patch tested? Jenkins unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #14350 from zsxwing/SPARK-16715.
* [SPARK-16166][CORE] Also take off-heap memory usage into consideration in ↵jerryshao2016-07-257-9/+26
| | | | | | | | | | | | | | | | | | | | | | | log and webui display ## What changes were proposed in this pull request? Currently in the log and UI display, only on-heap storage memory is calculated and displayed, ``` 16/06/27 13:41:52 INFO MemoryStore: Block rdd_5_0 stored as values in memory (estimated size 17.8 KB, free 665.9 MB) ``` <img width="1232" alt="untitled" src="https://cloud.githubusercontent.com/assets/850797/16369960/53fb614e-3c6e-11e6-8fa3-7ffe65abcb49.png"> With [SPARK-13992](https://issues.apache.org/jira/browse/SPARK-13992) off-heap memory is supported for data persistence, so here change to also take off-heap storage memory into consideration. ## How was this patch tested? Unit test and local verification. Author: jerryshao <sshao@hortonworks.com> Closes #13920 from jerryshao/SPARK-16166.
* [SQL][DOC] Fix a default name for parquet compressionTakeshi YAMAMURO2016-07-251-1/+1
| | | | | | | | | ## What changes were proposed in this pull request? This pr is to fix a wrong description for parquet default compression. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #14351 from maropu/FixParquetDoc.
* Revert "[SPARK-15271][MESOS] Allow force pulling executor docker images"Josh Rosen2016-07-2514-110/+29
| | | | This reverts commit 978cd5f125eb5a410bad2e60bf8385b11cf1b978.
* [SPARK-16485][DOC][ML] Fixed several inline formatting in ml features docShuai Lin2016-07-251-2/+2
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fixed several inline formatting in ml features doc. Before: <img width="475" alt="screen shot 2016-07-14 at 12 24 57 pm" src="https://cloud.githubusercontent.com/assets/717363/16827974/1e1b6e04-49be-11e6-8aa9-4a0cb6cd3b4e.png"> After: <img width="404" alt="screen shot 2016-07-14 at 12 25 48 pm" src="https://cloud.githubusercontent.com/assets/717363/16827976/2576510a-49be-11e6-96dd-92a1fa464d36.png"> ## How was this patch tested? Genetate the docs locally by `SKIP_API=1 jekyll build` and view it in the browser. Author: Shuai Lin <linshuai2012@gmail.com> Closes #14194 from lins05/fix-docs-formatting.
* [SPARK-15271][MESOS] Allow force pulling executor docker imagesPhilipp Hoffmann2016-07-2514-29/+110
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Mesos agents by default will not pull docker images which are cached locally already. In order to run Spark executors from mutable tags like `:latest` this commit introduces a Spark setting `spark.mesos.executor.docker.forcePullImage`. Setting this flag to true will tell the Mesos agent to force pull the docker image (default is `false` which is consistent with the previous implementation and Mesos' default behaviour). ## How was this patch tested? I ran a sample application including this change on a Mesos cluster and verified the correct behaviour for both, with and without, force pulling the executor image. As expected the image is being force pulled if the flag is set. Author: Philipp Hoffmann <mail@philipphoffmann.de> Closes #13051 from philipphoffmann/force-pull-image.
* [SPARK-16685] Remove audit-release scripts.Reynold Xin2016-07-2524-1020/+0
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch removes dev/audit-release. It was initially created to do basic release auditing. They have been unused by for the last one year+. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #14342 from rxin/SPARK-16685.
* [SPARK-16653][ML][OPTIMIZER] update ANN convergence tolerance param default ↵WeichenXu2016-07-252-6/+6
| | | | | | | | | | | | | | | | | | | to 1e-6 ## What changes were proposed in this pull request? replace ANN convergence tolerance param default from 1e-4 to 1e-6 so that it will be the same with other algorithms in MLLib which use LBFGS as optimizer. ## How was this patch tested? Existing Test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14286 from WeichenXu123/update_ann_tol.
* [SPARKR][DOCS] fix broken url in docFelix Cheung2016-07-253-63/+62
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix broken url, also, sparkR.session.stop doc page should have it in the header, instead of saying "sparkR.stop" ![image](https://cloud.githubusercontent.com/assets/8969467/17080129/26d41308-50d9-11e6-8967-79d6c920313f.png) Data type section is in the middle of a list of gapply/gapplyCollect subsections: ![image](https://cloud.githubusercontent.com/assets/8969467/17080122/f992d00a-50d8-11e6-8f2c-fd5786213920.png) ## How was this patch tested? manual test Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #14329 from felixcheung/rdoclinkfix.
* [SPARK-16703][SQL] Remove extra whitespace in SQL generation for window ↵Cheng Lian2016-07-256-9/+40
| | | | | | | | | | | | | | | | | | | | | | | | | | | | functions ## What changes were proposed in this pull request? This PR fixes a minor formatting issue of `WindowSpecDefinition.sql` when no partitioning expressions are present. Before: ```sql ( ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) ``` After: ```sql (ORDER BY `a` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) ``` ## How was this patch tested? New test case added in `ExpressionSQLBuilderSuite`. Author: Cheng Lian <lian@databricks.com> Closes #14334 from liancheng/window-spec-sql-format.
* [SPARK-16698][SQL] Field names having dots should be allowed for datasources ↵hyukjinkwon2016-07-252-1/+16
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | based on FileFormat ## What changes were proposed in this pull request? It seems this is a regression assuming from https://issues.apache.org/jira/browse/SPARK-16698. Field name having dots throws an exception. For example the codes below: ```scala val path = "/tmp/path" val json =""" {"a.b":"data"}""" spark.sparkContext .parallelize(json :: Nil) .saveAsTextFile(path) spark.read.json(path).collect() ``` throws an exception as below: ``` Unable to resolve a.b given [a.b]; org.apache.spark.sql.AnalysisException: Unable to resolve a.b given [a.b]; at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1$$anonfun$apply$5.apply(LogicalPlan.scala:134) at scala.Option.getOrElse(Option.scala:121) ``` This problem was introduced in https://github.com/apache/spark/commit/17eec0a71ba8713c559d641e3f43a1be726b037c#diff-27c76f96a7b2733ecfd6f46a1716e153R121 When extracting the data columns, it does not count that it can contains dots in field names. Actually, it seems the fields name are not expected as quoted when defining schema. So, It not have to consider whether this is wrapped with quotes because the actual schema (inferred or user-given schema) would not have the quotes for fields. For example, this throws an exception. (**Loading JSON from RDD is fine**) ```scala val json =""" {"a.b":"data"}""" val rdd = spark.sparkContext.parallelize(json :: Nil) spark.read.schema(StructType(Seq(StructField("`a.b`", StringType, true)))) .json(rdd).select("`a.b`").printSchema() ``` as below: ``` cannot resolve '```a.b```' given input columns: [`a.b`]; org.apache.spark.sql.AnalysisException: cannot resolve '```a.b```' given input columns: [`a.b`]; at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42) ``` ## How was this patch tested? Unit tests in `FileSourceStrategySuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14339 from HyukjinKwon/SPARK-16698-regression.
* [SPARK-16668][TEST] Test parquet reader for row groups containing both ↵Sameer Agarwal2016-07-251-0/+29
| | | | | | | | | | | | | | | | | dictionary and plain encoded pages ## What changes were proposed in this pull request? This patch adds an explicit test for [SPARK-14217] by setting the parquet dictionary and page size the generated parquet file spans across 3 pages (within a single row group) where the first page is dictionary encoded and the remaining two are plain encoded. ## How was this patch tested? 1. ParquetEncodingSuite 2. Also manually tested that this test fails without https://github.com/apache/spark/pull/12279 Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14304 from sameeragarwal/hybrid-encoding-test.
* [SPARK-16691][SQL] move BucketSpec to catalyst module and use it in CatalogTableWenchen Fan2016-07-2521-78/+78
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? It's weird that we have `BucketSpec` to abstract bucket info, but don't use it in `CatalogTable`. This PR moves `BucketSpec` into catalyst module. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14331 from cloud-fan/check.
* [SPARK-16660][SQL] CreateViewCommand should not take CatalogTableWenchen Fan2016-07-256-127/+116
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `CreateViewCommand` only needs some information of a `CatalogTable`, but not all of them. We have some tricks(e.g. we need to check the table type is `VIEW`, we need to make `CatalogColumn.dataType` nullable) to allow it to take a `CatalogTable`. This PR cleans it up and only pass in necessary information to `CreateViewCommand`. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14297 from cloud-fan/minor2.
* [SPARK-16674][SQL] Avoid per-record type dispatch in JDBC when readinghyukjinkwon2016-07-251-116/+129
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `JDBCRDD.compute` is doing type dispatch for each row to read appropriate values. It might not have to be done like this because the schema is already kept in `JDBCRDD`. So, appropriate converters can be created first according to the schema, and then apply them to each row. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14313 from HyukjinKwon/SPARK-16674.
* [SPARK-16648][SQL] Make ignoreNullsExpr a child expression of First and LastCheng Lian2016-07-253-4/+16
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Default `TreeNode.withNewChildren` implementation doesn't work for `Last` and when both constructor arguments are the same, e.g.: ```sql LAST_VALUE(FALSE) -- The 2nd argument defaults to FALSE LAST_VALUE(FALSE, FALSE) LAST_VALUE(TRUE, TRUE) ``` This is because although `Last` is a unary expression, both of its constructor arguments, `child` and `ignoreNullsExpr`, are `Expression`s. When they have the same value, `TreeNode.withNewChildren` treats both of them as child nodes by mistake. `First` is also affected by this issue in exactly the same way. This PR fixes this issue by making `ignoreNullsExpr` a child expression of `First` and `Last`. ## How was this patch tested? New test case added in `WindowQuerySuite`. Author: Cheng Lian <lian@databricks.com> Closes #14295 from liancheng/spark-16648-last-value.
* [SPARK-16699][SQL] Fix performance bug in hash aggregate on long string keysQifan Pu2016-07-241-2/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | In the following code in `VectorizedHashMapGenerator.scala`: ``` def hashBytes(b: String): String = { val hash = ctx.freshName("hash") s""" |int $result = 0; |for (int i = 0; i < $b.length; i++) { | ${genComputeHash(ctx, s"$b[i]", ByteType, hash)} | $result = ($result ^ (0x9e3779b9)) + $hash + ($result << 6) + ($result >>> 2); |} """.stripMargin } ``` when b=input.getBytes(), the current 2.0 code results in getBytes() being called n times, n being length of input. getBytes() involves memory copy is thus expensive and causes a performance degradation. Fix is to evaluate getBytes() before the for loop. Performance bug, no additional test added. Author: Qifan Pu <qifan.pu@gmail.com> Closes #14337 from ooq/SPARK-16699. (cherry picked from commit d226dce12babcd9f30db033417b2b9ce79f44312) Signed-off-by: Reynold Xin <rxin@databricks.com>
* [SPARK-5581][CORE] When writing sorted map output file, avoid open / …Brian Cho2016-07-247-156/+192
| | | | | | | | | | | | | | | | | | | | | | …close between each partition ## What changes were proposed in this pull request? Replace commitAndClose with separate commit and close to avoid opening and closing the file between partitions. ## How was this patch tested? Run existing unit tests, add a few unit tests regarding reverts. Observed a ~20% reduction in total time in tasks on stages with shuffle writes to many partitions. JoshRosen Author: Brian Cho <bcho@fb.com> Closes #13382 from dafrista/separatecommit-master.
* [SPARK-16645][SQL] rename CatalogStorageFormat.serdeProperties to propertiesWenchen Fan2016-07-2516-65/+65
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? we also store data source table options in this field, it's unreasonable to call it `serdeProperties`. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14283 from cloud-fan/minor1.
* [SPARK-16416][CORE] force eager creation of loggers to avoid shutdown hook ↵Mikael Ståldal2016-07-248-0/+10
| | | | | | | | | | | | | | | | conflicts ## What changes were proposed in this pull request? Force eager creation of loggers to avoid shutdown hook conflicts. ## How was this patch tested? Manually tested with a project using Log4j 2, verified that the shutdown hook conflict issue was solved. Author: Mikael Ståldal <mikael.staldal@magine.com> Closes #14320 from mikaelstaldal/shutdown-hook-logging.
* [PYSPARK] add picklable SparseMatrix in pyspark.ml.commonWeichenXu2016-07-241-0/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? add `SparseMatrix` class whick support pickler. ## How was this patch tested? Existing test. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14265 from WeichenXu123/picklable_py.
* [SPARK-16463][SQL] Support `truncate` option in Overwrite mode for JDBC ↵Dongjoon Hyun2016-07-249-4/+70
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | DataFrameWriter ## What changes were proposed in this pull request? This PR adds a boolean option, `truncate`, for `SaveMode.Overwrite` of JDBC DataFrameWriter. If this option is `true`, it try to take advantage of `TRUNCATE TABLE` instead of `DROP TABLE`. This is a trivial option, but will provide great **convenience** for BI tool users based on RDBMS tables generated by Spark. **Goal** - Without `CREATE/DROP` privilege, we can save dataframe to database. Sometime these are not allowed for security. - It will preserve the existing table information, so users can add and keep some additional `INDEX` and `CONSTRAINT`s for the table. - Sometime, `TRUNCATE` is faster than the combination of `DROP/CREATE`. **Supported DBMS** The following is `truncate`-option support table. Due to the different behavior of `TRUNCATE TABLE` among DBMSs, it's not always safe to use `TRUNCATE TABLE`. Spark will ignore the `truncate` option for **unknown** and **some** DBMS with **default CASCADING** behavior. Newly added JDBCDialect should implement corresponding function to support `truncate` option additionally. Spark Dialects | `truncate` OPTION SUPPORT ---------------|------------------------------- MySQLDialect | O PostgresDialect | X DB2Dialect | O MsSqlServerDialect | O DerbyDialect | O OracleDialect | O **Before (TABLE with INDEX case)**: SparkShell & MySQL CLI are interleaved intentionally. ```scala scala> val (url, prop)=("jdbc:mysql://localhost:3306/temp?useSSL=false", new java.util.Properties) scala> prop.setProperty("user","root") scala> df.write.mode("overwrite").jdbc(url, "table_with_index", prop) scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop) mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | | NULL | | +-------+------------+------+-----+---------+-------+ mysql> CREATE UNIQUE INDEX idx_id ON table_with_index(id); mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | PRI | NULL | | +-------+------------+------+-----+---------+-------+ scala> spark.range(10).write.mode("overwrite").jdbc(url, "table_with_index", prop) mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | | NULL | | +-------+------------+------+-----+---------+-------+ ``` **After (TABLE with INDEX case)** ```scala scala> spark.range(10).write.mode("overwrite").option("truncate", true).jdbc(url, "table_with_index", prop) mysql> DESC table_with_index; +-------+------------+------+-----+---------+-------+ | Field | Type | Null | Key | Default | Extra | +-------+------------+------+-----+---------+-------+ | id | bigint(20) | NO | PRI | NULL | | +-------+------------+------+-----+---------+-------+ ``` **Error Handling** - In case of exceptions, Spark will not retry. Users should turn off the `truncate` option. - In case of schema change: - If one of the column names changes, this will raise exceptions intuitively. - If there exists only type difference, this will work like Append mode. ## How was this patch tested? Pass the Jenkins tests with a updated testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14086 from dongjoon-hyun/SPARK-16410.