aboutsummaryrefslogtreecommitdiff
path: root/sql/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17289][SQL] Fix a bug to satisfy sort requirements in partial ↵Takeshi YAMAMURO2016-08-302-2/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | aggregations ## What changes were proposed in this pull request? Partial aggregations are generated in `EnsureRequirements`, but the planner fails to check if partial aggregation satisfies sort requirements. For the following query: ``` val df2 = (0 to 1000).map(x => (x % 2, x.toString)).toDF("a", "b").createOrReplaceTempView("t2") spark.sql("select max(b) from t2 group by a").explain(true) ``` Now, the SortAggregator won't insert Sort operator before partial aggregation, this will break sort-based partial aggregation. ``` == Physical Plan == SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17]) +- *Sort [a#5 ASC], false, 0 +- Exchange hashpartitioning(a#5, 200) +- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19]) +- LocalTableScan [a#5, b#6] ``` Actually, a correct plan is: ``` == Physical Plan == SortAggregate(key=[a#5], functions=[max(b#6)], output=[max(b)#17]) +- *Sort [a#5 ASC], false, 0 +- Exchange hashpartitioning(a#5, 200) +- SortAggregate(key=[a#5], functions=[partial_max(b#6)], output=[a#5, max#19]) +- *Sort [a#5 ASC], false, 0 +- LocalTableScan [a#5, b#6] ``` ## How was this patch tested? Added tests in `PlannerSuite`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #14865 from maropu/SPARK-17289.
* [SPARK-17063] [SQL] Improve performance of MSCK REPAIR TABLE with Hive metastoreDavies Liu2016-08-293-29/+150
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR split the the single `createPartitions()` call into smaller batches, which could prevent Hive metastore from OOM (caused by millions of partitions). It will also try to gather all the fast stats (number of files and total size of all files) in parallel to avoid the bottle neck of listing the files in metastore sequential, which is controlled by spark.sql.gatherFastStats (enabled by default). ## How was this patch tested? Tested locally with 10000 partitions and 100 files with embedded metastore, without gathering fast stats in parallel, adding partitions took 153 seconds, after enable that, gathering the fast stats took about 34 seconds, adding these partitions took 25 seconds (most of the time spent in object store), 59 seconds in total, 2.5X faster (with larger cluster, gathering will much faster). Author: Davies Liu <davies@databricks.com> Closes #14607 from davies/repair_batch.
* [SPARK-17271][SQL] Planner adds un-necessary Sort even if child ordering is ↵Tejas Patil2016-08-282-2/+49
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | semantically same as required ordering ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-17271 Planner is adding un-needed SORT operation due to bug in the way comparison for `SortOrder` is done at https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala#L253 `SortOrder` needs to be compared semantically because `Expression` within two `SortOrder` can be "semantically equal" but not literally equal objects. eg. In case of `sql("SELECT * FROM table1 a JOIN table2 b ON a.col1=b.col1")` Expression in required SortOrder: ``` AttributeReference( name = "col1", dataType = LongType, nullable = false ) (exprId = exprId, qualifier = Some("a") ) ``` Expression in child SortOrder: ``` AttributeReference( name = "col1", dataType = LongType, nullable = false ) (exprId = exprId) ``` Notice that the output column has a qualifier but the child attribute does not but the inherent expression is the same and hence in this case we can say that the child satisfies the required sort order. This PR includes following changes: - Added a `semanticEquals` method to `SortOrder` so that it can compare underlying child expressions semantically (and not using default Object.equals) - Fixed `EnsureRequirements` to use semantic comparison of SortOrder ## How was this patch tested? - Added a test case to `PlannerSuite`. Ran rest tests in `PlannerSuite` Author: Tejas Patil <tejasp@fb.com> Closes #14841 from tejasapatil/SPARK-17271_sort_order_equals_bug.
* [SPARK-15382][SQL] Fix a bug in sampling with replacementTakeshi YAMAMURO2016-08-272-0/+8
| | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This pr to fix a bug below in sampling with replacement ``` val df = Seq((1, 0), (2, 0), (3, 0)).toDF("a", "b") df.sample(true, 2.0).withColumn("c", monotonically_increasing_id).select($"c").show +---+ | c| +---+ | 0| | 1| | 1| | 1| | 2| +---+ ``` ## How was this patch tested? Added a test in `DataFrameSuite`. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #14800 from maropu/FixSampleBug.
* [SPARK-17235][SQL] Support purging of old logs in MetadataLogpetermaxlee2016-08-263-4/+43
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds a purge interface to MetadataLog, and an implementation in HDFSMetadataLog. The purge function is currently unused, but I will use it to purge old execution and file source logs in follow-up patches. These changes are required in a production structured streaming job that runs for a long period of time. ## How was this patch tested? Added a unit test case in HDFSMetadataLogSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14802 from petermaxlee/SPARK-17235.
* [SPARK-17246][SQL] Add BigDecimal literalHerman van Hovell2016-08-262-1/+29
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds parser support for `BigDecimal` literals. If you append the suffix `BD` to a valid number then this will be interpreted as a `BigDecimal`, for example `12.0E10BD` will interpreted into a BigDecimal with scale -9 and precision 3. This is useful in situations where you need exact values. ## How was this patch tested? Added tests to `ExpressionParserSuite`, `ExpressionSQLBuilderSuite` and `SQLQueryTestSuite`. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14819 from hvanhovell/SPARK-17246.
* [SPARK-17165][SQL] FileStreamSource should not track the list of seen files ↵petermaxlee2016-08-265-36/+285
| | | | | | | | | | | | | | | | indefinitely ## What changes were proposed in this pull request? Before this change, FileStreamSource uses an in-memory hash set to track the list of files processed by the engine. The list can grow indefinitely, leading to OOM or overflow of the hash set. This patch introduces a new user-defined option called "maxFileAge", default to 24 hours. If a file is older than this age, FileStreamSource will purge it from the in-memory map that was used to track the list of files that have been processed. ## How was this patch tested? Added unit tests for the underlying utility, and also added an end-to-end test to validate the purge in FileStreamSourceSuite. Also verified the new test cases would fail when the timeout was set to a very large number. Author: petermaxlee <petermaxlee@gmail.com> Closes #14728 from petermaxlee/SPARK-17165.
* [SPARK-17192][SQL] Issue Exception when Users Specify the Partitioning ↵gatorsmile2016-08-262-22/+20
| | | | | | | | | | | | | | | | | | Columns without a Given Schema ### What changes were proposed in this pull request? Address the comments by yhuai in the original PR: https://github.com/apache/spark/pull/14207 First, issue an exception instead of logging a warning when users specify the partitioning columns without a given schema. Second, refactor the codes a little. ### How was this patch tested? Fixed the test cases. Author: gatorsmile <gatorsmile@gmail.com> Closes #14572 from gatorsmile/followup16552.
* [SPARK-16216][SQL][FOLLOWUP] Enable timestamp type tests for JSON and verify ↵hyukjinkwon2016-08-263-8/+26
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | all unsupported types in CSV ## What changes were proposed in this pull request? This PR enables the tests for `TimestampType` for JSON and unifies the logics for verifying schema when writing in CSV. In more details, this PR, - Enables the tests for `TimestampType` for JSON and This was disabled due to an issue in `DatatypeConverter.parseDateTime` which parses dates incorrectly, for example as below: ```scala val d = javax.xml.bind.DatatypeConverter.parseDateTime("0900-01-01T00:00:00.000").getTime println(d.toString) ``` ``` Fri Dec 28 00:00:00 KST 899 ``` However, since we use `FastDateFormat`, it seems we are safe now. ```scala val d = FastDateFormat.getInstance("yyyy-MM-dd'T'HH:mm:ss.SSS").parse("0900-01-01T00:00:00.000") println(d) ``` ``` Tue Jan 01 00:00:00 PST 900 ``` - Verifies all unsupported types in CSV There is a separate logics to verify the schemas in `CSVFileFormat`. This is actually not quite correct enough because we don't support `NullType` and `CalanderIntervalType` as well `StructType`, `ArrayType`, `MapType`. So, this PR adds both types. ## How was this patch tested? Tests in `JsonHadoopFsRelation` and `CSVSuite` Author: hyukjinkwon <gurwls223@gmail.com> Closes #14829 from HyukjinKwon/SPARK-16216-followup.
* [SPARK-17187][SQL] Supports using arbitrary Java object as internal ↵Sean Zhong2016-08-252-0/+315
| | | | | | | | | | | | | | | | | | | | | | | | | aggregation buffer object ## What changes were proposed in this pull request? This PR introduces an abstract class `TypedImperativeAggregate` so that an aggregation function of TypedImperativeAggregate can use **arbitrary** user-defined Java object as intermediate aggregation buffer object. **This has advantages like:** 1. It now can support larger category of aggregation functions. For example, it will be much easier to implement aggregation function `percentile_approx`, which has a complex aggregation buffer definition. 2. It can be used to avoid doing serialization/de-serialization for every call of `update` or `merge` when converting domain specific aggregation object to internal Spark-Sql storage format. 3. It is easier to integrate with other existing monoid libraries like algebird, and supports more aggregation functions with high performance. Please see `org.apache.spark.sql.TypedImperativeAggregateSuite.TypedMaxAggregate` to find an example of how to defined a `TypedImperativeAggregate` aggregation function. Please see Java doc of `TypedImperativeAggregate` and Jira ticket SPARK-17187 for more information. ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Author: Yin Huai <yhuai@databricks.com> Closes #14753 from clockfly/object_aggregation_buffer_try_2.
* [SPARK-17229][SQL] PostgresDialect shouldn't widen float and short types ↵Josh Rosen2016-08-252-1/+10
| | | | | | | | | | | | | | | | | | during reads ## What changes were proposed in this pull request? When reading float4 and smallint columns from PostgreSQL, Spark's `PostgresDialect` widens these types to Decimal and Integer rather than using the narrower Float and Short types. According to https://www.postgresql.org/docs/7.1/static/datatype.html#DATATYPE-TABLE, Postgres maps the `smallint` type to a signed two-byte integer and the `real` / `float4` types to single precision floating point numbers. This patch fixes this by adding more special-cases to `getCatalystType`, similar to what was done for the Derby JDBC dialect. I also fixed a similar problem in the write path which causes Spark to create integer columns in Postgres for what should have been ShortType columns. ## How was this patch tested? New test cases in `PostgresIntegrationSuite` (which I ran manually because Jenkins can't run it right now). Author: Josh Rosen <joshrosen@databricks.com> Closes #14796 from JoshRosen/postgres-jdbc-type-fixes.
* [SPARK-16991][SPARK-17099][SPARK-17120][SQL] Fix Outer Join Elimination when ↵gatorsmile2016-08-253-0/+116
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Filter's isNotNull Constraints Unable to Filter Out All Null-supplying Rows ### What changes were proposed in this pull request? This PR is to fix an incorrect outer join elimination when filter's `isNotNull` constraints is unable to filter out all null-supplying rows. For example, `isnotnull(coalesce(b#227, c#238))`. Users can hit this error when they try to use `using/natural outer join`, which is converted to a normal outer join with a `coalesce` expression on the `using columns`. For example, ```Scala val a = Seq((1, 2), (2, 3)).toDF("a", "b") val b = Seq((2, 5), (3, 4)).toDF("a", "c") val c = Seq((3, 1)).toDF("a", "d") val ab = a.join(b, Seq("a"), "fullouter") ab.join(c, "a").explain(true) ``` The dataframe `ab` is doing `using full-outer join`, which is converted to a normal outer join with a `coalesce` expression. Constraints inference generates a `Filter` with constraints `isnotnull(coalesce(b#227, c#238))`. Then, it triggers a wrong outer join elimination and generates a wrong result. ``` Project [a#251, b#227, c#237, d#247] +- Join Inner, (a#251 = a#246) :- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237] : +- Join FullOuter, (a#226 = a#236) : :- Project [_1#223 AS a#226, _2#224 AS b#227] : : +- LocalRelation [_1#223, _2#224] : +- Project [_1#233 AS a#236, _2#234 AS c#237] : +- LocalRelation [_1#233, _2#234] +- Project [_1#243 AS a#246, _2#244 AS d#247] +- LocalRelation [_1#243, _2#244] == Optimized Logical Plan == Project [a#251, b#227, c#237, d#247] +- Join Inner, (a#251 = a#246) :- Project [coalesce(a#226, a#236) AS a#251, b#227, c#237] : +- Filter isnotnull(coalesce(a#226, a#236)) : +- Join FullOuter, (a#226 = a#236) : :- LocalRelation [a#226, b#227] : +- LocalRelation [a#236, c#237] +- LocalRelation [a#246, d#247] ``` **A note to the `Committer`**, please also give the credit to dongjoon-hyun who submitted another PR for fixing this issue. https://github.com/apache/spark/pull/14580 ### How was this patch tested? Added test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14661 from gatorsmile/fixOuterJoinElimination.
* [SPARK-12978][SQL] Skip unnecessary final group-by when input data already ↵Takeshi YAMAMURO2016-08-258-224/+257
| | | | | | | | | | | | | | | | | | | | | | | | | | | clustered with group-by keys This ticket targets the optimization to skip an unnecessary group-by operation below; Without opt.: ``` == Physical Plan == TungstenAggregate(key=[col0#159], functions=[(sum(col1#160),mode=Final,isDistinct=false),(avg(col2#161),mode=Final,isDistinct=false)], output=[col0#159,sum(col1)#177,avg(col2)#178]) +- TungstenAggregate(key=[col0#159], functions=[(sum(col1#160),mode=Partial,isDistinct=false),(avg(col2#161),mode=Partial,isDistinct=false)], output=[col0#159,sum#200,sum#201,count#202L]) +- TungstenExchange hashpartitioning(col0#159,200), None +- InMemoryColumnarTableScan [col0#159,col1#160,col2#161], InMemoryRelation [col0#159,col1#160,col2#161], true, 10000, StorageLevel(true, true, false, true, 1), ConvertToUnsafe, None ``` With opt.: ``` == Physical Plan == TungstenAggregate(key=[col0#159], functions=[(sum(col1#160),mode=Complete,isDistinct=false),(avg(col2#161),mode=Final,isDistinct=false)], output=[col0#159,sum(col1)#177,avg(col2)#178]) +- TungstenExchange hashpartitioning(col0#159,200), None +- InMemoryColumnarTableScan [col0#159,col1#160,col2#161], InMemoryRelation [col0#159,col1#160,col2#161], true, 10000, StorageLevel(true, true, false, true, 1), ConvertToUnsafe, None ``` Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #10896 from maropu/SkipGroupbySpike.
* [SPARK-17215][SQL] Method `SQLContext.parseDataType(dataTypeString: String)` ↵jiangxingbo2016-08-241-10/+0
| | | | | | | | | | | | | | | | | could be removed. ## What changes were proposed in this pull request? Method `SQLContext.parseDataType(dataTypeString: String)` could be removed, we should use `SparkSession.parseDataType(dataTypeString: String)` instead. This require updating PySpark. ## How was this patch tested? Existing test cases. Author: jiangxingbo <jiangxb1987@gmail.com> Closes #14790 from jiangxb1987/parseDataType.
* [SPARK-17190][SQL] Removal of HiveSharedStategatorsmile2016-08-252-15/+46
| | | | | | | | | | | | | | ### What changes were proposed in this pull request? Since `HiveClient` is used to interact with the Hive metastore, it should be hidden in `HiveExternalCatalog`. After moving `HiveClient` into `HiveExternalCatalog`, `HiveSharedState` becomes a wrapper of `HiveExternalCatalog`. Thus, removal of `HiveSharedState` becomes straightforward. After removal of `HiveSharedState`, the reflection logic is directly applied on the choice of `ExternalCatalog` types, based on the configuration of `CATALOG_IMPLEMENTATION`. ~~`HiveClient` is also used/invoked by the other entities besides HiveExternalCatalog, we defines the following two APIs: getClient and getNewClient~~ ### How was this patch tested? The existing test cases Author: gatorsmile <gatorsmile@gmail.com> Closes #14757 from gatorsmile/removeHiveClient.
* [SPARK-16216][SQL] Read/write timestamps and dates in ISO 8601 and ↵hyukjinkwon2016-08-2415-70/+384
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | dateFormat/timestampFormat option for CSV and JSON ## What changes were proposed in this pull request? ### Default - ISO 8601 Currently, CSV datasource is writing `Timestamp` and `Date` as numeric form and JSON datasource is writing both as below: - CSV ``` // TimestampType 1414459800000000 // DateType 16673 ``` - Json ``` // TimestampType 1970-01-01 11:46:40.0 // DateType 1970-01-01 ``` So, for CSV we can't read back what we write and for JSON it becomes ambiguous because the timezone is being missed. So, this PR make both **write** `Timestamp` and `Date` in ISO 8601 formatted string (please refer the [ISO 8601 specification](https://www.w3.org/TR/NOTE-datetime)). - For `Timestamp` it becomes as below: (`yyyy-MM-dd'T'HH:mm:ss.SSSZZ`) ``` 1970-01-01T02:00:01.000-01:00 ``` - For `Date` it becomes as below (`yyyy-MM-dd`) ``` 1970-01-01 ``` ### Custom date format option - `dateFormat` This PR also adds the support to write and read dates and timestamps in a formatted string as below: - **DateType** - With `dateFormat` option (e.g. `yyyy/MM/dd`) ``` +----------+ | date| +----------+ |2015/08/26| |2014/10/27| |2016/01/28| +----------+ ``` ### Custom date format option - `timestampFormat` - **TimestampType** - With `dateFormat` option (e.g. `dd/MM/yyyy HH:mm`) ``` +----------------+ | date| +----------------+ |2015/08/26 18:00| |2014/10/27 18:30| |2016/01/28 20:00| +----------------+ ``` ## How was this patch tested? Unit tests were added in `CSVSuite` and `JsonSuite`. For JSON, existing tests cover the default cases. Author: hyukjinkwon <gurwls223@gmail.com> Closes #14279 from HyukjinKwon/SPARK-16216-json-csv.
* [SPARK-17186][SQL] remove catalog table type INDEXWenchen Fan2016-08-231-5/+3
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Actually Spark SQL doesn't support index, the catalog table type `INDEX` is from Hive. However, most operations in Spark SQL can't handle index table, e.g. create table, alter table, etc. Logically index table should be invisible to end users, and Hive also generates special table name for index table to avoid users accessing it directly. Hive has special SQL syntax to create/show/drop index tables. At Spark SQL side, although we can describe index table directly, but the result is unreadable, we should use the dedicated SQL syntax to do it(e.g. `SHOW INDEX ON tbl`). Spark SQL can also read index table directly, but the result is always empty.(Can hive read index table directly?) This PR remove the table type `INDEX`, to make it clear that Spark SQL doesn't support index currently. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14752 from cloud-fan/minor2.
* [SPARK-13286] [SQL] add the next expression of SQLException as causeDavies Liu2016-08-231-2/+13
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Some JDBC driver (for example PostgreSQL) does not use the underlying exception as cause, but have another APIs (getNextException) to access that, so it it's included in the error logging, making us hard to find the root cause, especially in batch mode. This PR will pull out the next exception and add it as cause (if it's different) or suppressed (if there is another different cause). ## How was this patch tested? Can't reproduce this on the default JDBC driver, so did not add a regression test. Author: Davies Liu <davies@databricks.com> Closes #14722 from davies/keep_cause.
* [SPARK-17199] Use CatalystConf.resolver for case-sensitivity comparisonJacek Laskowski2016-08-233-20/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Use `CatalystConf.resolver` consistently for case-sensitivity comparison (removed dups). ## How was this patch tested? Local build. Waiting for Jenkins to ensure clean build and test. Author: Jacek Laskowski <jacek@japila.pl> Closes #14771 from jaceklaskowski/17199-catalystconf-resolver.
* [SPARK-17188][SQL] Moves class QuantileSummaries to project catalyst for ↵Sean Zhong2016-08-232-375/+1
| | | | | | | | | | | | | | | | implementing percentile_approx ## What changes were proposed in this pull request? This is a sub-task of [SPARK-16283](https://issues.apache.org/jira/browse/SPARK-16283) (Implement percentile_approx SQL function), which moves class QuantileSummaries to project catalyst so that it can be reused when implementing aggregation function `percentile_approx`. ## How was this patch tested? This PR only does class relocation, class implementation is not changed. Author: Sean Zhong <seanzhong@databricks.com> Closes #14754 from clockfly/move_QuantileSummaries_to_catalyst.
* [SPARK-17144][SQL] Removal of useless CreateHiveTableAsSelectLogicalPlangatorsmile2016-08-231-18/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? `CreateHiveTableAsSelectLogicalPlan` is a dead code after refactoring. ## How was this patch tested? N/A Author: gatorsmile <gatorsmile@gmail.com> Closes #14707 from gatorsmile/removeCreateHiveTable.
* [SPARK-17162] Range does not support SQL generationEric Liang2016-08-223-3/+5
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The range operator previously didn't support SQL generation, which made it not possible to use in views. ## How was this patch tested? Unit tests. cc hvanhovell Author: Eric Liang <ekl@databricks.com> Closes #14724 from ericl/spark-17162.
* [MINOR][SQL] Fix some typos in comments and test hintsSean Zhong2016-08-223-7/+7
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix some typos in comments and test hints ## How was this patch tested? N/A. Author: Sean Zhong <seanzhong@databricks.com> Closes #14755 from clockfly/fix_minor_typo.
* [SPARK-17115][SQL] decrease the threshold when split expressionsDavies Liu2016-08-222-2/+53
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In 2.0, we change the threshold of splitting expressions from 16K to 64K, which cause very bad performance on wide table, because the generated method can't be JIT compiled by default (above the limit of 8K bytecode). This PR will decrease it to 1K, based on the benchmark results for a wide table with 400 columns of LongType. It also fix a bug around splitting expression in whole-stage codegen (it should not split them). ## How was this patch tested? Added benchmark suite. Author: Davies Liu <davies@databricks.com> Closes #14692 from davies/split_exprs.
* [SPARK-16498][SQL] move hive hack for data source table into HiveExternalCatalogWenchen Fan2016-08-2114-500/+92
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark SQL doesn't have its own meta store yet, and use hive's currently. However, hive's meta store has some limitations(e.g. columns can't be too many, not case-preserving, bad decimal type support, etc.), so we have some hacks to successfully store data source table metadata into hive meta store, i.e. put all the information in table properties. This PR moves these hacks to `HiveExternalCatalog`, tries to isolate hive specific logic in one place. changes overview: 1. **before this PR**: we need to put metadata(schema, partition columns, etc.) of data source tables to table properties before saving it to external catalog, even the external catalog doesn't use hive metastore(e.g. `InMemoryCatalog`) **after this PR**: the table properties tricks are only in `HiveExternalCatalog`, the caller side doesn't need to take care of it anymore. 2. **before this PR**: because the table properties tricks are done outside of external catalog, so we also need to revert these tricks when we read the table metadata from external catalog and use it. e.g. in `DescribeTableCommand` we will read schema and partition columns from table properties. **after this PR**: The table metadata read from external catalog is exactly the same with what we saved to it. bonus: now we can create data source table using `SessionCatalog`, if schema is specified. breaks: `schemaStringLengthThreshold` is not configurable anymore. `hive.default.rcfile.serde` is not configurable anymore. ## How was this patch tested? existing tests. Author: Wenchen Fan <wenchen@databricks.com> Closes #14155 from cloud-fan/catalog-table.
* [SPARK-17098][SQL] Fix `NullPropagation` optimizer to handle `COUNT(NULL) ↵Dongjoon Hyun2016-08-212-0/+47
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | OVER` correctly ## What changes were proposed in this pull request? Currently, `NullPropagation` optimizer replaces `COUNT` on null literals in a bottom-up fashion. During that, `WindowExpression` is not covered properly. This PR adds the missing propagation logic. **Before** ```scala scala> sql("SELECT COUNT(1 + NULL) OVER ()").show java.lang.UnsupportedOperationException: Cannot evaluate expression: cast(0 as bigint) windowspecdefinition(ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) ``` **After** ```scala scala> sql("SELECT COUNT(1 + NULL) OVER ()").show +----------------------------------------------------------------------------------------------+ |count((1 + CAST(NULL AS INT))) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)| +----------------------------------------------------------------------------------------------+ | 0| +----------------------------------------------------------------------------------------------+ ``` ## How was this patch tested? Pass the Jenkins test with a new test case. Author: Dongjoon Hyun <dongjoon@apache.org> Closes #14689 from dongjoon-hyun/SPARK-17098.
* [SPARK-17124][SQL] RelationalGroupedDataset.agg should preserve order and ↵petermaxlee2016-08-212-2/+14
| | | | | | | | | | | | | | | | | | | allow multiple aggregates per column ## What changes were proposed in this pull request? This patch fixes a longstanding issue with one of the RelationalGroupedDataset.agg function. Even though the signature accepts vararg of pairs, the underlying implementation turns the seq into a map, and thus not order preserving nor allowing multiple aggregates per column. This change also allows users to use this function to run multiple different aggregations for a single column, e.g. ``` agg("age" -> "max", "age" -> "count") ``` ## How was this patch tested? Added a test case in DataFrameAggregateSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14697 from petermaxlee/SPARK-17124.
* [SPARK-17104][SQL] LogicalRelation.newInstance should follow the semantics ↵Liang-Chi Hsieh2016-08-201-2/+9
| | | | | | | | | | | | | | | | of MultiInstanceRelation ## What changes were proposed in this pull request? Currently `LogicalRelation.newInstance()` simply creates another `LogicalRelation` object with the same parameters. However, the `newInstance()` method inherited from `MultiInstanceRelation` should return a copy of object with unique expression ids. Current `LogicalRelation.newInstance()` can cause failure when doing self-join. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14682 from viirya/fix-localrelation.
* [SPARK-17150][SQL] Support SQL generation for inline tablespetermaxlee2016-08-201-0/+3
| | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds support for SQL generation for inline tables. With this, it would be possible to create a view that depends on inline tables. ## How was this patch tested? Added a test case in LogicalPlanToSQLSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14709 from petermaxlee/SPARK-17150.
* [SPARK-17158][SQL] Change error message for out of range numeric literalsSrinath Shankar2016-08-191-3/+3
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Modifies error message for numeric literals to Numeric literal <literal> does not fit in range [min, max] for type <T> ## How was this patch tested? Fixed up the error messages for literals.sql in SqlQueryTestSuite and re-ran via sbt. Also fixed up error messages in ExpressionParserSuite Author: Srinath Shankar <srinath@databricks.com> Closes #14721 from srinathshankar/sc4296.
* [SPARK-17149][SQL] array.sql for testing array related functionspetermaxlee2016-08-194-16/+240
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch creates array.sql in SQLQueryTestSuite for testing array related functions, including: - indexing - array creation - size - array_contains - sort_array ## How was this patch tested? The patch itself is about adding tests. Author: petermaxlee <petermaxlee@gmail.com> Closes #14708 from petermaxlee/SPARK-17149.
* [SPARK-16994][SQL] Whitelist operators for predicate pushdownReynold Xin2016-08-192-1/+12
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch changes predicate pushdown optimization rule (PushDownPredicate) from using a blacklist to a whitelist. That is to say, operators must be explicitly allowed. This approach is more future-proof: previously it was possible for us to introduce a new operator and then render the optimization rule incorrect. This also fixes the bug that previously we allowed pushing filter beneath limit, which was incorrect. That is to say, before this patch, the optimizer would rewrite ``` select * from (select * from range(10) limit 5) where id > 3 to select * from range(10) where id > 3 limit 5 ``` ## How was this patch tested? - a unit test case in FilterPushdownSuite - an end-to-end test in limit.sql Author: Reynold Xin <rxin@databricks.com> Closes #14713 from rxin/SPARK-16994.
* [SPARK-16947][SQL] Support type coercion and foldable expression for inline ↵petermaxlee2016-08-192-0/+193
| | | | | | | | | | | | | | | | | | | tables ## What changes were proposed in this pull request? This patch improves inline table support with the following: 1. Support type coercion. 2. Support using foldable expressions. Previously only literals were supported. 3. Improve error message handling. 4. Improve test coverage. ## How was this patch tested? Added a new unit test suite ResolveInlineTablesSuite and a new file-based end-to-end test inline-table.sql. Author: petermaxlee <petermaxlee@gmail.com> Closes #14676 from petermaxlee/SPARK-16947.
* [SPARK-17117][SQL] 1 / NULL should not fail analysispetermaxlee2016-08-182-20/+76
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch fixes the problem described in SPARK-17117, i.e. "SELECT 1 / NULL" throws an analysis exception: ``` org.apache.spark.sql.AnalysisException: cannot resolve '(1 / NULL)' due to data type mismatch: differing types in '(1 / NULL)' (int and null). ``` The problem is that division type coercion did not take null type into account. ## How was this patch tested? A unit test for the type coercion, and a few end-to-end test cases using SQLQueryTestSuite. Author: petermaxlee <petermaxlee@gmail.com> Closes #14695 from petermaxlee/SPARK-17117.
* [SPARK-17069] Expose spark.range() as table-valued function in SQLEric Liang2016-08-182-0/+107
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This adds analyzer rules for resolving table-valued functions, and adds one builtin implementation for range(). The arguments for range() are the same as those of `spark.range()`. ## How was this patch tested? Unit tests. cc hvanhovell Author: Eric Liang <ekl@databricks.com> Closes #14656 from ericl/sc-4309.
* [SPARK-16391][SQL] Support partial aggregation for reduceGroupsReynold Xin2016-08-183-5/+146
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch introduces a new private ReduceAggregator interface that is a subclass of Aggregator. ReduceAggregator only requires a single associative and commutative reduce function. ReduceAggregator is also used to implement KeyValueGroupedDataset.reduceGroups in order to support partial aggregation. Note that the pull request was initially done by viirya. ## How was this patch tested? Covered by original tests for reduceGroups, as well as a new test suite for ReduceAggregator. Author: Reynold Xin <rxin@databricks.com> Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14576 from rxin/reduceAggregator.
* [SPARK-16995][SQL] TreeNodeException when flat mapping ↵Liang-Chi Hsieh2016-08-181-0/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | RelationalGroupedDataset created from DataFrame containing a column created with lit/expr ## What changes were proposed in this pull request? A TreeNodeException is thrown when executing the following minimal example in Spark 2.0. import spark.implicits._ case class test (x: Int, q: Int) val d = Seq(1).toDF("x") d.withColumn("q", lit(0)).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show d.withColumn("q", expr("0")).as[test].groupByKey(_.x).flatMapGroups{case (x, iter) => List[Int]()}.show The problem is at `FoldablePropagation`. The rule will do `transformExpressions` on `LogicalPlan`. The query above contains a `MapGroups` which has a parameter `dataAttributes:Seq[Attribute]`. One attributes in `dataAttributes` will be transformed to an `Alias(literal(0), _)` in `FoldablePropagation`. `Alias` is not an `Attribute` and causes the error. We can't easily detect such type inconsistency during transforming expressions. A direct approach to this problem is to skip doing `FoldablePropagation` on object operators as they should not contain such expressions. ## How was this patch tested? Jenkins tests. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #14648 from viirya/flat-mapping.
* [SPARK-17096][SQL][STREAMING] Improve exception string reported through the ↵Tathagata Das2016-08-174-14/+10
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | StreamingQueryListener ## What changes were proposed in this pull request? Currently, the stackTrace (as `Array[StackTraceElements]`) reported through StreamingQueryListener.onQueryTerminated is useless as it has the stack trace of where StreamingQueryException is defined, not the stack trace of underlying exception. For example, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.stackTrace` will have ``` org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:211) org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124) ``` This is basically useless, as it is location where the StreamingQueryException was defined. What we want is Here is the right way to reason about what should be posted as through StreamingQueryListener.onQueryTerminated - The actual exception could either be a SparkException, or an arbitrary exception. - SparkException reports the relevant executor stack trace of a failed task as a string in the the exception message. The `Array[StackTraceElements]` returned by `SparkException.stackTrace()` is mostly irrelevant. - For any arbitrary exception, the `Array[StackTraceElements]` returned by `exception.stackTrace()` may be relevant. - When there is an error in a streaming query, it's hard to reason whether the `Array[StackTraceElements]` is useful or not. In fact, it is not clear whether it is even useful to report the stack trace as this array of Java objects. It may be sufficient to report the strack trace as a string, along with the message. This is how Spark reported executor stra - Hence, this PR simplifies the API by removing the array `stackTrace` from `QueryTerminated`. Instead the `exception` returns a string containing the message and the stack trace of the actual underlying exception that failed the streaming query (i.e. not that of the StreamingQueryException). If anyone is interested in the actual stack trace as an array, can always access them through `streamingQuery.exception` which returns the exception object. With this change, if a streaming query fails because of a / by zero exception in a task, the `QueryTerminated.exception` will be ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 1 in stage 0.0 failed 1 times, most recent failure: Lost task 1.0 in stage 0.0 (TID 1, localhost): java.lang.ArithmeticException: / by zero at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply$mcII$sp(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.streaming.StreamingQueryListenerSuite$$anonfun$5$$anonfun$apply$mcV$sp$4$$anonfun$apply$mcV$sp$5.apply(StreamingQueryListenerSuite.scala:153) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source) at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43) at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:370) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:232) at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:226) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$24.apply(RDD.scala:803) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:319) at org.apache.spark.rdd.RDD.iterator(RDD.scala:283) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:86) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1429) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1417) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1416) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1416) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:802) ... ``` It contains the relevant executor stack trace. In a case non-SparkException, if the streaming source MemoryStream throws an exception, exception message will have the relevant stack trace. ``` java.lang.RuntimeException: this is the exception message at org.apache.spark.sql.execution.streaming.MemoryStream.getBatch(memory.scala:103) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:316) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$5.apply(StreamExecution.scala:313) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) at scala.collection.Iterator$class.foreach(Iterator.scala:893) at scala.collection.AbstractIterator.foreach(Iterator.scala:1336) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241) at org.apache.spark.sql.execution.streaming.StreamProgress.flatMap(StreamProgress.scala:25) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatch(StreamExecution.scala:313) at org.apache.spark.sql.execution.streaming.StreamExecution$$anonfun$org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches$1.apply$mcZ$sp(StreamExecution.scala:197) at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:43) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runBatches(StreamExecution.scala:187) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:124) ``` Note that this change in the public `QueryTerminated` class is okay as the APIs are still experimental. ## How was this patch tested? Unit tests that test whether the right information is present in the exception message reported through QueryTerminated object. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14675 from tdas/SPARK-17096.
* [SPARK-16736][CORE][SQL] purge superfluous fs callsSteve Loughran2016-08-171-15/+17
| | | | | | | | | | | | | | | A review of the code, working back from Hadoop's `FileSystem.exists()` and `FileSystem.isDirectory()` code, then removing uses of the calls when superfluous. 1. delete is harmless if called on a nonexistent path, so don't do any checks before deletes 1. any `FileSystem.exists()` check before `getFileStatus()` or `open()` is superfluous as the operation itself does the check. Instead the `FileNotFoundException` is caught and triggers the downgraded path. When a `FileNotFoundException` was thrown before, the code still creates a new FNFE with the error messages. Though now the inner exceptions are nested, for easier diagnostics. Initially, relying on Jenkins test runs. One troublespot here is that some of the codepaths are clearly error situations; it's not clear that they have coverage anyway. Trying to create the failure conditions in tests would be ideal, but it will also be hard. Author: Steve Loughran <stevel@apache.org> Closes #14371 from steveloughran/cloud/SPARK-16736-superfluous-fs-calls.
* [SPARK-17102][SQL] bypass UserDefinedGenerator for json format checkWenchen Fan2016-08-171-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? We use reflection to convert `TreeNode` to json string, and currently don't support arbitrary object. `UserDefinedGenerator` takes a function object, so we should skip json format test for it, or the tests can be flacky, e.g. `DataFrameSuite.simple explode`, this test always fail with scala 2.10(branch 1.6 builds with scala 2.10 by default), but pass with scala 2.11(master branch builds with scala 2.11 by default). ## How was this patch tested? N/A Author: Wenchen Fan <wenchen@databricks.com> Closes #14679 from cloud-fan/json.
* [SPARK-17106] [SQL] Simplify the SubqueryExpression interfaceHerman van Hovell2016-08-174-36/+20
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current subquery expression interface contains a little bit of technical debt in the form of a few different access paths to get and set the query contained by the expression. This is confusing to anyone who goes over this code. This PR unifies these access paths. ## How was this patch tested? (Existing tests) Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14685 from hvanhovell/SPARK-17106.
* [SPARK-15285][SQL] Generated SpecificSafeProjection.apply method grows ↵Kazuaki Ishizaki2016-08-171-0/+40
| | | | | | | | | | | | | | | | | | beyond 64 KB ## What changes were proposed in this pull request? This PR splits the generated code for ```SafeProjection.apply``` by using ```ctx.splitExpressions()```. This is because the large code body for ```NewInstance``` may grow beyond 64KB bytecode size for ```apply()``` method. Here is [the original PR](https://github.com/apache/spark/pull/13243) for SPARK-15285. However, it breaks a build with Scala 2.10 since Scala 2.10 does not a case class with large number of members. Thus, it was reverted by [this commit](https://github.com/apache/spark/commit/fa244e5a90690d6a31be50f2aa203ae1a2e9a1cf). ## How was this patch tested? Added new tests by using `DefinedByConstructorParams` instead of case class for scala-2.10 Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #14670 from kiszk/SPARK-15285-2.
* [SPARK-17068][SQL] Make view-usage visible during analysisHerman van Hovell2016-08-163-5/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR adds a field to subquery alias in order to make the usage of views in a resolved `LogicalPlan` more visible (and more understandable). For example, the following view and query: ```sql create view constants as select 1 as id union all select 1 union all select 42 select * from constants; ``` ...now yields the following analyzed plan: ``` Project [id#39] +- SubqueryAlias c, `default`.`constants` +- Project [gen_attr_0#36 AS id#39] +- SubqueryAlias gen_subquery_0 +- Union :- Union : :- Project [1 AS gen_attr_0#36] : : +- OneRowRelation$ : +- Project [1 AS gen_attr_1#37] : +- OneRowRelation$ +- Project [42 AS gen_attr_2#38] +- OneRowRelation$ ``` ## How was this patch tested? Added tests for the two code paths in `SessionCatalogSuite` (sql/core) and `HiveMetastoreCatalogSuite` (sql/hive) Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14657 from hvanhovell/SPARK-17068.
* [SPARK-17084][SQL] Rename ParserUtils.assert to validateHerman van Hovell2016-08-161-3/+2
| | | | | | | | | | | | ## What changes were proposed in this pull request? This PR renames `ParserUtils.assert` to `ParserUtils.validate`. This is done because this method is used to check requirements, and not to check if the program is in an invalid state. ## How was this patch tested? Simple rename. Compilation should do. Author: Herman van Hovell <hvanhovell@databricks.com> Closes #14665 from hvanhovell/SPARK-17084.
* [SPARK-17034][SQL] adds expression UnresolvedOrdinal to represent the ↵Sean Zhong2016-08-162-6/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ordinals in GROUP BY or ORDER BY ## What changes were proposed in this pull request? This PR adds expression `UnresolvedOrdinal` to represent the ordinal in GROUP BY or ORDER BY, and fixes the rules when resolving ordinals. Ordinals in GROUP BY or ORDER BY like `1` in `order by 1` or `group by 1` should be considered as unresolved before analysis. But in current code, it uses `Literal` expression to store the ordinal. This is inappropriate as `Literal` itself is a resolved expression, it gives the user a wrong message that the ordinals has already been resolved. ### Before this change Ordinal is stored as `Literal` expression ``` scala> sc.setLogLevel("TRACE") scala> sql("select a from t group by 1 order by 1") ... 'Sort [1 ASC], true +- 'Aggregate [1], ['a] +- 'UnresolvedRelation `t ``` For query: ``` scala> Seq(1).toDF("a").createOrReplaceTempView("t") scala> sql("select count(a), a from t group by 2 having a > 0").show ``` During analysis, the intermediate plan before applying rule `ResolveAggregateFunctions` is: ``` 'Filter ('a > 0) +- Aggregate [2], [count(1) AS count(1)#83L, a#81] +- LocalRelation [value#7 AS a#9] ``` Before this PR, rule `ResolveAggregateFunctions` believes all expressions of `Aggregate` have already been resolved, and tries to resolve the expressions in `Filter` directly. But this is wrong, as ordinal `2` in Aggregate is not really resolved! ### After this change Ordinals are stored as `UnresolvedOrdinal`. ``` scala> sc.setLogLevel("TRACE") scala> sql("select a from t group by 1 order by 1") ... 'Sort [unresolvedordinal(1) ASC], true +- 'Aggregate [unresolvedordinal(1)], ['a] +- 'UnresolvedRelation `t` ``` ## How was this patch tested? Unit tests. Author: Sean Zhong <seanzhong@databricks.com> Closes #14616 from clockfly/spark-16955.
* [SPARK-16916][SQL] serde/storage properties should not have limitationsWenchen Fan2016-08-153-11/+0
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `CatalogStorageFormat.properties` can be used in 2 ways: 1. for hive tables, it stores the serde properties. 2. for data source tables, it stores the data source options, e.g. `path`, `skipHiveMetadata`, etc. however, both of them have nothing to do with data source properties, e.g. `spark.sql.sources.provider`, so they should not have limitations about data source properties. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #14506 from cloud-fan/table-prop.
* [SPARK-17065][SQL] Improve the error message when encountering an ↵Shixiong Zhu2016-08-151-39/+52
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | incompatible DataSourceRegister ## What changes were proposed in this pull request? Add an instruction to ask the user to remove or upgrade the incompatible DataSourceRegister in the error message. ## How was this patch tested? Test command: ``` build/sbt -Dscala-2.10 package SPARK_SCALA_VERSION=2.10 bin/spark-shell --packages ai.h2o:sparkling-water-core_2.10:1.6.5 scala> Seq(1).toDS().write.format("parquet").save("foo") ``` Before: ``` java.util.ServiceConfigurationError: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at java.util.ServiceLoader.fail(ServiceLoader.java:232) at java.util.ServiceLoader.access$100(ServiceLoader.java:185) at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384) at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404) at java.util.ServiceLoader$1.next(ServiceLoader.java:480) ... Caused by: java.lang.NoClassDefFoundError: org/apache/spark/Logging at java.lang.ClassLoader.defineClass1(Native Method) at java.lang.ClassLoader.defineClass(ClassLoader.java:760) at java.security.SecureClassLoader.defineClass(SecureClassLoader.java:142) at java.net.URLClassLoader.defineClass(URLClassLoader.java:467) at java.net.URLClassLoader.access$100(URLClassLoader.java:73) at java.net.URLClassLoader$1.run(URLClassLoader.java:368) at java.net.URLClassLoader$1.run(URLClassLoader.java:362) at java.security.AccessController.doPrivileged(Native Method) ... ``` After: ``` java.lang.ClassNotFoundException: Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: org.apache.spark.sql.sources.DataSourceRegister: Provider org.apache.spark.h2o.DefaultSource could not be instantiated at org.apache.spark.sql.execution.datasources.DataSource.lookupDataSource(DataSource.scala:178) at org.apache.spark.sql.execution.datasources.DataSource.providingClass$lzycompute(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.providingClass(DataSource.scala:79) at org.apache.spark.sql.execution.datasources.DataSource.write(DataSource.scala:441) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:213) at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:196) ... ``` Author: Shixiong Zhu <shixiong@databricks.com> Closes #14651 from zsxwing/SPARK-17065.
* [SPARK-16671][CORE][SQL] Consolidate code to do variable substitution.Marcelo Vanzin2016-08-153-101/+18
| | | | | | | | | | | | | | | | Both core and sql have slightly different code that does variable substitution of config values. This change refactors that code and encapsulates the logic of reading config values and expading variables in a new helper class, which can be configured so that both core and sql can use it without losing existing functionality, and allows for easier testing and makes it easier to add more features in the future. Tested with existing and new unit tests, and by running spark-shell with some configs referencing variables and making sure it behaved as expected. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14468 from vanzin/SPARK-16671.
* [SPARK-16966][SQL][CORE] App Name is a randomUUID even when "spark.app.name" ↵Sean Owen2016-08-132-4/+8
| | | | | | | | | | | | | | | | | | | exists ## What changes were proposed in this pull request? Don't override app name specified in `SparkConf` with a random app name. Only set it if the conf has no app name even after options have been applied. See also https://github.com/apache/spark/pull/14602 This is similar to Sherry302 's original proposal in https://github.com/apache/spark/pull/14556 ## How was this patch tested? Jenkins test, with new case reproducing the bug Author: Sean Owen <sowen@cloudera.com> Closes #14630 from srowen/SPARK-16966.2.
* [SPARK-16968] Add additional options in jdbc when creating a new tableGraceH2016-08-133-12/+51
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In the PR, we just allow the user to add additional options when create a new table in JDBC writer. The options can be table_options or partition_options. E.g., "CREATE TABLE t (name string) ENGINE=InnoDB DEFAULT CHARSET=utf8" Here is the usage example: ``` df.write.option("createTableOptions", "ENGINE=InnoDB DEFAULT CHARSET=utf8").jdbc(...) ``` ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) will apply test result soon. Author: GraceH <93113783@qq.com> Closes #14559 from GraceH/jdbc_options.