aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19459] Support for nested char/varchar fields in ORCHerman van Hovell2017-02-233-19/+100
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR is a small follow-up on https://github.com/apache/spark/pull/16804. This PR also adds support for nested char/varchar fields in orc. ## How was this patch tested? I have added a regression test to the OrcSourceSuite. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #17030 from hvanhovell/SPARK-19459-follow-up.
* [SPARK-19691][SQL] Fix ClassCastException when calculating percentile of ↵Takeshi Yamamuro2017-02-233-37/+45
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | decimal column ## What changes were proposed in this pull request? This pr fixed a class-cast exception below; ``` scala> spark.range(10).selectExpr("cast (id as decimal) as x").selectExpr("percentile(x, 0.5)").collect() java.lang.ClassCastException: org.apache.spark.sql.types.Decimal cannot be cast to java.lang.Number at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:141) at org.apache.spark.sql.catalyst.expressions.aggregate.Percentile.update(Percentile.scala:58) at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.update(interfaces.scala:514) at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171) at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$1$$anonfun$applyOrElse$1.apply(AggregationIterator.scala:171) at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:187) at org.apache.spark.sql.execution.aggregate.AggregationIterator$$anonfun$generateProcessRow$1.apply(AggregationIterator.scala:181) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.processInputs(ObjectAggregationIterator.scala:151) at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.<init>(ObjectAggregationIterator.scala:78) at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec$$anonfun$doExecute$1$$anonfun$2.apply(ObjectHashAggregateExec.scala:109) at ``` This fix simply converts catalyst values (i.e., `Decimal`) into scala ones by using `CatalystTypeConverters`. ## How was this patch tested? Added a test in `DataFrameSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17028 from maropu/SPARK-19691.
* [SPARK-19695][SQL] Throw an exception if a `columnNameOfCorruptRecord` field ↵Takeshi Yamamuro2017-02-224-4/+56
| | | | | | | | | | | | | | violates requirements in json formats ## What changes were proposed in this pull request? This pr comes from #16928 and fixed a json behaviour along with the CSV one. ## How was this patch tested? Added tests in `JsonSuite`. Author: Takeshi Yamamuro <yamamuro@apache.org> Closes #17023 from maropu/SPARK-19695.
* [SPARK-16122][CORE] Add rest api for job environmentuncleGen2017-02-223-0/+71
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? add rest api for job environment. ## How was this patch tested? existing ut. Author: uncleGen <hustyugm@gmail.com> Closes #16949 from uncleGen/SPARK-16122.
* [SPARK-15615][SQL] Add an API to load DataFrame from Dataset[String] storing ↵pj.fanning2017-02-221-2/+18
| | | | | | | | | | | | | | | | | | | | | JSON ## What changes were proposed in this pull request? SPARK-15615 proposes replacing the sqlContext.read.json(rdd) with a dataset equivalent. SPARK-15463 adds a CSV API for reading from Dataset[String] so this keeps the API consistent. I am deprecating the existing RDD based APIs. ## How was this patch tested? There are existing tests. I left most tests to use the existing APIs as they delegate to the new json API. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: pj.fanning <pj.fanning@workday.com> Author: PJ Fanning <pjfanning@users.noreply.github.com> Closes #16895 from pjfanning/SPARK-15615.
* [SPARK-19658][SQL] Set NumPartitions of RepartitionByExpression In ParserXiao Li2017-02-2210-32/+74
| | | | | | | | | | | | | | ### What changes were proposed in this pull request? Currently, if `NumPartitions` is not set in RepartitionByExpression, we will set it using `spark.sql.shuffle.partitions` during Planner. However, this is not following the general resolution process. This PR is to set it in `Parser` and then `Optimizer` can use the value for plan optimization. ### How was this patch tested? Added a test case. Author: Xiao Li <gatorsmile@gmail.com> Closes #16988 from gatorsmile/resolveRepartition.
* [SPARK-19554][UI,YARN] Allow SHS URL to be used for tracking in YARN RM.Marcelo Vanzin2017-02-226-6/+167
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Allow an application to use the History Server URL as the tracking URL in the YARN RM, so there's still a link to the web UI somewhere in YARN even if the driver's UI is disabled. This is useful, for example, if an admin wants to disable the driver UI by default for applications, since it's harder to secure it (since it involves non trivial ssl certificate and auth management that admins may not want to expose to user apps). This needs to be opt-in, because of the way the YARN proxy works, so a new configuration was added to enable the option. The YARN RM will proxy requests to live AMs instead of redirecting the client, so pages in the SHS UI will not render correctly since they'll reference invalid paths in the RM UI. The proxy base support in the SHS cannot be used since that would prevent direct access to the SHS. So, to solve this problem, for the feature to work end-to-end, a new YARN-specific filter was added that detects whether the requests come from the proxy and redirects the client appropriatly. The SHS admin has to add this filter manually if they want the feature to work. Tested with new unit test, and by running with the documented configuration set in a test cluster. Also verified the driver UI is used when it's enabled. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16946 from vanzin/SPARK-19554.
* [SPARK-19666][SQL] Skip a property without getter in Java schema inference ↵hyukjinkwon2017-02-225-32/+54
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | and allow empty bean in encoder creation ## What changes were proposed in this pull request? This PR proposes to fix two. **Skip a property without a getter in beans** Currently, if we use a JavaBean without the getter as below: ```java public static class BeanWithoutGetter implements Serializable { private String a; public void setA(String a) { this.a = a; } } BeanWithoutGetter bean = new BeanWithoutGetter(); List<BeanWithoutGetter> data = Arrays.asList(bean); spark.createDataFrame(data, BeanWithoutGetter.class).show(); ``` - Before It throws an exception as below: ``` java.lang.NullPointerException at org.spark_project.guava.reflect.TypeToken.method(TypeToken.java:465) at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:126) at org.apache.spark.sql.catalyst.JavaTypeInference$$anonfun$2.apply(JavaTypeInference.scala:125) ``` - After ``` ++ || ++ || ++ ``` **Supports empty bean in encoder creation** ```java public static class EmptyBean implements Serializable {} EmptyBean bean = new EmptyBean(); List<EmptyBean> data = Arrays.asList(bean); spark.createDataset(data, Encoders.bean(EmptyBean.class)).show(); ``` - Before throws an exception as below: ``` java.lang.UnsupportedOperationException: Cannot infer type for class EmptyBean because it is not bean-compliant at org.apache.spark.sql.catalyst.JavaTypeInference$.org$apache$spark$sql$catalyst$JavaTypeInference$$serializerFor(JavaTypeInference.scala:436) at org.apache.spark.sql.catalyst.JavaTypeInference$.serializerFor(JavaTypeInference.scala:341) ``` - After ``` ++ || ++ || ++ ``` ## How was this patch tested? Unit test in `JavaDataFrameSuite`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #17013 from HyukjinKwon/SPARK-19666.
* [SPARK-19616][SPARKR] weightCol and aggregationDepth should be improved for ↵wm624@hotmail.com2017-02-228-19/+50
| | | | | | | | | | | | | | | | | some SparkR APIs ## What changes were proposed in this pull request? This is a follow-up PR of #16800 When doing SPARK-19456, we found that "" should be consider a NULL column name and should not be set. aggregationDepth should be exposed as an expert parameter. ## How was this patch tested? Existing tests. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16945 from wangmiao1981/svc.
* [SPARK-19405][STREAMING] Support for cross-account Kinesis reads via STSAdam Budde2017-02-2217-83/+407
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | - Add dependency on aws-java-sdk-sts - Replace SerializableAWSCredentials with new SerializableCredentialsProvider interface - Make KinesisReceiver take SerializableCredentialsProvider as argument and pass credential provider to KCL - Add new implementations of KinesisUtils.createStream() that take STS arguments - Make JavaKinesisStreamSuite test the entire KinesisUtils Java API - Update KCL/AWS SDK dependencies to 1.7.x/1.11.x ## What changes were proposed in this pull request? [JIRA link with detailed description.](https://issues.apache.org/jira/browse/SPARK-19405) * Replace SerializableAWSCredentials with new SerializableKCLAuthProvider class that takes 5 optional config params for configuring AWS auth and returns the appropriate credential provider object * Add new public createStream() APIs for specifying these parameters in KinesisUtils ## How was this patch tested? * Manually tested using explicit keypair and instance profile to read data from Kinesis stream in separate account (difficult to write a test orchestrating creation and assumption of IAM roles across separate accounts) * Expanded JavaKinesisStreamSuite to test the entire Java API in KinesisUtils ## License acknowledgement This contribution is my original work and that I license the work to the project under the project’s open source license. Author: Budde <budde@amazon.com> Closes #16744 from budde/master.
* [SPARK-13721][SQL] Make GeneratorOuter unresolved.Bogdan Raducanu2017-02-222-2/+8
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is a small change to make GeneratorOuter always unresolved. It is mostly no-op change but makes it more clear since GeneratorOuter shouldn't survive analysis phase. This requires also handling in ResolveAliases rule. ## How was this patch tested? Existing generator tests. Author: Bogdan Raducanu <bogdan@databricks.com> Author: Reynold Xin <rxin@databricks.com> Closes #17026 from bogdanrdc/PR16958.
* [SPARK-19679][ML] Destroy broadcasted object without blockingZheng RuiFeng2017-02-223-3/+3
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Destroy broadcasted object without blocking use `find mllib -name '*.scala' | xargs -i bash -c 'egrep "destroy" -n {} && echo {}'` ## How was this patch tested? existing tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #17016 from zhengruifeng/destroy_without_block.
* [SPARK-19694][ML] Add missing 'setTopicDistributionCol' for LDAModelZheng RuiFeng2017-02-221-0/+3
| | | | | | | | | | | ## What changes were proposed in this pull request? Add missing 'setTopicDistributionCol' for LDAModel ## How was this patch tested? existing tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #17021 from zhengruifeng/lda_outputCol.
* [SPARK-19670][SQL][TEST] Enable Bucketed Table Reading and Writing Testing ↵Xiao Li2017-02-214-21/+101
| | | | | | | | | | | | | | Without Hive Support ### What changes were proposed in this pull request? Bucketed table reading and writing does not need Hive support. We can move the test cases from `sql/hive` to `sql/core`. After this PR, we can improve the test case coverage. Bucket table reading and writing can be tested with and without Hive support. ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #17004 from gatorsmile/mvTestCaseForBuckets.
* [SPARK-19652][UI] Do auth checks for REST API access.Marcelo Vanzin2017-02-217-45/+123
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | The REST API has a security filter that performs auth checks based on the UI root's security manager. That works fine when the UI root is the app's UI, but not when it's the history server. In the SHS case, all users would be allowed to see all applications through the REST API, even if the UI itself wouldn't be available to them. This change adds auth checks for each app access through the API too, so that only authorized users can see the app's data. The change also modifies the existing security filter to use `HttpServletRequest.getRemoteUser()`, which is used in other places. That is not necessarily the same as the principal's name; for example, when using Hadoop's SPNEGO auth filter, the remote user strips the realm information, which then matches the user name registered as the owner of the application. I also renamed the UIRootFromServletContext trait to a more generic name since I'm using it to store more context information now. Tested manually with an authentication filter enabled. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16978 from vanzin/SPARK-19652.
* [SPARK-19626][YARN] Using the correct config to set credentials update timeKent Yao2017-02-211-1/+1
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In https://github.com/apache/spark/pull/14065, we introduced a configurable credential manager for Spark running on YARN. Also two configs `spark.yarn.credentials.renewalTime` and `spark.yarn.credentials.updateTime` were added, one is for the credential renewer and the other updater. But now we just query `spark.yarn.credentials.renewalTime` by mistake during CREDENTIALS UPDATING, where should be actually `spark.yarn.credentials.updateTime` . This PR fixes this mistake. ## How was this patch tested? existing test cc jerryshao vanzin Author: Kent Yao <yaooqinn@hotmail.com> Closes #16955 from yaooqinn/cred_update.
* [SPARK-19337][ML][DOC] Documentation and examples for LinearSVCYuhao Yang2017-02-214-0/+196
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Documentation and examples (Java, scala, python, R) for LinearSVC ## How was this patch tested? local doc generation Author: Yuhao Yang <yuhao.yang@intel.com> Closes #16968 from hhbyyh/mlsvmdoc.
* [SPARK-18922][TESTS] Fix new test failures on Windows due to path and ↵hyukjinkwon2017-02-205-28/+35
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | resource not closed ## What changes were proposed in this pull request? This PR proposes to fix new test failures on WIndows as below: **Before** ``` KafkaRelationSuite: - test late binding start offsets *** FAILED *** (7 seconds, 679 milliseconds) Cause: java.nio.file.FileSystemException: C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log -> C:\projects\spark\target\tmp\spark-4c4b0cd1-4cb7-4908-949d-1b0cc8addb50\topic-4-0\00000000000000000000.log.deleted: The process cannot access the file because it is being used by another process. KafkaSourceSuite: - deserialization of initial offset with Spark 2.1.0 *** FAILED *** (3 seconds, 542 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-97ef64fc-ae61-4ce3-ac59-287fd38bd824 - deserialization of initial offset written by Spark 2.1.0 *** FAILED *** (60 milliseconds) java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/external/kafka-0-10-sql/target/scala-2.11/test-classes/kafka-source-initial-offset-version-2.1.0.b HiveDDLSuite: - partitioned table should always put partition columns at the end of table schema *** FAILED *** (657 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-f1b83d09-850a-4bba-8e43-a2a28dfaa757; DDLSuite: - create a data source table without schema *** FAILED *** (94 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-a3f3c161-afae-4d6f-9182-e8642f77062b; - SET LOCATION for managed table *** FAILED *** (219 milliseconds) org.apache.spark.sql.catalyst.errors.package$TreeNodeException: execute, tree: Exchange SinglePartit +- *HashAggregate(keys=[], functions=[partial_count(1)], output=[count#99367L]) +- *FileScan parquet default.tbl[] Batched: true, Format: Parquet, Location: InMemoryFileIndex[file:/C:projectsspark arget mpspark-15be2f2f-4ea9-4c47-bfee-1b7b49363033], PartitionFilters: [], PushedFilters: [], ReadSchema: struct<> - insert data to a data source table which has a not existed location should succeed *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-34987671-e8d1-4624-ba5b-db1012e1246b; - insert into a data source table with no existed partition location should succeed *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4c6ccfbf-4091-4032-9fbc-3d40c58267d5; - read data from a data source table which has a not existed location should succeed *** FAILED *** (0 milliseconds) - read data from a data source table with no existed partition location should succeed *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-6af39e37-abd1-44e8-ac68-e2dfcf67a2f3; InputOutputMetricsSuite: - output metrics on records written *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-cd69ee77-88f2-4202-bed6-19c0ee05ef55\InputOutputMetricsSuite, expected: file:/// - output metrics on records written - new Hadoop API *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-b69e8fcb-047b-4de8-9cdf-5f026efb6762\InputOutputMetricsSuite, expected: file:/// ``` **After** ``` KafkaRelationSuite: - test late binding start offsets !!! CANCELED !!! (62 milliseconds) KafkaSourceSuite: - deserialization of initial offset with Spark 2.1.0 (5 seconds, 341 milliseconds) - deserialization of initial offset written by Spark 2.1.0 (910 milliseconds) HiveDDLSuite: - partitioned table should always put partition columns at the end of table schema (2 seconds) DDLSuite: - create a data source table without schema (828 milliseconds) - SET LOCATION for managed table (406 milliseconds) - insert data to a data source table which has a not existed location should succeed (406 milliseconds) - insert into a data source table with no existed partition location should succeed (453 milliseconds) - read data from a data source table which has a not existed location should succeed (94 milliseconds) - read data from a data source table with no existed partition location should succeed (265 milliseconds) InputOutputMetricsSuite: - output metrics on records written (172 milliseconds) - output metrics on records written - new Hadoop API (297 milliseconds) ``` ## How was this patch tested? Fixed tests in `InputOutputMetricsSuite`, `KafkaRelationSuite`, `KafkaSourceSuite`, `DDLSuite.scala` and `HiveDDLSuite`. Manually tested via AppVeyor as below: `InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/ex8nvwa6tsh7rmto `KafkaRelationSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/h8dlcowew52y8ncw `KafkaSourceSuite`: https://ci.appveyor.com/project/spark-test/spark/build/634-20170219-windows-test/job/9ybgjl7yeubxcre4 `DDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/635-20170219-windows-test `HiveDDLSuite`: https://ci.appveyor.com/project/spark-test/spark/build/633-20170219-windows-test/job/up6o9n47er087ltb Author: hyukjinkwon <gurwls223@gmail.com> Closes #16999 from HyukjinKwon/windows-fix.
* [SPARK-19508][CORE] Improve error message when binding service failsLiang-Chi Hsieh2017-02-201-6/+21
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Utils provides a helper function to bind service on port. This function can bind the service to a random free port. However, if the binding fails on a random free port, the retrying and final exception messages look confusing. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. 17/02/06 16:25:43 WARN Utils: Service 'sparkDriver' could not bind on port 0. Attempting port 1. ... 17/02/06 16:25:43 ERROR SparkContext: Error initializing SparkContext. java.net.BindException: Can't assign requested address: Service 'sparkDriver' failed after 16 retries (starting from 0)! Consider explicitly setting the appropriate port for the service 'sparkDriver' (for example spark.ui.port for SparkUI) to an available port or increasing spark.port.maxRetries. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16851 from viirya/better-log-message.
* [SPARK-19669][HOTFIX][SQL] sessionState access privileges compiled failed in ↵windpiger2017-02-201-1/+1
| | | | | | | | | | | | | | | | | TestSQLContext ## What changes were proposed in this pull request? In [SPARK-19669](https://github.com/apache/spark/commit/0733a54a4517b82291efed9ac7f7407d9044593c) change the sessionState access privileges from private to public, this lead to the compile failed in TestSQLContext this pr is a hotfix for this. ## How was this patch tested? N/A Author: windpiger <songjun@outlook.com> Closes #17008 from windpiger/hotfixcompile.
* [SPARK-19669][SQL] Open up visibility for sharedState, sessionState, and a ↵Reynold Xin2017-02-203-8/+27
| | | | | | | | | | | | | | | | few other functions ## What changes were proposed in this pull request? To ease debugging, most of Spark SQL internals have public level visibility. Two of the most important internal states, sharedState and sessionState, however, are package private. It would make more sense to open these up as well with clear documentation that they are internal. In addition, users currently have way to set active/default SparkSession, but no way to actually get them back. We should open those up as well. ## How was this patch tested? N/A - only visibility change. Author: Reynold Xin <rxin@databricks.com> Closes #17002 from rxin/SPARK-19669.
* [SPARK-15453][SQL][FOLLOW-UP] FileSourceScanExec to extract `outputOrdering` ↵Xiao Li2017-02-201-92/+137
| | | | | | | | | | | | | | information ### What changes were proposed in this pull request? `outputOrdering` is also dependent on whether the bucket has more than one files. The test cases fail when we try to move them to sql/core. This PR is to fix the test cases introduced in https://github.com/apache/spark/pull/14864 and add a test case to verify [the related logics](https://github.com/tejasapatil/spark/blob/070c24994747c0479fb2520774ede27ff1cf8cac/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala#L197-L206). ### How was this patch tested? N/A Author: Xiao Li <gatorsmile@gmail.com> Closes #16994 from gatorsmile/bucketingTS.
* [SPARK-19646][CORE][STREAMING] binaryRecords replicates records in scala APISean Owen2017-02-204-156/+53
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Use `BytesWritable.copyBytes`, not `getBytes`, because `getBytes` returns the underlying array, which may be reused when repeated reads don't need a different size, as is the case with binaryRecords APIs ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16974 from srowen/SPARK-19646.
* [SPARK-19563][SQL] avoid unnecessary sort in FileFormatWriterWenchen Fan2017-02-191-99/+90
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In `FileFormatWriter`, we will sort the input rows by partition columns and bucket id and sort columns, if we want to write data out partitioned or bucketed. However, if the data is already sorted, we will sort it again, which is unnecssary. This PR removes the sorting logic in `FileFormatWriter` and use `SortExec` instead. We will not add `SortExec` if the data is already sorted. ## How was this patch tested? I did a micro benchmark manually ``` val df = spark.range(10000000).select($"id", $"id" % 10 as "part").sort("part") spark.time(df.write.partitionBy("part").parquet("/tmp/test")) ``` The result was about 6.4 seconds before this PR, and is 5.7 seconds afterwards. close https://github.com/apache/spark/pull/16724 Author: Wenchen Fan <wenchen@databricks.com> Closes #16898 from cloud-fan/writer.
* [SPARK-19598][SQL] Remove the alias parameter in UnresolvedRelationwindpiger2017-02-1912-75/+51
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Remove the alias parameter in `UnresolvedRelation`, and use `SubqueryAlias` to replace it. This can simplify some `match case` situations. For example, the broadcast hint pull request can have one fewer case https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala#L57-L61 ## How was this patch tested? add some unit tests Author: windpiger <songjun@outlook.com> Closes #16956 from windpiger/removeUnresolveTableAlias.
* [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 featuresSean Owen2017-02-1945-1574/+662
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code. ## How was this patch tested? Jenkins tests Author: Sean Owen <sowen@cloudera.com> Closes #16964 from srowen/SPARK-19534.
* [SPARK-19533][EXAMPLES] Convert Java tests to use lambdas, Java 8 featuresSean Owen2017-02-1952-1018/+380
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Convert Java tests to use lambdas, Java 8 features. ## How was this patch tested? Jenkins tests. Author: Sean Owen <sowen@cloudera.com> Closes #16961 from srowen/SPARK-19533.
* [SPARK-19450] Replace askWithRetry with askSync.jinxing2017-02-1924-119/+58
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated. As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218): >askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it. Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`. ## How was this patch tested? This PR doesn't change code logic, existing unit test can cover. Author: jinxing <jinxing@meituan.com> Closes #16790 from jinxing64/SPARK-19450.
* [SPARK-19550][BUILD][WIP] Addendum: select Java 1.7 for scalac 2.10, stillSean Owen2017-02-191-1/+3
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Go back to selecting source/target 1.7 for Scala 2.10 builds, because the SBT-based build for 2.10 won't work otherwise. ## How was this patch tested? Existing tests, but, we need to verify this vs what the SBT build would exactly run on Jenkins Author: Sean Owen <sowen@cloudera.com> Closes #16983 from srowen/SPARK-19550.3.
* [SPARK-19447] Make Range operator generate "recordsRead" metricAla Luszczak2017-02-186-155/+125
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The Range was modified to produce "recordsRead" metric instead of "generated rows". The tests were updated and partially moved to SQLMetricsSuite. ## How was this patch tested? Unit tests. Author: Ala Luszczak <ala@databricks.com> Closes #16960 from ala/range-records-read.
* [SPARK-19263] DAGScheduler should avoid sending conflicting task set.jinxing2017-02-183-3/+91
| | | | | | | | | | | | | | | | | | | | | | | | | In current `DAGScheduler handleTaskCompletion` code, when event.reason is `Success`, it will first do `stage.pendingPartitions -= task.partitionId`, which maybe a bug when `FetchFailed` happens. **Think about below** 1. Stage 0 runs and generates shuffle output data. 2. Stage 1 reads the output from stage 0 and generates more shuffle data. It has two tasks: ShuffleMapTask1 and ShuffleMapTask2, and these tasks are launched on executorA. 3. ShuffleMapTask1 fails to fetch blocks locally and sends a FetchFailed to the driver. The driver marks executorA as lost and updates failedEpoch; 4. The driver resubmits stage 0 so the missing output can be re-generated, and then once it completes, resubmits stage 1 with ShuffleMapTask1x and ShuffleMapTask2x. 5. ShuffleMapTask2 (from the original attempt of stage 1) successfully finishes on executorA and sends Success back to driver. This causes DAGScheduler::handleTaskCompletion to remove partition 2 from stage.pendingPartitions (line 1149), but it does not add the partition to the set of output locations (line 1192), because the task’s epoch is less than the failure epoch for the executor (because of the earlier failure on executor A) 6. ShuffleMapTask1x successfully finishes on executorB, causing the driver to remove partition 1 from stage.pendingPartitions. Combined with the previous step, this means that there are no more pending partitions for the stage, so the DAGScheduler marks the stage as finished (line 1196). However, the shuffle stage is not available (line 1215) because the completion for ShuffleMapTask2 was ignored because of its epoch, so the DAGScheduler resubmits the stage. 7. ShuffleMapTask2x is still running, so when TaskSchedulerImpl::submitTasks is called for the re-submitted stage, it throws an error, because there’s an existing active task set **In this fix** If a task completion is from a previous stage attempt and the epoch is too low (i.e., it was from a failed executor), don't remove the corresponding partition from pendingPartitions. Author: jinxing <jinxing@meituan.com> Author: jinxing <jinxing6042@126.com> Closes #16620 from jinxing64/SPARK-19263.
* [MLLIB][TYPO] Replace LeastSquaresAggregator with LogisticAggregatorMoussa Taifi2017-02-181-1/+1
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Replace LeastSquaresAggregator with LogisticAggregator in the require statement of the merge op. ## How was this patch tested? Simple message fix. Author: Moussa Taifi <moutai10@gmail.com> Closes #16903 from moutai/master.
* [SPARK-19550] Follow-up: fixed a typo that fails the ↵Shuai Lin2017-02-181-1/+1
| | | | | | | | | | | | | | | | dev/make-distribution.sh script. ## What changes were proposed in this pull request? Fixed a typo in `dev/make-distribution.sh` script that sets the MAVEN_OPTS variable, introduced [here](https://github.com/apache/spark/commit/0e24054#diff-ba2c046d92a1d2b5b417788bfb5cb5f8R149). ## How was this patch tested? Run `dev/make-distribution.sh` manually. Author: Shuai Lin <linshuai2012@gmail.com> Closes #16984 from lins05/fix-spark-make-distribution-after-removing-java7.
* [SPARK-19639][SPARKR][EXAMPLE] Add spark.svmLinear example and update vignetteswm624@hotmail.com2017-02-173-0/+65
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We recently add the spark.svmLinear API for SparkR. We need to add an example and update the vignettes. ## How was this patch tested? Manually run example. Author: wm624@hotmail.com <wm624@hotmail.com> Closes #16969 from wangmiao1981/example.
* [SPARK-19617][SS] Fix the race condition when starting and stopping a query ↵Shixiong Zhu2017-02-178-107/+64
| | | | | | | | | | | | | | | | | | | | | | | | | | quickly ## What changes were proposed in this pull request? The streaming thread in StreamExecution uses the following ways to check if it should exit: - Catch an InterruptException. - `StreamExecution.state` is TERMINATED. When starting and stopping a query quickly, the above two checks may both fail: - Hit [HADOOP-14084](https://issues.apache.org/jira/browse/HADOOP-14084) and swallow InterruptException - StreamExecution.stop is called before `state` becomes `ACTIVE`. Then [runBatches](https://github.com/apache/spark/blob/dcc2d540a53f0bd04baead43fdee1c170ef2b9f3/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala#L252) changes the state from `TERMINATED` to `ACTIVE`. If the above cases both happen, the query will hang forever. This PR changes `state` to `AtomicReference` and uses`compareAndSet` to make sure we only change the state from `INITIALIZING` to `ACTIVE`. It also removes the `runUninterruptibly` hack from ``HDFSMetadata`, because HADOOP-14084 won't cause any problem after we fix the race condition. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16947 from zsxwing/SPARK-19617.
* [SPARKR][EXAMPLES] update examples to stop spark sessionFelix Cheung2017-02-1715-18/+46
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? stop session at end of example ## How was this patch tested? manual Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16973 from felixcheung/rexamples.
* [SPARK-18285][SPARKR] SparkR approxQuantile supports input multiple columnsYanbo Liang2017-02-173-14/+31
| | | | | | | | | | | | ## What changes were proposed in this pull request? SparkR ```approxQuantile``` supports input multiple columns. ## How was this patch tested? Unit test. Author: Yanbo Liang <ybliang8@gmail.com> Closes #16951 from yanboliang/spark-19619.
* [SPARK-19517][SS] KafkaSource fails to initialize partition offsetsRoberto Agostino Vitillo2017-02-174-7/+131
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes a bug in `KafkaSource` with the (de)serialization of the length of the JSON string that contains the initial partition offsets. ## How was this patch tested? I ran the test suite for spark-sql-kafka-0-10. Author: Roberto Agostino Vitillo <ra.vitillo@gmail.com> Closes #16857 from vitillo/kafka_source_fix.
* [SPARK-18986][CORE] ExternalAppendOnlyMap shouldn't fail when forced to ↵Liang-Chi Hsieh2017-02-172-5/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | spill before calling its iterator ## What changes were proposed in this pull request? `ExternalAppendOnlyMap.forceSpill` now uses an assert to check if an iterator is not null in the map. However, the assertion is only true after the map is asked for iterator. Before it, if another memory consumer asks more memory than currently available, `ExternalAppendOnlyMap.forceSpill` is also be called too. In this case, we will see failure like this: [info] java.lang.AssertionError: assertion failed [info] at scala.Predef$.assert(Predef.scala:156) [info] at org.apache.spark.util.collection.ExternalAppendOnlyMap.forceSpill(ExternalAppendOnlyMap.scala:196) [info] at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:111) [info] at org.apache.spark.util.collection.ExternalAppendOnlyMapSuite$$anonfun$13.apply$mcV$sp(ExternalAppendOnlyMapSuite.scala:294) This fixing is motivated by http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-AssertionError-assertion-failed-tc20277.html. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16387 from viirya/fix-externalappendonlymap.
* [SPARK-19500] [SQL] Fix off-by-one bug in BytesToBytesMapDavies Liu2017-02-172-2/+43
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Radix sort require that half of array as free (as temporary space), so we use 0.5 as the scale factor to make sure that BytesToBytesMap will not have more items than 1/2 of capacity. Turned out this is not true, the current implementation of append() could leave 1 more item than the threshold (1/2 of capacity) in the array, which break the requirement of radix sort (fail the assert in 2.2, or fail to insert into InMemorySorter in 2.1). This PR fix the off-by-one bug in BytesToBytesMap. This PR also fix a bug that the array will never grow if it fail to grow once (stay as initial capacity), introduced by #15722 . ## How was this patch tested? Added regression test. Author: Davies Liu <davies@databricks.com> Closes #16844 from davies/off_by_one.
* [SPARK-19622][WEBUI] Fix a http error in a paged table when using a `Go` ↵Stan Zhai2017-02-171-2/+3
| | | | | | | | | | | | | | | | | | button to search. ## What changes were proposed in this pull request? The search function of paged table is not available because of we don't skip the hash data of the reqeust path. ![](https://issues.apache.org/jira/secure/attachment/12852996/screenshot-1.png) ## How was this patch tested? Tested manually with my browser. Author: Stan Zhai <zhaishidan@haizhi.com> Closes #16953 from stanzhai/fix-webui-paged-table.
* [MINOR][PYTHON] Fix typo docstring: 'top' -> 'topic'Rolando Espinoza2017-02-171-1/+1
| | | | | | | | | | ## What changes were proposed in this pull request? Fix typo in docstring. Author: Rolando Espinoza <rndmax84@gmail.com> Closes #16967 from rolando/pyspark-doc-typo.
* [BUILD] Close stale PRshyukjinkwon2017-02-170-0/+0
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to close stale PRs. What I mean by "stale" here includes that there are some review comments by reviewers but the author looks inactive without any answer to them more than a month. I left some comments roughly a week ago to ping and the author looks still inactive in these PR below These below includes some PR suggested to be closed and a PR against another branch which seems obviously inappropriate. Given the comments in the last three PRs below, they are probably worth being taken over by anyone who is interested in it. Closes #7963 Closes #8374 Closes #11192 Closes #11374 Closes #11692 Closes #12243 Closes #12583 Closes #12620 Closes #12675 Closes #12697 Closes #12800 Closes #13715 Closes #14266 Closes #15053 Closes #15159 Closes #15209 Closes #15264 Closes #15267 Closes #15871 Closes #15861 Closes #16319 Closes #16324 Closes #16890 Closes #12398 Closes #12933 Closes #14517 ## How was this patch tested? N/A Author: hyukjinkwon <gurwls223@gmail.com> Closes #16937 from HyukjinKwon/stale-prs-close.
* [SPARK-18120][SPARK-19557][SQL] Call QueryExecutionListener callback methods ↵Wenchen Fan2017-02-163-16/+142
| | | | | | | | | | | | | | | | | | for DataFrameWriter methods ## What changes were proposed in this pull request? We only notify `QueryExecutionListener` for several `Dataset` operations, e.g. collect, take, etc. We should also do the notification for `DataFrameWriter` operations. ## How was this patch tested? new regression test close https://github.com/apache/spark/pull/16664 Author: Wenchen Fan <wenchen@databricks.com> Closes #16962 from cloud-fan/insert.
* [SPARK-18352][SQL] Support parsing multiline json filesNathan Howell2017-02-1617-231/+740
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory. Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired. These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing. I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one. ## How was this patch tested? New and existing unit tests. No performance or load tests have been run. Author: Nathan Howell <nhowell@godaddy.com> Closes #16386 from NathanHowell/SPARK-18352.
* [SPARK-19550][HOTFIX][BUILD] Use JAVA_HOME/bin/java if JAVA_HOME is set in ↵Sean Owen2017-02-161-1/+7
| | | | | | | | | | | | | | | | | dev/mima ## What changes were proposed in this pull request? Use JAVA_HOME/bin/java if JAVA_HOME is set in dev/mima script to run MiMa This follows on https://github.com/apache/spark/pull/16871 -- it's a slightly separate issue, but, is currently causing a build failure. ## How was this patch tested? Manually tested. Author: Sean Owen <sowen@cloudera.com> Closes #16957 from srowen/SPARK-19550.2.
* [SPARK-19436][SQL] Add missing tests for approxQuantileZheng RuiFeng2017-02-163-23/+88
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1, check the behavior with illegal `quantiles` and `relativeError` 2, add tests for `relativeError` > 1 3, update tests for `null` data 4, update some docs for javadoc8 ## How was this patch tested? local test in spark-shell Author: Zheng RuiFeng <ruifengz@foxmail.com> Author: Ruifeng Zheng <ruifengz@foxmail.com> Closes #16776 from zhengruifeng/fix_approxQuantile.
* [MINOR][BUILD] Fix javadoc8 breakhyukjinkwon2017-02-161-5/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? These error below seems caused by unidoc that does not understand double commented block. ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected [error] * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction = [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:69: error: class, interface, or enum expected [error] * MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt; mappingFunction = [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected [error] * new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:70: error: class, interface, or enum expected [error] * new MapGroupsWithStateFunction&lt;String, Integer, Integer, String&gt;() { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: illegal character: '#' [error] * &#64;Override [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:72: error: class, interface, or enum expected [error] * &#64;Override [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:73: error: class, interface, or enum expected [error] * public String call(String key, Iterator&lt;Integer&gt; value, KeyedState&lt;Integer&gt; state) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:76: error: class, interface, or enum expected [error] * boolean shouldRemove = ...; // Decide whether to remove the state [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:77: error: class, interface, or enum expected [error] * if (shouldRemove) { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:79: error: class, interface, or enum expected [error] * } else { [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:81: error: class, interface, or enum expected [error] * state.update(newState); // Set the new state [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:82: error: class, interface, or enum expected [error] * } [error] ^ [error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:85: error: class, interface, or enum expected [error] * state.update(initialState); [error] ^ [error] .../forked/spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:86: error: class, interface, or enum expected [error] * } [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:90: error: class, interface, or enum expected [error] * </code></pre> [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:92: error: class, interface, or enum expected [error] * tparam S User-defined type of the state to be stored for each key. Must be encodable into [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:93: error: class, interface, or enum expected [error] * Spark SQL types (see {link Encoder} for more details). [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:94: error: class, interface, or enum expected [error] * since 2.1.1 [error] ^ ``` And another link seems unrecognisable. ``` .../spark/sql/core/target/java/org/apache/spark/sql/KeyedState.java:16: error: reference not found [error] * That is, in every batch of the {link streaming.StreamingQuery StreamingQuery}, [error] ``` Note that this PR does not fix the two breaks as below: ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:43: error: unexpected content [error] * see {link DataFrameStatsFunctions.approxQuantile(col:Str* approxQuantile} for [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/DataFrameStatFunctions.java:52: error: bad use of '>' [error] * param relativeError The relative target precision to achieve (>= 0). [error] ^ [error] ``` because these seem probably fixed soon in https://github.com/apache/spark/pull/16776 and I intended to avoid potential conflicts. ## How was this patch tested? Manually via `jekyll build` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16926 from HyukjinKwon/javadoc-break.
* [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 supportSean Owen2017-02-16101-1186/+513
| | | | | | | | | | | | | | | | | | | | | | | | - Move external/java8-tests tests into core, streaming, sql and remove - Remove MaxPermGen and related options - Fix some reflection / TODOs around Java 8+ methods - Update doc references to 1.7/1.8 differences - Remove Java 7/8 related build profiles - Update some plugins for better Java 8 compatibility - Fix a few Java-related warnings For the future: - Update Java 8 examples to fully use Java 8 - Update Java tests to use lambdas for simplicity - Update Java internal implementations to use lambdas ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16871 from srowen/SPARK-19493.
* [SPARK-18871][SQL][TESTS] New test cases for IN/NOT IN subquery 3rd batchKevin Yu2017-02-166-0/+1297
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is 3ird batch of test case for IN/NOT IN subquery. In this PR, it has these test files: `in-having.sql` `in-joins.sql` `in-multiple-columns.sql` These are the queries and results from running on DB2. [in-having DB2 version](https://github.com/apache/spark/files/772668/in-having.sql.db2.txt) [output of in-having](https://github.com/apache/spark/files/772670/in-having.sql.db2.out.txt) [in-joins DB2 version](https://github.com/apache/spark/files/772672/in-joins.sql.db2.txt) [output of in-joins](https://github.com/apache/spark/files/772673/in-joins.sql.db2.out.txt) [in-multiple-columns DB2 version](https://github.com/apache/spark/files/772678/in-multiple-columns.sql.db2.txt) [output of in-multiple-columns](https://github.com/apache/spark/files/772680/in-multiple-columns.sql.db2.out.txt) ## How was this patch tested? This pr is adding new test cases. We compare the result from spark with the result from another RDBMS(We used DB2 LUW). If the results are the same, we assume the result is correct. Author: Kevin Yu <qyu@us.ibm.com> Closes #16841 from kevinyu98/spark-18871-33.