aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
...
* [SPARK-16930][YARN] Fix a couple of races in cluster app initialization.Marcelo Vanzin2016-08-173-59/+66
| | | | | | | | | | | | | | | | | | There are two narrow races that could cause the ApplicationMaster to miss when the user application instantiates the SparkContext, which could cause app failures when nothing was wrong with the app. It was also possible for a failing application to get stuck in the loop that waits for the context for a long time, instead of failing quickly. The change uses a promise to track the SparkContext instance, which gets rid of the races and allows for some simplification of the code. Tested with existing unit tests, and a new one being added to test the timeout code. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14542 from vanzin/SPARK-16930.
* [SPARK-17102][SQL] bypass UserDefinedGenerator for json format checkWenchen Fan2016-08-171-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We use reflection to convert `TreeNode` to json string, and currently don't support arbitrary object. `UserDefinedGenerator` takes a function object, so we should skip json format test for it, or the tests can be flacky, e.g. `DataFrameSuite.simple explode`, this test always fail with scala 2.10(branch 1.6 builds with scala 2.10 by default), but pass with scala 2.11(master branch builds with scala 2.11 by default). ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14679 from cloud-fan/json.
* [SPARK-17106] [SQL] Simplify the SubqueryExpression interfaceHerman van Hovell2016-08-178-74/+56
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code. This PR unifies these access paths. ## How was this patch tested? (Existing tests) Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14685 from hvanhovell/SPARK-17106.
* [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows ↵Kazuaki Ishizaki2016-08-172-6/+66
| | | | | | | | | | | | | | | | | | beyond 64 KB ## What changes were proposed in this pull request? This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method. Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf). ## How was this patch tested? Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10 Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #14670 from kiszk/SPARK-15285-2.
* [SPARK-16444][SPARKR] Isotonic Regression wrapper in SparkRwm624@hotmail.com2016-08-176-1/+277
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) Add Isotonic Regression wrapper in SparkR Wrappers in R and Scala are added. Unit tests Documentation ## How was this patch tested? Manually tested with sudo ./R/run-tests.sh (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) Author: wm624@hotmail.com <wm624@hotmail.com> Closes #14182 from wangmiao1981/isoR.
* [SPARK-17032][SQL] Add test cases for methods in ParserUtils.jiangxingbo2016-08-172-7/+128
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently methods in `ParserUtils` are tested indirectly, we should add test cases in `ParserUtilsSuite` to verify their integrity directly. ## How was this patch tested? New test cases in `ParserUtilsSuite` Author: jiangxingbo <jiangxb1987@gmail.com> Closes #14620 from jiangxb1987/parserUtils.
* [MINOR][DOC] Fix the descriptions for `properties` argument in the ↵mvervuurt2016-08-161-5/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | documenation for jdbc APIs ## What changes were proposed in this pull request? This should be credited to mvervuurt. The main purpose of this PR is - simply to include the change for the same instance in `DataFrameReader` just to match up. - just avoid duplicately verifying the PR (as I already did). The documentation for both should be the same because both assume the `properties` should be the same `dict` for the same option. ## How was this patch tested? Manually building Python documentation. This will produce the output as below: - `DataFrameReader` ![2016-08-17 11 12 00](https://cloud.githubusercontent.com/assets/6477701/17722764/b3f6568e-646f-11e6-8b75-4fb672f3f366.png) - `DataFrameWriter` ![2016-08-17 11 12 10](https://cloud.githubusercontent.com/assets/6477701/17722765/b58cb308-646f-11e6-841a-32f19800d139.png) Closes #14624 Author: hyukjinkwon <gurwls223@gmail.com> Author: mvervuurt <m.a.vervuurt@gmail.com> Closes #14677 from HyukjinKwon/typo-python.
* [SPARK-17068][SQL] Make view-usage visible during analysisHerman van Hovell2016-08-1620-71/+94
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable). For example, the following view and query: ```sql create view constants as select 1 as id union all select 1 union all select 42 select * from constants; ``` ...now yields the following analyzed plan: ``` Project [id#39] +- SubqueryAlias c, `default`.`constants` +- Project [gen_attr_0#36 AS id#39] +- SubqueryAlias gen_subquery_0 +- Union :- Union : :- Project [1 AS gen_attr_0#36] : : +- OneRowRelation$ : +- Project [1 AS gen_attr_1#37] : +- OneRowRelation$ +- Project [42 AS gen_attr_2#38] +- OneRowRelation$ ``` ## How was this patch tested? Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive) Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14657 from hvanhovell/SPARK-17068.
* [SPARK-17084][SQL] Rename ParserUtils.assert to validateHerman van Hovell2016-08-163-12/+11
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state. ## How was this patch tested? Simple rename. Compilation should do. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14665 from hvanhovell/SPARK-17084.
* [SPARK-17089][DOCS] Remove api doc link for mapReduceTriplets operatorsandy2016-08-161-3/+2
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Remove the api doc link for mapReduceTriplets operator because in latest api they are remove so when user link to that api they will not get mapReduceTriplets there so its more good to remove than confuse the user. ## How was this patch tested? Run all the test cases ![screenshot from 2016-08-16 23-08-25](https://cloud.githubusercontent.com/assets/8075390/17709393/8cfbf75a-6406-11e6-98e6-38f7b319d833.png) Author: sandy <phalodi@gmail.com> Closes #14669 from phalodi/SPARK-17089.
* [SPARK-16519][SPARKR] Handle SparkR RDD generics that create warnings in R ↵Felix Cheung2016-08-1617-287/+312
| | | | | | | | | | | | | | | | | CMD check ## What changes were proposed in this pull request? Rename RDD functions for now to avoid CRAN check warnings. Some RDD functions are sharing generics with DataFrame functions (hence the problem) so after the renames we need to add new generics, for now. ## How was this patch tested? unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #14626 from felixcheung/rrddfunctions.
* [MINOR][SPARKR] spark.glm weightCol should in the signature.Yanbo Liang2016-08-161-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? Fix the issue that ```spark.glm``` ```weightCol``` should in the signature. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14641 from yanboliang/weightCol.
* [SPARK-17035] [SQL] [PYSPARK] Improve Timestamp not to lose precision for ↵Dongjoon Hyun2016-08-162-1/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | all cases ## What changes were proposed in this pull request? `PySpark` loses `microsecond` precision for some corner cases during converting `Timestamp` into `Long`. For example, for the following `datetime.max` value should be converted a value whose last 6 digits are '999999'. This PR improves the logic not to lose precision for all cases. **Corner case** ```python >>> datetime.datetime.max datetime.datetime(9999, 12, 31, 23, 59, 59, 999999) ``` **Before** ```python >>> from datetime import datetime >>> from pyspark.sql import Row >>> from pyspark.sql.types import StructType, StructField, TimestampType >>> schema = StructType([StructField("dt", TimestampType(), False)]) >>> [schema.toInternal(row) for row in [{"dt": datetime.max}]] [(253402329600000000,)] ``` **After** ```python >>> [schema.toInternal(row) for row in [{"dt": datetime.max}]] [(253402329599999999,)] ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14631 from dongjoon-hyun/SPARK-17035.
* [MINOR][DOC] Correct code snippet results in quick start documentationlinbojin2016-08-161-5/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? As README.md file is updated over time. Some code snippet outputs are not correct based on new README.md file. For example: ``` scala> textFile.count() res0: Long = 126 ``` should be ``` scala> textFile.count() res0: Long = 99 ``` This pr is to add comments to point out this problem so that new spark learners have a correct reference. Also, fixed a samll bug, inside current documentation, the outputs of linesWithSpark.count() without and with cache are different (one is 15 and the other is 19) ``` scala> val linesWithSpark = textFile.filter(line => line.contains("Spark")) linesWithSpark: org.apache.spark.rdd.RDD[String] = MapPartitionsRDD[2] at filter at <console>:27 scala> textFile.filter(line => line.contains("Spark")).count() // How many lines contain "Spark"? res3: Long = 15 ... scala> linesWithSpark.cache() res7: linesWithSpark.type = MapPartitionsRDD[2] at filter at <console>:27 scala> linesWithSpark.count() res8: Long = 19 ``` ## How was this patch tested? manual test: run `$ SKIP_API=1 jekyll serve --watch` Author: linbojin <linbojin203@gmail.com> Closes #14645 from linbojin/quick-start-documentation.
* [SPARK-16964][SQL] Remove private[hive] from sql.hive.execution packageHerman van Hovell2016-08-163-6/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is a small follow-up to https://github.com/apache/spark/pull/14554. This also widens the visibility of a few (similar) Hive classes. ## How was this patch tested? No test. Only a visibility change. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14654 from hvanhovell/SPARK-16964-hive.
* [SPARK-17034][SQL] adds expression UnresolvedOrdinal to represent the ↵Sean Zhong2016-08-167-19/+175
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ordinals in GROUP BY or ORDER BY ## What changes were proposed in this pull request? This PR adds expression `UnresolvedOrdinal` to represent the ordinal in GROUP BY or ORDER BY, and fixes the rules when resolving ordinals. Ordinals in GROUP BY or ORDER BY like `1` in `order by 1` or `group by 1` should be considered as unresolved before analysis. But in current code, it uses `Literal` expression to store the ordinal. This is inappropriate as `Literal` itself is a resolved expression, it gives the user a wrong message that the ordinals has already been resolved. ### Before this change Ordinal is stored as `Literal` expression ``` scala> sc.setLogLevel("TRACE") scala> sql("select a from t group by 1 order by 1") ... 'Sort [1 ASC], true +- 'Aggregate [1], ['a] +- 'UnresolvedRelation `t ``` For query: ``` scala> Seq(1).toDF("a").createOrReplaceTempView("t") scala> sql("select count(a), a from t group by 2 having a > 0").show ``` During analysis, the intermediate plan before applying rule `ResolveAggregateFunctions` is: ``` 'Filter ('a > 0) +- Aggregate [2], [count(1) AS count(1)#83L, a#81] +- LocalRelation [value#7 AS a#9] ``` Before this PR, rule `ResolveAggregateFunctions` believes all expressions of `Aggregate` have already been resolved, and tries to resolve the expressions in `Filter` directly. But this is wrong, as ordinal `2` in Aggregate is not really resolved! ### After this change Ordinals are stored as `UnresolvedOrdinal`. ``` scala> sc.setLogLevel("TRACE") scala> sql("select a from t group by 1 order by 1") ... 'Sort [unresolvedordinal(1) ASC], true +- 'Aggregate [unresolvedordinal(1)], ['a] +- 'UnresolvedRelation `t` ``` ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #14616 from clockfly/spark-16955.
* [SPARK-16916][SQL] serde/storage properties should not have limitationsWenchen Fan2016-08-153-11/+0
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `CatalogStorageFormat.properties` can be used in 2 ways: 1. for hive tables, it stores the serde properties. 2. for data source tables, it stores the data source options, e.g. `path`, `skipHiveMetadata`, etc. however, both of them have nothing to do with data source properties, e.g. `spark.sql.sources.provider`, so they should not have limitations about data source properties. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #14506 from cloud-fan/table-prop.
* [SPARK-17065][SQL] Improve the error message when encountering an ↵Shixiong Zhu2016-08-151-39/+52
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | incompatible DataSourceRegister ## What changes were proposed in this pull request? Add an instruction to ask the user to remove or upgrade the incompatible DataSourceRegister in the error message. ## How was this patch tested? Test command: ``` build/sbt -Dscala-2.10 package SPARK_SCALA_VERSION=2.10 bin/spark-shell --packages ai.h2o:sparkling-water-core_2.10:1.6.5 scala> Seq(1).toDS().write.format("parquet").save("foo") ``` Before: ``` java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at java.util.ServiceLoader.fail(ServiceLoader.java:232) at java.util.ServiceLoader.access$100(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) ... Caused by: java.lang.NoClassDefFoundError: org/apache/spark/Logging at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:760) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:467) at java.net.URLClassLoader.access$100(URLClassLoader.java:73) at java.net.URLClassLoader$1.run(URLClassLoader.java:368) at java.net.URLClassLoader$1.run(URLClassLoader.java:362) at java.security.AccessController.doPrivileged(Native Method) ... ``` After: ``` java.lang.ClassNotFoundException: Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at org.apache.spark.sql.execution.datasources.DataSource.lookupDataSource(DataSource.scala:178) at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:441) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:213) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:196) ... ``` Author: Shixiong Zhu <shixiong@databricks.com> Closes #14651 from zsxwing/SPARK-17065.
* [SPARK-16700][PYSPARK][SQL] create DataFrame from dict/Row with schemaDavies Liu2016-08-154-28/+62
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In 2.0, we verify the data type against schema for every row for safety, but with performance cost, this PR make it optional. When we verify the data type for StructType, it does not support all the types we support in infer schema (for example, dict), this PR fix that to make them consistent. For Row object which is created using named arguments, the order of fields are sorted by name, they may be not different than the order in provided schema, this PR fix that by ignore the order of fields in this case. ## How was this patch tested? Created regression tests for them. Author: Davies Liu <davies@databricks.com> Closes #14469 from davies/py_dict.
* [SPARK-16671][CORE][SQL] Consolidate code to do variable substitution.Marcelo Vanzin2016-08-159-228/+312
| | | | | | | | | | | | | | | | Both core and sql have slightly different code that does variable substitution of config values. This change refactors that code and encapsulates the logic of reading config values and expading variables in a new helper class, which can be configured so that both core and sql can use it without losing existing functionality, and allows for easier testing and makes it easier to add more features in the future. Tested with existing and new unit tests, and by running spark-shell with some configs referencing variables and making sure it behaved as expected. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14468 from vanzin/SPARK-16671.
* [SPARK-16508][SPARKR] Split docs for arrange and orderBy methodsJunyang Qian2016-08-154-15/+17
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR splits arrange and orderBy methods according to their functionality (the former for sorting sparkDataFrame and the latter for windowSpec). ## How was this patch tested? ![screen shot 2016-08-06 at 6 39 19 pm](https://cloud.githubusercontent.com/assets/15318264/17459969/51eade28-5c05-11e6-8ca1-8d8a8e344bab.png) ![screen shot 2016-08-06 at 6 39 29 pm](https://cloud.githubusercontent.com/assets/15318264/17459966/51e3c246-5c05-11e6-8d35-3e905ca48676.png) ![screen shot 2016-08-06 at 6 40 02 pm](https://cloud.githubusercontent.com/assets/15318264/17459967/51e650ec-5c05-11e6-8698-0f037f5199ff.png) Author: Junyang Qian <junyangq@databricks.com> Closes #14522 from junyangq/SPARK-16508-0.
* [SPARK-16934][ML][MLLIB] Update LogisticCostAggregator serialization code to ↵WeichenXu2016-08-151-16/+20
| | | | | | | | | | | | | | | | | | | make it consistent with LinearRegression ## What changes were proposed in this pull request? Update LogisticCostAggregator serialization code to make it consistent with #14109 ## How was this patch tested? MLlib 2.0: ![image](https://cloud.githubusercontent.com/assets/19235986/17649601/5e2a79ac-61ee-11e6-833c-3bd8b5250470.png) After this PR: ![image](https://cloud.githubusercontent.com/assets/19235986/17649599/52b002ae-61ee-11e6-9402-9feb3439880f.png) Author: WeichenXu <WeichenXu123@outlook.com> Closes #14520 from WeichenXu123/improve_logistic_regression_costfun.
* [TRIVIAL][ML] Fix LogisticRegression typo in error message.Yanbo Liang2016-08-151-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? Fix ```LogisticRegression``` typo in error message. ## How was this patch tested? Docs change, no new tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14633 from yanboliang/lr-typo.
* [SPARK-11714][MESOS] Make Spark on Mesos honor port restrictions on coarse ↵Stavros Kontopoulos2016-08-156-25/+336
| | | | | | | | | | | | | grain mode - Make mesos coarse grained scheduler accept port offers and pre-assign ports Previous attempt was for fine grained: https://github.com/apache/spark/pull/10808 Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com> Author: Stavros Kontopoulos <stavros.kontopoulos@typesafe.com> Closes #11157 from skonto/honour_ports_coarse.
* [WIP][MINOR][TYPO] Fix several trivival typosZhenglai Zhang2016-08-142-2/+2
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? * Fixed one typo `"overriden"` as `"overridden"`, also make sure no other same typo. * Fixed one typo `"lowcase"` as `"lowercase"`, also make sure no other same typo. ## How was this patch tested? Since the change is very tiny, so I just make sure compilation is successful. I am new to the spark community, please feel free to let me do other necessary steps. Thanks in advance! ---- Updated: Found another typo `lowcase` later and fixed then in the same patch Author: Zhenglai Zhang <zhenglaizhang@hotmail.com> Closes #14622 from zhenglaizhang/fixtypo.
* [SPARK-17027][ML] Avoid integer overflow in PolynomialExpansion.getPolySizezero3232016-08-142-4/+30
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Replaces custom choose function with o.a.commons.math3.CombinatoricsUtils.binomialCoefficient ## How was this patch tested? Spark unit tests Author: zero323 <zero323@users.noreply.github.com> Closes #14614 from zero323/SPARK-17027.
* [SPARK-16966][SQL][CORE] App Name is a randomUUID even when "spark.app.name" ↵Sean Owen2016-08-132-4/+8
| | | | | | | | | | | | | | | | | | | exists ## What changes were proposed in this pull request? Don't override app name specified in `SparkConf` with a random app name. Only set it if the conf has no app name even after options have been applied. See also https://github.com/apache/spark/pull/14602 This is similar to Sherry302 's original proposal in https://github.com/apache/spark/pull/14556 ## How was this patch tested? Jenkins test, with new case reproducing the bug Author: Sean Owen <sowen@cloudera.com> Closes #14630 from srowen/SPARK-16966.2.
* [SPARK-17023][BUILD] Upgrade to Kafka 0.10.0.1 releaseLuciano Resende2016-08-131-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? Update Kafka streaming connector to use Kafka 0.10.0.1 release ## How was this patch tested? Tested via Spark unit and integration tests Author: Luciano Resende <lresende@apache.org> Closes #14606 from lresende/kafka-upgrade.
* [SPARK-16968] Add additional options in jdbc when creating a new tableGraceH2016-08-133-12/+51
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In the PR, we just allow the user to add additional options when create a new table in JDBC writer. The options can be table_options or partition_options. E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" Here is the usage example: ``` df.write.option("createTableOptions", "ENGINE=InnoDB DEFAULT CHARSET=utf8").jdbc(...) ``` ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) will apply test result soon. Author: GraceH <93113783@qq.com> Closes #14559 from GraceH/jdbc_options.
* [MINOR][CORE] fix warnings on depreciated methods in ↵Xin Ren2016-08-132-5/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | MesosClusterSchedulerSuite and DiskBlockObjectWriterSuite ## What changes were proposed in this pull request? Fixed warnings below after scanning through warnings during build: ``` [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala:34: imported `Utils' is permanently hidden by definition of object Utils in package mesos [warn] import org.apache.spark.scheduler.cluster.mesos.Utils [warn] ^ ``` and ``` [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:113: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:119: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:131: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:135: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ ``` ## How was this patch tested? Tested manually on local laptop. Author: Xin Ren <iamshrek@126.com> Closes #14609 from keypointt/suiteWarnings.
* [SPARK-12370][DOCUMENTATION] Documentation should link to examples …Jagadeesan2016-08-136-28/+28
| | | | | | | | | | | | ## What changes were proposed in this pull request? When documentation is built is should reference examples from the same build. There are times when the docs have links that point to files in the GitHub head which may not be valid on the current release. Changed that in URLs to make them point to the right tag in git using ```SPARK_VERSION_SHORT``` …from its own release version] [Streaming programming guide] Author: Jagadeesan <as2@us.ibm.com> Closes #14596 from jagadeesanas2/SPARK-12370.
* [DOC] add config option spark.ui.enabled into documentWeichenXu2016-08-121-0/+7
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The configuration doc lost the config option `spark.ui.enabled` (default value is `true`) I think this option is important because many cases we would like to turn it off. so I add it. ## How was this patch tested? N/A Author: WeichenXu <WeichenXu123@outlook.com> Closes #14604 from WeichenXu123/add_doc_param_spark_ui_enabled.
* [SPARK-16771][SQL] WITH clause should not fall into infinite loop.Dongjoon Hyun2016-08-126-18/+88
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR changes the CTE resolving rule to use only **forward-declared** tables in order to prevent infinite loops. More specifically, new logic is like the following. * Resolve CTEs in `WITH` clauses first before replacing the main SQL body. * When resolving CTEs, only forward-declared CTEs or base tables are referenced. - Self-referencing is not allowed any more. - Cross-referencing is not allowed any more. **Reported Error Scenarios** ```scala scala> sql("WITH t AS (SELECT 1 FROM t) SELECT * FROM t") java.lang.StackOverflowError ... scala> sql("WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1, t2") java.lang.StackOverflowError ... ``` Note that `t`, `t1`, and `t2` are not declared in database. Spark falls into infinite loops before resolving table names. ## How was this patch tested? Pass the Jenkins tests with new two testcases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14397 from dongjoon-hyun/SPARK-16771-TREENODE.
* [SPARK-17033][ML][MLLIB] GaussianMixture should use treeAggregate to improve ↵Yanbo Liang2016-08-121-1/+2
| | | | | | | | | | | | | | | performance ## What changes were proposed in this pull request? ```GaussianMixture``` should use ```treeAggregate``` rather than ```aggregate``` to improve performance and scalability. In my test of dataset with 200 features and 1M instance, I found there is 20% increased performance. BTW, we should destroy broadcast variable ```compute``` at the end of each iteration. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14621 from yanboliang/spark-17033.
* [SPARK-16598][SQL][TEST] Added a test case for verifying the table ↵gatorsmile2016-08-121-0/+8
| | | | | | | | | | | | | | identifier parsing #### What changes were proposed in this pull request? So far, the test cases of `TableIdentifierParserSuite` do not cover the quoted cases. We should add one for avoiding regression. #### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #14244 from gatorsmile/quotedIdentifiers.
* [MINOR][DOC] Fix style in examples across documentationhyukjinkwon2016-08-125-47/+47
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes the documentation as below: - Python has 4 spaces and Java and Scala has 2 spaces (See https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide). - Avoid excessive parentheses and curly braces for anonymous functions. (See https://github.com/databricks/scala-style-guide#anonymous) ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #14593 from HyukjinKwon/minor-documentation.
* [SPARK-16985] Change dataFormat from yyyyMMddHHmm to yyyyMMddHHmmsshongshen2016-08-124-5/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In our cluster, sometimes the sql output maybe overrided. When I submit some sql, all insert into the same table, and the sql will cost less one minute, here is the detail, 1 sql1, 11:03 insert into table. 2 sql2, 11:04:11 insert into table. 3 sql3, 11:04:48 insert into table. 4 sql4, 11:05 insert into table. 5 sql5, 11:06 insert into table. The sql3's output file will override the sql2's output file. here is the log: ``` 16/05/04 11:04:11 INFO hive.SparkHiveHadoopWriter: XXfinalPath=hdfs://tl-sng-gdt-nn-tdw.tencent-distribute.com:54310/tmp/assorz/tdw-tdwadmin/20160504/04559505496526517_-1_1204544348/10000/_tmp.p_20160428/attempt_201605041104_0001_m_000000_1 16/05/04 11:04:48 INFO hive.SparkHiveHadoopWriter: XXfinalPath=hdfs://tl-sng-gdt-nn-tdw.tencent-distribute.com:54310/tmp/assorz/tdw-tdwadmin/20160504/04559505496526517_-1_212180468/10000/_tmp.p_20160428/attempt_201605041104_0001_m_000000_1 ``` The reason is the output file use SimpleDateFormat("yyyyMMddHHmm"), if two sql insert into the same table in the same minute, the output will be overrite. I think we should change dateFormat to "yyyyMMddHHmmss", in our cluster, we can't finished a sql in one second. ## 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) Author: hongshen <shenh062326@126.com> Closes #14574 from shenh062326/SPARK-16985.
* [SPARK-17013][SQL] Parse negative numeric literalspetermaxlee2016-08-115-55/+37
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch updates the SQL parser to parse negative numeric literals as numeric literals, instead of unary minus of positive literals. This allows the parser to parse the minimal value for each data type, e.g. "-32768S". ## How was this patch tested? Updated test cases. Author: petermaxlee <petermaxlee@gmail.com> Closes #14608 from petermaxlee/SPARK-17013.
* [SPARK-16975][SQL] Column-partition path starting '_' should be handled ↵Dongjoon Hyun2016-08-125-4/+14
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | correctly ## What changes were proposed in this pull request? Currently, Spark ignores path names starting with underscore `_` and `.`. This causes read-failures for the column-partitioned file data sources whose partition column names starts from '_', e.g. `_col`. **Before** ```scala scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet") scala> spark.read.parquet("/tmp/parquet") org.apache.spark.sql.AnalysisException: Unable to infer schema for ParquetFormat at /tmp/parquet20. It must be specified manually; ``` **After** ```scala scala> spark.range(10).withColumn("_locality_code", $"id").write.partitionBy("_locality_code").save("/tmp/parquet") scala> spark.read.parquet("/tmp/parquet") res2: org.apache.spark.sql.DataFrame = [id: bigint, _locality_code: int] ``` ## How was this patch tested? Pass the Jenkins with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14585 from dongjoon-hyun/SPARK-16975-PARQUET.
* [MINOR][ML] Rename TreeEnsembleModels to TreeEnsembleModel for PySparkYanbo Liang2016-08-112-6/+6
| | | | | | | | | | | | ## What changes were proposed in this pull request? Fix the typo of ```TreeEnsembleModels``` for PySpark, it should ```TreeEnsembleModel``` which will be consistent with Scala. What's more, it represents a tree ensemble model, so ```TreeEnsembleModel``` should be more reasonable. This should not be used public, so it will not involve breaking change. ## How was this patch tested? No new tests, should pass existing ones. Author: Yanbo Liang <ybliang8@gmail.com> Closes #14454 from yanboliang/TreeEnsembleModel.
* [SPARK-16434][SQL] Avoid per-record type dispatch in JSON when readinghyukjinkwon2016-08-125-216/+297
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `JacksonParser.parse` is doing type-based dispatch for each row to convert the tokens to appropriate values for Spark. It might not have to be done like this because the schema is already kept. So, appropriate converters can be created first according to the schema once, and then apply them to each row. This PR corrects `JacksonParser` so that it creates all converters for the schema once and then applies them to each row rather than type dispatching for every row. Benchmark was proceeded with the codes below: #### Parser tests **Before** ```scala test("Benchmark for JSON converter") { val N = 500 << 8 val row = """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" val data = List.fill(N)(row) val dummyOption = new JSONOptions(Map.empty[String, String]) val schema = InferSchema.infer(spark.sparkContext.parallelize(Seq(row)), "", dummyOption) val factory = new JsonFactory() val benchmark = new Benchmark("JSON converter", N) benchmark.addCase("convert JSON file", 10) { _ => data.foreach { input => val parser = factory.createParser(input) parser.nextToken() JacksonParser.convertRootField(factory, parser, schema) } } benchmark.run() } ``` ``` JSON converter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ convert JSON file 1697 / 1807 0.1 13256.9 1.0X ``` **After** ```scala test("Benchmark for JSON converter") { val N = 500 << 8 val row = """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" val data = List.fill(N)(row) val dummyOption = new JSONOptions(Map.empty[String, String], new SQLConf()) val schema = InferSchema.infer(spark.sparkContext.parallelize(Seq(row)), dummyOption) val benchmark = new Benchmark("JSON converter", N) benchmark.addCase("convert JSON file", 10) { _ => val parser = new JacksonParser(schema, dummyOption) data.foreach { input => parser.parse(input) } } benchmark.run() } ``` ``` JSON converter: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ convert JSON file 1401 / 1461 0.1 10947.4 1.0X ``` It seems parsing time is improved by roughly ~20% #### End-to-End test ```scala test("Benchmark for JSON reader") { val N = 500 << 8 val row = """{"struct":{"field1": true, "field2": 92233720368547758070}, "structWithArrayFields":{"field1":[4, 5, 6], "field2":["str1", "str2"]}, "arrayOfString":["str1", "str2"], "arrayOfInteger":[1, 2147483647, -2147483648], "arrayOfLong":[21474836470, 9223372036854775807, -9223372036854775808], "arrayOfBigInteger":[922337203685477580700, -922337203685477580800], "arrayOfDouble":[1.2, 1.7976931348623157E308, 4.9E-324, 2.2250738585072014E-308], "arrayOfBoolean":[true, false, true], "arrayOfNull":[null, null, null, null], "arrayOfStruct":[{"field1": true, "field2": "str1"}, {"field1": false}, {"field3": null}], "arrayOfArray1":[[1, 2, 3], ["str1", "str2"]], "arrayOfArray2":[[1, 2, 3], [1.1, 2.1, 3.1]] }""" val df = spark.sqlContext.read.json(spark.sparkContext.parallelize(List.fill(N)(row))) withTempPath { path => df.write.format("json").save(path.getCanonicalPath) val benchmark = new Benchmark("JSON reader", N) benchmark.addCase("reading JSON file", 10) { _ => spark.read.format("json").load(path.getCanonicalPath).collect() } benchmark.run() } } ``` **Before** ``` JSON reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading JSON file 6485 / 6924 0.0 50665.0 1.0X ``` **After** ``` JSON reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading JSON file 6350 / 6529 0.0 49609.3 1.0X ``` ## How was this patch tested? Existing test cases should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14102 from HyukjinKwon/SPARK-16434.
* [SPARK-13081][PYSPARK][SPARK_SUBMIT] Allow set pythonExec of driver and ↵Jeff Zhang2016-08-118-8/+72
| | | | | | | | | | | | executor through conf… Before this PR, user have to export environment variable to specify the python of driver & executor which is not so convenient for users. This PR is trying to allow user to specify python through configuration "--pyspark-driver-python" & "--pyspark-executor-python" Manually test in local & yarn mode for pyspark-shell and pyspark batch mode. Author: Jeff Zhang <zjffdu@apache.org> Closes #13146 from zjffdu/SPARK-13081.
* [SPARK-17022][YARN] Handle potential deadlock in driver handling messagesWangTaoTheTonic2016-08-111-3/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We directly send RequestExecutors to AM instead of transfer it to yarnShedulerBackend first, to avoid potential deadlock. ## How was this patch tested? manual tests Author: WangTaoTheTonic <wangtao111@huawei.com> Closes #14605 from WangTaoTheTonic/lock.
* [SPARK-16868][WEB UI] Fix executor be both dead and alive on executor ui.huangzhaowei2016-08-111-0/+4
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In a heavy pressure of the spark application, since the executor will register it to driver block manager twice(because of heart beats), the executor will show as picture show: ![image](https://cloud.githubusercontent.com/assets/7404824/17467245/c1359094-5d4e-11e6-843a-f6d6347e1bf6.png) ## How was this patch tested? NA Details in: [SPARK-16868](https://issues.apache.org/jira/browse/SPARK-16868) Author: huangzhaowei <carlmartinmax@gmail.com> Closes #14530 from SaintBacchus/SPARK-16868.
* [SPARK-13602][CORE] Add shutdown hook to DriverRunner to prevent driver ↵Bryan Cutler2016-08-112-50/+142
| | | | | | | | | | | | | | | | | | process leak ## What changes were proposed in this pull request? Added shutdown hook to DriverRunner to kill the driver process in case the Worker JVM exits suddenly and the `WorkerWatcher` was unable to properly catch this. Did some cleanup to consolidate driver state management and setting of finalized vars within the running thread. ## How was this patch tested? Added unit tests to verify that final state and exception variables are set accordingly for successfull, failed, and errors in the driver process. Retrofitted existing test to verify killing of mocked process ends with the correct state and stops properly Manually tested (with deploy-mode=cluster) that the shutdown hook is called by forcibly exiting the `Worker` and various points in the code with the `WorkerWatcher` both disabled and enabled. Also, manually killed the driver through the ui and verified that the `DriverRunner` interrupted, killed the process and exited properly. Author: Bryan Cutler <cutlerb@gmail.com> Closes #11746 from BryanCutler/DriverRunner-shutdown-hook-SPARK-13602.
* [SPARK-17018][SQL] literals.sql for testing literal parsingpetermaxlee2016-08-115-62/+476
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds literals.sql for testing literal parsing end-to-end in SQL. ## How was this patch tested? The patch itself is only about adding test cases. Author: petermaxlee <petermaxlee@gmail.com> Closes #14598 from petermaxlee/SPARK-17018-2.
* [SPARK-17021][SQL] simplify the constructor parameters of QuantileSummariesWenchen Fan2016-08-111-10/+11
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1. `sampled` doesn't need to be `ArrayBuffer`, we never update it, but assign new value 2. `count` doesn't need to be `var`, we never mutate it. 3. `headSampled` doesn't need to be in constructor, we never pass a non-empty `headSampled` to constructor ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14603 from cloud-fan/simply.
* [SPARK-16958] [SQL] Reuse subqueries within the same queryDavies Liu2016-08-117-49/+215
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? There could be multiple subqueries that generate same results, we could re-use the result instead of running it multiple times. This PR also cleanup up how we run subqueries. For SQL query ```sql select id,(select avg(id) from t) from t where id > (select avg(id) from t) ``` The explain is ``` == Physical Plan == *Project [id#15L, Subquery subquery29 AS scalarsubquery()#35] : +- Subquery subquery29 : +- *HashAggregate(keys=[], functions=[avg(id#15L)]) : +- Exchange SinglePartition : +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)]) : +- *Range (0, 1000, splits=4) +- *Filter (cast(id#15L as double) > Subquery subquery29) : +- Subquery subquery29 : +- *HashAggregate(keys=[], functions=[avg(id#15L)]) : +- Exchange SinglePartition : +- *HashAggregate(keys=[], functions=[partial_avg(id#15L)]) : +- *Range (0, 1000, splits=4) +- *Range (0, 1000, splits=4) ``` The visualized plan: ![reuse-subquery](https://cloud.githubusercontent.com/assets/40902/17573229/e578d93c-5f0d-11e6-8a3c-0150d81d3aed.png) ## How was this patch tested? Existing tests. Author: Davies Liu <davies@databricks.com> Closes #14548 from davies/subq.
* [SPARK-16952] don't lookup spark home directory when executor uri is setMichael Gummelt2016-08-112-7/+26
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? remove requirement to set spark.mesos.executor.home when spark.executor.uri is used ## How was this patch tested? unit tests Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #14552 from mgummelt/fix-spark-home.
* [SPARK-16886][EXAMPLES][DOC] Fix some examples to be consistent and ↵hyukjinkwon2016-08-115-122/+124
| | | | | | | | | | | | | | | | | | | | | | | | | | | indentation in documentation ## What changes were proposed in this pull request? Originally this PR was based on #14491 but I realised that fixing examples are more sensible rather than comments. This PR fixes three things below: - Fix two wrong examples in `structured-streaming-programming-guide.md`. Loading via `read.load(..)` without `as` will be `Dataset<Row>` not `Dataset<String>` in Java. - Fix indentation across `structured-streaming-programming-guide.md`. Python has 4 spaces and Scala and Java have double spaces. These are inconsistent across the examples. - Fix `StructuredNetworkWordCountWindowed` and `StructuredNetworkWordCount` in Java and Scala to initially load `DataFrame` and `Dataset<Row>` to be consistent with the comments and some examples in `structured-streaming-programming-guide.md` and to match Scala and Java to Python one (Python one loads it as `DataFrame` initially). ## How was this patch tested? N/A Closes https://github.com/apache/spark/pull/14491 Author: hyukjinkwon <gurwls223@gmail.com> Author: Ganesh Chand <ganeshchand@Ganeshs-MacBook-Pro-2.local> Closes #14564 from HyukjinKwon/SPARK-16886.