aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19496][SQL] to_date udf to return null when input date is invalidwindpiger2017-02-132-4/+75
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently the udf `to_date` has different return value with an invalid date input. ``` SELECT to_date('2015-07-22', 'yyyy-dd-MM') -> return `2016-10-07` SELECT to_date('2014-31-12') -> return null ``` As discussed in JIRA [SPARK-19496](https://issues.apache.org/jira/browse/SPARK-19496), we should return null in both situations when the input date is invalid ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16870 from windpiger/to_date.
* [SPARK-19448][SQL] optimize some duplication functions between ↵windpiger2017-02-117-157/+88
| | | | | | | | | | | | | | | | | | HiveClientImpl and HiveUtils ## What changes were proposed in this pull request? There are some duplicate functions between `HiveClientImpl` and `HiveUtils`, we can merge them to one place. such as: `toHiveTable` 、`toHivePartition`、`fromHivePartition`. And additional modify is change `MetastoreRelation.attributes` to `MetastoreRelation.dataColKeys` https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/MetastoreRelation.scala#L234 ## How was this patch tested? N/A Author: windpiger <songjun@outlook.com> Closes #16787 from windpiger/todoInMetaStoreRelation.
* [SPARK-19548][SQL] Support Hive UDFs which return typed Lists/MapsHerman van Hovell2017-02-1011-57/+250
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds support for Hive UDFs that return fully typed java Lists or Maps, for example `List<String>` or `Map<String, Integer>`. It is also allowed to nest these structures, for example `Map<String, List<Integer>>`. Raw collections or collections using wildcards are still not supported, and cannot be supported due to the lack of type information. ## How was this patch tested? Modified existing tests in `HiveUDFSuite`, and I have added test cases for raw collection and collection using wildcards. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16886 from hvanhovell/SPARK-19548.
* [SPARK-19549] Allow providing reason for stage/job cancellingAla Luszczak2017-02-101-4/+15
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change add an optional argument to `SparkContext.cancelStage()` and `SparkContext.cancelJob()` functions, which allows the caller to provide exact reason for the cancellation. ## How was this patch tested? Adds unit test. Author: Ala Luszczak <ala@databricks.com> Closes #16887 from ala/cancel.
* [SPARK-19459][SQL] Add Hive datatype (char/varchar) to StructField metadataHerman van Hovell2017-02-106-25/+76
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Reading from an existing ORC table which contains `char` or `varchar` columns can fail with a `ClassCastException` if the table metadata has been created using Spark. This is caused by the fact that spark internally replaces `char` and `varchar` columns with a `string` column. This PR fixes this by adding the hive type to the `StructField's` metadata under the `HIVE_TYPE_STRING` key. This is picked up by the `HiveClient` and the ORC reader, see https://github.com/apache/spark/pull/16060 for more details on how the metadata is used. ## How was this patch tested? Added a regression test to `OrcSourceSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16804 from hvanhovell/SPARK-19459.
* [SPARK-19543] from_json fails when the input row is emptyBurak Yavuz2017-02-102-1/+9
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Using from_json on a column with an empty string results in: java.util.NoSuchElementException: head of empty list. This is because `parser.parse(input)` may return `Nil` when `input.trim.isEmpty` ## How was this patch tested? Regression test in `JsonExpressionsSuite` Author: Burak Yavuz <brkyvz@gmail.com> Closes #16881 from brkyvz/json-fix.
* [SPARK-19025][SQL] Remove SQL builder for operatorsjiangxingbo2017-02-096-2196/+3
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? With the new approach of view resolution, we can get rid of SQL generation on view creation, so let's remove SQL builder for operators. Note that, since all sql generation for operators is defined in one file (org.apache.spark.sql.catalyst.SQLBuilder), it’d be trivial to recover it in the future. ## How was this patch tested? N/A Author: jiangxingbo <jiangxb1987@gmail.com> Closes #16869 from jiangxb1987/SQLBuilder.
* [SPARK-19512][SQL] codegen for compare structs failsBogdan Raducanu2017-02-093-4/+24
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Set currentVars to null in GenerateOrdering.genComparisons before genCode is called. genCode ignores INPUT_ROW if currentVars is not null and in genComparisons we want it to use INPUT_ROW. ## How was this patch tested? Added test with 2 queries in WholeStageCodegenSuite Author: Bogdan Raducanu <bogdan.rdc@gmail.com> Closes #16852 from bogdanrdc/SPARK-19512.
* [SPARK-19514] Making range interruptible.Ala Luszczak2017-02-093-6/+52
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Previously range operator could not be interrupted. For example, using DAGScheduler.cancelStage(...) on a query with range might have been ineffective. This change adds periodic checks of TaskContext.isInterrupted to codegen version, and InterruptibleOperator to non-codegen version. I benchmarked the performance of codegen version on a sample query `spark.range(1000L * 1000 * 1000 * 10).count()` and there is no measurable difference. ## How was this patch tested? Adds a unit test. Author: Ala Luszczak <ala@databricks.com> Closes #16872 from ala/SPARK-19514b.
* [SPARK-19265][SQL][FOLLOW-UP] Configurable `tableRelationCache` maximum sizeLiwei Lin2017-02-094-3/+33
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable. In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`. ## How was this patch tested? new test case: `test("conf entry: checkValue()")` Author: Liwei Lin <lwlin7@gmail.com> Closes #16736 from lw-lin/conf.
* [SPARK-19359][SQL] renaming partition should not leave useless directoriesWenchen Fan2017-02-093-15/+84
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Hive metastore is not case-preserving and keep partition columns with lower case names. If Spark SQL creates a table with upper-case partition column names using `HiveExternalCatalog`, when we rename partition, it first calls the HiveClient to renamePartition, which will create a new lower case partition path, then Spark SQL renames the lower case path to upper-case. However, when we rename a nested path, different file systems have different behaviors. e.g. in jenkins, renaming `a=1/b=2` to `A=2/B=2` will success, but leave an empty directory `a=1`. in mac os, the renaming doesn't work as expected and result to `a=1/B=2`. This PR renames the partition directory recursively from the first partition column in `HiveExternalCatalog`, to be most compatible with different file systems. ## How was this patch tested? new regression test Author: Wenchen Fan <wenchen@databricks.com> Closes #16837 from cloud-fan/partition.
* [SPARK-18872][SQL][TESTS] New test cases for EXISTS subquery (Aggregate, ↵Dilip Biswal2017-02-096-0/+885
| | | | | | | | | | | | | | | | | | | | | | | | | | Having, Orderby, Limit) ## What changes were proposed in this pull request? This PR adds the second set of tests for EXISTS subquery. File name | Brief description ------------------------| ----------------- exists-aggregate.sql |Tests aggregate expressions in outer query and EXISTS subquery. exists-having.sql|Tests HAVING clause in subquery. exists-orderby-limit.sql|Tests EXISTS subquery support with ORDER BY and LIMIT clauses. DB2 results are attached here as reference : [exists-aggregate-db2.txt](https://github.com/apache/spark/files/743287/exists-aggregate-db2.txt) [exists-having-db2.txt](https://github.com/apache/spark/files/743286/exists-having-db2.txt) [exists-orderby-limit-db2.txt](https://github.com/apache/spark/files/743288/exists-orderby-limit-db2.txt) ## How the patch was tested. The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #16760 from dilipbiswal/exists-pr2.
* [SPARK-19279][SQL][FOLLOW-UP] Infer Schema for Hive Serde Tablesgatorsmile2017-02-083-1/+53
| | | | | | | | | | | | ### What changes were proposed in this pull request? `table.schema` is always not empty for partitioned tables, because `table.schema` also contains the partitioned columns, even if the original table does not have any column. This PR is to fix the issue. ### How was this patch tested? Added a test case Author: gatorsmile <gatorsmile@gmail.com> Closes #16848 from gatorsmile/inferHiveSerdeSchema.
* [SPARK-19409][BUILD][TEST-MAVEN] Fix ParquetAvroCompatibilitySuite failure ↵Dongjoon Hyun2017-02-081-0/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | due to test dependency on avro ## What changes were proposed in this pull request? After using Apache Parquet 1.8.2, `ParquetAvroCompatibilitySuite` fails on **Maven** test. It is because `org.apache.parquet.avro.AvroParquetWriter` in the test code used new `avro 1.8.0` specific class, `LogicalType`. This PR aims to fix the test dependency of `sql/core` module to use avro 1.8.0. https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/2530/consoleFull ``` ParquetAvroCompatibilitySuite: *** RUN ABORTED *** java.lang.NoClassDefFoundError: org/apache/avro/LogicalType at org.apache.parquet.avro.AvroParquetWriter.writeSupport(AvroParquetWriter.java:144) ``` ## How was this patch tested? Pass the existing test with **Maven**. ``` $ build/mvn -Pyarn -Phadoop-2.7 -Pkinesis-asl -Phive -Phive-thriftserver test ... [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 02:07 h [INFO] Finished at: 2017-02-04T05:41:43+00:00 [INFO] Final Memory: 77M/987M [INFO] ------------------------------------------------------------------------ ``` Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16795 from dongjoon-hyun/SPARK-19409-2.
* [SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and ↵Sean Owen2017-02-084-27/+11
| | | | | | | | | | | | | | | | | | | earlier ## What changes were proposed in this pull request? - Remove support for Hadoop 2.5 and earlier - Remove reflection and code constructs only needed to support multiple versions at once - Update docs to reflect newer versions - Remove older versions' builds and profiles. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16810 from srowen/SPARK-19464.
* [SPARK-19488][SQL] fix csv infer schema when the field is Nan/Inf etcwindpiger2017-02-082-1/+13
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? when csv infer schema, it does not use user defined csvoptions to parse the field, such as `inf`, `-inf` which are should be parsed to DoubleType this pr add `options.nanValue`, `options.negativeInf`, `options.positiveIn` to check if the field is a DoubleType ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16834 from windpiger/fixinferInfSchemaCsv.
* [SPARK-18873][SQL][TEST] New test cases for scalar subquery (part 1 of 2) - ↵Nattavut Sutyanyong2017-02-072-0/+328
| | | | | | | | | | | | | | scalar subquery in SELECT clause ## What changes were proposed in this pull request? This PR adds new test cases for scalar subquery in SELECT clause. ## How was this patch tested? The test result is compared with the result run from another SQL engine (in this case is IBM DB2). If the result are equivalent, we assume the result is correct. Author: Nattavut Sutyanyong <nsy.can@gmail.com> Closes #16712 from nsyca/18873.
* [SPARK-19499][SS] Add more notes in the comments of Sink.addBatch()CodingCat2017-02-071-1/+4
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? addBatch method in Sink trait is supposed to be a synchronous method to coordinate with the fault-tolerance design in StreamingExecution (being different with the compute() method in DStream) We need to add more notes in the comments of this method to remind the developers ## How was this patch tested? existing tests Author: CodingCat <zhunansjtu@gmail.com> Closes #16840 from CodingCat/SPARK-19499.
* [SPARK-19413][SS] MapGroupsWithState for arbitrary stateful operationsTathagata Das2017-02-0718-36/+1059
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `mapGroupsWithState` is a new API for arbitrary stateful operations in Structured Streaming, similar to `DStream.mapWithState` *Requirements* - Users should be able to specify a function that can do the following - Access the input row corresponding to a key - Access the previous state corresponding to a key - Optionally, update or remove the state - Output any number of new rows (or none at all) *Proposed API* ``` // ------------ New methods on KeyValueGroupedDataset ------------ class KeyValueGroupedDataset[K, V] { // Scala friendly def mapGroupsWithState[S: Encoder, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => U) def flatMapGroupsWithState[S: Encode, U: Encoder](func: (K, Iterator[V], KeyedState[S]) => Iterator[U]) // Java friendly def mapGroupsWithState[S, U](func: MapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U]) def flatMapGroupsWithState[S, U](func: FlatMapGroupsWithStateFunction[K, V, S, R], stateEncoder: Encoder[S], resultEncoder: Encoder[U]) } // ------------------- New Java-friendly function classes ------------------- public interface MapGroupsWithStateFunction<K, V, S, R> extends Serializable { R call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception; } public interface FlatMapGroupsWithStateFunction<K, V, S, R> extends Serializable { Iterator<R> call(K key, Iterator<V> values, state: KeyedState<S>) throws Exception; } // ---------------------- Wrapper class for state data ---------------------- trait State[S] { def exists(): Boolean def get(): S // throws Exception is state does not exist def getOption(): Option[S] def update(newState: S): Unit def remove(): Unit // exists() will be false after this } ``` Key Semantics of the State class - The state can be null. - If the state.remove() is called, then state.exists() will return false, and getOption will returm None. - After that state.update(newState) is called, then state.exists() will return true, and getOption will return Some(...). - None of the operations are thread-safe. This is to avoid memory barriers. *Usage* ``` val stateFunc = (word: String, words: Iterator[String, runningCount: KeyedState[Long]) => { val newCount = words.size + runningCount.getOption.getOrElse(0L) runningCount.update(newCount) (word, newCount) } dataset // type is Dataset[String] .groupByKey[String](w => w) // generates KeyValueGroupedDataset[String, String] .mapGroupsWithState[Long, (String, Long)](stateFunc) // returns Dataset[(String, Long)] ``` ## How was this patch tested? New unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #16758 from tdas/mapWithState.
* [SPARK-19397][SQL] Make option names of LIBSVM and TEXT case insensitivegatorsmile2017-02-083-4/+63
| | | | | | | | | | | | | | ### What changes were proposed in this pull request? Prior to Spark 2.1, the option names are case sensitive for all the formats. Since Spark 2.1, the option key names become case insensitive except the format `Text` and `LibSVM `. This PR is to fix these issues. Also, add a check to know whether the input option vector type is legal for `LibSVM`. ### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #16737 from gatorsmile/libSVMTextOptions.
* [SPARK-18609][SPARK-18841][SQL] Fix redundant Alias removal in the optimizerHerman van Hovell2017-02-079-115/+302
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The optimizer tries to remove redundant alias only projections from the query plan using the `RemoveAliasOnlyProject` rule. The current rule identifies removes such a project and rewrites the project's attributes in the **entire** tree. This causes problems when parts of the tree are duplicated (for instance a self join on a temporary view/CTE) and the duplicated part contains the alias only project, in this case the rewrite will break the tree. This PR fixes these problems by using a blacklist for attributes that are not to be moved, and by making sure that attribute remapping is only done for the parent tree, and not for unrelated parts of the query plan. The current tree transformation infrastructure works very well if the transformation at hand requires little or a global contextual information. In this case we need to know both the attributes that were not to be moved, and we also needed to know which child attributes were modified. This cannot be done easily using the current infrastructure, and solutions typically involves transversing the query plan multiple times (which is super slow). I have moved around some code in `TreeNode`, `QueryPlan` and `LogicalPlan`to make this much more straightforward; this basically allows you to manually traverse the tree. This PR subsumes the following PRs by windpiger: Closes https://github.com/apache/spark/pull/16267 Closes https://github.com/apache/spark/pull/16255 ## How was this patch tested? I have added unit tests to `RemoveRedundantAliasAndProjectSuite` and I have added integration tests to the `SQLQueryTestSuite.union` and `SQLQueryTestSuite.cte` test cases. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16757 from hvanhovell/SPARK-18609.
* [SPARK-19495][SQL] Make SQLConf slightly more extensibleReynold Xin2017-02-075-137/+139
| | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request makes SQLConf slightly more extensible by removing the visibility limitations on the build* functions. ## How was this patch tested? N/A - there are no logic changes and everything should be covered by existing unit tests. Author: Reynold Xin <rxin@databricks.com> Closes #16835 from rxin/SPARK-19495.
* [SPARK-16609] Add to_date/to_timestamp with format functionsanabranch2017-02-075-8/+161
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pull request adds two new user facing functions: - `to_date` which accepts an expression and a format and returns a date. - `to_timestamp` which accepts an expression and a format and returns a timestamp. For example, Given a date in format: `2016-21-05`. (YYYY-dd-MM) ### Date Function *Previously* ``` to_date(unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp")) ``` *Current* ``` to_date(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Timestamp Function *Previously* ``` unix_timestamp(lit("2016-21-05"), "yyyy-dd-MM").cast("timestamp") ``` *Current* ``` to_timestamp(lit("2016-21-05"), "yyyy-dd-MM") ``` ### Tasks - [X] Add `to_date` to Scala Functions - [x] Add `to_date` to Python Functions - [x] Add `to_date` to SQL Functions - [X] Add `to_timestamp` to Scala Functions - [x] Add `to_timestamp` to Python Functions - [x] Add `to_timestamp` to SQL Functions - [x] Add function to R ## How was this patch tested? - [x] Add Functions to `DateFunctionsSuite` - Test new `ParseToTimestamp` Expression (*not necessary*) - Test new `ParseToDate` Expression (*not necessary*) - [x] Add test for R - [x] Add test for Python in test.py Please review http://spark.apache.org/contributing.html before opening a pull request. Author: anabranch <wac.chambers@gmail.com> Author: Bill Chambers <bill@databricks.com> Author: anabranch <bill@databricks.com> Closes #16138 from anabranch/SPARK-16609.
* [SPARK-19447] Fixing input metrics for range operator.Ala Luszczak2017-02-076-75/+350
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change introduces a new metric "number of generated rows". It is used exclusively for Range, which is a leaf in the query tree, yet doesn't read any input data, and therefore cannot report "recordsRead". Additionally the way in which the metrics are reported by the JIT-compiled version of Range was changed. Previously, it was immediately reported that all the records were produced. This could be confusing for a user monitoring execution progress in the UI. Now, the metric is updated gradually. In order to avoid negative impact on Range performance, the code generation was reworked. The values are now produced in batches in the tighter inner loop, while the metrics are updated in the outer loop. The change also contains a number of unit tests, which should help ensure the correctness of metrics for various input sources. ## How was this patch tested? Unit tests. Author: Ala Luszczak <ala@databricks.com> Closes #16829 from ala/SPARK-19447.
* [SPARK-19118][SQL] Percentile support for frequency distribution tablegagan taneja2017-02-072-55/+141
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? I have a frequency distribution table with following entries Age, No of person 21, 10 22, 15 23, 18 .. .. 30, 14 Moreover it is common to have data in frequency distribution format to further calculate Percentile, Median. With current implementation It would be very difficult and complex to find the percentile. Therefore i am proposing enhancement to current Percentile and Approx Percentile implementation to take frequency distribution column into consideration ## How was this patch tested? 1) Enhanced /sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala to cover the additional functionality 2) Run some performance benchmark test with 20 million row in local environment and did not see any performance degradation Please review http://spark.apache.org/contributing.html before opening a pull request. Author: gagan taneja <tanejagagan@gagans-MacBook-Pro.local> Closes #16497 from tanejagagan/branch-18940.
* [SPARK-16101][SQL] Refactoring CSV schema inference path to be consistent ↵hyukjinkwon2017-02-0711-246/+271
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | with JSON ## What changes were proposed in this pull request? This PR refactors CSV schema inference path to be consistent with JSON data source and moves some filtering codes having the similar/same logics into `CSVUtils`. It makes the methods in classes have consistent arguments with JSON ones. (this PR renames `.../json/InferSchema.scala` → `.../json/JsonInferSchema.scala`) `CSVInferSchema` and `JsonInferSchema` ``` scala private[csv] object CSVInferSchema { ... def infer( csv: Dataset[String], caseSensitive: Boolean, options: CSVOptions): StructType = { ... ``` ``` scala private[sql] object JsonInferSchema { ... def infer( json: RDD[String], columnNameOfCorruptRecord: String, configOptions: JSONOptions): StructType = { ... ``` These allow schema inference from `Dataset[String]` directly, meaning the similar functionalities that use `JacksonParser`/`JsonInferSchema` for JSON can be easily implemented by `UnivocityParser`/`CSVInferSchema` for CSV. This completes refactoring CSV datasource and they are now pretty consistent. ## How was this patch tested? Existing tests should cover this and ``` ./dev/change-scala-version.sh 2.10 ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16680 from HyukjinKwon/SPARK-16101-schema-inference.
* [SPARK-18601][SQL] Simplify Create/Get complex expression pairs in optimizerEyal Farago2017-02-077-4/+427
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? It often happens that a complex object (struct/map/array) is created only to get elements from it in an subsequent expression. We can add an optimizer rule for this. ## How was this patch tested? unit-tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Eyal Farago <eyal@nrgene.com> Author: eyal farago <eyal.farago@gmail.com> Closes #16043 from eyalfa/SPARK-18601.
* [SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it ↵uncleGen2017-02-061-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | from uri scheme ## What changes were proposed in this pull request? ``` Caused by: java.lang.IllegalArgumentException: Wrong FS: s3a://**************/checkpoint/7b2231a3-d845-4740-bfa3-681850e5987f/metadata, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649) at org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:82) at org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:606) at org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:824) at org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:601) at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421) at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426) at org.apache.spark.sql.execution.streaming.StreamMetadata$.read(StreamMetadata.scala:51) at org.apache.spark.sql.execution.streaming.StreamExecution.<init>(StreamExecution.scala:100) at org.apache.spark.sql.streaming.StreamingQueryManager.createQuery(StreamingQueryManager.scala:232) at org.apache.spark.sql.streaming.StreamingQueryManager.startQuery(StreamingQueryManager.scala:269) at org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:262) ``` Can easily replicate on spark standalone cluster by providing checkpoint location uri scheme anything other than "file://" and not overriding in config. WorkAround --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in sparkConf or spark-default.conf ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes #16815 from uncleGen/SPARK-19407.
* [SPARK-19441][SQL] Remove IN type coercion from PromoteStringsgatorsmile2017-02-071-9/+0
| | | | | | | | | | | | ### What changes were proposed in this pull request? The removed codes for `IN` are not reachable, because the previous rule `InConversion` already resolves the type coercion issues. ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #16783 from gatorsmile/typeCoercionIn.
* [SPARK-19472][SQL] Parser should not mistake CASE WHEN(...) for a function callHerman van Hovell2017-02-062-1/+3
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The SQL parser can mistake a `WHEN (...)` used in `CASE` for a function call. This happens in cases like the following: ```sql select case when (1) + case when 1 > 0 then 1 else 0 end = 2 then 1 else 0 end from tb ``` This PR fixes this by re-organizing the case related parsing rules. ## How was this patch tested? Added a regression test to the `ExpressionParserSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #16821 from hvanhovell/SPARK-19472.
* [SPARK-19080][SQL] simplify data source analysisWenchen Fan2017-02-0720-199/+126
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current way of resolving `InsertIntoTable` and `CreateTable` is convoluted: sometimes we replace them with concrete implementation commands during analysis, sometimes during planning phase. And the error checking logic is also a mess: we may put it in extended analyzer rules, or extended checking rules, or `CheckAnalysis`. This PR simplifies the data source analysis: 1. `InsertIntoTable` and `CreateTable` are always unresolved and need to be replaced by concrete implementation commands during analysis. 2. The error checking logic is mainly in 2 rules: `PreprocessTableCreation` and `PreprocessTableInsertion`. ## How was this patch tested? existing test. Author: Wenchen Fan <wenchen@databricks.com> Closes #16269 from cloud-fan/ddl.
* [SPARK-17213][SQL][FOLLOWUP] Re-enable Parquet filter tests for binary and ↵hyukjinkwon2017-02-061-4/+2
| | | | | | | | | | | | | | | | | | string ## What changes were proposed in this pull request? This PR proposes to enable the tests for Parquet filter pushdown with binary and string. This was disabled in https://github.com/apache/spark/pull/16106 due to Parquet's issue but it is now revived in https://github.com/apache/spark/pull/16791 after upgrading Parquet to 1.8.2. ## How was this patch tested? Manually tested `ParquetFilterSuite` via IDE. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16817 from HyukjinKwon/SPARK-17213.
* [SPARK-19409][SPARK-17213] Cleanup Parquet workarounds/hacks due to bugs of ↵Cheng Lian2017-02-062-33/+2
| | | | | | | | | | | | | | | | old Parquet versions ## What changes were proposed in this pull request? We've already upgraded parquet-mr to 1.8.2. This PR does some further cleanup by removing a workaround of PARQUET-686 and a hack due to PARQUET-363 and PARQUET-278. All three Parquet issues are fixed in parquet-mr 1.8.2. ## How was this patch tested? Existing unit tests. Author: Cheng Lian <lian@databricks.com> Closes #16791 from liancheng/parquet-1.8.2-cleanup.
* [SPARK-19279][SQL] Infer Schema for Hive Serde Tables and Block Creating a ↵gatorsmile2017-02-0610-194/+304
| | | | | | | | | | | | | | | | | | | | | | | | | Hive Table With an Empty Schema ### What changes were proposed in this pull request? So far, we allow users to create a table with an empty schema: `CREATE TABLE tab1`. This could break many code paths if we enable it. Thus, we should follow Hive to block it. For Hive serde tables, some serde libraries require the specified schema and record it in the metastore. To get the list, we need to check `hive.serdes.using.metastore.for.schema,` which contains a list of serdes that require user-specified schema. The default values are - org.apache.hadoop.hive.ql.io.orc.OrcSerde - org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe - org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe - org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe - org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe - org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe - org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe - org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe ### How was this patch tested? Added test cases for both Hive and data source tables Author: gatorsmile <gatorsmile@gmail.com> Closes #16636 from gatorsmile/fixEmptyTableSchema.
* [SPARK-19425][SQL] Make ExtractEquiJoinKeys support UDT columnsLiang-Chi Hsieh2017-02-043-0/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? DataFrame.except doesn't work for UDT columns. It is because `ExtractEquiJoinKeys` will run `Literal.default` against UDT. However, we don't handle UDT in `Literal.default` and an exception will throw like: java.lang.RuntimeException: no default for type org.apache.spark.ml.linalg.VectorUDT3bfc3ba7 at org.apache.spark.sql.catalyst.expressions.Literal$.default(literals.scala:179) at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:117) at org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys$$anonfun$4.apply(patterns.scala:110) More simple fix is just let `Literal.default` handle UDT by its sql type. So we can use more efficient join type on UDT. Besides `except`, this also fixes other similar scenarios, so in summary this fixes: * `except` on two Datasets with UDT * `intersect` on two Datasets with UDT * `Join` with the join conditions using `<=>` on UDT columns ## 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 #16765 from viirya/df-except-for-udt.
* [SPARK-19446][SQL] Remove unused findTightestCommonType in TypeCoercionhyukjinkwon2017-02-033-23/+12
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to - remove unused `findTightestCommonType` in `TypeCoercion` as suggested in https://github.com/apache/spark/pull/16777#discussion_r99283834 - rename `findTightestCommonTypeOfTwo ` to `findTightestCommonType`. - fix comments accordingly The usage was removed while refactoring/fixing in several JIRAs such as SPARK-16714, SPARK-16735 and SPARK-16646 ## How was this patch tested? Existing tests. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16786 from HyukjinKwon/SPARK-19446.
* [SPARK-18909][SQL] The error messages in `ExpressionEncoder.toRow/fromRow` ↵Dongjoon Hyun2017-02-031-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | are too verbose ## What changes were proposed in this pull request? In `ExpressionEncoder.toRow` and `fromRow`, we catch the exception and output `treeString` of serializer/deserializer expressions in the error message. However, encoder can be very complex and the serializer/deserializer expressions can be very large trees and blow up the log files(e.g. generate over 500mb logs for this single error message.) As a first attempt, this PR try to use `simpleString` instead. **BEFORE** ```scala scala> :paste // Entering paste mode (ctrl-D to finish) case class TestCaseClass(value: Int) import spark.implicits._ Seq(TestCaseClass(1)).toDS().collect() // Exiting paste mode, now interpreting. java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException newInstance(class TestCaseClass) +- assertnotnull(input[0, int, false], - field (class: "scala.Int", name: "value"), - root class: "TestCaseClass") +- input[0, int, false] at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303) ... ``` **AFTER** ```scala ... // Exiting paste mode, now interpreting. java.lang.RuntimeException: Error while decoding: java.lang.NullPointerException newInstance(class TestCaseClass) at org.apache.spark.sql.catalyst.encoders.ExpressionEncoder.fromRow(ExpressionEncoder.scala:303) ... ``` ## How was this patch tested? Manual. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #16701 from dongjoon-hyun/SPARK-18909-EXPR-ERROR.
* [SPARK-19411][SQL] Remove the metadata used to mark optional columns in ↵Liang-Chi Hsieh2017-02-036-137/+22
| | | | | | | | | | | | | | | | | | merged Parquet schema for filter predicate pushdown ## What changes were proposed in this pull request? There is a metadata introduced before to mark the optional columns in merged Parquet schema for filter predicate pushdown. As we upgrade to Parquet 1.8.2 which includes the fix for the pushdown of optional columns, we don't need this metadata now. ## 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 #16756 from viirya/remove-optional-metadata.
* [SPARK-14352][SQL] approxQuantile should support multi columnsZheng RuiFeng2017-02-012-3/+49
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? 1, add the multi-cols support based on current private api 2, add the multi-cols support to pyspark ## How was this patch tested? unit tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Author: Ruifeng Zheng <ruifengz@foxmail.com> Closes #12135 from zhengruifeng/quantile4multicols.
* [SPARK-19296][SQL] Deduplicate url and table in JdbcUtilshyukjinkwon2017-02-012-32/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR deduplicates arguments, `url` and `table` in `JdbcUtils` with `JDBCOptions`. It avoids to use duplicated arguments, for example, as below: from ```scala val jdbcOptions = new JDBCOptions(url, table, map) JdbcUtils.saveTable(ds, url, table, jdbcOptions) ``` to ```scala val jdbcOptions = new JDBCOptions(url, table, map) JdbcUtils.saveTable(ds, jdbcOptions) ``` ## How was this patch tested? Running unit test in `JdbcSuite`/`JDBCWriteSuite` Building with Scala 2.10 as below: ``` ./dev/change-scala-version.sh 2.10 ./build/mvn -Pyarn -Phadoop-2.4 -Dscala-2.10 -DskipTests clean package ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16753 from HyukjinKwon/SPARK-19296.
* [SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings ↵hyukjinkwon2017-02-0116-42/+42
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | in Scala/Java APIs generation ## What changes were proposed in this pull request? This PR proposes three things as below: - Support LaTex inline-formula, `\( ... \)` in Scala API documentation It seems currently, ``` \( ... \) ``` are rendered as they are, for example, <img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png"> It seems mistakenly more backslashes were added. - Fix warnings Scaladoc/Javadoc generation This PR fixes t two types of warnings as below: ``` [warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException". [warn] /** [warn] ^ ``` ``` [warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution [warn] * `${var}`, `${system:var}` and `${env:var}`. [warn] ^ ``` - Fix Javadoc8 break ``` [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found [error] * E.g., {link VectorUDT} for vector features. [error] ^ [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found [error] * E.g., {link VectorUDT} for vector features. [error] ^ [error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found [error] * E.g., {link VectorUDT} for vector features. [error] ^ [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found [error] * Note that, this rule must be run after {link PreprocessTableInsertion}. [error] ^ ``` ## How was this patch tested? Manually via `sbt unidoc` and `jeykil build`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16741 from HyukjinKwon/warn-and-break.
* [SPARK-19378][SS] Ensure continuity of stateOperator and eventTime metrics ↵Burak Yavuz2017-01-312-13/+64
| | | | | | | | | | | | | | | | even if there is no new data in trigger ## What changes were proposed in this pull request? In StructuredStreaming, if a new trigger was skipped because no new data arrived, we suddenly report nothing for the metrics `stateOperator`. We could however easily report the metrics from `lastExecution` to ensure continuity of metrics. ## How was this patch tested? Regression test in `StreamingQueryStatusAndProgressSuite` Author: Burak Yavuz <brkyvz@gmail.com> Closes #16716 from brkyvz/state-agg.
* [SPARK-19406][SQL] Fix function to_json to respect user-provided optionsgatorsmile2017-01-302-2/+24
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ### What changes were proposed in this pull request? Currently, the function `to_json` allows users to provide options for generating JSON. However, it does not pass it to `JacksonGenerator`. Thus, it ignores the user-provided options. This PR is to fix it. Below is an example. ```Scala val df = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a") val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm") df.select(to_json($"a", options)).show(false) ``` The current output is like ``` +--------------------------------------+ |structtojson(a) | +--------------------------------------+ |{"_1":"2015-08-26T18:00:00.000-07:00"}| +--------------------------------------+ ``` After the fix, the output is like ``` +-------------------------+ |structtojson(a) | +-------------------------+ |{"_1":"26/08/2015 18:00"}| +-------------------------+ ``` ### How was this patch tested? Added test cases for both `from_json` and `to_json` Author: gatorsmile <gatorsmile@gmail.com> Closes #16745 from gatorsmile/toJson.
* [SPARK-19385][SQL] During canonicalization, `NOT(...(l, r))` should not ↵Liwei Lin2017-01-292-9/+43
| | | | | | | | | | | | | | | | | | | | | | expect such cases that l.hashcode > r.hashcode ## What changes were proposed in this pull request? During canonicalization, `NOT(...(l, r))` should not expect such cases that `l.hashcode > r.hashcode`. Take the rule `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` for example, it should never be matched since `GreaterThan(l, r)` itself would be re-written as `GreaterThan(r, l)` given `l.hashcode > r.hashcode` after canonicalization. This patch consolidates rules like `case NOT(GreaterThan(l, r)) if l.hashcode > r.hashcode` and `case NOT(GreaterThan(l, r))`. ## How was this patch tested? This patch expanded the `NOT` test case to cover both cases where: - `l.hashcode > r.hashcode` - `l.hashcode < r.hashcode` Author: Liwei Lin <lwlin7@gmail.com> Closes #16719 from lw-lin/canonicalize.
* [SPARK-18872][SQL][TESTS] New test cases for EXISTS subqueryDilip Biswal2017-01-294-0/+589
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds the first set of tests for EXISTS subquery. File name | Brief description ------------------------| ----------------- exists-basic.sql |Tests EXISTS and NOT EXISTS subqueries with both correlated and local predicates. exists-within-and-or.sql|Tests EXISTS and NOT EXISTS subqueries embedded in AND or OR expression. DB2 results are attached here as reference : [exists-basic-db2.txt](https://github.com/apache/spark/files/733031/exists-basic-db2.txt) [exists-and-or-db2.txt](https://github.com/apache/spark/files/733030/exists-and-or-db2.txt) ## How was this patch tested? This patch is adding tests. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #16710 from dilipbiswal/exist-basic.
* [SPARK-19152][SQL][FOLLOWUP] simplify CreateHiveTableAsSelectCommandWenchen Fan2017-01-284-42/+49
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? After https://github.com/apache/spark/pull/16552 , `CreateHiveTableAsSelectCommand` becomes very similar to `CreateDataSourceTableAsSelectCommand`, and we can further simplify it by only creating table in the table-not-exist branch. This PR also adds hive provider checking in DataStream reader/writer, which is missed in #16552 ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #16693 from cloud-fan/minor.
* [SPARK-19359][SQL] Revert Clear useless path after rename a partition with ↵gatorsmile2017-01-282-71/+0
| | | | | | | | | | | | | | | | | | | | | | | | | upper-case by HiveExternalCatalog ### What changes were proposed in this pull request? This PR is to revert the changes made in https://github.com/apache/spark/pull/16700. It could cause the data loss after partition rename, because we have a bug in the file renaming. Not all the OSs have the same behaviors. For example, on mac OS, if we renaming a path from `.../tbl/a=5/b=6` to `.../tbl/A=5/B=6`. The result is `.../tbl/a=5/B=6`. The expected result is `.../tbl/A=5/B=6`. Thus, renaming on mac OS is not recursive. However, the systems used in Jenkin does not have such an issue. Although this PR is not the root cause, it exposes an existing issue on the code `tablePath.getFileSystem(hadoopConf).rename(wrongPath, rightPath)` --- Hive metastore is not case preserving and keep partition columns with lower case names. If SparkSQL create a table with upper-case partion name use HiveExternalCatalog, when we rename partition, it first call the HiveClient to renamePartition, which will create a new lower case partition path, then SparkSql rename the lower case path to the upper-case. while if the renamed partition contains more than one depth partition ,e.g. A=1/B=2, hive renamePartition change to a=1/b=2, then SparkSql rename it to A=1/B=2, but the a=1 still exists in the filesystem, we should also delete it. ### How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #16728 from gatorsmile/revert-pr-16700.
* [SPARK-19359][SQL] clear useless path after rename a partition with ↵windpiger2017-01-272-0/+71
| | | | | | | | | | | | | | | | | | | upper-case by HiveExternalCatalog ## What changes were proposed in this pull request? Hive metastore is not case preserving and keep partition columns with lower case names. If SparkSQL create a table with upper-case partion name use HiveExternalCatalog, when we rename partition, it first call the HiveClient to renamePartition, which will create a new lower case partition path, then SparkSql rename the lower case path to the upper-case. while if the renamed partition contains more than one depth partition ,e.g. A=1/B=2, hive renamePartition change to a=1/b=2, then SparkSql rename it to A=1/B=2, but the a=1 still exists in the filesystem, we should also delete it. ## How was this patch tested? unit test added Author: windpiger <songjun@outlook.com> Closes #16700 from windpiger/clearUselessPathAfterRenamPartition.
* [SPARK-12970][DOCS] Fix the example in SturctType APIs for Scala and Javahyukjinkwon2017-01-272-16/+18
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes both, javadoc8 break ``` [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/FindHiveSerdeTable.java:3: error: reference not found [error] * Replaces {link SimpleCatalogRelation} with {link MetastoreRelation} if its table provider is hive. ``` and the example in `StructType` as a self-contained example as below: ```scala import org.apache.spark.sql._ import org.apache.spark.sql.types._ val struct = StructType( StructField("a", IntegerType, true) :: StructField("b", LongType, false) :: StructField("c", BooleanType, false) :: Nil) // Extract a single StructField. val singleField = struct("b") // singleField: StructField = StructField(b,LongType,false) // If this struct does not have a field called "d", it throws an exception. struct("d") // java.lang.IllegalArgumentException: Field "d" does not exist. // ... // Extract multiple StructFields. Field names are provided in a set. // A StructType object will be returned. val twoFields = struct(Set("b", "c")) // twoFields: StructType = // StructType(StructField(b,LongType,false), StructField(c,BooleanType,false)) // Any names without matching fields will throw an exception. // For the case shown below, an exception is thrown due to "d". struct(Set("b", "c", "d")) // java.lang.IllegalArgumentException: Field "d" does not exist. // ... ``` ```scala import org.apache.spark.sql._ import org.apache.spark.sql.types._ val innerStruct = StructType( StructField("f1", IntegerType, true) :: StructField("f2", LongType, false) :: StructField("f3", BooleanType, false) :: Nil) val struct = StructType( StructField("a", innerStruct, true) :: Nil) // Create a Row with the schema defined by struct val row = Row(Row(1, 2, true)) ``` Also, now when the column is missing, it throws an exception rather than ignoring. ## How was this patch tested? Manually via `sbt unidoc`. - Scaladoc <img width="665" alt="2017-01-26 12 54 13" src="https://cloud.githubusercontent.com/assets/6477701/22297905/1245620e-e362-11e6-9e22-43bb8d9871af.png"> - Javadoc <img width="722" alt="2017-01-26 12 54 27" src="https://cloud.githubusercontent.com/assets/6477701/22297899/0fd87e0c-e362-11e6-9033-7590bda1aea6.png"> <img width="702" alt="2017-01-26 12 54 32" src="https://cloud.githubusercontent.com/assets/6477701/22297900/0fe14154-e362-11e6-9882-768381c53163.png"> Author: hyukjinkwon <gurwls223@gmail.com> Closes #16703 from HyukjinKwon/SPARK-12970.
* [SPARK-19338][SQL] Add UDF names in explainTakeshi YAMAMURO2017-01-264-27/+44
| | | | | | | | | | | | | ## What changes were proposed in this pull request? This pr added a variable for a UDF name in `ScalaUDF`. Then, if the variable filled, `DataFrame#explain` prints the name. ## How was this patch tested? Added a test in `UDFSuite`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16707 from maropu/SPARK-19338.