aboutsummaryrefslogtreecommitdiff
path: root/sql/catalyst
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-6134][SQL] Fix wrong datatype for casting FloatType and default ↵Liang-Chi Hsieh2015-03-041-2/+2
| | | | | | | | | | | | | | LongType value in defaultPrimitive In `CodeGenerator`, the casting on `FloatType` should use `FloatType` instead of `IntegerType`. Besides, `defaultPrimitive` for `LongType` should be `-1L` instead of `1L`. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #4870 from viirya/codegen_type and squashes the following commits: 76311dd [Liang-Chi Hsieh] Fix wrong datatype for casting on FloatType. Fix the wrong value for LongType in defaultPrimitive.
* [SPARK-5950][SQL]Insert array into a metastore table saved as parquet should ↵Yin Huai2015-03-023-2/+181
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | work when using datasource api This PR contains the following changes: 1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However, the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values). 2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types. 3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings. 4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust. 5. Update the equality check of JSON relation. Since JSON does not really cares nullability, `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables. JIRA: https://issues.apache.org/jira/browse/SPARK-5950 Thanks viirya for the initial work in #4729. cc marmbrus liancheng Author: Yin Huai <yhuai@databricks.com> Closes #4826 from yhuai/insertNullabilityCheck and squashes the following commits: 3b61a04 [Yin Huai] Revert change on equals. 80e487e [Yin Huai] asNullable in UDT. 587d88b [Yin Huai] Make methods private. 0cb7ea2 [Yin Huai] marmbrus's comments. 3cec464 [Yin Huai] Cheng's comments. 486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck d3747d1 [Yin Huai] Remove unnecessary change. 8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck 8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check. 0eb5578 [Yin Huai] Fix tests. f6ed813 [Yin Huai] Update old parquet path. e4f397c [Yin Huai] Unit tests. b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check. 8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data. bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data. 0a703e7 [Yin Huai] Test failed again since we cannot read correct content. 9a26611 [Yin Huai] Make InsertIntoTable happy. 8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability 4ec17fd [Yin Huai] Failed test.
* [Minor] Fix doc typo for describing primitiveTerm effectiveness conditionLiang-Chi Hsieh2015-03-021-1/+1
| | | | | | | | | | It should be `true` instead of `false`? Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #4762 from viirya/doc_fix and squashes the following commits: 2e37482 [Liang-Chi Hsieh] Fix doc.
* [SPARK-5999][SQL] Remove duplicate Literal matching blockLiang-Chi Hsieh2015-02-251-0/+2
| | | | | | | | Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #4760 from viirya/dup_literal and squashes the following commits: 06e7516 [Liang-Chi Hsieh] Remove duplicate Literal matching block.
* [SPARK-5910][SQL] Support for as in selectExprMichael Armbrust2015-02-241-1/+1
| | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #4736 from marmbrus/asExprs and squashes the following commits: 5ba97e4 [Michael Armbrust] [SPARK-5910][SQL] Support for as in selectExpr
* [SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecutionMichael Armbrust2015-02-234-97/+128
| | | | | | | | | | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #4684 from marmbrus/explainAnalysis and squashes the following commits: afbaa19 [Michael Armbrust] fix python d93278c [Michael Armbrust] fix hive e5fa0a4 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis 52119f2 [Michael Armbrust] more tests 82a5431 [Michael Armbrust] fix tests 25753d2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis aee1e6a [Michael Armbrust] fix hive b23a844 [Michael Armbrust] newline de8dc51 [Michael Armbrust] more comments acf620a [Michael Armbrust] [SPARK-5873][SQL] Show partially analyzed plans in query execution
* [SPARK-5875][SQL]logical.Project should not be resolved if it contains ↵Yin Huai2015-02-172-1/+22
| | | | | | | | | | | | aggregates or generators https://issues.apache.org/jira/browse/SPARK-5875 has a case to reproduce the bug and explain the root cause. Author: Yin Huai <yhuai@databricks.com> Closes #4663 from yhuai/projectResolved and squashes the following commits: 472f7b6 [Yin Huai] If a logical.Project has any AggregateExpression or Generator, it's resolved field should be false.
* [SPARK-5853][SQL] Schema support in Row.Reynold Xin2015-02-163-5/+14
| | | | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #4640 from rxin/SPARK-5853 and squashes the following commits: 9c6f569 [Reynold Xin] [SPARK-5853][SQL] Schema support in Row.
* [SQL] Optimize arithmetic and predicate operatorskai2015-02-168-260/+290
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Existing implementation of arithmetic operators and BinaryComparison operators have redundant type checking codes, e.g.: Expression.n2 is used by Add/Subtract/Multiply. (1) n2 always checks left.dataType == right.dataType. However, this checking should be done once when we resolve expression types; (2) n2 requires dataType is a NumericType. This can be done once. This PR optimizes arithmetic and predicate operators by removing such redundant type-checking codes. Some preliminary benchmarking on 10G TPC-H data over 5 r3.2xlarge EC2 machines shows that this PR can reduce the query time by 5.5% to 11%. The benchmark queries follow the template below, where OP is plus/minus/times/divide/remainder/bitwise and/bitwise or/bitwise xor. SELECT l_returnflag, l_linestatus, SUM(l_quantity OP cnt1), SUM(l_quantity OP cnt2), ...., SUM(l_quantity OP cnt700) FROM ( SELECT l_returnflag, l_linestatus, l_quantity, 1 AS cnt1, 2 AS cnt2, ..., 700 AS cnt700 FROM lineitem WHERE l_shipdate <= '1998-09-01' ) GROUP BY l_returnflag, l_linestatus; Author: kai <kaizeng@eecs.berkeley.edu> Closes #4472 from kai-zeng/arithmetic-optimize and squashes the following commits: fef0cf1 [kai] Merge branch 'master' of github.com:apache/spark into arithmetic-optimize 4b3a1bb [kai] chmod a-x 5a41e49 [kai] chmod a-x Expression.scala cb37c94 [kai] rebase onto spark master 7f6e968 [kai] chmod 100755 -> 100644 6cddb46 [kai] format 7490dbc [kai] fix unresolved-expression exception for EqualTo 9c40bc0 [kai] fix bitwisenot 3cbd363 [kai] clean up test code ca47801 [kai] override evalInternal for bitwise ops 8fa84a1 [kai] add bitwise or and xor 6892fc4 [kai] revert override evalInternal f8eba24 [kai] override evalInternal 31ccdd4 [kai] rewrite all bitwise op and remove evalInternal 86297e2 [kai] generalized cb92ae1 [kai] bitwise-and: override eval 97a7d6c [kai] bitwise-and: override evalInternal using and func 0906c39 [kai] add bitwise test 62abbbc [kai] clean up predicate and arithmetic b34d58d [kai] add caching and benmark option 12c5b32 [kai] override eval 1cd7571 [kai] fix sqrt and maxof 03fd0c3 [kai] fix predicate 16fd84c [kai] optimize + - * / % -(unary) abs < > <= >= fd95823 [kai] remove unnecessary type checking 24d062f [kai] test suite
* [SPARK-5746][SQL] Check invalid cases for the write path of data source APIYin Huai2015-02-161-3/+10
| | | | | | | | | | | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-5746 liancheng marmbrus Author: Yin Huai <yhuai@databricks.com> Closes #4617 from yhuai/insertOverwrite and squashes the following commits: 8e3019d [Yin Huai] Fix compilation error. 499e8e7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite e76e85a [Yin Huai] Address comments. ac31b3c [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite f30bdad [Yin Huai] Use toDF. 99da57e [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite 6b7545c [Yin Huai] Add a pre write check to the data source API. a88c516 [Yin Huai] DDLParser will take a parsering function to take care CTAS statements.
* [SPARK-5833] [SQL] Adds REFRESH TABLE commandCheng Lian2015-02-161-0/+10
| | | | | | | | | | | | | | Lifts `HiveMetastoreCatalog.refreshTable` to `Catalog`. Adds `RefreshTable` command to refresh (possibly cached) metadata in external data sources tables. <!-- Reviewable:start --> [<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4624) <!-- Reviewable:end --> Author: Cheng Lian <lian@databricks.com> Closes #4624 from liancheng/refresh-table and squashes the following commits: 8d1aa4c [Cheng Lian] Adds REFRESH TABLE command
* [SQL] Initial support for reporting location of error in sql stringMichael Armbrust2015-02-166-15/+111
| | | | | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #4587 from marmbrus/position and squashes the following commits: 0810052 [Michael Armbrust] fix tests 395c019 [Michael Armbrust] Merge remote-tracking branch 'marmbrus/position' into position e155dce [Michael Armbrust] more errors f3efa51 [Michael Armbrust] Update AnalysisException.scala d45ff60 [Michael Armbrust] [SQL] Initial support for reporting location of error in sql string
* [SQL] [Minor] Update the SpecificMutableRow.copyCheng Hao2015-02-161-3/+4
| | | | | | | | | | When profiling the Join / Aggregate queries via VisualVM, I noticed lots of `SpecificMutableRow` objects created, as well as the `MutableValue`, since the `SpecificMutableRow` are mostly used in data source implementation, but the `copy` method could be called multiple times in upper modules (e.g. in Join / aggregation etc.), duplicated instances created should be avoid. Author: Cheng Hao <hao.cheng@intel.com> Closes #4619 from chenghao-intel/specific_mutable_row and squashes the following commits: 9300d23 [Cheng Hao] update the SpecificMutableRow.copy
* [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFramesReynold Xin2015-02-131-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | - The old implicit would convert RDDs directly to DataFrames, and that added too many methods. - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed Python changes: - toDataFrame -> toDF - Dsl -> functions package - addColumn -> withColumn - renameColumn -> withColumnRenamed - add toDF functions to RDD on SQLContext init - add flatMap to DataFrame Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4556 from rxin/SPARK-5752 and squashes the following commits: 5ef9910 [Reynold Xin] More fix 61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752 ff5832c [Reynold Xin] Fix python 749c675 [Reynold Xin] count(*) fixes. 5806df0 [Reynold Xin] Fix build break again. d941f3d [Reynold Xin] Fixed explode compilation break. fe1267a [Davies Liu] flatMap c4afb8e [Reynold Xin] style d9de47f [Davies Liu] add comment b783994 [Davies Liu] add comment for toDF e2154e5 [Davies Liu] schema() -> schema 3a1004f [Davies Liu] Dsl -> functions, toDF() fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames 97dd47c [Davies Liu] fix mistake 6168f74 [Davies Liu] fix test 1fc0199 [Davies Liu] fix test a075cd5 [Davies Liu] clean up, toPandas 663d314 [Davies Liu] add test for agg('*') 9e214d5 [Reynold Xin] count(*) fixes. 1ed7136 [Reynold Xin] Fix build break again. 921b2e3 [Reynold Xin] Fixed explode compilation break. 14698d4 [Davies Liu] flatMap ba3e12d [Reynold Xin] style d08c92d [Davies Liu] add comment 5c8b524 [Davies Liu] add comment for toDF a4e5e66 [Davies Liu] schema() -> schema d377fc9 [Davies Liu] Dsl -> functions, toDF() 6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed 807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
* [SPARK-5642] [SQL] Apply column pruning on unused aggregation fieldsDaoyuan Wang2015-02-132-2/+44
| | | | | | | | | | | | | | select k from (select key k, max(value) v from src group by k) t Author: Daoyuan Wang <daoyuan.wang@intel.com> Author: Michael Armbrust <michael@databricks.com> Closes #4415 from adrian-wang/groupprune and squashes the following commits: 5d2d8a3 [Daoyuan Wang] address Michael's comments 61f8ef7 [Daoyuan Wang] add a unit test 80ddcc6 [Daoyuan Wang] keep project b69d385 [Daoyuan Wang] add a prune rule for grouping set
* [SPARK-3365][SQL]Wrong schema generated for List typetianyi2015-02-122-15/+20
| | | | | | | | | | | | | | | | | | | | | | This PR fix the issue SPARK-3365. The reason is Spark generated wrong schema for the type `List` in `ScalaReflection.scala` for example: the generated schema for type `Seq[String]` is: ``` {"name":"x","type":{"type":"array","elementType":"string","containsNull":true},"nullable":true,"metadata":{}}` ``` the generated schema for type `List[String]` is: ``` {"name":"x","type":{"type":"struct","fields":[]},"nullable":true,"metadata":{}}` ``` Author: tianyi <tianyi.asiainfo@gmail.com> Closes #4581 from tianyi/SPARK-3365 and squashes the following commits: a097e86 [tianyi] change the order of resolution in ScalaReflection.scala
* [SPARK-3299][SQL]Public API in SQLContext to list tablesYin Huai2015-02-121-0/+37
| | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-3299 Author: Yin Huai <yhuai@databricks.com> Closes #4547 from yhuai/tables and squashes the following commits: 6c8f92e [Yin Huai] Add tableNames. acbb281 [Yin Huai] Update Python test. 7793dcb [Yin Huai] Fix scala test. 572870d [Yin Huai] Address comments. aba2e88 [Yin Huai] Format. 12c86df [Yin Huai] Add tables() to SQLContext to return a DataFrame containing existing tables.
* [SPARK-5573][SQL] Add explode to dataframesMichael Armbrust2015-02-121-0/+19
| | | | | | | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #4546 from marmbrus/explode and squashes the following commits: eefd33a [Michael Armbrust] whitespace a8d496c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into explode 4af740e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explode dc86a5c [Michael Armbrust] simple version d633d01 [Michael Armbrust] add scala specific 950707a [Michael Armbrust] fix comments ba8854c [Michael Armbrust] [SPARK-5573][SQL] Add explode to dataframes
* [SQL] Improve error messagesMichael Armbrust2015-02-1211-95/+158
| | | | | | | | | | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Author: wangfei <wangfei1@huawei.com> Closes #4558 from marmbrus/errorMessages and squashes the following commits: 5e5ab50 [Michael Armbrust] Merge pull request #15 from scwf/errorMessages fa38881 [wangfei] fix for grouping__id f279a71 [wangfei] make right references for ScriptTransformation d29fbde [Michael Armbrust] extra case 1a797b4 [Michael Armbrust] comments d4e9015 [Michael Armbrust] add comment af9e668 [Michael Armbrust] no braces 34eb3a4 [Michael Armbrust] more work 6197cd5 [Michael Armbrust] [SQL] Better error messages for analysis failures
* [SPARK-3688][SQL] More inline comments for LogicalPlan.Reynold Xin2015-02-115-42/+115
| | | | | | | | | | | As a follow-up to https://github.com/apache/spark/pull/4524 Author: Reynold Xin <rxin@databricks.com> Closes #4539 from rxin/SPARK-3688 and squashes the following commits: 5ac56c7 [Reynold Xin] exists da8eea4 [Reynold Xin] [SPARK-3688][SQL] More inline comments for LogicalPlan.
* [SPARK-3688][SQL]LogicalPlan can't resolve column correctllytianyi2015-02-111-18/+32
| | | | | | | | | | | | | | | This PR fixed the resolving problem described in https://issues.apache.org/jira/browse/SPARK-3688 ``` CREATE TABLE t1(x INT); CREATE TABLE t2(a STRUCT<x: INT>, k INT); SELECT a.x FROM t1 a JOIN t2 b ON a.x = b.k; ``` Author: tianyi <tianyi.asiainfo@gmail.com> Closes #4524 from tianyi/SPARK-3688 and squashes the following commits: 237a256 [tianyi] resolve a name with table.column pattern first.
* [SPARK-5454] More robust handling of self joinsMichael Armbrust2015-02-113-27/+24
| | | | | | | | | | | | | Also I fix a bunch of bad output in test cases. Author: Michael Armbrust <michael@databricks.com> Closes #4520 from marmbrus/selfJoin and squashes the following commits: 4f4a85c [Michael Armbrust] comments 49c8e26 [Michael Armbrust] fix tests 6fc38de [Michael Armbrust] fix style 55d64b3 [Michael Armbrust] fix dataframe selfjoins
* [SQL] Add an exception for analysis errors.Michael Armbrust2015-02-103-15/+43
| | | | | | | | | | | | | | | Also start from the bottom so we show the first error instead of the top error. Author: Michael Armbrust <michael@databricks.com> Closes #4439 from marmbrus/analysisException and squashes the following commits: 45862a0 [Michael Armbrust] fix hive test a773bba [Michael Armbrust] Merge remote-tracking branch 'origin/master' into analysisException f88079f [Michael Armbrust] update more cases fede90a [Michael Armbrust] newline fbf4bc3 [Michael Armbrust] move to sql 6235db4 [Michael Armbrust] [SQL] Add an exception for analysis errors.
* [SQL] Add toString to DataFrame/ColumnMichael Armbrust2015-02-102-0/+32
| | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #4436 from marmbrus/dfToString and squashes the following commits: 8a3c35f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into dfToString b72a81b [Michael Armbrust] add toString
* [SPARK-2096][SQL] support dot notation on array of structWenchen Fan2015-02-094-18/+51
| | | | | | | | | | | | | | ~~The rule is simple: If you want `a.b` work, then `a` must be some level of nested array of struct(level 0 means just a StructType). And the result of `a.b` is same level of nested array of b-type. An optimization is: the resolve chain looks like `Attribute -> GetItem -> GetField -> GetField ...`, so we could transmit the nested array information between `GetItem` and `GetField` to avoid repeated computation of `innerDataType` and `containsNullList` of that nested array.~~ marmbrus Could you take a look? to evaluate `a.b`, if `a` is array of struct, then `a.b` means get field `b` on each element of `a`, and return a result of array. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #2405 from cloud-fan/nested-array-dot and squashes the following commits: 08a228a [Wenchen Fan] support dot notation on array of struct
* [SPARK-5614][SQL] Predicate pushdown through Generate.Lu Yan2015-02-092-1/+87
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Now in Catalyst's rules, predicates can not be pushed through "Generate" nodes. Further more, partition pruning in HiveTableScan can not be applied on those queries involves "Generate". This makes such queries very inefficient. In practice, it finds patterns like ```scala Filter(predicate, Generate(generator, _, _, _, grandChild)) ``` and splits the predicate into 2 parts by referencing the generated column from Generate node or not. And a new Filter will be created for those conjuncts can be pushed beneath Generate node. If nothing left for the original Filter, it will be removed. For example, physical plan for query ```sql select len, bk from s_server lateral view explode(len_arr) len_table as len where len > 5 and day = '20150102'; ``` where 'day' is a partition column in metastore is like this in current version of Spark SQL: > Project [len, bk] > > Filter ((len > "5") && "(day = "20150102")") > > Generate explode(len_arr), true, false > > HiveTableScan [bk, len_arr, day], (MetastoreRelation default, s_server, None), None But theoretically the plan should be like this > Project [len, bk] > > Filter (len > "5") > > Generate explode(len_arr), true, false > > HiveTableScan [bk, len_arr, day], (MetastoreRelation default, s_server, None), Some(day = "20150102") Where partition pruning predicates can be pushed to HiveTableScan nodes. Author: Lu Yan <luyan02@baidu.com> Closes #4394 from ianluyan/ppd and squashes the following commits: a67dce9 [Lu Yan] Fix English grammar. 7cea911 [Lu Yan] Revised based on @marmbrus's opinions ffc59fc [Lu Yan] [SPARK-5614][SQL] Predicate pushdown through Generate.
* [SPARK-5675][SQL] XyzType companion object should subclass XyzTypeReynold Xin2015-02-091-12/+73
| | | | | | | | | | | | | | | Otherwise, the following will always return false in Java. ```scala dataType instanceof StringType ``` Author: Reynold Xin <rxin@databricks.com> Closes #4463 from rxin/type-companion-object and squashes the following commits: 04d5d8d [Reynold Xin] Comment. 976e11e [Reynold Xin] [SPARK-5675][SQL]StringType case object should be subclass of StringType class
* [SPARK-5643][SQL] Add a show method to print the content of a DataFrame in ↵Reynold Xin2015-02-083-3/+79
| | | | | | | | | | | | | | | | | | | | | | | tabular format. An example: ``` year month AVG('Adj Close) MAX('Adj Close) 1980 12 0.503218 0.595103 1981 01 0.523289 0.570307 1982 02 0.436504 0.475256 1983 03 0.410516 0.442194 1984 04 0.450090 0.483521 ``` Author: Reynold Xin <rxin@databricks.com> Closes #4416 from rxin/SPARK-5643 and squashes the following commits: d0e0d6e [Reynold Xin] [SQL] Minor update to data source and statistics documentation. 269da83 [Reynold Xin] Updated isLocal comment. 2cf3c27 [Reynold Xin] Moved logic into optimizer. 1a04d8b [Reynold Xin] [SPARK-5643][SQL] Add a show method to print the content of a DataFrame in columnar format.
* [SPARK-5278][SQL] Introduce UnresolvedGetField and complete the check of ↵Wenchen Fan2015-02-069-76/+64
| | | | | | | | | | | | | | | ambiguous reference to fields When the `GetField` chain(`a.b.c.d.....`) is interrupted by `GetItem` like `a.b[0].c.d....`, then the check of ambiguous reference to fields is broken. The reason is that: for something like `a.b[0].c.d`, we first parse it to `GetField(GetField(GetItem(Unresolved("a.b"), 0), "c"), "d")`. Then in `LogicalPlan#resolve`, we resolve `"a.b"` and build a `GetField` chain from bottom(the relation). But for the 2 outer `GetFiled`, we have to resolve them in `Analyzer` or do it in `GetField` lazily, check data type of child, search needed field, etc. which is similar to what we have done in `LogicalPlan#resolve`. So in this PR, the fix is just copy the same logic in `LogicalPlan#resolve` to `Analyzer`, which is simple and quick, but I do suggest introduce `UnresolvedGetFiled` like I explained in https://github.com/apache/spark/pull/2405. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #4068 from cloud-fan/simple and squashes the following commits: a6857b5 [Wenchen Fan] fix import order 8411c40 [Wenchen Fan] use UnresolvedGetField
* [SQL][Minor] Remove cache keyword in SqlParserwangfei2015-02-061-1/+0
| | | | | | | | | | Since cache keyword already defined in `SparkSQLParser` and `SqlParser` of catalyst is a more general parser which should not cover keywords related to underlying compute engine, to remove cache keyword in `SqlParser`. Author: wangfei <wangfei1@huawei.com> Closes #4393 from scwf/remove-cache-keyword and squashes the following commits: 10ade16 [wangfei] remove cache keyword in sql parser
* [SPARK-5640] Synchronize ScalaReflection where necessaryTobias Schlatter2015-02-061-2/+3
| | | | | | | | Author: Tobias Schlatter <tobias@meisch.ch> Closes #4431 from gzm0/sync-scala-refl and squashes the following commits: c5da21e [Tobias Schlatter] [SPARK-5640] Synchronize ScalaReflection where necessary
* [SPARK-5182] [SPARK-5528] [SPARK-5509] [SPARK-3575] [SQL] Parquet data ↵Cheng Lian2015-02-052-3/+72
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | source improvements This PR adds three major improvements to Parquet data source: 1. Partition discovery While reading Parquet files resides in Hive style partition directories, `ParquetRelation2` automatically discovers partitioning information and infers partition column types. This is also a partial work for [SPARK-5182] [1], which aims to provide first class partitioning support for the data source API. Related code in this PR can be easily extracted to the data source API level in future versions. 1. Schema merging When enabled, Parquet data source collects schema information from all Parquet part-files and tries to merge them. Exceptions are thrown when incompatible schemas are detected. This feature is controlled by data source option `parquet.mergeSchema`, and is enabled by default. 1. Metastore Parquet table conversion moved to analysis phase This greatly simplifies the conversion logic. `ParquetConversion` strategy can be removed once the old Parquet implementation is removed in the future. This version of Parquet data source aims to entirely replace the old Parquet implementation. However, the old version hasn't been removed yet. Users can fall back to the old version by turning off SQL configuration `spark.sql.parquet.useDataSourceApi`. Other JIRA tickets fixed as side effects in this PR: - [SPARK-5509] [3]: `EqualTo` now uses a proper `Ordering` to compare binary types. - [SPARK-3575] [4]: Metastore schema is now preserved and passed to `ParquetRelation2` via data source option `parquet.metastoreSchema`. TODO: - [ ] More test cases for partition discovery - [x] Fix write path after data source write support (#4294) is merged It turned out to be non-trivial to fall back to old Parquet implementation on the write path when Parquet data source is enabled. Since we're planning to include data source write support in 1.3.0, I simply ignored two test cases involving Parquet insertion for now. - [ ] Fix outdated comments and documentations PS: This PR looks big, but more than a half of the changed lines in this PR are trivial changes to test cases. To test Parquet with and without the new data source, almost all Parquet test cases are moved into wrapper driver functions. This introduces hundreds of lines of changes. [1]: https://issues.apache.org/jira/browse/SPARK-5182 [2]: https://issues.apache.org/jira/browse/SPARK-5528 [3]: https://issues.apache.org/jira/browse/SPARK-5509 [4]: https://issues.apache.org/jira/browse/SPARK-3575 <!-- Reviewable:start --> [<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4308) <!-- Reviewable:end --> Author: Cheng Lian <lian@databricks.com> Closes #4308 from liancheng/parquet-partition-discovery and squashes the following commits: b6946e6 [Cheng Lian] Fixes MiMA issues, addresses comments 8232e17 [Cheng Lian] Write support for Parquet data source a49bd28 [Cheng Lian] Fixes spelling typo in trait name "CreateableRelationProvider" 808380f [Cheng Lian] Fixes issues introduced while rebasing 50dd8d1 [Cheng Lian] Addresses @rxin's comment, fixes UDT schema merging adf2aae [Cheng Lian] Fixes compilation error introduced while rebasing 4e0175f [Cheng Lian] Fixes Python Parquet API, we need Py4J array to call varargs method 0d8ec1d [Cheng Lian] Adds more test cases b35c8c6 [Cheng Lian] Fixes some typos and outdated comments dd704fd [Cheng Lian] Fixes Python Parquet API 596c312 [Cheng Lian] Uses switch to control whether use Parquet data source or not 7d0f7a2 [Cheng Lian] Fixes Metastore Parquet table conversion a1896c7 [Cheng Lian] Fixes all existing Parquet test suites except for ParquetMetastoreSuite 5654c9d [Cheng Lian] Draft version of Parquet partition discovery and schema merging
* [SPARK-5135][SQL] Add support for describe table to DDL in SQLContextOopsOutOfMemory2015-02-051-2/+24
| | | | | | | | | | | | Hi, rxin marmbrus I considered your suggestion (in #4127) and now re-write it. This is now up-to-date. Could u please review it ? Author: OopsOutOfMemory <victorshengli@126.com> Closes #4227 from OopsOutOfMemory/describe and squashes the following commits: 053826f [OopsOutOfMemory] describe
* [SPARK-5602][SQL] Better support for creating DataFrame from local data ↵Reynold Xin2015-02-043-11/+30
| | | | | | | | | | | | | | collection 1. Added methods to create DataFrames from Seq[Product] 2. Added executeTake to avoid running a Spark job on LocalRelations. Author: Reynold Xin <rxin@databricks.com> Closes #4372 from rxin/localDataFrame and squashes the following commits: f696858 [Reynold Xin] style checker. 839ef7f [Reynold Xin] [SPARK-5602][SQL] Better support for creating DataFrame from local data collection.
* [SQL] Correct the default size of TimestampType and expose NumericTypeYin Huai2015-02-042-4/+4
| | | | | | | | | Author: Yin Huai <yhuai@databricks.com> Closes #4314 from yhuai/minor and squashes the following commits: d3870a7 [Yin Huai] Update test. 6e4b0c0 [Yin Huai] Two minor changes.
* [SPARK-5367][SQL] Support star expression in udfswangfei2015-02-041-0/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | A follow up for #4163: support `select array(key, *) from src` Since array(key, *) will not go into this case ``` case Alias(f UnresolvedFunction(_, args), name) if containsStar(args) => val expandedArgs = args.flatMap { case s: Star => s.expand(child.output, resolver) case o => o :: Nil } ``` here added a case to cover the corner case of array. /cc liancheng Author: wangfei <wangfei1@huawei.com> Author: scwf <wangfei1@huawei.com> Closes #4353 from scwf/udf-star1 and squashes the following commits: 4350d17 [wangfei] minor fix a7cd191 [wangfei] minor fix 0942fb1 [wangfei] follow up: support select array(key, *) from src 6ae00db [wangfei] also fix problem with array da1da09 [scwf] minor fix f87b5f9 [scwf] added test case 587bf7e [wangfei] compile fix eb93c16 [wangfei] fix star resolve issue in udf
* [SPARK-5579][SQL][DataFrame] Support for project/filter using SQL expressionsReynold Xin2015-02-031-0/+10
| | | | | | | | | | | | | | ```scala df.selectExpr("abs(colA)", "colB") df.filter("age > 21") ``` Author: Reynold Xin <rxin@databricks.com> Closes #4348 from rxin/SPARK-5579 and squashes the following commits: 2baeef2 [Reynold Xin] Fix Python. b416372 [Reynold Xin] [SPARK-5579][SQL][DataFrame] Support for project/filter using SQL expressions.
* [SPARK-4508] [SQL] build native date type to conform behavior to HiveDaoyuan Wang2015-02-0310-61/+112
| | | | | | | | | | | | | | | | | | The previous #3732 is reverted due to some test failure. Have fixed that. Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #4325 from adrian-wang/datenative and squashes the following commits: 096e20d [Daoyuan Wang] fix for mixed timezone 0ed0fdc [Daoyuan Wang] fix test data a2fdd4e [Daoyuan Wang] getDate c37832b [Daoyuan Wang] row to catalyst f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion 024c9a6 [Daoyuan Wang] clean some import order d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally 374abd5 [Daoyuan Wang] spark native date type support
* [SPARK-5383][SQL] Support alias for udtfswangfei2015-02-033-3/+42
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Add support for alias of udtfs, such as ``` select stack(2, key, value, key, value) as (a, b) from src limit 5; select a, b from (select stack(2, key, value, key, value) as (a, b) from src) t limit 5 ``` Author: wangfei <wangfei1@huawei.com> Author: scwf <wangfei1@huawei.com> Author: Fei Wang <wangfei1@huawei.com> Closes #4186 from scwf/multi-alias-names and squashes the following commits: c35e922 [wangfei] fix conflicts adc8311 [wangfei] minor format fix 2783aed [wangfei] convert it to a Generate instead of leaving it inside of a Project clause a87668a [wangfei] minor improvement b25d9b3 [wangfei] resolve conflicts d38f041 [wangfei] style fix 8cfcebf [wangfei] minor improvement 12a239e [wangfei] fix test case 050177f [wangfei] added extendedCheckRules 3d69329 [wangfei] added CheckMultiAlias to analyzer 324150d [wangfei] added multi alias node 74f5a81 [Fei Wang] imports order fix 5bc3f59 [scwf] style fix 3daec28 [scwf] support alias for udfs with multi output columns
* [SPARK-5550] [SQL] Support the case insensitive for UDFCheng Hao2015-02-031-4/+32
| | | | | | | | | | | | | | | SQL in HiveContext, should be case insensitive, however, the following query will fail. ```scala udf.register("random0", () => { Math.random()}) assert(sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0) ``` Author: Cheng Hao <hao.cheng@intel.com> Closes #4326 from chenghao-intel/udf_case_sensitive and squashes the following commits: 485cf66 [Cheng Hao] Support the case insensitive for UDF
* [SPARK-5501][SPARK-5420][SQL] Write support for the data source APIYin Huai2015-02-021-2/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This PR aims to support `INSERT INTO/OVERWRITE TABLE tableName` and `CREATE TABLE tableName AS SELECT` for the data source API (partitioned tables are not supported). In this PR, I am also adding the support of `IF NOT EXISTS` for our ddl parser. The current semantic of `IF NOT EXISTS` is explained as follows. * For a `CREATE TEMPORARY TABLE` statement, it does not `IF NOT EXISTS` for now. * For a `CREATE TABLE` statement (we are creating a metastore table), if there is an existing table having the same name ... * when `IF NOT EXISTS` clause is used, we will do nothing. * when `IF NOT EXISTS` clause is not used, the user will see an exception saying the table already exists. TODOs: - [x] CTAS support - [x] Programmatic APIs - [ ] Python API (another PR) - [x] More unit tests - [ ] Documents (another PR) marmbrus liancheng rxin Author: Yin Huai <yhuai@databricks.com> Closes #4294 from yhuai/writeSupport and squashes the following commits: 3db1539 [Yin Huai] save does not take overwrite. 1c98881 [Yin Huai] Fix test. 142372a [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupport 34e1bfb [Yin Huai] Address comments. 1682ca6 [Yin Huai] Better support for CTAS statements. e789d64 [Yin Huai] For the Scala API, let users to use tuples to provide options. 0128065 [Yin Huai] Short hand versions of save and load. 66ebd74 [Yin Huai] Formatting. 9203ec2 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupport e5d29f2 [Yin Huai] Programmatic APIs. 1a719a5 [Yin Huai] CREATE TEMPORARY TABLE with IF NOT EXISTS is not allowed for now. 909924f [Yin Huai] Add saveAsTable for the data source API to DataFrame. 95a7c71 [Yin Huai] Fix bug when handling IF NOT EXISTS clause in a CREATE TEMPORARY TABLE statement. d37b19c [Yin Huai] Cheng's comments. fd6758c [Yin Huai] Use BeforeAndAfterAll. 7880891 [Yin Huai] Support CREATE TABLE AS SELECT STATEMENT and the IF NOT EXISTS clause. cb85b05 [Yin Huai] Initial write support. 2f91354 [Yin Huai] Make INSERT OVERWRITE/INTO statements consistent between HiveQL and SqlParser.
* [SQL] Improve DataFrame API error reportingReynold Xin2015-02-021-1/+2
| | | | | | | | | | | | | | | | | | 1. Throw UnsupportedOperationException if a Column is not computable. 2. Perform eager analysis on DataFrame so we can catch errors when they happen (not when an action is run). Author: Reynold Xin <rxin@databricks.com> Author: Davies Liu <davies@databricks.com> Closes #4296 from rxin/col-computability and squashes the following commits: 6527b86 [Reynold Xin] Merge pull request #8 from davies/col-computability fd92bc7 [Reynold Xin] Merge branch 'master' into col-computability f79034c [Davies Liu] fix python tests 5afe1ff [Reynold Xin] Fix scala test. 17f6bae [Reynold Xin] Various fixes. b932e86 [Reynold Xin] Added eager analysis for error reporting. e6f00b8 [Reynold Xin] [SQL][API] ComputableColumn vs IncomputableColumn
* Revert "[SPARK-4508] [SQL] build native date type to conform behavior to Hive"Patrick Wendell2015-02-0210-112/+61
| | | | This reverts commit 1646f89d967913ee1f231d9606f8502d13c25804.
* [SPARK-4508] [SQL] build native date type to conform behavior to HiveDaoyuan Wang2015-02-0210-61/+112
| | | | | | | | | | | | | | | | | | Store daysSinceEpoch as an Int value(4 bytes) to represent DateType, instead of using java.sql.Date(8 bytes as Long) in catalyst row. This ensures the same comparison behavior of Hive and Catalyst. Subsumes #3381 I thinks there are already some tests in JavaSQLSuite, and for python it will not affect python's datetime class. Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #3732 from adrian-wang/datenative and squashes the following commits: 0ed0fdc [Daoyuan Wang] fix test data a2fdd4e [Daoyuan Wang] getDate c37832b [Daoyuan Wang] row to catalyst f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion 024c9a6 [Daoyuan Wang] clean some import order d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally 374abd5 [Daoyuan Wang] spark native date type support
* [SPARK-5212][SQL] Add support of schema-less, custom field delimiter and ↵Liang-Chi Hsieh2015-02-021-1/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | SerDe for HiveQL transform This pr adds the support of schema-less syntax, custom field delimiter and SerDe for HiveQL's transform. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #4014 from viirya/schema_less_trans and squashes the following commits: ac2d1fe [Liang-Chi Hsieh] Refactor codes for comments. a137933 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into schema_less_trans aa10fbd [Liang-Chi Hsieh] Add Hive golden answer files again. 575f695 [Liang-Chi Hsieh] Add Hive golden answer files for new unit tests. a422562 [Liang-Chi Hsieh] Use createQueryTest for unit tests and remove unnecessary imports. ccb71e3 [Liang-Chi Hsieh] Refactor codes for comments. 37bd391 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into schema_less_trans 6000889 [Liang-Chi Hsieh] Wrap input and output schema into ScriptInputOutputSchema. 21727f7 [Liang-Chi Hsieh] Move schema-less output to proper place. Use multilines instead of a long line SQL. 9a6dc04 [Liang-Chi Hsieh] setRecordReaderID is introduced in 0.13.1, use reflection API to call it. 7a14f31 [Liang-Chi Hsieh] Fix bug. 799b5e1 [Liang-Chi Hsieh] Call getSerializedClass instead of using Text. be2c3fc [Liang-Chi Hsieh] Fix style. 32d3046 [Liang-Chi Hsieh] Add SerDe support. ab22f7b [Liang-Chi Hsieh] Fix style. 7a48e42 [Liang-Chi Hsieh] Add support of custom field delimiter. b1729d9 [Liang-Chi Hsieh] Fix style. ccee49e [Liang-Chi Hsieh] Add unit test. f561c37 [Liang-Chi Hsieh] Add support of schema-less script transformation.
* [SPARK-5262] [SPARK-5244] [SQL] add coalesce in SQLParser and widen types ↵Daoyuan Wang2015-02-013-0/+45
| | | | | | | | | | | | | | | for parameters of coalesce I'll add test case in #4040 Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #4057 from adrian-wang/coal and squashes the following commits: 4d0111a [Daoyuan Wang] address Yin's comments c393e18 [Daoyuan Wang] fix rebase conflicts e47c03a [Daoyuan Wang] add coalesce in parser c74828d [Daoyuan Wang] cast types for coalesce
* [Minor][SQL] Little refactor DataFrame related codesLiang-Chi Hsieh2015-02-011-1/+1
| | | | | | | | | | | | | | | Simplify some codes related to DataFrame. * Calling `toAttributes` instead of a `map`. * Original `createDataFrame` creates the `StructType` and its attributes in a redundant way. Refactored it to create `StructType` and call `toAttributes` on it directly. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #4298 from viirya/refactor_df and squashes the following commits: 1d61c64 [Liang-Chi Hsieh] Revert it. f36efb5 [Liang-Chi Hsieh] Relax the constraint of toDataFrame. 2c9f370 [Liang-Chi Hsieh] Just refactor DataFrame codes.
* [SPARK-5504] [sql] convertToCatalyst should support nested arraysJoseph K. Bradley2015-01-302-3/+11
| | | | | | | | | | | | | | After the recent refactoring, convertToCatalyst in ScalaReflection does not recurse on Arrays. It should. The test suite modification made the test fail before the fix in ScalaReflection. The fix makes the test suite succeed. CC: marmbrus Author: Joseph K. Bradley <joseph@databricks.com> Closes #4295 from jkbradley/SPARK-5504 and squashes the following commits: 6b7276d [Joseph K. Bradley] Fixed issue in ScalaReflection.convertToCatalyst with Arrays with non-primitive types. Modified test suite so it failed before the fix and works after the fix.
* [SQL] Support df("*") to select all columns in a data frame.Reynold Xin2015-01-293-19/+40
| | | | | | | | | | | This PR makes Star a trait, and provides two implementations: UnresolvedStar (used for *, tblName.*) and ResolvedStar (used for df("*")). Author: Reynold Xin <rxin@databricks.com> Closes #4283 from rxin/df-star and squashes the following commits: c9cba3e [Reynold Xin] Removed mapFunction in UnresolvedStar. 1a3a1d7 [Reynold Xin] [SQL] Support df("*") to select all columns in a data frame.
* [SPARK-5373][SQL] Literal in agg grouping expressions leads to incorrect resultwangfei2015-01-291-4/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | `select key, count( * ) from src group by key, 1` will get the wrong answer. e.g. for this table ``` val testData2 = TestSQLContext.sparkContext.parallelize( TestData2(1, 1) :: TestData2(1, 2) :: TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: TestData2(3, 2) :: Nil, 2).toSchemaRDD testData2.registerTempTable("testData2") ``` result of `SELECT a, count(1) FROM testData2 GROUP BY a, 1` is ``` [1,1] [2,2] [3,1] ``` Author: wangfei <wangfei1@huawei.com> Closes #4169 from scwf/agg-bug and squashes the following commits: 05751db [wangfei] fix bugs when literal in agg grouping expressioons