aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-15541] Casting ConcurrentHashMap to ConcurrentMap (master branch)Maciej Brynski2016-08-021-3/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Casting ConcurrentHashMap to ConcurrentMap allows to run code compiled with Java 8 on Java 7 ## How was this patch tested? Compilation. Existing automatic tests Author: Maciej Brynski <maciej.brynski@adpilot.pl> Closes #14459 from maver1ck/spark-15541-master.
* [SPARK-16558][EXAMPLES][MLLIB] examples/mllib/LDAExample should use MLVector ↵Xusen Yin2016-08-021-2/+3
| | | | | | | | | | | | | | | | instead of MLlib Vector ## What changes were proposed in this pull request? mllib.LDAExample uses ML pipeline and MLlib LDA algorithm. The former transforms original data into MLVector format, while the latter uses MLlibVector format. ## How was this patch tested? Test manually. Author: Xusen Yin <yinxusen@gmail.com> Closes #14212 from yinxusen/SPARK-16558.
* [SPARK-16851][ML] Incorrect threshould length in 'setThresholds()' evoke ↵Zheng RuiFeng2016-08-021-1/+6
| | | | | | | | | | | | | | Exception ## What changes were proposed in this pull request? Add a length checking for threshoulds' length in method `setThreshoulds()` of classification models. ## How was this patch tested? unit tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #14457 from zhengruifeng/check_setThresholds.
* [SPARK-16850][SQL] Improve type checking error message for greatest/leastpetermaxlee2016-08-022-2/+15
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Greatest/least function does not have the most friendly error message for data types. This patch improves the error message to not show the Seq type, and use more human readable data types. Before: ``` org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST (ArrayBuffer(DecimalType(2,1), StringType)).; line 1 pos 7 ``` After: ``` org.apache.spark.sql.AnalysisException: cannot resolve 'greatest(CAST(1.0 AS DECIMAL(2,1)), "1.0")' due to data type mismatch: The expressions should all have the same type, got GREATEST(decimal(2,1), string).; line 1 pos 7 ``` ## How was this patch tested? Manually verified the output and also added unit tests to ConditionalExpressionSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14453 from petermaxlee/SPARK-16850.
* [SPARK-16734][EXAMPLES][SQL] Revise examples of all language bindingsCheng Lian2016-08-029-117/+137
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR makes various minor updates to examples of all language bindings to make sure they are consistent with each other. Some typos and missing parts (JDBC example in Scala/Java/Python) are also fixed. ## How was this patch tested? Manually tested. Author: Cheng Lian <lian@databricks.com> Closes #14368 from liancheng/revise-examples.
* [SPARK-16793][SQL] Set the temporary warehouse path to sc'conf in TestHive.jiangxingbo2016-08-013-25/+21
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? With SPARK-15034, we could use the value of spark.sql.warehouse.dir to set the warehouse location. In TestHive, we can now simply set the temporary warehouse path in sc's conf, and thus, param "warehousePath" could be removed. ## How was this patch tested? exsiting testsuites. Author: jiangxingbo <jiangxingbo@meituan.com> Closes #14401 from jiangxb1987/warehousePath.
* [SPARK-16828][SQL] remove MaxOf and MinOfWenchen Fan2016-08-014-175/+0
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? These 2 expressions are not needed anymore after we have `Greatest` and `Least`. This PR removes them and related tests. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14434 from cloud-fan/minor1.
* [SPARK-15869][STREAMING] Fix a potential NPE in ↵Shixiong Zhu2016-08-011-1/+1
| | | | | | | | | | | | | | | | StreamingJobProgressListener.getBatchUIData ## What changes were proposed in this pull request? Moved `asScala` to a `map` to avoid NPE. ## How was this patch tested? Existing unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #14443 from zsxwing/SPARK-15869.
* [SPARK-16774][SQL] Fix use of deprecated timestamp constructor & improve ↵Holden Karau2016-08-012-7/+10
| | | | | | | | | | | | | | | | | | | | timezone handling ## What changes were proposed in this pull request? Removes the deprecated timestamp constructor and incidentally fixes the use which was using system timezone rather than the one specified when working near DST. This change also causes the roundtrip tests to fail since it now actually uses all the timezones near DST boundaries where it didn't before. Note: this is only a partial the solution, longer term we should follow up with https://issues.apache.org/jira/browse/SPARK-16788 to avoid this problem & simplify our timezone handling code. ## How was this patch tested? New tests for two timezones added so even if user timezone happens to coincided with one, the other tests should still fail. Important note: this (temporarily) disables the round trip tests until we can fix the issue more thoroughly. Author: Holden Karau <holden@us.ibm.com> Closes #14398 from holdenk/SPARK-16774-fix-use-of-deprecated-timestamp-constructor.
* [SPARK-16791][SQL] cast struct with timestamp field failseyal farago2016-08-012-1/+11
| | | | | | | | | | | | | ## What changes were proposed in this pull request? a failing test case + fix to SPARK-16791 (https://issues.apache.org/jira/browse/SPARK-16791) ## How was this patch tested? added a failing test case to CastSuit, then fixed the Cast code and rerun the entire CastSuit Author: eyal farago <eyal farago> Author: Eyal Farago <eyal.farago@actimize.com> Closes #14400 from eyalfa/SPARK-16791_cast_struct_with_timestamp_field_fails.
* [SPARK-16776][STREAMING] Replace deprecated API in KafkaTestUtils for 0.10.0.hyukjinkwon2016-08-011-8/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR replaces the old Kafka API to 0.10.0 ones in `KafkaTestUtils`. The change include: - `Producer` to `KafkaProducer` - Change configurations to equalvant ones. (I referred [here](http://kafka.apache.org/documentation.html#producerconfigs) for 0.10.0 and [here](http://kafka.apache.org/082/documentation.html#producerconfigs ) for old, 0.8.2). This PR will remove the build warning as below: ```scala [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:71: class Producer in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.KafkaProducer instead. [WARNING] private var producer: Producer[String, String] = _ [WARNING] ^ [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:181: class Producer in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.KafkaProducer instead. [WARNING] producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) [WARNING] ^ [WARNING] .../spark/streaming/kafka010/KafkaTestUtils.scala:181: class ProducerConfig in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.ProducerConfig instead. [WARNING] producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) [WARNING] ^ [WARNING] .../spark/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala:182: class KeyedMessage in package producer is deprecated: This class has been deprecated and will be removed in a future release. Please use org.apache.kafka.clients.producer.ProducerRecord instead. [WARNING] producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) [WARNING] ^ [WARNING] four warnings found [WARNING] warning: [options] bootstrap class path not set in conjunction with -source 1.7 [WARNING] 1 warning ``` ## How was this patch tested? Existing tests that use `KafkaTestUtils` should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14416 from HyukjinKwon/SPARK-16776.
* [SPARK-16778][SQL][TRIVIAL] Fix deprecation warning with SQLContextHolden Karau2016-08-011-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Change to non-deprecated constructor for SQLContext. ## How was this patch tested? Existing tests Author: Holden Karau <holden@us.ibm.com> Closes #14406 from holdenk/SPARK-16778-fix-use-of-deprecated-SQLContext-constructor.
* [SPARK-16485][DOC][ML] Remove useless latex in a log messge.Shuai Lin2016-08-011-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Removed useless latex in a log messge. ## How was this patch tested? Check generated scaladoc. Author: Shuai Lin <linshuai2012@gmail.com> Closes #14380 from lins05/fix-docs-formatting.
* [SPARK-16726][SQL] Improve `Union/Intersect/Except` error messages on ↵Dongjoon Hyun2016-08-012-13/+46
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | incompatible types ## What changes were proposed in this pull request? Currently, `UNION` queries on incompatible types show misleading error messages, i.e., `unresolved operator Union`. We had better show a more correct message. This will help users in the situation of [SPARK-16704](https://issues.apache.org/jira/browse/SPARK-16704). **Before** ```scala scala> sql("select 1,2,3 union (select 1,array(2),3)") org.apache.spark.sql.AnalysisException: unresolved operator 'Union; scala> sql("select 1,2,3 intersect (select 1,array(2),3)") org.apache.spark.sql.AnalysisException: unresolved operator 'Intersect; scala> sql("select 1,2,3 except (select 1,array(2),3)") org.apache.spark.sql.AnalysisException: unresolved operator 'Except; ``` **After** ```scala scala> sql("select 1,2,3 union (select 1,array(2),3)") org.apache.spark.sql.AnalysisException: Union can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table; scala> sql("select 1,2,3 intersect (select 1,array(2),3)") org.apache.spark.sql.AnalysisException: Intersect can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the second column of the second table; scala> sql("select 1,2,3 except (select array(1),array(2),3)") org.apache.spark.sql.AnalysisException: Except can only be performed on tables with the compatible column types. ArrayType(IntegerType,false) <> IntegerType at the first column of the second table; ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14355 from dongjoon-hyun/SPARK-16726.
* [SPARK-16805][SQL] Log timezone when query result does not matchReynold Xin2016-07-311-0/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? It is useful to log the timezone when query result does not match, especially on build machines that have different timezone from AMPLab Jenkins. ## How was this patch tested? This is a test-only change. Author: Reynold Xin <rxin@databricks.com> Closes #14413 from rxin/SPARK-16805.
* [SPARK-16731][SQL] use StructType in CatalogTable and remove CatalogColumnWenchen Fan2016-07-3117-175/+120
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `StructField` has very similar semantic with `CatalogColumn`, except that `CatalogColumn` use string to express data type. I think it's reasonable to use `StructType` as the `CatalogTable.schema` and remove `CatalogColumn`. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14363 from cloud-fan/column.
* [SPARK-16813][SQL] Remove private[sql] and private[spark] from catalyst packageReynold Xin2016-07-3118-32/+32
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The catalyst package is meant to be internal, and as a result it does not make sense to mark things as private[sql] or private[spark]. It simply makes debugging harder when Spark developers need to inspect the plans at runtime. This patch removes all private[sql] and private[spark] visibility modifiers in org.apache.spark.sql.catalyst. ## How was this patch tested? N/A - just visibility changes. Author: Reynold Xin <rxin@databricks.com> Closes #14418 from rxin/SPARK-16813.
* [SPARK-16812] Open up SparkILoop.getAddedJarsReynold Xin2016-07-301-1/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch makes SparkILoop.getAddedJars a public developer API. It is a useful function to get the list of jars added. ## How was this patch tested? N/A - this is a simple visibility change. Author: Reynold Xin <rxin@databricks.com> Closes #14417 from rxin/SPARK-16812.
* [SPARK-16818] Exchange reuse incorrectly reuses scans over different sets of ↵Eric Liang2016-07-302-1/+36
| | | | | | | | | | | | | | | | | | | | partitions ## What changes were proposed in this pull request? This fixes a bug wherethe file scan operator does not take into account partition pruning in its implementation of `sameResult()`. As a result, executions may be incorrect on self-joins over the same base file relation. The patch here is minimal, but we should reconsider relying on `metadata` for implementing sameResult() in the future, as string representations may not be uniquely identifying. cc rxin ## How was this patch tested? Unit tests. Author: Eric Liang <ekl@databricks.com> Closes #14425 from ericl/spark-16818.
* [SPARK-16800][EXAMPLES][ML] Fix Java examples that fail to run due to exceptionBryan Cutler2016-07-3012-38/+49
| | | | | | | | | | | | ## What changes were proposed in this pull request? Some Java examples are using mllib.linalg.Vectors instead of ml.linalg.Vectors and causes an exception when run. Also there are some Java examples that incorrectly specify data types in the schema, also causing an exception. ## How was this patch tested? Ran corrected examples locally Author: Bryan Cutler <cutlerb@gmail.com> Closes #14405 from BryanCutler/java-examples-ml.Vectors-fix-SPARK-16800.
* [SPARK-16696][ML][MLLIB] destroy KMeans bcNewCenters when loop finished and ↵WeichenXu2016-07-303-9/+17
| | | | | | | | | | | | | | | | | | | | | | | | | | update code where should release unused broadcast/RDD in proper time ## What changes were proposed in this pull request? update unused broadcast in KMeans/Word2Vec, use destroy(false) to release memory in time. and several place destroy() update to destroy(false) so that it will be async-called, it will better than blocking called. and update bcNewCenters in KMeans to make it destroy in correct time. I use a list to store all historical `bcNewCenters` generated in each loop iteration and delay them to release at the end of loop. fix TODO in `BisectingKMeans.run` "unpersist old indices", Implements the pattern "persist current step RDD, and unpersist previous one" in the loop iteration. ## How was this patch tested? Existing tests. Author: WeichenXu <WeichenXu123@outlook.com> Closes #14333 from WeichenXu123/broadvar_unpersist_to_destroy.
* [SPARK-16694][CORE] Use for/foreach rather than map for Unit expressions ↵Sean Owen2016-07-3039-146/+125
| | | | | | | | | | | | | | | | whose side effects are required ## What changes were proposed in this pull request? Use foreach/for instead of map where operation requires execution of body, not actually defining a transformation ## How was this patch tested? Jenkins Author: Sean Owen <sowen@cloudera.com> Closes #14332 from srowen/SPARK-16694.
* [SPARK-16748][SQL] SparkExceptions during planning should not wrapped in ↵Tathagata Das2016-07-292-2/+16
| | | | | | | | | | | | | | TreeNodeException ## What changes were proposed in this pull request? We do not want SparkExceptions from job failures in the planning phase to create TreeNodeException. Hence do not wrap SparkException in TreeNodeException. ## How was this patch tested? New unit test Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14395 from tdas/SPARK-16748.
* [SPARK-16772][PYTHON][DOCS] Restore "datatype string" to Python API docstringsNicholas Chammas2016-07-292-12/+8
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR corrects [an error made in an earlier PR](https://github.com/apache/spark/pull/14393/files#r72843069). ## How was this patch tested? ```sh $ ./dev/lint-python PEP8 checks passed. rm -rf _build/* pydoc checks passed. ``` I also built the docs and confirmed that they looked good in my browser. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #14408 from nchammas/SPARK-16772.
* [SPARK-16761][DOC][ML] Fix doc link in docs/ml-guide.mdSun Dapeng2016-07-291-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix the link at http://spark.apache.org/docs/latest/ml-guide.html. ## How was this patch tested? None Author: Sun Dapeng <sdp@apache.org> Closes #14386 from sundapeng/doclink.
* [SPARK-16637] Unified containerizerMichael Gummelt2016-07-2918-79/+149
| | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? New config var: spark.mesos.docker.containerizer={"mesos","docker" (default)} This adds support for running docker containers via the Mesos unified containerizer: http://mesos.apache.org/documentation/latest/container-image/ The benefit is losing the dependency on `dockerd`, and all the costs which it incurs. I've also updated the supported Mesos version to 0.28.2 for support of the required protobufs. This is blocked on: https://github.com/apache/spark/pull/14167 ## How was this patch tested? - manually testing jobs submitted with both "mesos" and "docker" settings for the new config var. - spark/mesos integration test suite Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #14275 from mgummelt/unified-containerizer.
* [SPARK-16751] Upgrade derby to 10.12.1.1Adam Roberts2016-07-296-6/+6
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Version of derby upgraded based on important security info at VersionEye. Test scope added so we don't include it in our final package anyway. NB: I think this should be backported to all previous releases as it is a security problem https://www.versioneye.com/java/org.apache.derby:derby/10.11.1.1 The CVE number is 2015-1832. I also suggest we add a SECURITY tag for JIRAs ## How was this patch tested? Existing tests with the change making sure that we see no new failures. I checked derby 10.12.x and not derby 10.11.x is downloaded to our ~/.m2 folder. I then used dev/make-distribution.sh and checked the dist/jars folder for Spark 2.0: no derby jar is present. I don't know if this would also remove it from the assembly jar in our 1.x branches. Author: Adam Roberts <aroberts@uk.ibm.com> Closes #14379 from a-roberts/patch-4.
* [SPARK-16750][ML] Fix GaussianMixture training failed due to feature column ↵Yanbo Liang2016-07-2910-7/+19
| | | | | | | | | | | | | | | | type mistake ## What changes were proposed in this pull request? ML ```GaussianMixture``` training failed due to feature column type mistake. The feature column type should be ```ml.linalg.VectorUDT``` but got ```mllib.linalg.VectorUDT``` by mistake. See [SPARK-16750](https://issues.apache.org/jira/browse/SPARK-16750) for how to reproduce this bug. Why the unit tests did not complain this errors? Because some estimators/transformers missed calling ```transformSchema(dataset.schema)``` firstly during ```fit``` or ```transform```. I will also add this function to all estimators/transformers who missed in this PR. ## How was this patch tested? No new tests, should pass existing ones. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14378 from yanboliang/spark-16750.
* [SPARK-16664][SQL] Fix persist call on Data frames with more than 200…Wesley Tang2016-07-293-3/+12
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? f12f11e578169b47e3f8b18b299948c0670ba585 introduced this bug, missed foreach as map ## How was this patch tested? Test added Author: Wesley Tang <tangmingjun@mininglamp.com> Closes #14324 from breakdawn/master.
* [SPARK-16772] Correct API doc references to PySpark classes + formatting fixesNicholas Chammas2016-07-288-58/+75
| | | | | | | | | | | | | | | | | | ## What's Been Changed The PR corrects several broken or missing class references in the Python API docs. It also correct formatting problems. For example, you can see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.registerFunction) how Sphinx is not picking up the reference to `DataType`. That's because the reference is relative to the current module, whereas `DataType` is in a different module. You can also see [here](http://spark.apache.org/docs/2.0.0/api/python/pyspark.sql.html#pyspark.sql.SQLContext.createDataFrame) how the formatting for byte, tinyint, and so on is italic instead of monospace. That's because in ReST single backticks just make things italic, unlike in Markdown. ## Testing I tested this PR by [building the Python docs](https://github.com/apache/spark/tree/master/docs#generating-the-documentation-html) and reviewing the results locally in my browser. I confirmed that the broken or missing class references were resolved, and that the formatting was corrected. Author: Nicholas Chammas <nicholas.chammas@gmail.com> Closes #14393 from nchammas/python-docstring-fixes.
* [SPARK-16764][SQL] Recommend disabling vectorized parquet reader on ↵Sameer Agarwal2016-07-281-5/+19
| | | | | | | | | | | | | | | | OutOfMemoryError ## What changes were proposed in this pull request? We currently don't bound or manage the data array size used by column vectors in the vectorized reader (they're just bound by INT.MAX) which may lead to OOMs while reading data. As a short term fix, this patch intercepts the OutOfMemoryError exception and suggest the user to disable the vectorized parquet reader. ## How was this patch tested? Existing Tests Author: Sameer Agarwal <sameerag@cs.berkeley.edu> Closes #14387 from sameeragarwal/oom.
* [SPARK-16740][SQL] Fix Long overflow in LongToUnsafeRowMapSylvain Zimmer2016-07-281-1/+2
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid overflow of Long type causing a NegativeArraySizeException a few lines later. ## How was this patch tested? Unit tests for HashedRelationSuite still pass. I can confirm the python script I included in https://issues.apache.org/jira/browse/SPARK-16740 works fine with this patch. Unfortunately I don't have the knowledge/time to write a Scala test case for HashedRelationSuite right now. As the patch is pretty obvious I hope it can be included without this. Thanks! Author: Sylvain Zimmer <sylvain@sylvainzimmer.com> Closes #14373 from sylvinus/master.
* [SPARK-16639][SQL] The query with having condition that contains grouping by ↵Liang-Chi Hsieh2016-07-282-6/+31
| | | | | | | | | | | | | | | | | | | | | column should work ## What changes were proposed in this pull request? The query with having condition that contains grouping by column will be failed during analysis. E.g., create table tbl(a int, b string); select count(b) from tbl group by a + 1 having a + 1 = 2; Having condition should be able to use grouping by column. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14296 from viirya/having-contains-grouping-column.
* [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.