aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19359][SQL] clear useless path after rename a partition with ↵windpiger2017-01-272-0/+71
| | | | | | | | | | | | | | | | | | | upper-case by HiveExternalCatalog ## What changes were proposed in this pull request? Hive metastore is not case preserving and keep partition columns with lower case names. If SparkSQL create a table with upper-case partion name use HiveExternalCatalog, when we rename partition, it first call the HiveClient to renamePartition, which will create a new lower case partition path, then SparkSql rename the lower case path to the upper-case. while if the renamed partition contains more than one depth partition ,e.g. A=1/B=2, hive renamePartition change to a=1/b=2, then SparkSql rename it to A=1/B=2, but the a=1 still exists in the filesystem, we should also delete it. ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16700 from windpiger/clearUselessPathAfterRenamPartition.
* [SPARK-19336][ML][PYSPARK] LinearSVC Python APIwm624@hotmail.com2017-01-274-3/+158
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add Python API for the newly added LinearSVC algorithm. ## How was this patch tested? Add new doc string test. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16694 from wangmiao1981/ser.
* [SPARK-19365][CORE] Optimize RequestMessage serializationShixiong Zhu2017-01-274-27/+132
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Right now Netty PRC serializes `RequestMessage` using Java serialization, and the size of a single message (e.g., RequestMessage(..., "hello")`) is almost 1KB. This PR optimizes it by serializing `RequestMessage` manually (eliminate unnecessary information from most messages, e.g., class names of `RequestMessage`, `NettyRpcEndpointRef`, ...), and reduces the above message size to 100+ bytes. ## How was this patch tested? Jenkins I did a simple test to measure the improvement: Before ``` $ bin/spark-shell --master local-cluster[1,4,1024] ... scala> for (i <- 1 to 10) { | val start = System.nanoTime | val s = sc.parallelize(1 to 1000000, 10 * 1000).count() | val end = System.nanoTime | println(s"$i\t" + ((end - start)/1000/1000)) | } 1 6830 2 4353 3 3322 4 3107 5 3235 6 3139 7 3156 8 3166 9 3091 10 3029 ``` After: ``` $ bin/spark-shell --master local-cluster[1,4,1024] ... scala> for (i <- 1 to 10) { | val start = System.nanoTime | val s = sc.parallelize(1 to 1000000, 10 * 1000).count() | val end = System.nanoTime | println(s"$i\t" + ((end - start)/1000/1000)) | } 1 6431 2 3643 3 2913 4 2679 5 2760 6 2710 7 2747 8 2793 9 2679 10 2651 ``` I also captured the TCP packets for this test. Before this patch, the total size of TCP packets is ~1.5GB. After it, it reduces to ~1.2GB. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16706 from zsxwing/rpc-opt.
* [SPARK-19324][SPARKR] Spark VJM stdout output is getting dropped in SparkRFelix Cheung2017-01-272-4/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This affects mostly running job from the driver in client mode when results are expected to be through stdout (which should be somewhat rare, but possible) Before: ``` > a <- as.DataFrame(cars) > b <- group_by(a, "dist") > c <- count(b) > sparkR.callJMethod(c$countjc, "explain", TRUE) NULL ``` After: ``` > a <- as.DataFrame(cars) > b <- group_by(a, "dist") > c <- count(b) > sparkR.callJMethod(c$countjc, "explain", TRUE) count#11L NULL ``` Now, `column.explain()` doesn't seem very useful (we can get more extensive output with `DataFrame.explain()`) but there are other more complex examples with calls of `println` in Scala/JVM side, that are getting dropped. ## How was this patch tested? manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16670 from felixcheung/rjvmstdout.
* [SPARK-19333][SPARKR] Add Apache License headers to R filesFelix Cheung2017-01-272-0/+34
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? add header ## How was this patch tested? Manual run to check vignettes html is created properly Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16709 from felixcheung/rfilelicense.
* [SPARK-12970][DOCS] Fix the example in SturctType APIs for Scala and Javahyukjinkwon2017-01-272-16/+18
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes both, javadoc8 break ``` [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/FindHiveSerdeTable.java:3: error: reference not found [error] * Replaces {link SimpleCatalogRelation} with {link MetastoreRelation} if its table provider is hive. ``` and the example in `StructType` as a self-contained example as below: ```scala import org.apache.spark.sql._ import org.apache.spark.sql.types._ val struct = StructType( StructField("a", IntegerType, true) :: StructField("b", LongType, false) :: StructField("c", BooleanType, false) :: Nil) // Extract a single StructField. val singleField = struct("b") // singleField: StructField = StructField(b,LongType,false) // If this struct does not have a field called "d", it throws an exception. struct("d") // java.lang.IllegalArgumentException: Field "d" does not exist. // ... // Extract multiple StructFields. Field names are provided in a set. // A StructType object will be returned. val twoFields = struct(Set("b", "c")) // twoFields: StructType = // StructType(StructField(b,LongType,false), StructField(c,BooleanType,false)) // Any names without matching fields will throw an exception. // For the case shown below, an exception is thrown due to "d". struct(Set("b", "c", "d")) // java.lang.IllegalArgumentException: Field "d" does not exist. // ... ``` ```scala import org.apache.spark.sql._ import org.apache.spark.sql.types._ val innerStruct = StructType( StructField("f1", IntegerType, true) :: StructField("f2", LongType, false) :: StructField("f3", BooleanType, false) :: Nil) val struct = StructType( StructField("a", innerStruct, true) :: Nil) // Create a Row with the schema defined by struct val row = Row(Row(1, 2, true)) ``` Also, now when the column is missing, it throws an exception rather than ignoring. ## How was this patch tested? Manually via `sbt unidoc`. - Scaladoc <img width="665" alt="2017-01-26 12 54 13" src="https://cloud.githubusercontent.com/assets/6477701/22297905/1245620e-e362-11e6-9e22-43bb8d9871af.png"> - Javadoc <img width="722" alt="2017-01-26 12 54 27" src="https://cloud.githubusercontent.com/assets/6477701/22297899/0fd87e0c-e362-11e6-9033-7590bda1aea6.png"> <img width="702" alt="2017-01-26 12 54 32" src="https://cloud.githubusercontent.com/assets/6477701/22297900/0fe14154-e362-11e6-9882-768381c53163.png"> Author: hyukjinkwon <gurwls223@gmail.com> Closes #16703 from HyukjinKwon/SPARK-12970.
* [SPARK-18929][ML] Add Tweedie distribution in GLMactuaryzhang2017-01-262-83/+567
| | | | | | | | | | | | ## What changes were proposed in this pull request? I propose to add the full Tweedie family into the GeneralizedLinearRegression model. The Tweedie family is characterized by a power variance function. Currently supported distributions such as Gaussian, Poisson and Gamma families are a special case of the Tweedie https://en.wikipedia.org/wiki/Tweedie_distribution. yanboliang srowen sethah Author: actuaryzhang <actuaryzhang10@gmail.com> Author: Wayne Zhang <actuaryzhang10@gmail.com> Closes #16344 from actuaryzhang/tweedie.
* [SPARK-18788][SPARKR] Add API for getNumPartitionsFelix Cheung2017-01-267-31/+59
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? With doc to say this would convert DF into RDD ## How was this patch tested? unit tests, manual tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16668 from felixcheung/rgetnumpartitions.
* [SPARK-18821][SPARKR] Bisecting k-means wrapper in SparkRwm624@hotmail.com2017-01-267-5/+347
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add R wrapper for bisecting Kmeans. As JIRA is down, I will update title to link with corresponding JIRA later. ## How was this patch tested? Add new unit tests. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16566 from wangmiao1981/bk.
* [SPARK-18218][ML][MLLIB] Reduce shuffled data size of BlockMatrix ↵WeichenXu2017-01-262-13/+57
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | multiplication and solve potential OOM and low parallelism usage problem By split middle dimension in matrix multiplication ## What changes were proposed in this pull request? ### The problem in current block matrix mulitiplication As in JIRA https://issues.apache.org/jira/browse/SPARK-18218 described, block matrix multiplication in spark may cause some problem, suppose we have `M*N` dimensions matrix A multiply `N*P` dimensions matrix B, when N is much larger than M and P, then the following problem may occur: - when the middle dimension N is too large, it will cause reducer OOM. - even if OOM do not occur, it will still cause parallism too low. - when N is much large than M and P, and matrix A and B have many partitions, it may cause too many partition on M and P dimension, it will cause much larger shuffled data size. (I will expain this in detail in the following.) ### Key point of my improvement In this PR, I introduce `midDimSplitNum` parameter, and improve the algorithm, to resolve this problem. In order to understand the improvement in this PR, first let me give a simple case to explain how the current mulitiplication works and what cause the problems above: suppose we have block matrix A, contains 200 blocks (`2 numRowBlocks * 100 numColBlocks`), blocks arranged in 2 rows, 100 cols: ``` A00 A01 A02 ... A0,99 A10 A11 A12 ... A1,99 ``` and we have block matrix B, also contains 200 blocks (`100 numRowBlocks * 2 numColBlocks`), blocks arranged in 100 rows, 2 cols: ``` B00 B01 B10 B11 B20 B21 ... B99,0 B99,1 ``` Suppose all blocks in the two matrices are dense for now. Now we call A.multiply(B), suppose the generated `resultPartitioner` contains 2 rowPartitions and 2 colPartitions (can't be more partitions because the result matrix only contains `2 * 2` blocks), the current algorithm will contains two shuffle steps: **step-1** Step-1 will generate 4 reducer, I tag them as reducer-00, reducer-01, reducer-10, reducer-11, and shuffle data as following: ``` A00 A01 A02 ... A0,99 B00 B10 B20 ... B99,0 shuffled into reducer-00 A00 A01 A02 ... A0,99 B01 B11 B21 ... B99,1 shuffled into reducer-01 A10 A11 A12 ... A1,99 B00 B10 B20 ... B99,0 shuffled into reducer-10 A10 A11 A12 ... A1,99 B01 B11 B21 ... B99,1 shuffled into reducer-11 ``` and the shuffling above is a `cogroup` transform, note that each reducer contains **only one group**. **step-2** Step-2 will do an `aggregateByKey` transform on the result of step-1, will also generate 4 reducers, and generate the final result RDD, contains 4 partitions, each partition contains one block. The main problems are in step-1. Now we have only 4 reducers, but matrix A and B have 400 blocks in total, obviously the reducer number is too small. and, we can see that, each reducer contains only one group(the group concept in `coGroup` transform), each group contains 200 blocks. This is terrible because we know that `coGroup` transformer will load each group into memory when computing. It is un-extensable in the algorithm level. Suppose matrix A has 10000 cols blocks or more instead of 100? Than each reducer will load 20000 blocks into memory. It will easily cause reducer OOM. This PR try to resolve the problem described above. When matrix A with dimension M * N multiply matrix B with dimension N * P, the middle dimension N is the keypoint. If N is large, the current mulitiplication implementation works badly. In this PR, I introduce a `numMidDimSplits` parameter, represent how many splits it will cut on the middle dimension N. Still using the example described above, now we set `numMidDimSplits = 10`, now we can generate 40 reducers in **step-1**: the reducer-ij above now will be splited into 10 reducers: reducer-ij0, reducer-ij1, ... reducer-ij9, each reducer will receive 20 blocks. now the shuffle works as following: **reducer-000 to reducer-009** ``` A0,0 A0,10 A0,20 ... A0,90 B0,0 B10,0 B20,0 ... B90,0 shuffled into reducer-000 A0,1 A0,11 A0,21 ... A0,91 B1,0 B11,0 B21,0 ... B91,0 shuffled into reducer-001 A0,2 A0,12 A0,22 ... A0,92 B2,0 B12,0 B22,0 ... B92,0 shuffled into reducer-002 ... A0,9 A0,19 A0,29 ... A0,99 B9,0 B19,0 B29,0 ... B99,0 shuffled into reducer-009 ``` **reducer-010 to reducer-019** ``` A0,0 A0,10 A0,20 ... A0,90 B0,1 B10,1 B20,1 ... B90,1 shuffled into reducer-010 A0,1 A0,11 A0,21 ... A0,91 B1,1 B11,1 B21,1 ... B91,1 shuffled into reducer-011 A0,2 A0,12 A0,22 ... A0,92 B2,1 B12,1 B22,1 ... B92,1 shuffled into reducer-012 ... A0,9 A0,19 A0,29 ... A0,99 B9,1 B19,1 B29,1 ... B99,1 shuffled into reducer-019 ``` **reducer-100 to reducer-109** and **reducer-110 to reducer-119** is similar to the above, I omit to write them out. ### API for this optimized algorithm I add a new API as following: ``` def multiply( other: BlockMatrix, numMidDimSplits: Int // middle dimension split number, expained above ): BlockMatrix ``` ### Shuffled data size analysis (compared under the same parallelism) The optimization has some subtle influence on the total shuffled data size. Appropriate `numMidDimSplits` will significantly reduce the shuffled data size, but too large `numMidDimSplits` may increase the shuffled data in reverse. For now I don't want to introduce formula to make thing too complex, I only use a simple case to represent it here: Suppose we have two same size square matrices X and Y, both have `16 numRowBlocks * 16 numColBlocks`. X and Y are both dense matrix. Now let me analysis the shuffling data size in the following case: **case 1: X and Y both partitioned in 16 rowPartitions and 16 colPartitions, numMidDimSplits = 1** ShufflingDataSize = (16 * 16 * (16 + 16) + 16 * 16) blocks = 8448 blocks parallelism = 16 * 16 * 1 = 256 //use step-1 reducers number as the parallism because it cost most of the computation time in this algorithm. **case 2: X and Y both partitioned in 8 rowPartitions and 8 colPartitions, numMidDimSplits = 4** ShufflingDataSize = (8 * 8 * (32 + 32) + 16 * 16 * 4) blocks = 5120 blocks parallelism = 8 * 8 * 4 = 256 //use step-1 reducers number as the parallism because it cost most of the computation time in this algorithm. **The two cases above all have parallism = 256**, case 1 `numMidDimSplits = 1` is equivalent with current implementation in mllib, but case 2 shuffling data is 60.6% of case 1, **it shows that under the same parallelism, proper `numMidDimSplits` will significantly reduce the shuffling data size**. ## How was this patch tested? Test suites added. Running result: ![blockmatrix](https://cloud.githubusercontent.com/assets/19235986/21600989/5e162cc2-d1bf-11e6-868c-0ec29190b605.png) Author: WeichenXu <WeichenXu123@outlook.com> Closes #15730 from WeichenXu123/optim_block_matrix.
* [SPARK-19338][SQL] Add UDF names in explainTakeshi YAMAMURO2017-01-264-27/+44
| | | | | | | | | | | | | ## What changes were proposed in this pull request? This pr added a variable for a UDF name in `ScalaUDF`. Then, if the variable filled, `DataFrame#explain` prints the name. ## How was this patch tested? Added a test in `UDFSuite`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16707 from maropu/SPARK-19338.
* [SPARK-18936][SQL] Infrastructure for session local timezone support.Takuya UESHIN2017-01-2632-674/+1182
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? As of Spark 2.1, Spark SQL assumes the machine timezone for datetime manipulation, which is bad if users are not in the same timezones as the machines, or if different users have different timezones. We should introduce a session local timezone setting that is used for execution. An explicit non-goal is locale handling. ### Semantics Setting the session local timezone means that the timezone-aware expressions listed below should use the timezone to evaluate values, and also it should be used to convert (cast) between string and timestamp or between timestamp and date. - `CurrentDate` - `CurrentBatchTimestamp` - `Hour` - `Minute` - `Second` - `DateFormatClass` - `ToUnixTimestamp` - `UnixTimestamp` - `FromUnixTime` and below are implicitly timezone-aware through cast from timestamp to date: - `DayOfYear` - `Year` - `Quarter` - `Month` - `DayOfMonth` - `WeekOfYear` - `LastDay` - `NextDay` - `TruncDate` For example, if you have timestamp `"2016-01-01 00:00:00"` in `GMT`, the values evaluated by some of timezone-aware expressions are: ```scala scala> val df = Seq(new java.sql.Timestamp(1451606400000L)).toDF("ts") df: org.apache.spark.sql.DataFrame = [ts: timestamp] scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false) +-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+ |ts |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)| +-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+ |2016-01-01 00:00:00|2016 |1 |1 |0 |0 |0 | +-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+ ``` whereas setting the session local timezone to `"PST"`, they are: ```scala scala> spark.conf.set("spark.sql.session.timeZone", "PST") scala> df.selectExpr("cast(ts as string)", "year(ts)", "month(ts)", "dayofmonth(ts)", "hour(ts)", "minute(ts)", "second(ts)").show(truncate = false) +-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+ |ts |year(CAST(ts AS DATE))|month(CAST(ts AS DATE))|dayofmonth(CAST(ts AS DATE))|hour(ts)|minute(ts)|second(ts)| +-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+ |2015-12-31 16:00:00|2015 |12 |31 |16 |0 |0 | +-------------------+----------------------+-----------------------+----------------------------+--------+----------+----------+ ``` Notice that even if you set the session local timezone, it affects only in `DataFrame` operations, neither in `Dataset` operations, `RDD` operations nor in `ScalaUDF`s. You need to properly handle timezone by yourself. ### Design of the fix I introduced an analyzer to pass session local timezone to timezone-aware expressions and modified DateTimeUtils to take the timezone argument. ## How was this patch tested? Existing tests and added tests for timezone aware expressions. Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #16308 from ueshin/issues/SPARK-18350.
* [CORE][DOCS] Update a help message for --files in spark-submitTakeshi YAMAMURO2017-01-261-1/+2
| | | | | | | | | ## What changes were proposed in this pull request? This pr is to update a help message for `--files` in spark-submit because it seems users get confused about how to get full paths of the files that one adds via the option. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16698 from maropu/SparkFilesDoc.
* [SPARK-19220][UI] Make redirection to HTTPS apply to all URIs.Marcelo Vanzin2017-01-265-59/+155
| | | | | | | | | | | | | | | | | | | The redirect handler was installed only for the root of the server; any other context ended up being served directly through the HTTP port. Since every sub page (e.g. application UIs in the history server) is a separate servlet context, this meant that everything but the root was accessible via HTTP still. The change adds separate names to each connector, and binds contexts to specific connectors so that content is only served through the HTTPS connector when it's enabled. In that case, the only thing that binds to the HTTP connector is the redirect handler. Tested with new unit tests and by checking a live history server. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16582 from vanzin/SPARK-19220.
* [TESTS][SQL] Setup testdata at the beginning for tests to run independentlyDilip Biswal2017-01-254-13/+10
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In CachedTableSuite, we are not setting up the test data at the beginning. Some tests fail while trying to run individually. When running the entire suite they run fine. Here are some of the tests that fail - - test("SELECT star from cached table") - test("Self-join cached") As part of this simplified a couple of tests by calling a support method to count the number of InMemoryRelations. ## How was this patch tested? Ran the failing tests individually. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #16688 from dilipbiswal/cachetablesuite_simple.
* [SPARK-18020][STREAMING][KINESIS] Checkpoint SHARD_END to finish reading ↵Takeshi YAMAMURO2017-01-256-8/+117
| | | | | | | | | | | | | | closed shards ## What changes were proposed in this pull request? This pr is to fix an issue occurred when resharding Kinesis streams; the resharding makes the KCL throw an exception because Spark does not checkpoint `SHARD_END` when finishing reading closed shards in `KinesisRecordProcessor#shutdown`. This bug finally leads to stopping subscribing new split (or merged) shards. ## How was this patch tested? Added a test in `KinesisStreamSuite` to check if it works well when splitting/merging shards. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16213 from maropu/SPARK-18020.
* [SPARK-18495][UI] Document meaning of green dot in DAG visualizationuncleGen2017-01-251-1/+6
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? A green dot in the DAG visualization apparently means that the referenced RDD is cached. This is not documented anywhere except in this [blog post](https://databricks.com/blog/2015/06/22/understanding-your-spark-application-through-visualization.html). It would be good if the Web UI itself documented this somehow (perhaps in the tooltip?) so that the user can naturally learn what it means while using the Web UI. before pr: ![dingtalk20170125180158](https://cloud.githubusercontent.com/assets/7402327/22286167/37910ea2-e329-11e6-9aae-03dce6fceee2.png) ![dingtalk20170125180218](https://cloud.githubusercontent.com/assets/7402327/22286178/421f3132-e329-11e6-8283-ba6bbd15cfb0.png) after pr: ![dingtalk20170125175704](https://cloud.githubusercontent.com/assets/7402327/22286216/56a0050a-e329-11e6-813d-2be96b5ab7f1.png) ![dingtalk20170125175749](https://cloud.githubusercontent.com/assets/7402327/22286233/618cd646-e329-11e6-9ce5-10322b169dcb.png) ## How was this patch tested? Author: uncleGen <hustyugm@gmail.com> Closes #16702 from uncleGen/SPARK-18495.
* [SPARK-14804][SPARK][GRAPHX] Fix checkpointing of VertexRDD/EdgeRDDTathagata Das2017-01-253-2/+56
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? EdgeRDD/VertexRDD overrides checkpoint() and isCheckpointed() to forward these to the internal partitionRDD. So when checkpoint() is called on them, its the partitionRDD that actually gets checkpointed. However since isCheckpointed() also overridden to call partitionRDD.isCheckpointed, EdgeRDD/VertexRDD.isCheckpointed returns true even though this RDD is actually not checkpointed. This would have been fine except the RDD's internal logic for computing the RDD depends on isCheckpointed(). So for VertexRDD/EdgeRDD, since isCheckpointed is true, when computing Spark tries to read checkpoint data of VertexRDD/EdgeRDD even though they are not actually checkpointed. Through a crazy sequence of call forwarding, it reads checkpoint data of partitionsRDD and tries to cast it to types in Vertex/EdgeRDD. This leads to ClassCastException. The minimal fix that does not change any public behavior is to modify RDD internal to not use public override-able API for internal logic. ## How was this patch tested? New unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15396 from tdas/SPARK-14804.
* [SPARK-19064][PYSPARK] Fix pip installing of sub componentsHolden Karau2017-01-255-2/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix instalation of mllib and ml sub components, and more eagerly cleanup cache files during test script & make-distribution. ## How was this patch tested? Updated sanity test script to import mllib and ml sub-components. Author: Holden Karau <holden@us.ibm.com> Closes #16465 from holdenk/SPARK-19064-fix-pip-install-sub-components.
* [SPARK-19307][PYSPARK] Make sure user conf is propagated to SparkContext.Marcelo Vanzin2017-01-252-0/+23
| | | | | | | | | | | | The code was failing to propagate the user conf in the case where the JVM was already initialized, which happens when a user submits a python script via spark-submit. Tested with new unit test and by running a python script in a real cluster. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16682 from vanzin/SPARK-19307.
* [SPARK-19311][SQL] fix UDT hierarchy issuegmoehler2017-01-252-3/+110
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? acceptType() in UDT will no only accept the same type but also all base types ## How was this patch tested? Manual test using a set of generated UDTs fixing acceptType() in my user defined types Please review http://spark.apache.org/contributing.html before opening a pull request. Author: gmoehler <moehler@de.ibm.com> Closes #16660 from gmoehler/master.
* [SPARK-18863][SQL] Output non-aggregate expressions without GROUP BY in a ↵Nattavut Sutyanyong2017-01-254-51/+168
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | subquery does not yield an error ## What changes were proposed in this pull request? This PR will report proper error messages when a subquery expression contain an invalid plan. This problem is fixed by calling CheckAnalysis for the plan inside a subquery. ## How was this patch tested? Existing tests and two new test cases on 2 forms of subquery, namely, scalar subquery and in/exists subquery. ```` -- TC 01.01 -- The column t2b in the SELECT of the subquery is invalid -- because it is neither an aggregate function nor a GROUP BY column. select t1a, t2b from t1, t2 where t1b = t2c and t2b = (select max(avg) from (select t2b, avg(t2b) avg from t2 where t2a = t1.t1b ) ) ; -- TC 01.02 -- Invalid due to the column t2b not part of the output from table t2. select * from t1 where t1a in (select min(t2a) from t2 group by t2c having t2c in (select max(t3c) from t3 group by t3b having t3b > t2b )) ; ```` Author: Nattavut Sutyanyong <nsy.can@gmail.com> Closes #16572 from nsyca/18863.
* [SPARK-19313][ML][MLLIB] GaussianMixture should limit the number of featuressethah2017-01-254-6/+51
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The following test will fail on current master ````scala test("gmm fails on high dimensional data") { val ctx = spark.sqlContext import ctx.implicits._ val df = Seq( Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(0, 4), Array(3.0, 8.0)), Vectors.sparse(GaussianMixture.MAX_NUM_FEATURES + 1, Array(1, 5), Array(4.0, 9.0))) .map(Tuple1.apply).toDF("features") val gm = new GaussianMixture() intercept[IllegalArgumentException] { gm.fit(df) } } ```` Instead, you'll get an `ArrayIndexOutOfBoundsException` or something similar for MLlib. That's because the covariance matrix allocates an array of `numFeatures * numFeatures`, and in this case we get integer overflow. While there is currently a warning that the algorithm does not perform well for high number of features, we should perform an appropriate check to communicate this limitation to users. This patch adds a `require(numFeatures < GaussianMixture.MAX_NUM_FEATURES)` check to ML and MLlib algorithms. For the feature limitation, we can limit it such that we do not get numerical overflow to something like `math.sqrt(Integer.MaxValue).toInt` (about 46k) which eliminates the cryptic error. However in, for example WLS, we need to collect an array on the order of `numFeatures * numFeatures` to the driver and we therefore limit to 4096 features. We may want to keep that convention here for consistency. ## How was this patch tested? Unit tests in ML and MLlib. Author: sethah <seth.hendrickson16@gmail.com> Closes #16661 from sethah/gmm_high_dim.
* [SPARK-18750][YARN] Avoid using "mapValues" when allocating containers.Marcelo Vanzin2017-01-252-5/+93
| | | | | | | | | | That method is prone to stack overflows when the input map is really large; instead, use plain "map". Also includes a unit test that was tested and caused stack overflows without the fix. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16667 from vanzin/SPARK-18750.
* [SPARK-16046][DOCS] Aggregations in the Spark SQL programming guideaokolnychyi2017-01-246-0/+533
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - A separate subsection for Aggregations under “Getting Started” in the Spark SQL programming guide. It mentions which aggregate functions are predefined and how users can create their own. - Examples of using the `UserDefinedAggregateFunction` abstract class for untyped aggregations in Java and Scala. - Examples of using the `Aggregator` abstract class for type-safe aggregations in Java and Scala. - Python is not covered. - The PR might not resolve the ticket since I do not know what exactly was planned by the author. In total, there are four new standalone examples that can be executed via `spark-submit` or `run-example`. The updated Spark SQL programming guide references to these examples and does not contain hard-coded snippets. ## How was this patch tested? The patch was tested locally by building the docs. The examples were run as well. ![image](https://cloud.githubusercontent.com/assets/6235869/21292915/04d9d084-c515-11e6-811a-999d598dffba.png) Author: aokolnychyi <okolnychyyanton@gmail.com> Closes #16329 from aokolnychyi/SPARK-16046.
* [SPARK-19330][DSTREAMS] Also show tooltip for successful batchesLiwei Lin2017-01-241-2/+2
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? ### Before ![_streaming_before](https://cloud.githubusercontent.com/assets/15843379/22181462/1e45c20c-e0c8-11e6-831c-8bf69722a4ee.png) ### After ![_streaming_after](https://cloud.githubusercontent.com/assets/15843379/22181464/23f38a40-e0c8-11e6-9a87-e27b1ffb1935.png) ## How was this patch tested? Manually Author: Liwei Lin <lwlin7@gmail.com> Closes #16673 from lw-lin/streaming.
* [SPARK-19334][SQL] Fix the code injection vulnerability related to Generator ↵Kousuke Saruta2017-01-242-2/+25
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | functions. ## What changes were proposed in this pull request? Similar to SPARK-15165, codegen is in danger of arbitrary code injection. The root cause is how variable names are created by codegen. In GenerateExec#codeGenAccessor, a variable name is created like as follows. ``` val value = ctx.freshName(name) ``` The variable `value` is named based on the value of the variable `name` and the value of `name` is from schema given by users so an attacker can attack with queries like as follows. ``` SELECT inline(array(cast(struct(1) AS struct<`=new Object() { {f();} public void f() {throw new RuntimeException("This exception is injected.");} public int x;}.x`:int>))) ``` In the example above, a RuntimeException is thrown but an attacker can replace it with arbitrary code. ## How was this patch tested? Added a new test case. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #16681 from sarutak/SPARK-19334.
* [SPARK-19017][SQL] NOT IN subquery with more than one column may return ↵Nattavut Sutyanyong2017-01-245-6/+131
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | incorrect results ## What changes were proposed in this pull request? This PR fixes the code in Optimizer phase where the NULL-aware expression of a NOT IN query is expanded in Rule `RewritePredicateSubquery`. Example: The query select a1,b1 from t1 where (a1,b1) not in (select a2,b2 from t2); has the (a1, b1) = (a2, b2) rewritten from (before this fix): Join LeftAnti, ((isnull((_1#2 = a2#16)) || isnull((_2#3 = b2#17))) || ((_1#2 = a2#16) && (_2#3 = b2#17))) to (after this fix): Join LeftAnti, (((_1#2 = a2#16) || isnull((_1#2 = a2#16))) && ((_2#3 = b2#17) || isnull((_2#3 = b2#17)))) ## How was this patch tested? sql/test, catalyst/test and new test cases in SQLQueryTestSuite. Author: Nattavut Sutyanyong <nsy.can@gmail.com> Closes #16467 from nsyca/19017.
* [SPARK-19139][CORE] New auth mechanism for transport library.Marcelo Vanzin2017-01-2434-422/+1709
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This change introduces a new auth mechanism to the transport library, to be used when users enable strong encryption. This auth mechanism has better security than the currently used DIGEST-MD5. The new protocol uses symmetric key encryption to mutually authenticate the endpoints, and is very loosely based on ISO/IEC 9798. The new protocol falls back to SASL when it thinks the remote end is old. Because SASL does not support asking the server for multiple auth protocols, which would mean we could re-use the existing SASL code by just adding a new SASL provider, the protocol is implemented outside of the SASL API to avoid the boilerplate of adding a new provider. Details of the auth protocol are discussed in the included README.md file. This change partly undos the changes added in SPARK-13331; AES encryption is now decoupled from SASL authentication. The encryption code itself, though, has been re-used as part of this change. ## How was this patch tested? - Unit tests - Tested Spark 2.2 against Spark 1.6 shuffle service with SASL enabled - Tested Spark 2.2 against Spark 2.2 shuffle service with SASL fallback disabled Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16521 from vanzin/SPARK-19139.
* [SPARK-18036][ML][MLLIB] Fixing decision trees handling edge casesIlya Matiach2017-01-243-6/+51
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Decision trees/GBT/RF do not handle edge cases such as constant features or empty features. In the case of constant features we choose any arbitrary split instead of failing with a cryptic error message. In the case of empty features we fail with a better error message stating: DecisionTree requires number of features > 0, but was given an empty features vector Instead of the cryptic error message: java.lang.UnsupportedOperationException: empty.max ## How was this patch tested? Unit tests are added in the patch for: DecisionTreeRegressor GBTRegressor Random Forest Regressor Author: Ilya Matiach <ilmat@microsoft.com> Closes #16377 from imatiach-msft/ilmat/fix-decision-tree.
* [SPARK-17913][SQL] compare atomic and string type column may return ↵Wenchen Fan2017-01-247-7/+38
| | | | | | | | | | | | | | | | | | | | confusing result ## What changes were proposed in this pull request? Spark SQL follows MySQL to do the implicit type conversion for binary comparison: http://dev.mysql.com/doc/refman/5.7/en/type-conversion.html However, this may return confusing result, e.g. `1 = 'true'` will return true, `19157170390056973L = '19157170390056971'` will return true. I think it's more reasonable to follow postgres in this case, i.e. cast string to the type of the other side, but return null if the string is not castable to keep hive compatibility. ## How was this patch tested? newly added tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #15880 from cloud-fan/compare.
* [SPARK-14049][CORE] Add functionality in spark history sever API to query ↵Parag Chaudhari2017-01-247-5/+311
| | | | | | | | | | | | | | | | | | | | | applications by end time ## What changes were proposed in this pull request? Currently, spark history server REST API provides functionality to query applications by application start time range based on minDate and maxDate query parameters, but it lacks support to query applications by their end time. In this pull request we are proposing optional minEndDate and maxEndDate query parameters and filtering capability based on these parameters to spark history server REST API. This functionality can be used for following queries, 1. Applications finished in last 'x' minutes 2. Applications finished before 'y' time 3. Applications finished between 'x' time to 'y' time 4. Applications started from 'x' time and finished before 'y' time. For backward compatibility, we can keep existing minDate and maxDate query parameters as they are and they can continue support filtering based on start time range. ## How was this patch tested? Existing unit tests and 4 new unit tests. Author: Parag Chaudhari <paragpc@amazon.com> Closes #11867 from paragpc/master-SHS-query-by-endtime_2.
* [SPARK-19246][SQL] CataLogTable's partitionSchema order and exist checkwindpiger2017-01-241-4/+9
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? CataLogTable's partitionSchema should check if each column name in partitionColumnNames must match one and only one field in schema, if not we should throw an exception and CataLogTable's partitionSchema should keep order with partitionColumnNames ## How was this patch tested? N/A Author: windpiger <songjun@outlook.com> Closes #16606 from windpiger/checkPartionColNameWithSchema.
* [SPARK-19152][SQL] DataFrameWriter.saveAsTable support hive appendwindpiger2017-01-249-38/+83
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? After [SPARK-19107](https://issues.apache.org/jira/browse/SPARK-19107), we now can treat hive as a data source and create hive tables with DataFrameWriter and Catalog. However, the support is not completed, there are still some cases we do not support. This PR implement: DataFrameWriter.saveAsTable work with hive format with append mode ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16552 from windpiger/saveAsTableWithHiveAppend.
* delete useless var “j”Souljoy Zhuo2017-01-241-2/+0
| | | | | | | | the var “j” defined in "var j = 0" is useless for “def compress” Author: Souljoy Zhuo <zhuoshoujie@126.com> Closes #16676 from xiaoyesoso/patch-1.
* [DOCS] Fix typo in docsuncleGen2017-01-245-7/+7
| | | | | | | | | | | | ## What changes were proposed in this pull request? Fix typo in docs ## How was this patch tested? Author: uncleGen <hustyugm@gmail.com> Closes #16658 from uncleGen/typo-issue.
* [SPARK-18823][SPARKR] add support for assigning to columnFelix Cheung2017-01-242-13/+55
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Support for ``` df[[myname]] <- 1 df[[2]] <- df$eruptions ``` ## How was this patch tested? manual tests, unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16663 from felixcheung/rcolset.
* [SPARK-16101][HOTFIX] Fix the build with Scala 2.10 by explicit typed argumenthyukjinkwon2017-01-231-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? I goofed in https://github.com/apache/spark/pull/16669 which introduces the break in scala 2.10. This fixes ```bash [error] /home/jenkins/workspace/spark-master-compile-sbt-scala-2.10/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala:65: polymorphic expression cannot be instantiated to expected type; [error] found : [B >: org.apache.spark.sql.types.StructField](B, Int) => Int [error] required: org.apache.spark.sql.types.StructField => ? [error] fields.map(schema.indexOf).toArray [error] ^ [error] one error found [error] (sql/compile:compileIncremental) Compilation failed ``` ## How was this patch tested? Manually via ```bash ./dev/change-scala-version.sh 2.10 ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package ``` ``` [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 2.719 s] [INFO] Spark Project Tags ................................. SUCCESS [ 3.441 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 3.411 s] [INFO] Spark Project Networking ........................... SUCCESS [ 5.088 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 5.131 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 5.813 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 6.567 s] [INFO] Spark Project Core ................................. SUCCESS [01:39 min] [INFO] Spark Project ML Local Library ..................... SUCCESS [ 6.644 s] [INFO] Spark Project GraphX ............................... SUCCESS [ 11.304 s] [INFO] Spark Project Streaming ............................ SUCCESS [ 26.275 s] [INFO] Spark Project Catalyst ............................. SUCCESS [01:04 min] [INFO] Spark Project SQL .................................. SUCCESS [02:07 min] [INFO] Spark Project ML Library ........................... SUCCESS [01:20 min] [INFO] Spark Project Tools ................................ SUCCESS [ 8.755 s] [INFO] Spark Project Hive ................................. SUCCESS [ 51.141 s] [INFO] Spark Project REPL ................................. SUCCESS [ 13.688 s] [INFO] Spark Project YARN Shuffle Service ................. SUCCESS [ 7.211 s] [INFO] Spark Project YARN ................................. SUCCESS [ 10.908 s] [INFO] Spark Project Assembly ............................. SUCCESS [ 2.940 s] [INFO] Spark Project External Flume Sink .................. SUCCESS [ 4.386 s] [INFO] Spark Project External Flume ....................... SUCCESS [ 8.589 s] [INFO] Spark Project External Flume Assembly .............. SUCCESS [ 1.891 s] [INFO] Spark Integration for Kafka 0.8 .................... SUCCESS [ 8.458 s] [INFO] Spark Project Examples ............................. SUCCESS [ 17.706 s] [INFO] Spark Project External Kafka Assembly .............. SUCCESS [ 3.070 s] [INFO] Spark Integration for Kafka 0.10 ................... SUCCESS [ 11.227 s] [INFO] Spark Integration for Kafka 0.10 Assembly .......... SUCCESS [ 2.982 s] [INFO] Kafka 0.10 Source for Structured Streaming ......... SUCCESS [ 7.494 s] [INFO] Spark Project Java 8 Tests ......................... SUCCESS [ 3.748 s] [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ ``` and manual test `CSVSuite` with Scala 2.11 with my IDE. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16684 from HyukjinKwon/hot-fix-type-ensurance.
* [SPARK-19268][SS] Disallow adaptive query execution for streaming queriesShixiong Zhu2017-01-232-1/+17
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? As adaptive query execution may change the number of partitions in different batches, it may break streaming queries. Hence, we should disallow this feature in Structured Streaming. ## How was this patch tested? `test("SPARK-19268: Adaptive query execution should be disallowed")`. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16683 from zsxwing/SPARK-19268.
* [SPARK-9435][SQL] Reuse function in Java UDF to correctly support ↵hyukjinkwon2017-01-232-23/+68
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | expressions that require equality comparison between ScalaUDF ## What changes were proposed in this pull request? Currently, running the codes in Java ```java spark.udf().register("inc", new UDF1<Long, Long>() { Override public Long call(Long i) { return i + 1; } }, DataTypes.LongType); spark.range(10).toDF("x").createOrReplaceTempView("tmp"); Row result = spark.sql("SELECT inc(x) FROM tmp GROUP BY inc(x)").head(); Assert.assertEquals(7, result.getLong(0)); ``` fails as below: ``` org.apache.spark.sql.AnalysisException: expression 'tmp.`x`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;; Aggregate [UDF(x#19L)], [UDF(x#19L) AS UDF(x)#23L] +- SubqueryAlias tmp, `tmp` +- Project [id#16L AS x#19L] +- Range (0, 10, step=1, splits=Some(8)) at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:40) at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:57) ``` The root cause is because we were creating the function every time when it needs to build as below: ```scala scala> def inc(i: Int) = i + 1 inc: (i: Int)Int scala> (inc(_: Int)).hashCode res15: Int = 1231799381 scala> (inc(_: Int)).hashCode res16: Int = 2109839984 scala> (inc(_: Int)) == (inc(_: Int)) res17: Boolean = false ``` This seems leading to the comparison failure between `ScalaUDF`s created from Java UDF API, for example, in `Expression.semanticEquals`. In case of Scala one, it seems already fine. Both can be tested easily as below if any reviewer is more comfortable with Scala: ```scala val df = Seq((1, 10), (2, 11), (3, 12)).toDF("x", "y") val javaUDF = new UDF1[Int, Int] { override def call(i: Int): Int = i + 1 } // spark.udf.register("inc", javaUDF, IntegerType) // Uncomment this for Java API // spark.udf.register("inc", (i: Int) => i + 1) // Uncomment this for Scala API df.createOrReplaceTempView("tmp") spark.sql("SELECT inc(y) FROM tmp GROUP BY inc(y)").show() ``` ## How was this patch tested? Unit test in `JavaUDFSuite.java` and `./dev/lint-java`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16553 from HyukjinKwon/SPARK-9435.
* [SPARK-19272][SQL] Remove the param `viewOriginalText` from `CatalogTable`jiangxingbo2017-01-249-45/+39
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Hive will expand the view text, so it needs 2 fields: originalText and viewText. Since we don't expand the view text, but only add table properties, perhaps only a single field `viewText` is enough in CatalogTable. This PR brought in the following changes: 1. Remove the param `viewOriginalText` from `CatalogTable`; 2. Update the output of command `DescribeTableCommand`. ## How was this patch tested? Tested by exsiting test cases, also updated the failed test cases. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #16679 from jiangxb1987/catalogTable.
* [SPARK-19290][SQL] add a new extending interface in Analyzer for post-hoc ↵Wenchen Fan2017-01-236-55/+30
| | | | | | | | | | | | | | | | | | | | | resolution ## What changes were proposed in this pull request? To implement DDL commands, we added several analyzer rules in sql/hive module to analyze DDL related plans. However, our `Analyzer` currently only have one extending interface: `extendedResolutionRules`, which defines extra rules that will be run together with other rules in the resolution batch, and doesn't fit DDL rules well, because: 1. DDL rules may do some checking and normalization, but we may do it many times as the resolution batch will run rules again and again, until fixed point, and it's hard to tell if a DDL rule has already done its checking and normalization. It's fine because DDL rules are idempotent, but it's bad for analysis performance 2. some DDL rules may depend on others, and it's pretty hard to write `if` conditions to guarantee the dependencies. It will be good if we have a batch which run rules in one pass, so that we can guarantee the dependencies by rules order. This PR adds a new extending interface in `Analyzer`: `postHocResolutionRules`, which defines rules that will be run only once in a batch runs right after the resolution batch. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #16645 from cloud-fan/analyzer.
* [SPARK-17747][ML] WeightCol support non-double numeric datatypesZheng RuiFeng2017-01-239-38/+95
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1, add test for `WeightCol` in `MLTestingUtils.checkNumericTypes` 2, move datatype cast to `Predict.fit`, and supply algos' `train()` with casted dataframe ## How was this patch tested? local tests in spark-shell and unit tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #15314 from zhengruifeng/weightCol_support_int.
* [SPARK-19306][CORE] Fix inconsistent state in DiskBlockObject when expection ↵jerryshao2017-01-231-19/+25
| | | | | | | | | | | | | | | | occurred ## What changes were proposed in this pull request? In `DiskBlockObjectWriter`, when some errors happened during writing, it will call `revertPartialWritesAndClose`, if this method again failed due to some issues like out of disk, it will throw exception without resetting the state of this writer, also skipping the revert. So here propose to fix this issue to offer user a chance to recover from such issue. ## How was this patch tested? Existing test. Author: jerryshao <sshao@hortonworks.com> Closes #16657 from jerryshao/SPARK-19306.
* [SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge caseIlya Matiach2017-01-233-7/+44
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | [SPARK-16473][MLLIB] Fix BisectingKMeans Algorithm failing in edge case where no children exist in updateAssignments ## What changes were proposed in this pull request? Fix a bug in which BisectingKMeans fails with error: java.util.NoSuchElementException: key not found: 166 at scala.collection.MapLike$class.default(MapLike.scala:228) at scala.collection.AbstractMap.default(Map.scala:58) at scala.collection.MapLike$class.apply(MapLike.scala:141) at scala.collection.AbstractMap.apply(Map.scala:58) at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply$mcDJ$sp(BisectingKMeans.scala:338) at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337) at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1$$anonfun$2.apply(BisectingKMeans.scala:337) at scala.collection.TraversableOnce$$anonfun$minBy$1.apply(TraversableOnce.scala:231) at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:111) at scala.collection.immutable.List.foldLeft(List.scala:84) at scala.collection.LinearSeqOptimized$class.reduceLeft(LinearSeqOptimized.scala:125) at scala.collection.immutable.List.reduceLeft(List.scala:84) at scala.collection.TraversableOnce$class.minBy(TraversableOnce.scala:231) at scala.collection.AbstractTraversable.minBy(Traversable.scala:105) at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:337) at org.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$org$apache$spark$mllib$clustering$BisectingKMeans$$updateAssignments$1.apply(BisectingKMeans.scala:334) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) at scala.collection.Iterator$$anon$14.hasNext(Iterator.scala:389) ## How was this patch tested? The dataset was run against the code change to verify that the code works. I will try to add unit tests to the code. (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) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Ilya Matiach <ilmat@microsoft.com> Closes #16355 from imatiach-msft/ilmat/fix-kmeans.
* [SPARK-17455][MLLIB] Improve PAVA implementation in IsotonicRegressionz001qdp2017-01-232-71/+97
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? New implementation of the Pool Adjacent Violators Algorithm (PAVA) in mllib.IsotonicRegression, which used under the hood by ml.regression.IsotonicRegression. The previous implementation could have factorial complexity in the worst case. This implementation, which closely follows those in scikit-learn and the R `iso` package, runs in quadratic time in the worst case. ## How was this patch tested? Existing unit tests in both `mllib` and `ml` passed before and after this patch. Scaling properties were tested by running the `poolAdjacentViolators` method in [scala-benchmarking-template](https://github.com/sirthias/scala-benchmarking-template) with the input generated by ``` scala val x = (1 to length).toArray.map(_.toDouble) val y = x.reverse.zipWithIndex.map{ case (yi, i) => if (i % 2 == 1) yi - 1.5 else yi} val w = Array.fill(length)(1d) val input: Array[(Double, Double, Double)] = (y zip x zip w) map{ case ((y, x), w) => (y, x, w)} ``` Before this patch: | Input Length | Time (us) | | --: | --: | | 100 | 1.35 | | 200 | 3.14 | | 400 | 116.10 | | 800 | 2134225.90 | After this patch: | Input Length | Time (us) | | --: | --: | | 100 | 1.25 | | 200 | 2.53 | | 400 | 5.86 | | 800 | 10.55 | Benchmarking was also performed with randomly-generated y values, with similar results. Author: z001qdp <Nicholas.Eggert@target.com> Closes #15018 from neggert/SPARK-17455-isoreg-algo.
* [SPARK-14709][ML] spark.ml API for linear SVMYuhao2017-01-233-2/+789
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? jira: https://issues.apache.org/jira/browse/SPARK-14709 Provide API for SVM algorithm for DataFrames. As discussed in jira, the initial implementation uses OWL-QN with Hinge loss function. The API should mimic existing spark.ml.classification APIs. Currently only Binary Classification is supported. Multinomial support can be added in this or following release. ## How was this patch tested? new unit tests and simple manual test Author: Yuhao <yuhao.yang@intel.com> Author: Yuhao Yang <hhbyyh@gmail.com> Closes #15211 from hhbyyh/mlsvm.
* [SPARK-19284][SQL] append to partitioned datasource table should without ↵windpiger2017-01-233-5/+36
| | | | | | | | | | | | | | | custom partition location ## What changes were proposed in this pull request? when we append data to a existed partitioned datasource table, the InsertIntoHadoopFsRelationCommand.getCustomPartitionLocations currently return the same location with Hive default, it should return None. ## How was this patch tested? Author: windpiger <songjun@outlook.com> Closes #16642 from windpiger/appendSchema.
* [SPARK-19146][CORE] Drop more elements when stageData.taskData.size > ↵Yuming Wang2017-01-233-9/+47
| | | | | | | | | | | | | | | | retainedTasks ## What changes were proposed in this pull request? Drop more elements when `stageData.taskData.size > retainedTasks` to reduce the number of times on call drop function. ## How was this patch tested? Jenkins Author: Yuming Wang <wgyumg@gmail.com> Closes #16527 from wangyum/SPARK-19146.
* [SPARK-19218][SQL] Fix SET command to show a result correctly and in a ↵Dongjoon Hyun2017-01-233-3/+36
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | sorted order ## What changes were proposed in this pull request? This PR aims to fix the following two things. 1. `sql("SET -v").collect()` or `sql("SET -v").show()` raises the following exceptions for String configuration with default value, `null`. For the test, please see [Jenkins result](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/71539/testReport/) and https://github.com/apache/spark/commit/60953bf1f1ba144e709fdae3903a390ff9479fd0 in #16624 . ``` sbt.ForkMain$ForkError: java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException createexternalrow(input[0, string, false].toString, input[1, string, false].toString, input[2, string, false].toString, StructField(key,StringType,false), StructField(value,StringType,false), StructField(meaning,StringType,false)) :- input[0, string, false].toString : +- input[0, string, false] :- input[1, string, false].toString : +- input[1, string, false] +- input[2, string, false].toString +- input[2, string, false] ``` 2. Currently, `SET` and `SET -v` commands show unsorted result. We had better show a sorted result for UX. Also, this is compatible with Hive. **BEFORE** ``` scala> sql("set").show(false) ... |spark.driver.host |10.22.16.140 | |spark.driver.port |63893 | |spark.repl.class.uri |spark://10.22.16.140:63893/classes | ... |spark.app.name |Spark shell | |spark.driver.memory |4G | |spark.executor.id |driver | |spark.submit.deployMode |client | |spark.master |local[*] | |spark.home |/Users/dhyun/spark | |spark.sql.catalogImplementation|hive | |spark.app.id |local-1484333618945 | ``` **AFTER** ``` scala> sql("set").show(false) ... |spark.app.id |local-1484333925649 | |spark.app.name |Spark shell | |spark.driver.host |10.22.16.140 | |spark.driver.memory |4G | |spark.driver.port |64994 | |spark.executor.id |driver | |spark.jars | | |spark.master |local[*] | |spark.repl.class.uri |spark://10.22.16.140:64994/classes | |spark.sql.catalogImplementation|hive | |spark.submit.deployMode |client | ``` ## How was this patch tested? Jenkins with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16579 from dongjoon-hyun/SPARK-19218.