aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17817][PYSPARK] PySpark RDD Repartitioning Results in Highly Skewed ↵Liang-Chi Hsieh2016-10-112-3/+20
| | | | | | | | | | | | | | | | | | | | | | | | | | | Partition Sizes ## What changes were proposed in this pull request? Quoted from JIRA description: Calling repartition on a PySpark RDD to increase the number of partitions results in highly skewed partition sizes, with most having 0 rows. The repartition method should evenly spread out the rows across the partitions, and this behavior is correctly seen on the Scala side. Please reference the following code for a reproducible example of this issue: num_partitions = 20000 a = sc.parallelize(range(int(1e6)), 2) # start with 2 even partitions l = a.repartition(num_partitions).glom().map(len).collect() # get length of each partition min(l), max(l), sum(l)/len(l), len(l) # skewed! In Scala's `repartition` code, we will distribute elements evenly across output partitions. However, the RDD from Python is serialized as a single binary data, so the distribution fails. We need to convert the RDD in Python to java object before repartitioning. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #15389 from viirya/pyspark-rdd-repartition.
* [SPARK-17346][SQL][TESTS] Fix the flaky topic deletion in KafkaSourceStressSuiteShixiong Zhu2016-10-111-8/+1
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? A follow up Pr for SPARK-17346 to fix flaky `org.apache.spark.sql.kafka010.KafkaSourceStressSuite`. Test log: https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.4/1855/testReport/junit/org.apache.spark.sql.kafka010/KafkaSourceStressSuite/_It_is_not_a_test_/ Looks like deleting the Kafka internal topic `__consumer_offsets` is flaky. This PR just simply ignores internal topics. ## How was this patch tested? Existing tests. Author: Shixiong Zhu <shixiong@databricks.com> Closes #15384 from zsxwing/SPARK-17346-flaky-test.
* [SPARK-17821][SQL] Support And and Or in Expression CanonicalizeLiang-Chi Hsieh2016-10-112-0/+89
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently `Canonicalize` object doesn't support `And` and `Or`. So we can compare canonicalized form of predicates consistently. We should add the support. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #15388 from viirya/canonicalize-and-or.
* [SPARK-17864][SQL] Mark data type APIs as stable (not DeveloperApi)Reynold Xin2016-10-1131-94/+207
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The data type API has not been changed since Spark 1.3.0, and is ready for graduation. This patch marks them as stable APIs using the new InterfaceStability annotation. This patch also looks at the various files in the catalyst module (not the "package") and marks the remaining few classes appropriately as well. ## How was this patch tested? This is an annotation change. No functional changes. Author: Reynold Xin <rxin@databricks.com> Closes #15426 from rxin/SPARK-17864.
* [SPARK-17338][SQL][FOLLOW-UP] add global temp viewWenchen Fan2016-10-116-13/+24
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? address post hoc review comments for https://github.com/apache/spark/pull/14897 ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #15424 from cloud-fan/global-temp-view.
* [SPARK-17808][PYSPARK] Upgraded version of Pyrolite to 4.13Bryan Cutler2016-10-117-6/+14
| | | | | | | | | | | | ## What changes were proposed in this pull request? Upgraded to a newer version of Pyrolite which supports serialization of a BinaryType StructField for PySpark.SQL ## How was this patch tested? Added a unit test which fails with a raised ValueError when using the previous version of Pyrolite 4.9 and Python3 Author: Bryan Cutler <cutlerb@gmail.com> Closes #15386 from BryanCutler/pyrolite-upgrade-SPARK-17808.
* [SPARK-15957][ML] RFormula supports forcing to index labelYanbo Liang2016-10-102-4/+52
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? ```RFormula``` will index label only when it is string type currently. If the label is numeric type and we use ```RFormula``` to present a classification model, there is no label attributes in label column metadata. The label attributes are useful when making prediction for classification, so we can force to index label by ```StringIndexer``` whether it is numeric or string type for classification. Then SparkR wrappers can extract label attributes from label column metadata successfully. This feature can help us to fix bug similar with [SPARK-15153](https://issues.apache.org/jira/browse/SPARK-15153). For regression, we will still to keep label as numeric type. In this PR, we add a param ```indexLabel``` to control whether to force to index label for ```RFormula```. ## How was this patch tested? Unit tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #13675 from yanboliang/spark-15957.
* [SPARK-17844] Simplify DataFrame API for defining frame boundaries in window ↵Reynold Xin2016-10-106-10/+119
| | | | | | | | | | | | | | | | | | | | | | | | | | | | functions ## What changes were proposed in this pull request? When I was creating the example code for SPARK-10496, I realized it was pretty convoluted to define the frame boundaries for window functions when there is no partition column or ordering column. The reason is that we don't provide a way to create a WindowSpec directly with the frame boundaries. We can trivially improve this by adding rowsBetween and rangeBetween to Window object. As an example, to compute cumulative sum using the natural ordering, before this pr: ``` df.select('key, sum("value").over(Window.partitionBy(lit(1)).rowsBetween(Long.MinValue, 0))) ``` After this pr: ``` df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0))) ``` Note that you could argue there is no point specifying a window frame without partitionBy/orderBy -- but it is strange that only rowsBetween and rangeBetween are not the only two APIs not available. This also fixes https://issues.apache.org/jira/browse/SPARK-17656 (removing _root_.scala). ## How was this patch tested? Added test cases to compute cumulative sum in DataFrameWindowSuite for Scala/Java and tests.py for Python. Author: Reynold Xin <rxin@databricks.com> Closes #15412 from rxin/SPARK-17844.
* [SPARK-17719][SPARK-17776][SQL] Unify and tie up options in a single place ↵hyukjinkwon2016-10-1010-137/+182
| | | | | | | | | | | | | | | | | | | | | | | | | | | | in JDBC datasource package ## What changes were proposed in this pull request? This PR proposes to fix arbitrary usages among `Map[String, String]`, `Properties` and `JDBCOptions` instances for options in `execution/jdbc` package and make the connection properties exclude Spark-only options. This PR includes some changes as below: - Unify `Map[String, String]`, `Properties` and `JDBCOptions` in `execution/jdbc` package to `JDBCOptions`. - Move `batchsize`, `fetchszie`, `driver` and `isolationlevel` options into `JDBCOptions` instance. - Document `batchSize` and `isolationlevel` with marking both read-only options and write-only options. Also, this includes minor types and detailed explanation for some statements such as url. - Throw exceptions fast by checking arguments first rather than in execution time (e.g. for `fetchsize`). - Exclude Spark-only options in connection properties. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15292 from HyukjinKwon/SPARK-17719.
* [SPARK-17816][CORE] Fix ConcurrentModificationException issue in ↵Ergin Seyfe2016-10-103-42/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | BlockStatusesAccumulator ## What changes were proposed in this pull request? Change the BlockStatusesAccumulator to return immutable object when value method is called. ## How was this patch tested? Existing tests plus I verified this change by running a pipeline which consistently repro this issue. This is the stack trace for this exception: ` java.util.ConcurrentModificationException at java.util.ArrayList$Itr.checkForComodification(ArrayList.java:901) at java.util.ArrayList$Itr.next(ArrayList.java:851) at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:43) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59) at scala.collection.mutable.ListBuffer.$plus$plus$eq(ListBuffer.scala:183) at scala.collection.mutable.ListBuffer.$plus$plus$eq(ListBuffer.scala:45) at scala.collection.TraversableLike$class.to(TraversableLike.scala:590) at scala.collection.AbstractTraversable.to(Traversable.scala:104) at scala.collection.TraversableOnce$class.toList(TraversableOnce.scala:294) at scala.collection.AbstractTraversable.toList(Traversable.scala:104) at org.apache.spark.util.JsonProtocol$.accumValueToJson(JsonProtocol.scala:314) at org.apache.spark.util.JsonProtocol$$anonfun$accumulableInfoToJson$5.apply(JsonProtocol.scala:291) at org.apache.spark.util.JsonProtocol$$anonfun$accumulableInfoToJson$5.apply(JsonProtocol.scala:291) at scala.Option.map(Option.scala:146) at org.apache.spark.util.JsonProtocol$.accumulableInfoToJson(JsonProtocol.scala:291) at org.apache.spark.util.JsonProtocol$$anonfun$taskInfoToJson$12.apply(JsonProtocol.scala:283) at org.apache.spark.util.JsonProtocol$$anonfun$taskInfoToJson$12.apply(JsonProtocol.scala:283) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234) at scala.collection.immutable.List.foreach(List.scala:381) at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35) at scala.collection.mutable.ListBuffer.foreach(ListBuffer.scala:45) at scala.collection.TraversableLike$class.map(TraversableLike.scala:234) at scala.collection.AbstractTraversable.map(Traversable.scala:104) at org.apache.spark.util.JsonProtocol$.taskInfoToJson(JsonProtocol.scala:283) at org.apache.spark.util.JsonProtocol$.taskEndToJson(JsonProtocol.scala:145) at org.apache.spark.util.JsonProtocol$.sparkEventToJson(JsonProtocol.scala:76) ` Author: Ergin Seyfe <eseyfe@fb.com> Closes #15371 from seyfe/race_cond_jsonprotocal.
* [SPARK-16896][SQL] Handle duplicated field names in header consistently with ↵hyukjinkwon2016-10-113-11/+74
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | null or empty strings in CSV ## What changes were proposed in this pull request? Currently, CSV datasource allows to load duplicated empty string fields or fields having `nullValue` in the header. It'd be great if this can deal with normal fields as well. This PR proposes handling the duplicates consistently with the existing behaviour with considering case-sensitivity (`spark.sql.caseSensitive`) as below: data below: ``` fieldA,fieldB,,FIELDA,fielda,, 1,2,3,4,5,6,7 ``` is parsed as below: ```scala spark.read.format("csv").option("header", "true").load("test.csv").show() ``` - when `spark.sql.caseSensitive` is `false` (by default). ``` +-------+------+---+-------+-------+---+---+ |fieldA0|fieldB|_c2|FIELDA3|fieldA4|_c5|_c6| +-------+------+---+-------+-------+---+---+ | 1| 2| 3| 4| 5| 6| 7| +-------+------+---+-------+-------+---+---+ ``` - when `spark.sql.caseSensitive` is `true`. ``` +-------+------+---+-------+-------+---+---+ |fieldA0|fieldB|_c2| FIELDA|fieldA4|_c5|_c6| +-------+------+---+-------+-------+---+---+ | 1| 2| 3| 4| 5| 6| 7| +-------+------+---+-------+-------+---+---+ ``` **In more details**, There is a good reference about this problem, `read.csv()` in R. So, I initially wanted to propose the similar behaviour. In case of R, the CSV data below: ``` fieldA,fieldB,,fieldA,fieldA,, 1,2,3,4,5,6,7 ``` is parsed as below: ```r test <- read.csv(file="test.csv",header=TRUE,sep=",") > test fieldA fieldB X fieldA.1 fieldA.2 X.1 X.2 1 1 2 3 4 5 6 7 ``` However, Spark CSV datasource already is handling duplicated empty strings and `nullValue` as field names. So the data below: ``` ,,,fieldA,,fieldB, 1,2,3,4,5,6,7 ``` is parsed as below: ```scala spark.read.format("csv").option("header", "true").load("test.csv").show() ``` ``` +---+---+---+------+---+------+---+ |_c0|_c1|_c2|fieldA|_c4|fieldB|_c6| +---+---+---+------+---+------+---+ | 1| 2| 3| 4| 5| 6| 7| +---+---+---+------+---+------+---+ ``` R starts the number for each duplicate but Spark adds the number for its position for all fields for `nullValue` and empty strings. In terms of case-sensitivity, it seems R is case-sensitive as below: (it seems it is not configurable). ``` a,a,a,A,A 1,2,3,4,5 ``` is parsed as below: ```r test <- read.csv(file="test.csv",header=TRUE,sep=",") > test a a.1 a.2 A A.1 1 1 2 3 4 5 ``` ## How was this patch tested? Unit test in `CSVSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14745 from HyukjinKwon/SPARK-16896.
* [SPARK-17738][TEST] Fix flaky test in ColumnTypeSuiteDavies Liu2016-10-101-3/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The default buffer size is not big enough for randomly generated MapType. ## How was this patch tested? Ran the tests in 100 times, it never fail (it fail 8 times before the patch). Author: Davies Liu <davies@databricks.com> Closes #15395 from davies/flaky_map.
* [SPARK-14610][ML] Remove superfluous split for continuous features in ↵sethah2016-10-102-23/+52
| | | | | | | | | | | | | | | | decision tree training ## What changes were proposed in this pull request? A nonsensical split is produced from method `findSplitsForContinuousFeature` for decision trees. This PR removes the superfluous split and updates unit tests accordingly. Additionally, an assertion to check that the number of found splits is `> 0` is removed, and instead features with zero possible splits are ignored. ## How was this patch tested? A unit test was added to check that finding splits for a constant feature produces an empty array. Author: sethah <seth.hendrickson16@gmail.com> Closes #12374 from sethah/SPARK-14610.
* [SPARK-14082][MESOS] Enable GPU support with MesosTimothy Chen2016-10-105-23/+96
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Enable GPU resources to be used when running coarse grain mode with Mesos. ## How was this patch tested? Manual test with GPU. Author: Timothy Chen <tnachen@gmail.com> Closes #14644 from tnachen/gpu_mesos.
* [SPARK-17828][DOCS] Remove unused generate-changelist.pyAdam Roberts2016-10-101-148/+0
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? We can remove this file based on discussion at https://issues.apache.org/jira/browse/SPARK-17828 it's evident this file has been redundant for a while, JIRA release notes serves this purpose for us already. For ease of future reference you can find detailed release notes at, for example: http://spark.apache.org/downloads.html -> http://spark.apache.org/releases/spark-release-2-0-1.html -> "Detailed changes" which links to https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315420&version=12336857 ## How was this patch tested? Searched the codebase and saw nothing referencing this, hasn't been used in a while (probably manually invoked a long time ago) Author: Adam Roberts <aroberts@uk.ibm.com> Closes #15419 from a-roberts/patch-7.
* [SPARK-17830] Annotate spark.sql package with InterfaceStabilityReynold Xin2016-10-1019-22/+107
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch annotates the InterfaceStability level for top level classes in o.a.spark.sql and o.a.spark.sql.util packages, to experiment with this new annotation. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #15392 from rxin/SPARK-17830.
* [SPARK-17417][CORE] Fix # of partitions for Reliable RDD checkpointingDhruve Ashar2016-10-101-2/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? Currently the no. of partition files are limited to 10000 files (%05d format). If there are more than 10000 part files, the logic goes for a toss while recreating the RDD as it sorts them by string. More details can be found in the JIRA desc [here](https://issues.apache.org/jira/browse/SPARK-17417). ## How was this patch tested? I tested this patch by checkpointing a RDD and then manually renaming part files to the old format and tried to access the RDD. It was successfully created from the old format. Also verified loading a sample parquet file and saving it as multiple formats - CSV, JSON, Text, Parquet, ORC and read them successfully back from the saved files. I couldn't launch the unit test from my local box, so will wait for the Jenkins output. Author: Dhruve Ashar <dhruveashar@gmail.com> Closes #15370 from dhruve/bug/SPARK-17417.
* [HOT-FIX][SQL][TESTS] Remove unused function in `SparkSqlParserSuite`jiangxingbo2016-10-101-10/+0
| | | | | | | | | | | | | ## What changes were proposed in this pull request? The function `SparkSqlParserSuite.createTempViewUsing` is not used for now and causes build failure, this PR simply removes it. ## How was this patch tested? N/A Author: jiangxingbo <jiangxb1987@gmail.com> Closes #15418 from jiangxb1987/parserSuite.
* [SPARK-17338][SQL] add global temp viewWenchen Fan2016-10-1034-230/+906
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1. changes for `SessionCatalog`: 1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name. 2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved. 3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved. 4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views. 5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view. 6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views. 7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views. changes for SQL commands: 1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views 2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views. 3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc. changes for other public API 1. add a new method `dropGlobalTempView` in `Catalog` 2. `Catalog.findTable` can find global temp view 3. add a new method `createGlobalTempView` in `Dataset` ## How was this patch tested? new tests in `SQLViewSuite` Author: Wenchen Fan <wenchen@databricks.com> Closes #14897 from cloud-fan/global-temp-view.
* [SPARK-17741][SQL] Grammar to parse top level and nested data fields separatelyjiangxingbo2016-10-097-23/+195
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently we use the same rule to parse top level and nested data fields. For example: ``` create table tbl_x( id bigint, nested struct<col1:string,col2:string> ) ``` Shows both syntaxes. In this PR we split this rule in a top-level and nested rule. Before this PR, ``` sql("CREATE TABLE my_tab(column1: INT)") ``` works fine. After this PR, it will throw a `ParseException`: ``` scala> sql("CREATE TABLE my_tab(column1: INT)") org.apache.spark.sql.catalyst.parser.ParseException: no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27) ``` ## How was this patch tested? Add new testcases in `SparkSqlParserSuite`. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #15346 from jiangxb1987/cdt.
* [SPARK-17832][SQL] TableIdentifier.quotedString creates un-parseable names ↵jiangxingbo2016-10-093-3/+29
| | | | | | | | | | | | | | | | | | | | | | | | | | when name contains a backtick ## What changes were proposed in this pull request? The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example: ``` import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1")) parseTableIdentifier(complexName.unquotedString) // Does not work parseTableIdentifier(complexName.quotedString) // Does not work parseExpression(complexName.unquotedString) // Does not work parseExpression(complexName.quotedString) // Does not work ``` We should handle the backtick properly to make `quotedString` parseable. ## How was this patch tested? Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #15403 from jiangxb1987/backtick.
* [MINOR][SQL] Use resource path for test_script.shWeiqing Yang2016-10-086-12/+28
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR modified the test case `test("script")` to use resource path for `test_script.sh`. Make the test case portable (even in IntelliJ). ## How was this patch tested? Passed the test case. Before: Run `test("script")` in IntelliJ: ``` Caused by: org.apache.spark.SparkException: Subprocess exited with status 127. Error: bash: src/test/resources/test_script.sh: No such file or directory ``` After: Test passed. Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #15246 from weiqingy/hivetest.
* [SPARK-17768][CORE] Small (Sum,Count,Mean)Evaluator problems and suboptimalitiesSean Owen2016-10-0810-332/+203
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix: - GroupedMeanEvaluator and GroupedSumEvaluator are unused, as is the StudentTCacher support class - CountEvaluator can return a lower bound < 0, when counts can't be negative - MeanEvaluator will actually fail on exactly 1 datum (yields t-test with 0 DOF) - CountEvaluator uses a normal distribution, which may be an inappropriate approximation (leading to above) - Test for SumEvaluator asserts incorrect expected sums – e.g. after observing 10% of data has sum of 2, expectation should be 20, not 38 - CountEvaluator, MeanEvaluator have no unit tests to catch these - Duplication of distribution code across CountEvaluator, GroupedCountEvaluator - The stats in each could use a bit of documentation as I had to guess at them - (Code could use a few cleanups and optimizations too) ## How was this patch tested? Existing and new tests Author: Sean Owen <sowen@cloudera.com> Closes #15341 from srowen/SPARK-17768.
* [SPARK-17793][WEB UI] Sorting on the description on the Job or Stage page ↵Alex Bozarth2016-10-084-188/+49
| | | | | | | | | | | | | | | | | | | | | | | doesn’t always work ## What changes were proposed in this pull request? Added secondary sorting on stage name for the description column. This provide a clearer behavior in the common case where the Description column only comprises of Stage names instead of the option description value. ## How was this patch tested? manual testing and dev/run-tests Screenshots of sorting on both description and stage name as well as an example of both: ![screen shot 2016-10-04 at 1 09 39 pm](https://cloud.githubusercontent.com/assets/13952758/19135523/067b042e-8b1a-11e6-912e-e6371d006d21.png) ![screen shot 2016-10-04 at 1 09 51 pm](https://cloud.githubusercontent.com/assets/13952758/19135526/06960936-8b1a-11e6-85e9-8aaf694c5f7b.png) ![screen shot 2016-10-05 at 1 14 45 pm](https://cloud.githubusercontent.com/assets/13952758/19135525/069547da-8b1a-11e6-8692-6524c75c4c07.png) ![screen shot 2016-10-05 at 1 14 51 pm](https://cloud.githubusercontent.com/assets/13952758/19135524/0694b4d2-8b1a-11e6-92dc-c8aa514e4f62.png) ![screen shot 2016-10-05 at 4 42 52 pm](https://cloud.githubusercontent.com/assets/13952758/19135618/e232eafe-8b1a-11e6-88b3-ff0bbb26b7f8.png) Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #15366 from ajbozarth/spark17793.
* [MINOR][ML] remove redundant comment in LogisticRegressionwm624@hotmail.com2016-10-071-1/+0
| | | | | | | | | | | | ## What changes were proposed in this pull request? While adding R wrapper for LogisticRegression, I found one extra comment. It is minor and I just remove it. ## How was this patch tested? Unit tests Author: wm624@hotmail.com <wm624@hotmail.com> Closes #15391 from wangmiao1981/mlordoc.
* [HOTFIX][BUILD] Do not use contains in Option in JdbcRelationProviderhyukjinkwon2016-10-071-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes the fix the use of `contains` API which only exists from Scala 2.11. ## How was this patch tested? Manually checked: ```scala scala> val o: Option[Boolean] = None o: Option[Boolean] = None scala> o == Some(false) res17: Boolean = false scala> val o: Option[Boolean] = Some(true) o: Option[Boolean] = Some(true) scala> o == Some(false) res18: Boolean = false scala> val o: Option[Boolean] = Some(false) o: Option[Boolean] = Some(false) scala> o == Some(false) res19: Boolean = true ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #15393 from HyukjinKwon/hotfix.
* [SPARK-17806] [SQL] fix bug in join key rewritten in HashJoinDavies Liu2016-10-072-33/+79
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? In HashJoin, we try to rewrite the join key as Long to improve the performance of finding a match. The rewriting part is not well tested, has a bug that could cause wrong result when there are at least three integral columns in the joining key also the total length of the key exceed 8 bytes. ## How was this patch tested? Added unit test to covering the rewriting with different number of columns and different data types. Manually test the reported case and confirmed that this PR fix the bug. Author: Davies Liu <davies@databricks.com> Closes #15390 from davies/rewrite_key.
* [SPARK-17761][SQL] Remove MutableRowHerman van Hovell2016-10-0771-347/+343
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`. The code below illustrates the immutability issue with InternalRow: ```scala import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericMutableRow val struct = new GenericMutableRow(1) val row = InternalRow(struct, 1) println(row) scala> [[null], 1] struct.setInt(0, 42) println(row) scala> [[42], 1] ``` This might be somewhat controversial, so feedback is appreciated. ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15333 from hvanhovell/SPARK-17761.
* [SPARK-15621][SQL] Support spilling for Python UDFDavies Liu2016-10-073-7/+436
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? When execute a Python UDF, we buffer the input row into as queue, then pull them out to join with the result from Python UDF. In the case that Python UDF is slow or the input row is too wide, we could ran out of memory because of the queue. Since we can't flush all the buffers (sockets) between JVM and Python process from JVM side, we can't limit the rows in the queue, otherwise it could deadlock. This PR will manage the memory used by the queue, spill that into disk when there is no enough memory (also release the memory and disk space as soon as possible). ## How was this patch tested? Added unit tests. Also manually ran a workload with large input row and slow python UDF (with large broadcast) like this: ``` b = range(1<<24) add = udf(lambda x: x + len(b), IntegerType()) df = sqlContext.range(1, 1<<26, 1, 4) print df.select(df.id, lit("adf"*10000).alias("s"), add(df.id).alias("add")).groupBy(length("s")).sum().collect() ``` It ran out of memory (hang because of full GC) before the patch, ran smoothly after the patch. Author: Davies Liu <davies@databricks.com> Closes #15089 from davies/spill_udf.
* [SPARK-17665][SPARKR] Support options/mode all for read/write APIs and ↵hyukjinkwon2016-10-076-22/+160
| | | | | | | | | | | | | | | | | | | | options in other types ## What changes were proposed in this pull request? This PR includes the changes below: - Support `mode`/`options` in `read.parquet`, `write.parquet`, `read.orc`, `write.orc`, `read.text`, `write.text`, `read.json` and `write.json` APIs - Support other types (logical, numeric and string) as options for `write.df`, `read.df`, `read.parquet`, `write.parquet`, `read.orc`, `write.orc`, `read.text`, `write.text`, `read.json` and `write.json` ## How was this patch tested? Unit tests in `test_sparkSQL.R`/ `utils.R`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15239 from HyukjinKwon/SPARK-17665.
* [SPARK-16411][SQL][STREAMING] Add textFile to Structured Streaming.Prashant Sharma2016-10-072-1/+50
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Adds the textFile API which exists in DataFrameReader and serves same purpose. ## How was this patch tested? Added corresponding testcase. Author: Prashant Sharma <prashsh1@in.ibm.com> Closes #14087 from ScrapCodes/textFile.
* [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProviderhyukjinkwon2016-10-074-52/+74
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes cleaning up the confusing part in `createRelation` as discussed in https://github.com/apache/spark/pull/12601/files#r80627940 Also, this PR proposes the changes below: - Add documentation for `batchsize` and `isolationLevel`. - Move property names into `JDBCOptions` so that they can be managed in a single place. which were, `fetchsize`, `batchsize`, `isolationLevel` and `driver`. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Closes #15263 from HyukjinKwon/SPARK-14525.
* [SPARK-17707][WEBUI] Web UI prevents spark-submit application to be finishedSean Owen2016-10-074-9/+48
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This expands calls to Jetty's simple `ServerConnector` constructor to explicitly specify a `ScheduledExecutorScheduler` that makes daemon threads. It should otherwise result in exactly the same configuration, because the other args are copied from the constructor that is currently called. (I'm not sure we should change the Hive Thriftserver impl, but I did anyway.) This also adds `sc.stop()` to the quick start guide example. ## How was this patch tested? Existing tests; _pending_ at least manual verification of the fix. Author: Sean Owen <sowen@cloudera.com> Closes #15381 from srowen/SPARK-17707.
* [SPARK-17800] Introduce InterfaceStability annotationReynold Xin2016-10-071-0/+49
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch introduces three new annotations under InterfaceStability: - Stable - Evolving - Unstable This is inspired by Hadoop's InterfaceStability, and the first step towards switching over to a new API stability annotation framework. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #15374 from rxin/SPARK-17800.
* [SPARK-16827] Stop reporting spill metrics as shuffle metricsBrian Cho2016-10-071-1/+3
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix a bug where spill metrics were being reported as shuffle metrics. Eventually these spill metrics should be reported (SPARK-3577), but separate from shuffle metrics. The fix itself basically reverts the line to what it was in 1.6. ## How was this patch tested? Tested on a job that was reporting shuffle writes even for the final stage, when no shuffle writes should take place. After the change the job no longer shows these writes. Before: ![screen shot 2016-10-03 at 6 39 59 pm](https://cloud.githubusercontent.com/assets/1514239/19085897/dbf59a92-8a20-11e6-9f68-a978860c0d74.png) After: <img width="1052" alt="screen shot 2016-10-03 at 11 44 44 pm" src="https://cloud.githubusercontent.com/assets/1514239/19085903/e173a860-8a20-11e6-85e3-d47f9835f494.png"> Author: Brian Cho <bcho@fb.com> Closes #15347 from dafrista/shuffle-metrics.
* [SPARK-16960][SQL] Deprecate approxCountDistinct, toDegrees and toRadians ↵hyukjinkwon2016-10-074-33/+105
| | | | | | | | | | | | | | | | | | | | | according to FunctionRegistry ## What changes were proposed in this pull request? It seems `approxCountDistinct`, `toDegrees` and `toRadians` are also missed while matching the names to the ones in `FunctionRegistry`. (please see [approx_count_distinct](https://github.com/apache/spark/blob/5c2ae79bfcf448d8dc9217efafa1409997c739de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L244), [degrees](https://github.com/apache/spark/blob/5c2ae79bfcf448d8dc9217efafa1409997c739de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L203) and [radians](https://github.com/apache/spark/blob/5c2ae79bfcf448d8dc9217efafa1409997c739de/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala#L222) in `FunctionRegistry`). I took a scan between `functions.scala` and `FunctionRegistry` and it seems these are all left. For `countDistinct` and `sumDistinct`, they are not registered in `FunctionRegistry`. This PR deprecates `approxCountDistinct`, `toDegrees` and `toRadians` and introduces `approx_count_distinct`, `degrees` and `radians`. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon <gurwls223@gmail.com> Author: Hyukjin Kwon <gurwls223@gmail.com> Closes #14538 from HyukjinKwon/SPARK-16588-followup.
* [SPARK-17795][WEB UI] Sorting on stage or job tables doesn’t reload page ↵Alex Bozarth2016-10-075-23/+42
| | | | | | | | | | | | | | | | | | on that table ## What changes were proposed in this pull request? Added anchor on table header id to sorting links on job and stage tables. This make the page reload after a sort load the page at the sorted table. This only changes page load behavior so no UI changes ## How was this patch tested? manually tested and dev/run-tests Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #15369 from ajbozarth/spark17795.
* [SPARK-17782][STREAMING][BUILD] Add Kafka 0.10 project to build modulesHerman van Hovell2016-10-071-0/+11
| | | | | | | | | ## What changes were proposed in this pull request? This PR adds the Kafka 0.10 subproject to the build infrastructure. This makes sure Kafka 0.10 tests are only triggers when it or of its dependencies change. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15355 from hvanhovell/SPARK-17782.
* [SPARK-17805][PYSPARK] Fix in sqlContext.read.text when pass in list of pathsBryan Cutler2016-10-072-2/+8
| | | | | | | | | | | | ## What changes were proposed in this pull request? If given a list of paths, `pyspark.sql.readwriter.text` will attempt to use an undefined variable `paths`. This change checks if the param `paths` is a basestring and then converts it to a list, so that the same variable `paths` can be used for both cases ## How was this patch tested? Added unit test for reading list of files Author: Bryan Cutler <cutlerb@gmail.com> Closes #15379 from BryanCutler/sql-readtext-paths-SPARK-17805.
* [SPARK-17792][ML] L-BFGS solver for linear regression does not accept ↵sethah2016-10-062-14/+11
| | | | | | | | | | | | | | | | general numeric label column types ## What changes were proposed in this pull request? Before, we computed `instances` in LinearRegression in two spots, even though they did the same thing. One of them did not cast the label column to `DoubleType`. This patch consolidates the computation and always casts the label column to `DoubleType`. ## How was this patch tested? Added a unit test to check all solvers. This test failed before this patch. Author: sethah <seth.hendrickson16@gmail.com> Closes #15364 from sethah/linreg_numeric_type.
* [SPARK-17803][TESTS] Upgrade docker-client dependencyChristian Kadner2016-10-062-1/+2
| | | | | | | | | | | | | | | | | | | | | | | | [SPARK-17803: Docker integration tests don't run with "Docker for Mac"](https://issues.apache.org/jira/browse/SPARK-17803) ## What changes were proposed in this pull request? This PR upgrades the [docker-client](https://mvnrepository.com/artifact/com.spotify/docker-client) dependency from [3.6.6](https://mvnrepository.com/artifact/com.spotify/docker-client/3.6.6) to [5.0.2](https://mvnrepository.com/artifact/com.spotify/docker-client/5.0.2) to enable _Docker for Mac_ users to run the `docker-integration-tests` out of the box. The very latest docker-client version is [6.0.0](https://mvnrepository.com/artifact/com.spotify/docker-client/6.0.0) but that has one additional dependency and no usage yet. ## How was this patch tested? The code change was tested on Mac OS X Yosemite with both _Docker Toolbox_ as well as _Docker for Mac_ and on Linux Ubuntu 14.04. ``` $ build/mvn -Pyarn -Phadoop-2.6 -Dhadoop.version=2.6.0 -Phive -Phive-thriftserver -DskipTests clean package $ build/mvn -Pdocker-integration-tests -Pscala-2.11 -pl :spark-docker-integration-tests_2.11 clean compile test ``` Author: Christian Kadner <ckadner@us.ibm.com> Closes #15378 from ckadner/SPARK-17803_Docker_for_Mac.
* [SPARK-17780][SQL] Report Throwable to user in StreamExecutionShixiong Zhu2016-10-063-4/+37
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? When using an incompatible source for structured streaming, it may throw NoClassDefFoundError. It's better to just catch Throwable and report it to the user since the streaming thread is dying. ## How was this patch tested? `test("NoClassDefFoundError from an incompatible source")` Author: Shixiong Zhu <shixiong@databricks.com> Closes #15352 from zsxwing/SPARK-17780.
* [SPARK-17798][SQL] Remove redundant Experimental annotations in sql.streamingReynold Xin2016-10-063-60/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? I was looking through API annotations to catch mislabeled APIs, and realized DataStreamReader and DataStreamWriter classes are already annotated as Experimental, and as a result there is no need to annotate each method within them. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #15373 from rxin/SPARK-17798.
* [SPARK-17750][SQL] Fix CREATE VIEW with INTERVAL arithmetic.Dongjoon Hyun2016-10-064-1/+43
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions. ```scala scala> sql("CREATE TABLE dates (ts TIMESTAMP)") scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates") java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ... ``` ## How was this patch tested? Pass Jenkins with a new testcase. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #15318 from dongjoon-hyun/SPARK-17750.
* [BUILD] Closing some stale PRshyukjinkwon2016-10-060-0/+0
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to close some stale PRs and ones suggested to be closed by committer(s) or obviously inappropriate PRs (e.g. branch to branch). Closes #13458 Closes #15278 Closes #15294 Closes #15339 Closes #15283 ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #15356 from HyukjinKwon/closing-prs.
* [MINOR][ML] Avoid 2D array flatten in NB training.Yanbo Liang2016-10-051-4/+4
| | | | | | | | | | | | ## What changes were proposed in this pull request? Avoid 2D array flatten in ```NaiveBayes``` training, since flatten method might be expensive (It will create another array and copy data there). ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #15359 from yanboliang/nb-theta.
* [SPARK-17346][SQL][TEST-MAVEN] Generate the sql test jar to fix the maven buildShixiong Zhu2016-10-053-0/+42
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Generate the sql test jar to fix the maven build ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #15368 from zsxwing/sql-test-jar.
* [SPARK-17346][SQL] Add Kafka source for Structured StreamingShixiong Zhu2016-10-0521-23/+2268
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source. It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing tdas did most of work and part of them was inspired by koeninger's work. ### Introduction The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows: Column | Type ---- | ---- key | binary value | binary topic | string partition | int offset | long timestamp | long timestampType | int The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic. ### Configuration The user can use `DataStreamReader.option` to set the following configurations. Kafka Source's options | value | default | meaning ------ | ------- | ------ | ----- startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off. failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected. subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source. kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets. fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")` ### Usage * Subscribe to 1 topic ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1") .load() ``` * Subscribe to multiple topics ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribe", "topic1,topic2") .load() ``` * Subscribe to a pattern ```Scala spark .readStream .format("kafka") .option("kafka.bootstrap.servers", "host:port") .option("subscribePattern", "topic.*") .load() ``` ## How was this patch tested? The new unit tests. Author: Shixiong Zhu <shixiong@databricks.com> Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Shixiong Zhu <zsxwing@gmail.com> Author: cody koeninger <cody@koeninger.org> Closes #15102 from zsxwing/kafka-source.
* [SPARK-17758][SQL] Last returns wrong result in case of empty partitionHerman van Hovell2016-10-053-13/+184
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order: ``` - Partition 1 [Row1, Row2] - Partition 2 [Row3] - Partition 3 [] ``` In this case the `Last` function will currently return a null, instead of the value of `Row3`. This PR fixes this by adding a `valueSet` flag to the `Last` function. ## How was this patch tested? We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #15348 from hvanhovell/SPARK-17758.
* [SPARK-17778][TESTS] Mock SparkContext to reduce memory usage of ↵Shixiong Zhu2016-10-051-1/+4
| | | | | | | | | | | | | | | | BlockManagerSuite ## What changes were proposed in this pull request? Mock SparkContext to reduce memory usage of BlockManagerSuite ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #15350 from zsxwing/SPARK-17778.