aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17932][SQL] Support SHOW TABLES EXTENDED LIKE ↵jiangxingbo2016-11-306-25/+257
| | | | | | | | | | | | | | | | | | | | 'identifier_with_wildcards' statement ## What changes were proposed in this pull request? Currently we haven't implemented `SHOW TABLE EXTENDED` in Spark 2.0. This PR is to implement the statement. Goals: 1. Support `SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards'`; 2. Explicitly output an unsupported error message for `SHOW TABLES [EXTENDED] ... PARTITION` statement; 3. Improve test cases for `SHOW TABLES` statement. ## How was this patch tested? 1. Add new test cases in file `show-tables.sql`. 2. Modify tests for `SHOW TABLES` in `DDLSuite`. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #15958 from jiangxb1987/show-table-extended.
* [SPARK-17897][SQL] Fixed IsNotNull Constraint Inference Rulegatorsmile2016-11-303-7/+35
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? The `constraints` of an operator is the expressions that evaluate to `true` for all the rows produced. That means, the expression result should be neither `false` nor `unknown` (NULL). Thus, we can conclude that `IsNotNull` on all the constraints, which are generated by its own predicates or propagated from the children. The constraint can be a complex expression. For better usage of these constraints, we try to push down `IsNotNull` to the lowest-level expressions (i.e., `Attribute`). `IsNotNull` can be pushed through an expression when it is null intolerant. (When the input is NULL, the null-intolerant expression always evaluates to NULL.) Below is the existing code we have for `IsNotNull` pushdown. ```Scala private def scanNullIntolerantExpr(expr: Expression): Seq[Attribute] = expr match { case a: Attribute => Seq(a) case _: NullIntolerant | IsNotNull(_: NullIntolerant) => expr.children.flatMap(scanNullIntolerantExpr) case _ => Seq.empty[Attribute] } ``` **`IsNotNull` itself is not null-intolerant.** It converts `null` to `false`. If the expression does not include any `Not`-like expression, it works; otherwise, it could generate a wrong result. This PR is to fix the above function by removing the `IsNotNull` from the inference. After the fix, when a constraint has a `IsNotNull` expression, we infer new attribute-specific `IsNotNull` constraints if and only if `IsNotNull` appears in the root. Without the fix, the following test case will return empty. ```Scala val data = Seq[java.lang.Integer](1, null).toDF("key") data.filter("not key is not null").show() ``` Before the fix, the optimized plan is like ``` == Optimized Logical Plan == Project [value#1 AS key#3] +- Filter (isnotnull(value#1) && NOT isnotnull(value#1)) +- LocalRelation [value#1] ``` After the fix, the optimized plan is like ``` == Optimized Logical Plan == Project [value#1 AS key#3] +- Filter NOT isnotnull(value#1) +- LocalRelation [value#1] ``` ### How was this patch tested? Added a test Author: gatorsmile <gatorsmile@gmail.com> Closes #16067 from gatorsmile/isNotNull2.
* [SPARK-18612][MLLIB] Delete broadcasted variable in LBFGS CostFunAnthony Truchet2016-11-301-0/+3
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix a broadcasted variable leak occurring at each invocation of CostFun in L-BFGS. ## How was this patch tested? UTests + check that fixed fatal memory consumption on Criteo's use cases. This contribution is made on behalf of Criteo S.A. (http://labs.criteo.com/) under the terms of the Apache v2 License. Author: Anthony Truchet <a.truchet@criteo.com> Closes #16040 from AnthonyTruchet/SPARK-18612-lbfgs-cost-fun.
* [SPARK-18366][PYSPARK][ML] Add handleInvalid to Pyspark for ↵Sandeep Singh2016-11-301-14/+71
| | | | | | | | | | | | | | | QuantileDiscretizer and Bucketizer ## What changes were proposed in this pull request? added the new handleInvalid param for these transformers to Python to maintain API parity. ## How was this patch tested? existing tests testing is done with new doctests Author: Sandeep Singh <sandeep@techaddict.me> Closes #15817 from techaddict/SPARK-18366.
* [SPARK-18617][CORE][STREAMING] Close "kryo auto pick" feature for Spark ↵uncleGen2016-11-294-9/+65
| | | | | | | | | | | | | | | | Streaming ## What changes were proposed in this pull request? #15992 provided a solution to fix the bug, i.e. **receiver data can not be deserialized properly**. As zsxwing said, it is a critical bug, but we should not break APIs between maintenance releases. It may be a rational choice to close auto pick kryo serializer for Spark Streaming in the first step. I will continue #15992 to optimize the solution. ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes #16052 from uncleGen/SPARK-18617.
* [SPARK-18622][SQL] Fix the datatype of the Sum aggregate functionHerman van Hovell2016-11-303-3/+110
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The result of a `sum` aggregate function is typically a Decimal, Double or a Long. Currently the output dataType is based on input's dataType. The `FunctionArgumentConversion` rule will make sure that the input is promoted to the largest type, and that also ensures that the output uses a (hopefully) sufficiently large output dataType. The issue is that sum is in a resolved state when we cast the input type, this means that rules assuming that the dataType of the expression does not change anymore could have been applied in the mean time. This is what happens if we apply `WidenSetOperationTypes` before applying the casts, and this breaks analysis. The most straight forward and future proof solution is to make `sum` always output the widest dataType in its class (Long for IntegralTypes, Decimal for DecimalTypes & Double for FloatType and DoubleType). This PR implements that solution. We should move expression specific type casting rules into the given Expression at some point. ## How was this patch tested? Added (regression) tests to SQLQueryTestSuite's `union.sql`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16063 from hvanhovell/SPARK-18622.
* [SPARK-17680][SQL][TEST] Added a Testcase for Verifying Unicode Character ↵gatorsmile2016-11-301-0/+45
| | | | | | | | | | | | | | | | | | | Support for Column Names and Comments ### What changes were proposed in this pull request? Spark SQL supports Unicode characters for column names when specified within backticks(`). When the Hive support is enabled, the version of the Hive metastore must be higher than 0.12, See the JIRA: https://issues.apache.org/jira/browse/HIVE-6013 Hive metastore supports Unicode characters for column names since 0.13. In Spark SQL, table comments, and view comments always allow Unicode characters without backticks. BTW, a separate PR has been submitted for database and table name validation because we do not support Unicode characters in these two cases. ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #15255 from gatorsmile/unicodeSupport.
* [SPARK-18516][STRUCTURED STREAMING] Follow up PR to add ↵Tathagata Das2016-11-299-54/+127
| | | | | | | | | | | | | | | | | StreamingQuery.status to Python ## What changes were proposed in this pull request? - Add StreamingQueryStatus.json - Make it not case class (to avoid unnecessarily exposing implicit object StreamingQueryStatus, consistent with StreamingQueryProgress) - Add StreamingQuery.status to Python - Fix post-termination status ## How was this patch tested? New unit tests Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16075 from tdas/SPARK-18516-1.
* [SPARK-15819][PYSPARK][ML] Add KMeanSummary in KMeans of PySparkJeff Zhang2016-11-292-0/+56
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Add python api for KMeansSummary ## How was this patch tested? unit test added Author: Jeff Zhang <zjffdu@apache.org> Closes #13557 from zjffdu/SPARK-15819.
* [SPARK-18145] Update documentation for hive partition management in 2.1Eric Liang2016-11-291-0/+9
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This documents the partition handling changes for Spark 2.1 and how to migrate existing tables. ## How was this patch tested? Built docs locally. rxin Author: Eric Liang <ekl@databricks.com> Closes #16074 from ericl/spark-18145.
* [SPARK-18632][SQL] AggregateFunction should not implement ImplicitCastInputTypesHerman van Hovell2016-11-2924-63/+67
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `AggregateFunction` currently implements `ImplicitCastInputTypes` (which enables implicit input type casting). There are actually quite a few situations in which we don't need this, or require more control over our input. A recent example is the aggregate for `CountMinSketch` which should only take string, binary or integral types inputs. This PR removes `ImplicitCastInputTypes` from the `AggregateFunction` and makes a case-by-case decision on what kind of input validation we should use. ## How was this patch tested? Refactoring only. Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16066 from hvanhovell/SPARK-18632.
* [SPARK-18319][ML][QA2.1] 2.1 QA: API: Experimental, DeveloperApi, final, ↵Yuhao2016-11-2913-83/+7
| | | | | | | | | | | | | | | | | sealed audit ## What changes were proposed in this pull request? make a pass through the items marked as Experimental or DeveloperApi and see if any are stable enough to be unmarked. Also check for items marked final or sealed to see if they are stable enough to be opened up as APIs. Some discussions in the jira: https://issues.apache.org/jira/browse/SPARK-18319 ## How was this patch tested? existing ut Author: Yuhao <yuhao.yang@intel.com> Author: Yuhao Yang <hhbyyh@gmail.com> Closes #15972 from hhbyyh/experimental21.
* [SPARK-18516][SQL] Split state and progress in streamingTathagata Das2016-11-2926-1752/+1087
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR separates the status of a `StreamingQuery` into two separate APIs: - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available. - `recentProgress` - an array of statistics about the most recent microbatches that have executed. A recent progress contains the following information: ``` { "id" : "2be8670a-fce1-4859-a530-748f29553bb6", "name" : "query-29", "timestamp" : 1479705392724, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303, "durationMs" : { "triggerExecution" : 276, "queryPlanning" : 3, "getBatch" : 5, "getOffset" : 3, "addBatch" : 234, "walCommit" : 30 }, "currentWatermark" : 0, "stateOperators" : [ ], "sources" : [ { "description" : "KafkaSource[Subscribe[topic-14]]", "startOffset" : { "topic-14" : { "2" : 0, "4" : 1, "1" : 0, "3" : 0, "0" : 0 } }, "endOffset" : { "topic-14" : { "2" : 1, "4" : 2, "1" : 0, "3" : 0, "0" : 1 } }, "numRecords" : 3, "inputRowsPerSecond" : 230.76923076923077, "processedRowsPerSecond" : 10.869565217391303 } ] } ``` Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique. Author: Tathagata Das <tathagata.das1565@gmail.com> Author: Michael Armbrust <michael@databricks.com> Closes #15954 from marmbrus/queryProgress.
* [SPARK-18553][CORE] Fix leak of TaskSetManager following executor lossJosh Rosen2016-11-293-36/+121
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? _This is the master branch version of #15986; the original description follows:_ This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails. This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map. Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L338) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L527) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here. This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss. There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L523) in `removeExecutor`, so I'd appreciate a very careful review of these changes. ## How was this patch tested? I added a new unit test to `TaskSchedulerImplSuite`. /cc kayousterhout and markhamstra, who reviewed #15986. Author: Josh Rosen <joshrosen@databricks.com> Closes #16045 from JoshRosen/fix-leak-following-total-executor-loss-master.
* [SPARK-18614][SQL] Incorrect predicate pushdown from ExistenceJoinNattavut Sutyanyong2016-11-294-4/+37
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? ExistenceJoin should be treated the same as LeftOuter and LeftAnti, not InnerLike and LeftSemi. This is not currently exposed because the rewrite of [NOT] EXISTS OR ... to ExistenceJoin happens in rule RewritePredicateSubquery, which is in a separate rule set and placed after the rule PushPredicateThroughJoin. During the transformation in the rule PushPredicateThroughJoin, an ExistenceJoin never exists. The semantics of ExistenceJoin says we need to preserve all the rows from the left table through the join operation as if it is a regular LeftOuter join. The ExistenceJoin augments the LeftOuter operation with a new column called exists, set to true when the join condition in the ON clause is true and false otherwise. The filter of any rows will happen in the Filter operation above the ExistenceJoin. Example: A(c1, c2): { (1, 1), (1, 2) } // B can be any value as it is irrelevant in this example B(c1): { (NULL) } select A.* from A where exists (select 1 from B where A.c1 = A.c2) or A.c2=2 In this example, the correct result is all the rows from A. If the pattern ExistenceJoin around line 935 in Optimizer.scala is indeed active, the code will push down the predicate A.c1 = A.c2 to be a Filter on relation A, which will incorrectly filter the row (1,2) from A. ## How was this patch tested? Since this is not an exposed case, no new test cases is added. The scenario is discovered via a code review of another PR and confirmed to be valid with peer. Author: Nattavut Sutyanyong <nsy.can@gmail.com> Closes #16044 from nsyca/spark-18614.
* [SPARK-18631][SQL] Changed ExchangeCoordinator re-partitioning to avoid more ↵Mark Hamstra2016-11-292-37/+35
| | | | | | | | | | | | | | | | data skew ## What changes were proposed in this pull request? Re-partitioning logic in ExchangeCoordinator changed so that adding another pre-shuffle partition to the post-shuffle partition will not be done if doing so would cause the size of the post-shuffle partition to exceed the target partition size. ## How was this patch tested? Existing tests updated to reflect new expectations. Author: Mark Hamstra <markhamstra@gmail.com> Closes #16065 from markhamstra/SPARK-17064.
* [SPARK-18429][SQL] implement a new Aggregate for CountMinSketchwangzhenhua2016-11-298-8/+710
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR implements a new Aggregate to generate count min sketch, which is a wrapper of CountMinSketch. ## How was this patch tested? add test cases Author: wangzhenhua <wangzhenhua@huawei.com> Closes #15877 from wzhfy/cms.
* [SPARK-18498][SQL] Revise HDFSMetadataLog API for better testingTyson Condie2016-11-291-34/+66
| | | | | | | | | | | | Revise HDFSMetadataLog API such that metadata object serialization and final batch file write are separated. This will allow serialization checks without worrying about batch file name formats. marmbrus zsxwing Existing tests already ensure this API faithfully support core functionality i.e., creation of batch files. Author: Tyson Condie <tcondie@gmail.com> Closes #15924 from tcondie/SPARK-18498. Signed-off-by: Michael Armbrust <michael@databricks.com>
* [SPARK-18592][ML] Move DT/RF/GBT Param setter methods to subclassesYanbo Liang2016-11-297-90/+260
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Mainly two changes: * Move DT/RF/GBT Param setter methods to subclasses. * Deprecate corresponding setter methods in the model classes. See discussion here https://github.com/apache/spark/pull/15913#discussion_r89662469. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16017 from yanboliang/spark-18592.
* [SPARK-18615][DOCS] Switch to multi-line doc to avoid a genjavadoc bug for ↵hyukjinkwon2016-11-2924-43/+129
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | backticks ## What changes were proposed in this pull request? Currently, single line comment does not mark down backticks to `<code>..</code>` but prints as they are (`` `..` ``). For example, the line below: ```scala /** Return an RDD with the pairs from `this` whose keys are not in `other`. */ ``` So, we could work around this as below: ```scala /** * Return an RDD with the pairs from `this` whose keys are not in `other`. */ ``` - javadoc - **Before** ![2016-11-29 10 39 14](https://cloud.githubusercontent.com/assets/6477701/20693606/e64c8f90-b622-11e6-8dfc-4a029216e23d.png) - **After** ![2016-11-29 10 39 08](https://cloud.githubusercontent.com/assets/6477701/20693607/e7280d36-b622-11e6-8502-d2e21cd5556b.png) - scaladoc (this one looks fine either way) - **Before** ![2016-11-29 10 38 22](https://cloud.githubusercontent.com/assets/6477701/20693640/12c18aa8-b623-11e6-901a-693e2f6f8066.png) - **After** ![2016-11-29 10 40 05](https://cloud.githubusercontent.com/assets/6477701/20693642/14eb043a-b623-11e6-82ac-7cd0000106d1.png) I suspect this is related with SPARK-16153 and genjavadoc issue in ` typesafehub/genjavadoc#85`. ## How was this patch tested? I found them via ``` grep -r "\/\*\*.*\`" . | grep .scala ```` and then checked if each is in the public API documentation with manually built docs (`jekyll build`) with Java 7. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16050 from HyukjinKwon/javadoc-markdown.
* [MINOR][DOCS] Updates to the Accumulator example in the programming guide. ↵aokolnychyi2016-11-291-21/+33
| | | | | | | | | | | | | | | | | | | | | | | | Fixed typos, AccumulatorV2 in Java ## What changes were proposed in this pull request? This pull request contains updates to Scala and Java Accumulator code snippets in the programming guide. - For Scala, the pull request fixes the signature of the 'add()' method in the custom Accumulator, which contained two params (as the old AccumulatorParam) instead of one (as in AccumulatorV2). - The Java example was updated to use the AccumulatorV2 class since AccumulatorParam is marked as deprecated. - Scala and Java examples are more consistent now. ## How was this patch tested? This patch was tested manually by building the docs locally. ![image](https://cloud.githubusercontent.com/assets/6235869/20652099/77d98d18-b4f3-11e6-8565-a995fe8cf8e5.png) Author: aokolnychyi <okolnychyyanton@gmail.com> Closes #16024 from aokolnychyi/fixed_accumulator_example.
* [SPARK-3359][DOCS] Make javadoc8 working for unidoc/genjavadoc compatibility ↵hyukjinkwon2016-11-29125-524/+611
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | in Java API documentation ## What changes were proposed in this pull request? This PR make `sbt unidoc` complete with Java 8. This PR roughly includes several fixes as below: - Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` `` ```diff - * A column that will be computed based on the data in a [[DataFrame]]. + * A column that will be computed based on the data in a `DataFrame`. ``` - Fix throws annotations so that they are recognisable in javadoc - Fix URL links to `<a href="http..."></a>`. ```diff - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression. + * <a href="http://en.wikipedia.org/wiki/Decision_tree_learning"> + * Decision tree (Wikipedia)</a> model for regression. ``` ```diff - * see http://en.wikipedia.org/wiki/Receiver_operating_characteristic + * see <a href="http://en.wikipedia.org/wiki/Receiver_operating_characteristic"> + * Receiver operating characteristic (Wikipedia)</a> ``` - Fix < to > to - `greater than`/`greater than or equal to` or `less than`/`less than or equal to` where applicable. - Wrap it with `{{{...}}}` to print them in javadoc or use `{code ...}` or `{literal ..}`. Please refer https://github.com/apache/spark/pull/16013#discussion_r89665558 - Fix `</p>` complaint ## How was this patch tested? Manually tested by `jekyll build` with Java 7 and 8 ``` java version "1.7.0_80" Java(TM) SE Runtime Environment (build 1.7.0_80-b15) Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode) ``` ``` java version "1.8.0_45" Java(TM) SE Runtime Environment (build 1.8.0_45-b14) Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode) ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16013 from HyukjinKwon/SPARK-3359-errors-more.
* [SPARK-18188] add checksum for blocks of broadcastDavies Liu2016-11-291-0/+32
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? A TorrentBroadcast is serialized and compressed first, then splitted as fixed size blocks, if any block is corrupt when fetching from remote, the decompression/deserialization will fail without knowing which block is corrupt. Also, the corrupt block is kept in block manager and reported to driver, so other tasks (in same executor or from different executor) will also fail because of it. This PR add checksum for each block, and check it after fetching a block from remote executor, because it's very likely that the corruption happen in network. When the corruption happen, it will throw the block away and throw an exception to fail the task, which will be retried. Added a config for it: `spark.broadcast.checksum`, which is true by default. ## How was this patch tested? Existing tests. Author: Davies Liu <davies@databricks.com> Closes #15935 from davies/broadcast_checksum.
* [SPARK-18339][SPARK-18513][SQL] Don't push down current_timestamp for ↵Tyson Condie2016-11-289-33/+273
| | | | | | | | | | | | | | | | | | | | | | | | | | | filters in StructuredStreaming and persist batch and watermark timestamps to offset log. ## What changes were proposed in this pull request? For the following workflow: 1. I have a column called time which is at minute level precision in a Streaming DataFrame 2. I want to perform groupBy time, count 3. Then I want my MemorySink to only have the last 30 minutes of counts and I perform this by .where('time >= current_timestamp().cast("long") - 30 * 60) what happens is that the `filter` gets pushed down before the aggregation, and the filter happens on the source data for the aggregation instead of the result of the aggregation (where I actually want to filter). I guess the main issue here is that `current_timestamp` is non-deterministic in the streaming context and shouldn't be pushed down the filter. Does this require us to store the `current_timestamp` for each trigger of the streaming job, that is something to discuss. Furthermore, we want to persist current batch timestamp and watermark timestamp to the offset log so that these values are consistent across multiple executions of the same batch. brkyvz zsxwing tdas ## How was this patch tested? A test was added to StreamingAggregationSuite ensuring the above use case is handled. The test injects a stream of time values (in seconds) to a query that runs in complete mode and only outputs the (count) aggregation results for the past 10 seconds. Author: Tyson Condie <tcondie@gmail.com> Closes #15949 from tcondie/SPARK-18339.
* [SPARK-18544][SQL] Append with df.saveAsTable writes data to wrong locationEric Liang2016-11-283-9/+34
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We failed to properly propagate table metadata for existing tables for the saveAsTable command. This caused a downstream component to think the table was MANAGED, writing data to the wrong location. ## How was this patch tested? Unit test that fails before the patch. Author: Eric Liang <ekl@databricks.com> Closes #15983 from ericl/spark-18544.
* [SPARK-18058][SQL][TRIVIAL] Use dataType.sameResult(...) instead equality on ↵Herman van Hovell2016-11-284-6/+6
| | | | | | | | | | | | | | asNullable datatypes ## What changes were proposed in this pull request? This is absolutely minor. PR https://github.com/apache/spark/pull/15595 uses `dt1.asNullable == dt2.asNullable` expressions in a few places. It is however more efficient to call `dt1.sameType(dt2)`. I have replaced every instance of the first pattern with the second pattern (3/5 were introduced by #15595). ## How was this patch tested? Existing tests. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16041 from hvanhovell/SPARK-18058.
* [SPARK-18547][CORE] Propagate I/O encryption key when executors register.Marcelo Vanzin2016-11-2815-227/+166
| | | | | | | | | | | | | | | | | | | | | | | | This change modifies the method used to propagate encryption keys used during shuffle. Instead of relying on YARN's UserGroupInformation credential propagation, this change explicitly distributes the key using the messages exchanged between driver and executor during registration. When RPC encryption is enabled, this means key propagation is also secure. This allows shuffle encryption to work in non-YARN mode, which means that it's easier to write unit tests for areas of the code that are affected by the feature. The key is stored in the SecurityManager; because there are many instances of that class used in the code, the key is only guaranteed to exist in the instance managed by the SparkEnv. This path was chosen to avoid storing the key in the SparkConf, which would risk having the key being written to disk as part of the configuration (as, for example, is done when starting YARN applications). Tested by new and existing unit tests (which were moved from the YARN module to core), and by running apps with shuffle encryption enabled. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #15981 from vanzin/SPARK-18547.
* [SPARK-18588][SS][KAFKA] Ignore the flaky kafka testShixiong Zhu2016-11-281-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Ignore the flaky test to unblock other PRs while I'm debugging it. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16051 from zsxwing/ignore-flaky-kafka-test.
* [SPARK-16282][SQL] Follow-up: remove "percentile" from temp function ↵Shuai Lin2016-11-282-5/+1
| | | | | | | | | | | | | | | | detection after implementing it natively ## What changes were proposed in this pull request? In #15764 we added a mechanism to detect if a function is temporary or not. Hive functions are treated as non-temporary. Of the three hive functions, now "percentile" has been implemented natively, and "hash" has been removed. So we should update the list. ## How was this patch tested? Unit tests. Author: Shuai Lin <linshuai2012@gmail.com> Closes #16049 from lins05/update-temp-function-detect-hive-list.
* [SPARK-18523][PYSPARK] Make SparkContext.stop more reliableAlexander Shorin2016-11-281-2/+15
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes SparkContext broken state in which it may fall if spark driver get crashed or killed by OOM. ## How was this patch tested? 1. Start SparkContext; 2. Find Spark driver process and `kill -9` it; 3. Call `sc.stop()`; 4. Create new SparkContext after that; Without this patch you will crash on step 3 and won't be able to do step 4 without manual reset private attibutes or IPython notebook / shell restart. Author: Alexander Shorin <kxepal@apache.org> Closes #15961 from kxepal/18523-make-spark-context-stop-more-reliable.
* [SPARK-18403][SQL] Fix unsafe data false sharing issue in ↵Cheng Lian2016-11-292-74/+101
| | | | | | | | | | | | | | | | | | | | | | | | | | ObjectHashAggregateExec ## What changes were proposed in this pull request? This PR fixes a random OOM issue occurred while running `ObjectHashAggregateSuite`. This issue can be steadily reproduced under the following conditions: 1. The aggregation must be evaluated using `ObjectHashAggregateExec`; 2. There must be an input column whose data type involves `ArrayType` (an input column of `MapType` may even cause SIGSEGV); 3. Sort-based aggregation fallback must be triggered during evaluation. The root cause is that while falling back to sort-based aggregation, we must sort and feed already evaluated partial aggregation buffers living in the hash map to the sort-based aggregator using an external sorter. However, the underlying mutable byte buffer of `UnsafeRow`s produced by the iterator of the external sorter is reused and may get overwritten when the iterator steps forward. After the last entry is consumed, the byte buffer points to a block of uninitialized memory filled by `5a`. Therefore, while reading an `UnsafeArrayData` out of the `UnsafeRow`, `5a5a5a5a` is treated as array size and triggers a memory allocation for a ridiculously large array and immediately blows up the JVM with an OOM. To fix this issue, we only need to add `.copy()` accordingly. ## How was this patch tested? New regression test case added in `ObjectHashAggregateSuite`. Author: Cheng Lian <lian@databricks.com> Closes #15976 from liancheng/investigate-oom.
* [SPARK-18408][ML] API Improvements for LSHYun Ni2016-11-286-221/+306
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? (1) Change output schema to `Array of Vector` instead of `Vectors` (2) Use `numHashTables` as the dimension of Array (3) Rename `RandomProjection` to `BucketedRandomProjectionLSH`, `MinHash` to `MinHashLSH` (4) Make `randUnitVectors/randCoefficients` private (5) Make Multi-Probe NN Search and `hashDistance` private for future discussion Saved for future PRs: (1) AND-amplification and `numHashFunctions` as the dimension of Vector are saved for a future PR. (2) `hashDistance` and MultiProbe NN Search needs more discussion. The current implementation is just a backward compatible one. ## How was this patch tested? Related unit tests are modified to make sure the performance of LSH are ensured, and the outputs of the APIs meets expectation. Author: Yun Ni <yunn@uber.com> Author: Yunni <Euler57721@gmail.com> Closes #15874 from Yunni/SPARK-18408-yunn-api-improvements.
* [SPARK-18117][CORE] Add test for TaskSetBlacklistImran Rashid2016-11-283-9/+292
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This adds tests to verify the interaction between TaskSetBlacklist and TaskSchedulerImpl. TaskSetBlacklist was introduced by SPARK-17675 but it neglected to add these tests. This change does not fix any bugs -- it is just for increasing test coverage. ## How was this patch tested? Jenkins Author: Imran Rashid <irashid@cloudera.com> Closes #15644 from squito/taskset_blacklist_test_update.
* [SPARK-17680][SQL][TEST] Added test cases for InMemoryRelationKazuaki Ishizaki2016-11-281-2/+146
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request adds test cases for the following cases: - keep all data types with null or without null - access `CachedBatch` disabling whole stage codegen - access only some columns in `CachedBatch` This PR is a part of https://github.com/apache/spark/pull/15219. Here are motivations to add these tests. When https://github.com/apache/spark/pull/15219 is enabled, the first two cases are handled by specialized (generated) code. The third one is a pitfall. In general, even for now, it would be helpful to increase test coverage. ## How was this patch tested? added test suites itself Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #15462 from kiszk/columnartestsuites.
* [SPARK-16282][SQL] Implement percentile SQL function.jiangxingbo2016-11-285-2/+518
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Implement percentile SQL function. It computes the exact percentile(s) of expr at pc with range in [0, 1]. ## How was this patch tested? Add a new testsuite `PercentileSuite` to test percentile directly. Updated related testcases in `ExpressionToSQLSuite`. Author: jiangxingbo <jiangxb1987@gmail.com> Author: 蒋星博 <jiangxingbo@meituan.com> Author: jiangxingbo <jiangxingbo@meituan.com> Closes #14136 from jiangxb1987/percentile.
* [SQL][MINOR] DESC should use 'Catalog' as partition providerWenchen Fan2016-11-281-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `CatalogTable` has a parameter named `tracksPartitionsInCatalog`, and in `CatalogTable.toString` we use `"Partition Provider: Catalog"` to represent it. This PR fixes `DESC TABLE` to make it consistent with `CatalogTable.toString`. ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16035 from cloud-fan/minor.
* [SPARK-18602] Set the version of org.codehaus.janino:commons-compiler to ↵Yin Huai2016-11-287-5/+18
| | | | | | | | | | | | | | | | 3.0.0 to match the version of org.codehaus.janino:janino ## What changes were proposed in this pull request? org.codehaus.janino:janino depends on org.codehaus.janino:commons-compiler and we have been upgraded to org.codehaus.janino:janino 3.0.0. However, seems we are still pulling in org.codehaus.janino:commons-compiler 2.7.6 because of calcite. It looks like an accident because we exclude janino from calcite (see here https://github.com/apache/spark/blob/branch-2.1/pom.xml#L1759). So, this PR upgrades org.codehaus.janino:commons-compiler to 3.0.0. ## How was this patch tested? jenkins Author: Yin Huai <yhuai@databricks.com> Closes #16025 from yhuai/janino-commons-compile.
* [SPARK-18535][UI][YARN] Redact sensitive information from Spark logs and UIMark Grover2016-11-289-12/+81
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds a new property called `spark.secret.redactionPattern` that allows users to specify a scala regex to decide which Spark configuration properties and environment variables in driver and executor environments contain sensitive information. When this regex matches the property or environment variable name, its value is redacted from the environment UI and various logs like YARN and event logs. This change uses this property to redact information from event logs and YARN logs. It also, updates the UI code to adhere to this property instead of hardcoding the logic to decipher which properties are sensitive. Here's an image of the UI post-redaction: ![image](https://cloud.githubusercontent.com/assets/1709451/20506215/4cc30654-b007-11e6-8aee-4cde253fba2f.png) Here's the text in the YARN logs, post-redaction: ``HADOOP_CREDSTORE_PASSWORD -> *********(redacted)`` Here's the text in the event logs, post-redaction: ``...,"spark.executorEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)","spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)",...`` ## How was this patch tested? 1. Unit tests are added to ensure that redaction works. 2. A YARN job reading data off of S3 with confidential information (hadoop credential provider password) being provided in the environment variables of driver and executor. And, afterwards, logs were grepped to make sure that no mention of secret password was present. It was also ensure that the job was able to read the data off of S3 correctly, thereby ensuring that the sensitive information was being trickled down to the right places to read the data. 3. The event logs were checked to make sure no mention of secret password was present. 4. UI environment tab was checked to make sure there was no secret information being displayed. Author: Mark Grover <mark@apache.org> Closes #15971 from markgrover/master_redaction.
* [SPARK-17732][SQL] Revert ALTER TABLE DROP PARTITION should support comparatorsWenchen Fan2016-11-288-189/+34
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? https://github.com/apache/spark/pull/15704 will fail if we use int literal in `DROP PARTITION`, and we have reverted it in branch-2.1. This PR reverts it in master branch, and add a regression test for it, to make sure the master branch is healthy. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16036 from cloud-fan/revert.
* [SPARK-18597][SQL] Do not push-down join conditions to the right side of a ↵Herman van Hovell2016-11-284-3/+72
| | | | | | | | | | | | | | | | LEFT ANTI join ## What changes were proposed in this pull request? We currently push down join conditions of a Left Anti join to both sides of the join. This is similar to Inner, Left Semi and Existence (a specialized left semi) join. The problem is that this changes the semantics of the join; a left anti join filters out rows that matches the join condition. This PR fixes this by only pushing down conditions to the left hand side of the join. This is similar to the behavior of left outer join. ## How was this patch tested? Added tests to `FilterPushdownSuite.scala` and created a SQLQueryTestSuite file for left anti joins with a regression test. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16026 from hvanhovell/SPARK-18597.
* [SPARK-17783][SQL] Hide Credentials in CREATE and DESC FORMATTED/EXTENDED a ↵gatorsmile2016-11-285-8/+62
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | PERSISTENT/TEMP Table for JDBC ### What changes were proposed in this pull request? We should never expose the Credentials in the EXPLAIN and DESC FORMATTED/EXTENDED command. However, below commands exposed the credentials. In the related PR: https://github.com/apache/spark/pull/10452 > URL patterns to specify credential seems to be vary between different databases. Thus, we hide the whole `url` value if it contains the keyword `password`. We also hide the `password` property. Before the fix, the command outputs look like: ``` SQL CREATE TABLE tab1 USING org.apache.spark.sql.jdbc OPTIONS ( url 'jdbc:h2:mem:testdb0;user=testUser;password=testPass', dbtable 'TEST.PEOPLE', user 'testUser', password '$password') DESC FORMATTED tab1 DESC EXTENDED tab1 ``` Before the fix, - The output of SQL statement EXPLAIN ``` == Physical Plan == ExecutedCommand +- CreateDataSourceTableCommand CatalogTable( Table: `tab1` Created: Wed Nov 16 23:00:10 PST 2016 Last Access: Wed Dec 31 15:59:59 PST 1969 Type: MANAGED Provider: org.apache.spark.sql.jdbc Storage(Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass])), false ``` - The output of `DESC FORMATTED` ``` ... |Storage Desc Parameters: | | | | url |jdbc:h2:mem:testdb0;user=testUser;password=testPass | | | dbtable |TEST.PEOPLE | | | user |testUser | | | password |testPass | | +----------------------------+------------------------------------------------------------------+-------+ ``` - The output of `DESC EXTENDED` ``` |# Detailed Table Information|CatalogTable( Table: `default`.`tab1` Created: Wed Nov 16 23:00:10 PST 2016 Last Access: Wed Dec 31 15:59:59 PST 1969 Type: MANAGED Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)] Provider: org.apache.spark.sql.jdbc Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass]))| | ``` After the fix, - The output of SQL statement EXPLAIN ``` == Physical Plan == ExecutedCommand +- CreateDataSourceTableCommand CatalogTable( Table: `tab1` Created: Wed Nov 16 22:43:49 PST 2016 Last Access: Wed Dec 31 15:59:59 PST 1969 Type: MANAGED Provider: org.apache.spark.sql.jdbc Storage(Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###])), false ``` - The output of `DESC FORMATTED` ``` ... |Storage Desc Parameters: | | | | url |### | | | dbtable |TEST.PEOPLE | | | user |testUser | | | password |### | | +----------------------------+------------------------------------------------------------------+-------+ ``` - The output of `DESC EXTENDED` ``` |# Detailed Table Information|CatalogTable( Table: `default`.`tab1` Created: Wed Nov 16 22:43:49 PST 2016 Last Access: Wed Dec 31 15:59:59 PST 1969 Type: MANAGED Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)] Provider: org.apache.spark.sql.jdbc Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###]))| | ``` ### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #15358 from gatorsmile/maskCredentials.
* [SPARK-18118][SQL] fix a compilation error due to nested JavaBeans\nRemove ↵Herman van Hovell2016-11-281-1/+1
| | | | this reference.
* [SPARK-18118][SQL] fix a compilation error due to nested JavaBeansKazuaki Ishizaki2016-11-282-2/+437
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR avoids a compilation error due to more than 64KB Java byte code size. This error occur since generated java code `SpecificSafeProjection.apply()` for nested JavaBeans is too big. This PR avoids this compilation error by splitting a big code chunk into multiple methods by calling `CodegenContext.splitExpression` at `InitializeJavaBean.doGenCode` An object reference for JavaBean is stored to an instance variable `javaBean...`. Then, the instance variable will be referenced in the split methods. Generated code with this PR ```` /* 22098 */ private void apply130_0(InternalRow i) { ... /* 22125 */ boolean isNull238 = i.isNullAt(2); /* 22126 */ InternalRow value238 = isNull238 ? null : (i.getStruct(2, 3)); /* 22127 */ boolean isNull236 = false; /* 22128 */ test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value236 = null; /* 22129 */ if (!false && isNull238) { /* 22130 */ /* 22131 */ final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value239 = null; /* 22132 */ isNull236 = true; /* 22133 */ value236 = value239; /* 22134 */ } else { /* 22135 */ /* 22136 */ final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value241 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$Nesting1(); /* 22137 */ this.javaBean14 = value241; /* 22138 */ if (!false) { /* 22139 */ apply25_0(i); /* 22140 */ apply25_1(i); /* 22141 */ apply25_2(i); /* 22142 */ } /* 22143 */ isNull236 = false; /* 22144 */ value236 = value241; /* 22145 */ } /* 22146 */ this.javaBean.setField2(value236); /* 22147 */ /* 22148 */ } ... /* 22928 */ public java.lang.Object apply(java.lang.Object _i) { /* 22929 */ InternalRow i = (InternalRow) _i; /* 22930 */ /* 22931 */ final test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean value1 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean(); /* 22932 */ this.javaBean = value1; /* 22933 */ if (!false) { /* 22934 */ apply130_0(i); /* 22935 */ apply130_1(i); /* 22936 */ apply130_2(i); /* 22937 */ apply130_3(i); /* 22938 */ apply130_4(i); /* 22939 */ } /* 22940 */ if (false) { /* 22941 */ mutableRow.setNullAt(0); /* 22942 */ } else { /* 22943 */ /* 22944 */ mutableRow.update(0, value1); /* 22945 */ } /* 22946 */ /* 22947 */ return mutableRow; /* 22948 */ } ```` ## How was this patch tested? added a test suite into `JavaDatasetSuite.java` Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #16032 from kiszk/SPARK-18118.
* [SPARK-18604][SQL] Make sure CollapseWindow returns the attributes in the ↵Herman van Hovell2016-11-282-6/+9
| | | | | | | | | | | | | | same order. ## What changes were proposed in this pull request? The `CollapseWindow` optimizer rule changes the order of output attributes. This modifies the output of the plan, which the optimizer cannot do. This also breaks things like `collect()` for which we use a `RowEncoder` that assumes that the output attributes of the executed plan are equal to those outputted by the logical plan. ## How was this patch tested? I have updated an incorrect test in `CollapseWindowSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16027 from hvanhovell/SPARK-18604.
* [SPARK-18585][SQL] Use `ev.isNull = "false"` if possible for Janino to have ↵Takuya UESHIN2016-11-272-11/+6
| | | | | | | | | | | | | | | | a chance to optimize. ## What changes were proposed in this pull request? Janino can optimize `true ? a : b` into `a` or `false ? a : b` into `b`, or if/else with literal condition, so we should use literal as `ev.isNull` if possible. ## How was this patch tested? Existing tests. Author: Takuya UESHIN <ueshin@happy-camper.st> Closes #16008 from ueshin/issues/SPARK-18585.
* [SPARK-18482][SQL] make sure Spark can access the table metadata created by ↵Wenchen Fan2016-11-272-43/+251
| | | | | | | | | | | | | | | | | | older version of spark ## What changes were proposed in this pull request? In Spark 2.1, we did a lot of refactor for `HiveExternalCatalog` and related code path. These refactor may introduce external behavior changes and break backward compatibility. e.g. http://issues.apache.org/jira/browse/SPARK-18464 To avoid future compatibility problems of `HiveExternalCatalog`, this PR dumps some typical table metadata from tables created by 2.0, and test if they can recognized by current version of Spark. ## How was this patch tested? test only change Author: Wenchen Fan <wenchen@databricks.com> Closes #16003 from cloud-fan/test.
* [SPARK-18594][SQL] Name Validation of Databases/Tablesgatorsmile2016-11-274-27/+57
| | | | | | | | | | | | | | ### What changes were proposed in this pull request? Currently, the name validation checks are limited to table creation. It is enfored by Analyzer rule: `PreWriteCheck`. However, table renaming and database creation have the same issues. It makes more sense to do the checks in `SessionCatalog`. This PR is to add it into `SessionCatalog`. ### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #16018 from gatorsmile/nameValidate.
* [SPARK-17251][SQL] Improve `OuterReference` to be `NamedExpression`Dongjoon Hyun2016-11-263-2/+53
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently, `OuterReference` is not `NamedExpression`. So, it raises 'ClassCastException` when it used in projection lists of IN correlated subqueries. This PR aims to support that by making `OuterReference` as `NamedExpression` to show correct error messages. ```scala scala> sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES 1, 2 AS t1(a)") scala> sql("CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES 1 AS t2(b)") scala> sql("SELECT a FROM t1 WHERE a IN (SELECT a FROM t2)").show java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.OuterReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression ``` ## How was this patch tested? Pass the Jenkins test with new test cases. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16015 from dongjoon-hyun/SPARK-17251-2.
* [WIP][SQL][DOC] Fix incorrect `code` tagWeiqing Yang2016-11-262-2/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is to fix incorrect `code` tag in `sql-programming-guide.md` ## How was this patch tested? Manually. Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #15941 from weiqingy/fixtag.
* [SPARK-18481][ML] ML 2.1 QA: Remove deprecated methods for MLYanbo Liang2016-11-2616-107/+144
| | | | | | | | | | | | ## What changes were proposed in this pull request? Remove deprecated methods for ML. ## How was this patch tested? Existing tests. Author: Yanbo Liang <ybliang8@gmail.com> Closes #15913 from yanboliang/spark-18481.