Commit graph

270 commits

Author SHA1 Message Date
Cheng Hao e83f13e8d3 [SPARK-4152] [SQL] Avoid data change in CTAS while table already existed
CREATE TABLE t1 (a String);
CREATE TABLE t1 AS SELECT key FROM src; – throw exception
CREATE TABLE if not exists t1 AS SELECT key FROM src; – expect do nothing, currently it will overwrite the t1, which is incorrect.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #3013 from chenghao-intel/ctas_unittest and squashes the following commits:

194113e [Cheng Hao] fix bug in CTAS when table already existed
2014-11-03 13:59:43 -08:00
Cheng Lian c238fb423d [SPARK-4202][SQL] Simple DSL support for Scala UDF
This feature is based on an offline discussion with mengxr, hopefully can be useful for the new MLlib pipeline API.

For the following test snippet

```scala
case class KeyValue(key: Int, value: String)
val testData = sc.parallelize(1 to 10).map(i => KeyValue(i, i.toString)).toSchemaRDD
def foo(a: Int, b: String) => a.toString + b
```

the newly introduced DSL enables the following syntax

```scala
import org.apache.spark.sql.catalyst.dsl._
testData.select(Star(None), foo.call('key, 'value) as 'result)
```

which is equivalent to

```scala
testData.registerTempTable("testData")
sqlContext.registerFunction("foo", foo)
sql("SELECT *, foo(key, value) AS result FROM testData")
```

Author: Cheng Lian <lian@databricks.com>

Closes #3067 from liancheng/udf-dsl and squashes the following commits:

f132818 [Cheng Lian] Adds DSL support for Scala UDF
2014-11-03 13:20:33 -08:00
Davies Liu 24544fbce0 [SPARK-3594] [PySpark] [SQL] take more rows to infer schema or sampling
This patch will try to infer schema for RDD which has empty value (None, [], {}) in the first row. It will try first 100 rows and merge the types into schema, also merge fields of StructType together. If there is still NullType in schema, then it will show an warning, tell user to try with sampling.

If sampling is presented, it will infer schema from all the rows after sampling.

Also, add samplingRatio for jsonFile() and jsonRDD()

Author: Davies Liu <davies.liu@gmail.com>
Author: Davies Liu <davies@databricks.com>

Closes #2716 from davies/infer and squashes the following commits:

e678f6d [Davies Liu] Merge branch 'master' of github.com:apache/spark into infer
34b5c63 [Davies Liu] Merge branch 'master' of github.com:apache/spark into infer
567dc60 [Davies Liu] update docs
9767b27 [Davies Liu] Merge branch 'master' into infer
e48d7fb [Davies Liu] fix tests
29e94d5 [Davies Liu] let NullType inherit from PrimitiveType
ee5d524 [Davies Liu] Merge branch 'master' of github.com:apache/spark into infer
540d1d5 [Davies Liu] merge fields for StructType
f93fd84 [Davies Liu] add more tests
3603e00 [Davies Liu] take more rows to infer schema, or infer the schema by sampling the RDD
2014-11-03 13:17:09 -08:00
ravipesala 2b6e1ce6ee [SPARK-4207][SQL] Query which has syntax like 'not like' is not working in Spark SQL
Queries which has 'not like' is not working spark sql.

sql("SELECT * FROM records where value not like 'val%'")
 same query works in Spark HiveQL

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #3075 from ravipesala/SPARK-4207 and squashes the following commits:

35c11e7 [ravipesala] Supported 'not like' syntax in sql
2014-11-03 13:07:41 -08:00
Joseph K. Bradley ebd6480587 [SPARK-3572] [SQL] Internal API for User-Defined Types
This PR adds User-Defined Types (UDTs) to SQL. It is a precursor to using SchemaRDD as a Dataset for the new MLlib API. Currently, the UDT API is private since there is incomplete support (e.g., no Java or Python support yet).

Author: Joseph K. Bradley <joseph@databricks.com>
Author: Michael Armbrust <michael@databricks.com>
Author: Xiangrui Meng <meng@databricks.com>

Closes #3063 from marmbrus/udts and squashes the following commits:

7ccfc0d [Michael Armbrust] remove println
46a3aee [Michael Armbrust] Slightly easier to read test output.
6cc434d [Michael Armbrust] Recursively convert rows.
e369b91 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udts
15c10a6 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into sql-udt2
f3c72fe [Joseph K. Bradley] Fixing merge
e13cd8a [Joseph K. Bradley] Removed Vector UDTs
5817b2b [Joseph K. Bradley] style edits
30ce5b2 [Joseph K. Bradley] updates based on code review
d063380 [Joseph K. Bradley] Cleaned up Java UDT Suite, and added warning about element ordering when creating schema from Java Bean
a571bb6 [Joseph K. Bradley] Removed old UDT code (registry and Java UDTs).  Cleaned up other code.  Extended JavaUserDefinedTypeSuite
6fddc1c [Joseph K. Bradley] Made MyLabeledPoint into a Java Bean
20630bc [Joseph K. Bradley] fixed scalastyle
fa86b20 [Joseph K. Bradley] Removed Java UserDefinedType, and made UDTs private[spark] for now
8de957c [Joseph K. Bradley] Modified UserDefinedType to store Java class of user type so that registerUDT takes only the udt argument.
8b242ea [Joseph K. Bradley] Fixed merge error after last merge.  Note: Last merge commit also removed SQL UDT examples from mllib.
7f29656 [Joseph K. Bradley] Moved udt case to top of all matches.  Small cleanups
b028675 [Xiangrui Meng] allow any type in UDT
4500d8a [Xiangrui Meng] update example code
87264a5 [Xiangrui Meng] remove debug code
3143ac3 [Xiangrui Meng] remove unnecessary changes
cfbc321 [Xiangrui Meng] support UDT in parquet
db16139 [Joseph K. Bradley] Added more doc for UserDefinedType.  Removed unused code in Suite
759af7a [Joseph K. Bradley] Added more doc to UserDefineType
63626a4 [Joseph K. Bradley] Updated ScalaReflectionsSuite per @marmbrus suggestions
51e5282 [Joseph K. Bradley] fixed 1 test
f025035 [Joseph K. Bradley] Cleanups before PR.  Added new tests
85872f6 [Michael Armbrust] Allow schema calculation to be lazy, but ensure its available on executors.
dff99d6 [Joseph K. Bradley] Added UDTs for Vectors in MLlib, plus DatasetExample using the UDTs
cd60cb4 [Joseph K. Bradley] Trying to get other SQL tests to run
34a5831 [Joseph K. Bradley] Added MLlib dependency on SQL.
e1f7b9c [Joseph K. Bradley] blah
2f40c02 [Joseph K. Bradley] renamed UDT types
3579035 [Joseph K. Bradley] udt annotation now working
b226b9e [Joseph K. Bradley] Changing UDT to annotation
fea04af [Joseph K. Bradley] more cleanups
964b32e [Joseph K. Bradley] some cleanups
893ee4c [Joseph K. Bradley] udt finallly working
50f9726 [Joseph K. Bradley] udts
04303c9 [Joseph K. Bradley] udts
39f8707 [Joseph K. Bradley] removed old udt suite
273ac96 [Joseph K. Bradley] basic UDT is working, but deserialization has yet to be done
8bebf24 [Joseph K. Bradley] commented out convertRowToScala for debugging
53de70f [Joseph K. Bradley] more udts...
982c035 [Joseph K. Bradley] still working on UDTs
19b2f60 [Joseph K. Bradley] still working on UDTs
0eaeb81 [Joseph K. Bradley] Still working on UDTs
105c5a3 [Joseph K. Bradley] Adding UserDefinedType to SQL, not done yet.
2014-11-02 17:56:00 -08:00
Michael Armbrust 9c0eb57c73 [SPARK-3247][SQL] An API for adding data sources to Spark SQL
This PR introduces a new set of APIs to Spark SQL to allow other developers to add support for reading data from new sources in `org.apache.spark.sql.sources`.

New sources must implement the interface `BaseRelation`, which is responsible for describing the schema of the data.  BaseRelations have three `Scan` subclasses, which are responsible for producing an RDD containing row objects.  The [various Scan interfaces](https://github.com/marmbrus/spark/blob/foreign/sql/core/src/main/scala/org/apache/spark/sql/sources/package.scala#L50) allow for optimizations such as column pruning and filter push down, when the underlying data source can handle these operations.

By implementing a class that inherits from RelationProvider these data sources can be accessed using using pure SQL.  I've used the functionality to update the JSON support so it can now be used in this way as follows:

```sql
CREATE TEMPORARY TABLE jsonTableSQL
USING org.apache.spark.sql.json
OPTIONS (
  path '/home/michael/data.json'
)
```

Further example usage can be found in the test cases: https://github.com/marmbrus/spark/tree/foreign/sql/core/src/test/scala/org/apache/spark/sql/sources

There is also a library that uses this new API to read avro data available here:
https://github.com/marmbrus/sql-avro

Author: Michael Armbrust <michael@databricks.com>

Closes #2475 from marmbrus/foreign and squashes the following commits:

1ed6010 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into foreign
ab2c31f [Michael Armbrust] fix test
1d41bb5 [Michael Armbrust] unify argument names
5b47901 [Michael Armbrust] Remove sealed, more filter types
fab154a [Michael Armbrust] Merge remote-tracking branch 'origin/master' into foreign
e3e690e [Michael Armbrust] Add hook for extraStrategies
a70d602 [Michael Armbrust] Fix style, more tests, FilteredSuite => PrunedFilteredSuite
70da6d9 [Michael Armbrust] Modify API to ease binary compatibility and interop with Java
7d948ae [Michael Armbrust] Fix equality of AttributeReference.
5545491 [Michael Armbrust] Address comments
5031ac3 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into foreign
22963ef [Michael Armbrust] package objects compile wierdly...
b069146 [Michael Armbrust] traits => abstract classes
34f836a [Michael Armbrust] Make @DeveloperApi
0d74bcf [Michael Armbrust] Add documention on object life cycle
3e06776 [Michael Armbrust] remove line wraps
de3b68c [Michael Armbrust] Remove empty file
360cb30 [Michael Armbrust] style and java api
2957875 [Michael Armbrust] add override
0fd3a07 [Michael Armbrust] Draft of data sources API
2014-11-02 15:08:35 -08:00
Matei Zaharia 23f966f475 [SPARK-3930] [SPARK-3933] Support fixed-precision decimal in SQL, and some optimizations
- Adds optional precision and scale to Spark SQL's decimal type, which behave similarly to those in Hive 13 (https://cwiki.apache.org/confluence/download/attachments/27362075/Hive_Decimal_Precision_Scale_Support.pdf)
- Replaces our internal representation of decimals with a Decimal class that can store small values in a mutable Long, saving memory in this situation and letting some operations happen directly on Longs

This is still marked WIP because there are a few TODOs, but I'll remove that tag when done.

Author: Matei Zaharia <matei@databricks.com>

Closes #2983 from mateiz/decimal-1 and squashes the following commits:

35e6b02 [Matei Zaharia] Fix issues after merge
227f24a [Matei Zaharia] Review comments
31f915e [Matei Zaharia] Implement Davies's suggestions in Python
eb84820 [Matei Zaharia] Support reading/writing decimals as fixed-length binary in Parquet
4dc6bae [Matei Zaharia] Fix decimal support in PySpark
d1d9d68 [Matei Zaharia] Fix compile error and test issues after rebase
b28933d [Matei Zaharia] Support decimal precision/scale in Hive metastore
2118c0d [Matei Zaharia] Some test and bug fixes
81db9cb [Matei Zaharia] Added mutable Decimal that will be more efficient for small precisions
7af0c3b [Matei Zaharia] Add optional precision and scale to DecimalType, but use Unlimited for now
ec0a947 [Matei Zaharia] Make the result of AVG on Decimals be Decimal, not Double
2014-11-01 19:29:14 -07:00
Xiangrui Meng 1d4f355203 [SPARK-3569][SQL] Add metadata field to StructField
Add `metadata: Metadata` to `StructField` to store extra information of columns. `Metadata` is a simple wrapper over `Map[String, Any]` with value types restricted to Boolean, Long, Double, String, Metadata, and arrays of those types. SerDe is via JSON.

Metadata is preserved through simple operations like `SELECT`.

marmbrus liancheng

Author: Xiangrui Meng <meng@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #2701 from mengxr/structfield-metadata and squashes the following commits:

dedda56 [Xiangrui Meng] merge remote
5ef930a [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata
c35203f [Xiangrui Meng] Merge pull request #1 from marmbrus/pr/2701
886b85c [Michael Armbrust] Expose Metadata and MetadataBuilder through the public scala and java packages.
589f314 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata
1e2abcf [Xiangrui Meng] change default value of metadata to None in python
611d3c2 [Xiangrui Meng] move metadata from Expr to NamedExpr
ddfcfad [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata
a438440 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata
4266f4d [Xiangrui Meng] add StructField.toString back for backward compatibility
3f49aab [Xiangrui Meng] remove StructField.toString
24a9f80 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into structfield-metadata
473a7c5 [Xiangrui Meng] merge master
c9d7301 [Xiangrui Meng] organize imports
1fcbf13 [Xiangrui Meng] change metadata type in StructField for Scala/Java
60cc131 [Xiangrui Meng] add doc and header
60614c7 [Xiangrui Meng] add metadata
e42c452 [Xiangrui Meng] merge master
93518fb [Xiangrui Meng] support metadata in python
905bb89 [Xiangrui Meng] java conversions
618e349 [Xiangrui Meng] make tests work in scala
61b8e0f [Xiangrui Meng] merge master
7e5a322 [Xiangrui Meng] do not output metadata in StructField.toString
c41a664 [Xiangrui Meng] merge master
d8af0ed [Xiangrui Meng] move tests to SQLQuerySuite
67fdebb [Xiangrui Meng] add test on join
d65072e [Xiangrui Meng] remove Map.empty
367d237 [Xiangrui Meng] add test
c194d5e [Xiangrui Meng] add metadata field to StructField and Attribute
2014-11-01 14:37:00 -07:00
Cheng Lian 23468e7e96 [SPARK-2220][SQL] Fixes remaining Hive commands
This PR adds support for the `ADD FILE` Hive command, and removes `ShellCommand` and `SourceCommand`. The reason is described in [this SPARK-2220 comment](https://issues.apache.org/jira/browse/SPARK-2220?focusedCommentId=14191841&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14191841).

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #3038 from liancheng/hive-commands and squashes the following commits:

6db61e0 [Cheng Lian] Fixes remaining Hive commands
2014-10-31 11:34:51 -07:00
ravipesala ea465af12d [SPARK-4154][SQL] Query does not work if it has "not between " in Spark SQL and HQL
if the query contains "not between" does not work like.
SELECT * FROM src where key not between 10 and 20'

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #3017 from ravipesala/SPARK-4154 and squashes the following commits:

65fc89e [ravipesala] Handled admin comments
32e6d42 [ravipesala] 'not between' is not working
2014-10-31 11:33:20 -07:00
Anant d31517a3cd [SPARK-4108][SQL] Fixed usage of deprecated in sql/catalyst/types/datatypes
Fixed usage of deprecated in sql/catalyst/types/datatypes to have versio...n parameter

Author: Anant <anant.asty@gmail.com>

Closes #2970 from anantasty/SPARK-4108 and squashes the following commits:

e92cb01 [Anant] Fixed usage of deprecated in sql/catalyst/types/datatypes to have version parameter
2014-10-30 23:02:42 -07:00
ravipesala 9b6ebe33db [SPARK-4120][SQL] Join of multiple tables with syntax like SELECT .. FROM T1,T2,T3.. does not work in SparkSQL
Right now it works for only 2 tables like below query.
sql("SELECT * FROM records1 as a,records2 as b where a.key=b.key ")

But it does not work for more than 2 tables like below query
sql("SELECT * FROM records1 as a,records2 as b,records3 as c where a.key=b.key and a.key=c.key").

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #2987 from ravipesala/multijoin and squashes the following commits:

429b005 [ravipesala] Support multiple joins
2014-10-30 17:15:45 -07:00
Cheng Hao 4b55482abf [SPARK-3343] [SQL] Add serde support for CTAS
Currently, `CTAS` (Create Table As Select) doesn't support specifying the `SerDe` in HQL. This PR will pass down the `ASTNode` into the physical operator `execution.CreateTableAsSelect`, which will extract the `CreateTableDesc` object via Hive `SemanticAnalyzer`. In the meantime, I also update the `HiveMetastoreCatalog.createTable` to optionally support the `CreateTableDesc` for table creation.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #2570 from chenghao-intel/ctas_serde and squashes the following commits:

e011ef5 [Cheng Hao] shim for both 0.12 & 0.13.1
cfb3662 [Cheng Hao] revert to hive 0.12
c8a547d [Cheng Hao] Support SerDe properties within CTAS
2014-10-28 14:36:06 -07:00
Daoyuan Wang 47a40f60d6 [SPARK-3988][SQL] add public API for date type
Add json and python api for date type.
By using Pickle, `java.sql.Date` was serialized as calendar, and recognized in python as `datetime.datetime`.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #2901 from adrian-wang/spark3988 and squashes the following commits:

c51a24d [Daoyuan Wang] convert datetime to date
5670626 [Daoyuan Wang] minor line combine
f760d8e [Daoyuan Wang] fix indent
444f100 [Daoyuan Wang] fix a typo
1d74448 [Daoyuan Wang] fix scala style
8d7dd22 [Daoyuan Wang] add json and python api for date type
2014-10-28 13:43:25 -07:00
ravipesala 5807cb40ae [SPARK-3814][SQL] Support for Bitwise AND(&), OR(|) ,XOR(^), NOT(~) in Spark HQL and SQL
Currently there is no support of Bitwise & , | in Spark HiveQl and Spark SQL as well. So this PR support the same.
I am closing https://github.com/apache/spark/pull/2926 as it has conflicts to merge. And also added support for Bitwise AND(&), OR(|) ,XOR(^), NOT(~) And I handled all review comments in that PR

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #2961 from ravipesala/SPARK-3814-NEW4 and squashes the following commits:

a391c7a [ravipesala] Rebase with master
2014-10-28 13:36:06 -07:00
Cheng Hao 418ad83fe1 [SPARK-3911] [SQL] HiveSimpleUdf can not be optimized in constant folding
```
explain extended select cos(null) from src limit 1;
```
outputs:
```
 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5]
  MetastoreRelation default, src, None

== Optimized Logical Plan ==
Limit 1
 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5]
  MetastoreRelation default, src, None

== Physical Plan ==
Limit 1
 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5]
  HiveTableScan [], (MetastoreRelation default, src, None), None
```
After patching this PR it outputs
```
== Parsed Logical Plan ==
Limit 1
 Project ['cos(null) AS c_0#0]
  UnresolvedRelation None, src, None

== Analyzed Logical Plan ==
Limit 1
 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#0]
  MetastoreRelation default, src, None

== Optimized Logical Plan ==
Limit 1
 Project [null AS c_0#0]
  MetastoreRelation default, src, None

== Physical Plan ==
Limit 1
 Project [null AS c_0#0]
  HiveTableScan [], (MetastoreRelation default, src, None), None
```

Author: Cheng Hao <hao.cheng@intel.com>

Closes #2771 from chenghao-intel/hive_udf_constant_folding and squashes the following commits:

1379c73 [Cheng Hao] duplicate the PlanTest with catalyst/plans/PlanTest
1e52dda [Cheng Hao] add unit test for hive simple udf constant folding
01609ff [Cheng Hao] support constant folding for HiveSimpleUdf
2014-10-27 20:42:05 -07:00
Kousuke Saruta 3a9d66cf59 [SPARK-4061][SQL] We cannot use EOL character in the operand of LIKE predicate.
We cannot use EOL character like \n or \r in the operand of LIKE predicate.
So following condition is never true.

    -- someStr is 'hoge\nfuga'
    where someStr LIKE 'hoge_fuga'

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2908 from sarutak/spark-sql-like-match-modification and squashes the following commits:

d15798b [Kousuke Saruta] Remove test setting for thriftserver
f99a2f4 [Kousuke Saruta] Fixed LIKE predicate so that we can use EOL character as in a operand
2014-10-26 16:54:07 -07:00
Kousuke Saruta ace41e8bf2 [SPARK-3959][SPARK-3960][SQL] SqlParser fails to parse literal -9223372036854775808 (Long.MinValue). / We can apply unary minus only to literal.
SqlParser fails to parse -9223372036854775808 (Long.MinValue) so we cannot write queries such like as follows.

    SELECT value FROM someTable WHERE value > -9223372036854775808

Additionally, because of the wrong syntax definition, we cannot apply unary minus only to literal. So, we cannot write such expressions.

    -(value1 + value2) // Parenthesized expressions
    -column // Columns
    -MAX(column) // Functions

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2816 from sarutak/spark-sql-dsl-improvement2 and squashes the following commits:

32a5005 [Kousuke Saruta] Remove test setting for thriftserver
c2bab5e [Kousuke Saruta] Fixed SPARK-3959 and SPARK-3960
2014-10-26 16:40:29 -07:00
ravipesala 974d7b238b [SPARK-3483][SQL] Special chars in column names
Supporting special chars in column names by using back ticks. Closed https://github.com/apache/spark/pull/2804 and created this PR as it has merge conflicts

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #2927 from ravipesala/SPARK-3483-NEW and squashes the following commits:

f6329f3 [ravipesala] Rebased with master
2014-10-26 16:36:11 -07:00
Yin Huai 05308426f0 [SPARK-4052][SQL] Use scala.collection.Map for pattern matching instead of using Predef.Map (it is scala.collection.immutable.Map)
Please check https://issues.apache.org/jira/browse/SPARK-4052 for cases triggering this bug.

Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #2899 from yhuai/SPARK-4052 and squashes the following commits:

1188f70 [Yin Huai] Address liancheng's comments.
b6712be [Yin Huai] Use scala.collection.Map instead of Predef.Map (scala.collection.immutable.Map).
2014-10-26 16:30:15 -07:00
Kousuke Saruta d518bc24af [SPARK-3953][SQL][Minor] Confusable variable name.
In SqlParser.scala, there is following code.

    case d ~ p ~ r ~ f ~ g ~ h ~ o ~ l  =>
      val base = r.getOrElse(NoRelation)
      val withFilter = f.map(f => Filter(f, base)).getOrElse(base)

In the code above, there are 2 variables which have same name "f" in near place.
One is receiver "f" and other is bound variable "f".

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2807 from sarutak/SPARK-3953 and squashes the following commits:

4957c32 [Kousuke Saruta] Improved variable name in SqlParser.scala
2014-10-26 16:28:33 -07:00
Cheng Lian 2838bf8aad [SPARK-3537][SPARK-3914][SQL] Refines in-memory columnar table statistics
This PR refines in-memory columnar table statistics:

1. adds 2 more statistics for in-memory table columns: `count` and `sizeInBytes`
1. adds filter pushdown support for `IS NULL` and `IS NOT NULL`.
1. caches and propagates statistics in `InMemoryRelation` once the underlying cached RDD is materialized.

   Statistics are collected to driver side with an accumulator.

This PR also fixes SPARK-3914 by properly propagating in-memory statistics.

Author: Cheng Lian <lian@databricks.com>

Closes #2860 from liancheng/propagates-in-mem-stats and squashes the following commits:

0cc5271 [Cheng Lian] Restricts visibility of o.a.s.s.c.p.l.Statistics
c5ff904 [Cheng Lian] Fixes test table name conflict
a8c818d [Cheng Lian] Refines tests
1d01074 [Cheng Lian] Bug fix: shouldn't call STRING.actualSize on null string value
7dc6a34 [Cheng Lian] Adds more in-memory table statistics and propagates them properly
2014-10-26 16:10:09 -07:00
Liang-Chi Hsieh 0af7e514c6 [SPARK-3925][SQL] Do not consider the ordering of qualifiers during comparison
The orderings should not be considered during the comparison between old qualifiers and new qualifiers.

Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #2783 from viirya/full_qualifier_comp and squashes the following commits:

89f652c [Liang-Chi Hsieh] modification for comment.
abb5762 [Liang-Chi Hsieh] More comprehensive comparison of qualifiers.
2014-10-26 14:29:13 -07:00
Josh Rosen bf589fc717 [SPARK-3616] Add basic Selenium tests to WebUISuite
This patch adds Selenium tests for Spark's web UI.  To avoid adding extra
dependencies to the test environment, the tests use Selenium's HtmlUnitDriver,
which is pure-Java, instead of, say, ChromeDriver.

I added new tests to try to reproduce a few UI bugs reported on JIRA, namely
SPARK-3021, SPARK-2105, and SPARK-2527.  I wasn't able to reproduce these bugs;
I suspect that the older ones might have been fixed by other patches.

In order to use HtmlUnitDriver, I added an explicit dependency on the
org.apache.httpcomponents version of httpclient in order to prevent jets3t's
older version from taking precedence on the classpath.

I also upgraded ScalaTest to 2.2.1.

Author: Josh Rosen <joshrosen@apache.org>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #2474 from JoshRosen/webui-selenium-tests and squashes the following commits:

fcc9e83 [Josh Rosen] scalautils -> scalactic package rename
510e54a [Josh Rosen] [SPARK-3616] Add basic Selenium tests to WebUISuite.
2014-10-26 11:29:27 -07:00
Michael Armbrust 3a845d3c04 [SQL] Update Hive test harness for Hive 12 and 13
As part of the upgrade I also copy the newest version of the query tests, and whitelist a bunch of new ones that are now passing.

Author: Michael Armbrust <michael@databricks.com>

Closes #2936 from marmbrus/fix13tests and squashes the following commits:

d9cbdab [Michael Armbrust] Remove user specific tests
65801cd [Michael Armbrust] style and rat
8f6b09a [Michael Armbrust] Update test harness to work with both Hive 12 and 13.
f044843 [Michael Armbrust] Update Hive query tests and golden files to 0.13
2014-10-24 18:36:35 -07:00
Michael Armbrust 0e886610ee [SPARK-4050][SQL] Fix caching of temporary tables with projections.
Previously cached data was found by `sameResult` plan matching on optimized plans.  This technique however fails to locate the cached data when a temporary table with a projection is queried with a further reduced projection.  The failure is due to the fact that optimization will collapse the projections, producing a plan that no longer produces the sameResult as the cached data (though the cached data still subsumes the desired data).  For example consider the following previously failing test case.

```scala
sql("CACHE TABLE tempTable AS SELECT key FROM testData")
assertCached(sql("SELECT COUNT(*) FROM tempTable"))
```

In this PR I change the matching to occur after analysis instead of optimization, so that in the case of temporary tables, the plans will always match.  I think this should work generally, however, this error does raise questions about the need to do more thorough subsumption checking when locating cached data.

Another question is what sort of semantics we want to provide when uncaching data from temporary tables.  For example consider the following sequence of commands:

```scala
testData.select('key).registerTempTable("tempTable1")
testData.select('key).registerTempTable("tempTable2")
cacheTable("tempTable1")

// This obviously works.
assertCached(sql("SELECT COUNT(*) FROM tempTable1"))

// It seems good that this works ...
assertCached(sql("SELECT COUNT(*) FROM tempTable2"))

// ... but is this valid?
uncacheTable("tempTable2")

// Should this still be cached?
assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0)
```

Author: Michael Armbrust <michael@databricks.com>

Closes #2912 from marmbrus/cachingBug and squashes the following commits:

9c822d4 [Michael Armbrust] remove commented out code
5c72fb7 [Michael Armbrust] Add a test case / question about uncaching semantics.
63a23e4 [Michael Armbrust] Perform caching on analyzed instead of optimized plan.
03f1cfe [Michael Armbrust] Clean-up / add tests to SameResult suite.
2014-10-24 10:52:25 -07:00
Takuya UESHIN 7586e2e67a [SPARK-3969][SQL] Optimizer should have a super class as an interface.
Some developers want to replace `Optimizer` to fit their projects but can't do so because currently `Optimizer` is an `object`.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #2825 from ueshin/issues/SPARK-3969 and squashes the following commits:

abbc53c [Takuya UESHIN] Re-rename Optimizer object.
4d2e1bc [Takuya UESHIN] Rename Optimizer object.
9547a23 [Takuya UESHIN] Extract abstract class from Optimizer for developers to be able to replace Optimizer.
2014-10-20 17:09:12 -07:00
Takuya UESHIN 364d52b707 [SPARK-3966][SQL] Fix nullabilities of Cast related to DateType.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #2820 from ueshin/issues/SPARK-3966 and squashes the following commits:

ca4a745 [Takuya UESHIN] Fix nullabilities of Cast related to DateType.
2014-10-20 15:51:05 -07:00
Michael Armbrust e9c1afa87b [SPARK-3800][SQL] Clean aliases from grouping expressions
Author: Michael Armbrust <michael@databricks.com>

Closes #2658 from marmbrus/nestedAggs and squashes the following commits:

862b763 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs
3234521 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into nestedAggs
8b06fdc [Michael Armbrust] possible fix for grouping on nested fields
2014-10-20 15:32:17 -07:00
Cheng Lian 1b3ce61ce9 [SPARK-3906][SQL] Adds multiple join support for SQLContext
Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2767 from liancheng/multi-join and squashes the following commits:

9dc0d18 [Cheng Lian] Adds multiple join support for SQLContext
2014-10-20 15:29:54 -07:00
Takuya UESHIN ea054e1fc7 [SPARK-3986][SQL] Fix package names to fit their directory names.
Package names of 2 test suites are different from their directory names.
- `GeneratedEvaluationSuite`
- `GeneratedMutableEvaluationSuite`

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #2835 from ueshin/issues/SPARK-3986 and squashes the following commits:

fa2cc05 [Takuya UESHIN] Fix package names to fit their directory names.
2014-10-20 11:31:51 -07:00
Liquan Pei 9d9ca91fef [SQL]Small bug in unresolved.scala
name should throw exception with name instead of exprId.

Author: Liquan Pei <liquanpei@gmail.com>

Closes #2758 from Ishiihara/SparkSQL-bug and squashes the following commits:

aa36a3b [Liquan Pei] small bug
2014-10-13 13:49:11 -07:00
Michael Armbrust 371321cade [SQL] Add type checking debugging functions
Adds some functions that were very useful when trying to track down the bug from #2656.  This change also changes the tree output for query plans to include the `'` prefix to unresolved nodes and `!` prefix to nodes that refer to non-existent attributes.

Author: Michael Armbrust <michael@databricks.com>

Closes #2657 from marmbrus/debugging and squashes the following commits:

654b926 [Michael Armbrust] Clean-up, add tests
763af15 [Michael Armbrust] Add typeChecking debugging functions
8c69303 [Michael Armbrust] Add inputSet, references to QueryPlan. Improve tree string with a prefix to denote invalid or unresolved nodes.
fbeab54 [Michael Armbrust] Better toString, factories for AttributeSet.
2014-10-13 13:46:34 -07:00
Cheng Lian 56102dc2d8 [SPARK-2066][SQL] Adds checks for non-aggregate attributes with aggregation
This PR adds a new rule `CheckAggregation` to the analyzer to provide better error message for non-aggregate attributes with aggregation.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2774 from liancheng/non-aggregate-attr and squashes the following commits:

5246004 [Cheng Lian] Passes test suites
bf1878d [Cheng Lian] Adds checks for non-aggregate attributes with aggregation
2014-10-13 13:36:39 -07:00
Daoyuan Wang 2ac40da3f9 [SPARK-3407][SQL]Add Date type support
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #2344 from adrian-wang/date and squashes the following commits:

f15074a [Daoyuan Wang] remove outdated lines
2038085 [Daoyuan Wang] update return type
00fe81f [Daoyuan Wang] address lian cheng's comments
0df6ea1 [Daoyuan Wang] rebase and remove simple string
bb1b1ef [Daoyuan Wang] remove failing test
aa96735 [Daoyuan Wang] not cast for same type compare
30bf48b [Daoyuan Wang] resolve rebase conflict
617d1a8 [Daoyuan Wang] add date_udf case to white list
c37e848 [Daoyuan Wang] comment update
5429212 [Daoyuan Wang] change to long
f8f219f [Daoyuan Wang] revise according to Cheng Hao
0e0a4f5 [Daoyuan Wang] minor format
4ddcb92 [Daoyuan Wang] add java api for date
0e3110e [Daoyuan Wang] try to fix timezone issue
17fda35 [Daoyuan Wang] set test list
2dfbb5b [Daoyuan Wang] support date type
2014-10-13 13:33:12 -07:00
Daoyuan Wang 46db277cc1 [SPARK-3892][SQL] remove redundant type name
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #2747 from adrian-wang/typename and squashes the following commits:

2824216 [Daoyuan Wang] remove redundant typeName
fbaf340 [Daoyuan Wang] typename
2014-10-13 13:31:14 -07:00
Cheng Lian edf02da389 [SPARK-3654][SQL] Unifies SQL and HiveQL parsers
This PR is a follow up of #2590, and tries to introduce a top level SQL parser entry point for all SQL dialects supported by Spark SQL.

A top level parser `SparkSQLParser` is introduced to handle the syntaxes that all SQL dialects should recognize (e.g. `CACHE TABLE`, `UNCACHE TABLE` and `SET`, etc.). For all the syntaxes this parser doesn't recognize directly, it fallbacks to a specified function that tries to parse arbitrary input to a `LogicalPlan`. This function is typically another parser combinator like `SqlParser`. DDL syntaxes introduced in #2475 can be moved to here.

The `ExtendedHiveQlParser` now only handle Hive specific extensions.

Also took the chance to refactor/reformat `SqlParser` for better readability.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2698 from liancheng/gen-sql-parser and squashes the following commits:

ceada76 [Cheng Lian] Minor styling fixes
9738934 [Cheng Lian] Minor refactoring, removes optional trailing ";" in the parser
bb2ab12 [Cheng Lian] SET property value can be empty string
ce8860b [Cheng Lian] Passes test suites
e86968e [Cheng Lian] Removes debugging code
8bcace5 [Cheng Lian] Replaces digit.+ to rep1(digit) (Scala style checking doesn't like it)
d15d54f [Cheng Lian] Unifies SQL and HiveQL parsers
2014-10-09 18:25:06 -07:00
ravipesala ac30205287 [SPARK-3813][SQL] Support "case when" conditional functions in Spark SQL.
"case when" conditional function is already supported in Spark SQL but there is no support in SqlParser. So added parser support to it.

Author : ravipesala ravindra.pesalahuawei.com

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #2678 from ravipesala/SPARK-3813 and squashes the following commits:

70c75a7 [ravipesala] Fixed styles
713ea84 [ravipesala] Updated as per admin comments
709684f [ravipesala] Changed parser to support case when function.
2014-10-09 15:14:58 -07:00
Daoyuan Wang 0c0e09f567 [SPARK-3412][SQL]add missing row api
chenghao-intel assigned this to me, check PR #2284 for previous discussion

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #2529 from adrian-wang/rowapi and squashes the following commits:

c6594b2 [Daoyuan Wang] using boxed
7b7e6e3 [Daoyuan Wang] update pattern match
7a39456 [Daoyuan Wang] rename file and refresh getAs[T]
4c18c29 [Daoyuan Wang] remove setAs[T] and null judge
1614493 [Daoyuan Wang] add missing row api
2014-10-09 14:59:03 -07:00
Yash Datta 752e90f15e [SPARK-3711][SQL] Optimize where in clause filter queries
The In case class is replaced by a InSet class in case all the filters are literals, which uses a hashset instead of Sequence, thereby giving significant performance improvement (earlier the seq was using a worst case linear match (exists method) since expressions were assumed in the filter list) . Maximum improvement should be visible in case small percentage of large data matches the filter list.

Author: Yash Datta <Yash.Datta@guavus.com>

Closes #2561 from saucam/branch-1.1 and squashes the following commits:

4bf2d19 [Yash Datta] SPARK-3711: 1. Fix code style and import order             2. Fix optimization condition             3. Add tests for null in filter list             4. Add test case that optimization is not triggered in case of attributes in filter list
afedbcd [Yash Datta] SPARK-3711: 1. Add test cases for InSet class in ExpressionEvaluationSuite             2. Add class OptimizedInSuite on the lines of ConstantFoldingSuite, for the optimized In clause
0fc902f [Yash Datta] SPARK-3711: UnaryMinus will be handled by constantFolding
bd84c67 [Yash Datta] SPARK-3711: Incorporate review comments. Move optimization of In clause to Optimizer.scala by adding a rule. Add appropriate comments
430f5d1 [Yash Datta] SPARK-3711: Optimize the filter list in case of negative values as well
bee98aa [Yash Datta] SPARK-3711: Optimize where in clause filter queries
2014-10-09 13:17:13 -07:00
Cheng Hao 4ec931951f [SPARK-3707] [SQL] Fix bug of type coercion in DIV
Calling `BinaryArithmetic.dataType` will throws exception until it's resolved, but in type coercion rule `Division`, seems doesn't follow this.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #2559 from chenghao-intel/type_coercion and squashes the following commits:

199a85d [Cheng Hao] Simplify the divide rule
dc55218 [Cheng Hao] fix bug of type coercion in div
2014-10-08 17:52:27 -07:00
Cheng Lian a42cc08d21 [SPARK-3713][SQL] Uses JSON to serialize DataType objects
This PR uses JSON instead of `toString` to serialize `DataType`s. The latter is not only hard to parse but also flaky in many cases.

Since we already write schema information to Parquet metadata in the old style, we have to reserve the old `DataType` parser and ensure downward compatibility. The old parser is now renamed to `CaseClassStringParser` and moved into `object DataType`.

JoshRosen davies Please help review PySpark related changes, thanks!

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2563 from liancheng/datatype-to-json and squashes the following commits:

fc92eb3 [Cheng Lian] Reverts debugging code, simplifies primitive type JSON representation
438c75f [Cheng Lian] Refactors PySpark DataType JSON SerDe per comments
6b6387b [Cheng Lian] Removes debugging code
6a3ee3a [Cheng Lian] Addresses per review comments
dc158b5 [Cheng Lian] Addresses PEP8 issues
99ab4ee [Cheng Lian] Adds compatibility est case for Parquet type conversion
a983a6c [Cheng Lian] Adds PySpark support
f608c6e [Cheng Lian] De/serializes DataType objects from/to JSON
2014-10-08 17:04:49 -07:00
Kousuke Saruta a85f24accd [SPARK-3831] [SQL] Filter rule Improvement and bool expression optimization.
If we write the filter which is always FALSE like

    SELECT * from person WHERE FALSE;

200 tasks will run. I think, 1 task is enough.

And current optimizer cannot optimize the case NOT is duplicated like

    SELECT * from person WHERE NOT ( NOT (age > 30));

The filter rule above should be simplified

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2692 from sarutak/SPARK-3831 and squashes the following commits:

25f3e20 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3831
23c750c [Kousuke Saruta] Improved unsupported predicate test case
a11b9f3 [Kousuke Saruta] Modified NOT predicate test case in PartitionBatchPruningSuite
8ea872b [Kousuke Saruta] Fixed the number of tasks when the data of  LocalRelation is empty.
2014-10-08 17:03:47 -07:00
Renat Yusupov 90897ea5f2 [SPARK-3776][SQL] Wrong conversion to Catalyst for Option[Product]
Author: Renat Yusupov <re.yusupov@2gis.ru>

Closes #2641 from r3natko/feature/catalyst_option and squashes the following commits:

55d0c06 [Renat Yusupov] [SQL] SPARK-3776: Wrong conversion to Catalyst for Option[Product]
2014-10-05 17:56:34 -07:00
Cheng Lian 34b97a067d [SPARK-3645][SQL] Makes table caching eager by default and adds syntax for lazy caching
Although lazy caching for in-memory table seems consistent with the `RDD.cache()` API, it's relatively confusing for users who mainly work with SQL and not familiar with Spark internals. The `CACHE TABLE t; SELECT COUNT(*) FROM t;` pattern is also commonly seen just to ensure predictable performance.

This PR makes both the `CACHE TABLE t [AS SELECT ...]` statement and the `SQLContext.cacheTable()` API eager by default, and adds a new `CACHE LAZY TABLE t [AS SELECT ...]` syntax to provide lazy in-memory table caching.

Also, took the chance to make some refactoring: `CacheCommand` and `CacheTableAsSelectCommand` are now merged and renamed to `CacheTableCommand` since the former is strictly a special case of the latter. A new `UncacheTableCommand` is added for the `UNCACHE TABLE t` statement.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2513 from liancheng/eager-caching and squashes the following commits:

fe92287 [Cheng Lian] Makes table caching eager by default and adds syntax for lazy caching
2014-10-05 17:51:59 -07:00
Michael Armbrust 6a1d48f4f0 [SPARK-3212][SQL] Use logical plan matching instead of temporary tables for table caching
_Also addresses: SPARK-1671, SPARK-1379 and SPARK-3641_

This PR introduces a new trait, `CacheManger`, which replaces the previous temporary table based caching system.  Instead of creating a temporary table that shadows an existing table with and equivalent cached representation, the cached manager maintains a separate list of logical plans and their cached data.  After optimization, this list is searched for any matching plan fragments.  When a matching plan fragment is found it is replaced with the cached data.

There are several advantages to this approach:
 - Calling .cache() on a SchemaRDD now works as you would expect, and uses the more efficient columnar representation.
 - Its now possible to provide a list of temporary tables, without having to decide if a given table is actually just a  cached persistent table. (To be done in a follow-up PR)
 - In some cases it is possible that cached data will be used, even if a cached table was not explicitly requested.  This is because we now look at the logical structure instead of the table name.
 - We now correctly invalidate when data is inserted into a hive table.

Author: Michael Armbrust <michael@databricks.com>

Closes #2501 from marmbrus/caching and squashes the following commits:

63fbc2c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into caching.
0ea889e [Michael Armbrust] Address comments.
1e23287 [Michael Armbrust] Add support for cache invalidation for hive inserts.
65ed04a [Michael Armbrust] fix tests.
bdf9a3f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into caching
b4b77f2 [Michael Armbrust] Address comments
6923c9d [Michael Armbrust] More comments / tests
80f26ac [Michael Armbrust] First draft of improved semantics for Spark SQL caching.
2014-10-03 12:34:27 -07:00
ravipesala bbdf1de84f [SPARK-3371][SQL] Renaming a function expression with group by gives error
The following code gives error.
```
sqlContext.registerFunction("len", (s: String) => s.length)
sqlContext.sql("select len(foo) as a, count(1) from t1 group by len(foo)").collect()
```
Because SQl parser creates the aliases to the functions in grouping expressions with generated alias names. So if user gives the alias names to the functions inside projection then it does not match the generated alias name of grouping expression.
This kind of queries are working in Hive.
So the fix I have given that if user provides alias to the function in projection then don't generate alias in grouping expression,use the same alias.

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #2511 from ravipesala/SPARK-3371 and squashes the following commits:

9fb973f [ravipesala] Removed aliases to grouping expressions.
f8ace79 [ravipesala] Fixed the testcase issue
bad2fd0 [ravipesala] SPARK-3371 : Fixed Renaming a function expression with group by gives error
2014-10-01 23:53:21 -07:00
Venkata Ramana Gollamudi f84b228c40 [SPARK-3593][SQL] Add support for sorting BinaryType
BinaryType is derived from NativeType and added Ordering support.

Author: Venkata Ramana G <ramana.gollamudihuawei.com>

Author: Venkata Ramana Gollamudi <ramana.gollamudi@huawei.com>

Closes #2617 from gvramana/binarytype_sort and squashes the following commits:

1cf26f3 [Venkata Ramana Gollamudi] Supported Sorting of BinaryType
2014-10-01 15:57:09 -07:00
Venkata Ramana Gollamudi 1c62f97e94 [SPARK-3268][SQL] DoubleType, FloatType and DecimalType modulus support
Supported modulus operation using % operator on fractional datatypes FloatType, DoubleType and DecimalType
Example:
SELECT 1388632775.0 % 60 from tablename LIMIT 1

Author : Venkata Ramana Gollamudi ramana.gollamudihuawei.com

Author: Venkata Ramana Gollamudi <ramana.gollamudi@huawei.com>

Closes #2457 from gvramana/double_modulus_support and squashes the following commits:

79172a8 [Venkata Ramana Gollamudi] Add hive cache to testcase
c09bd5b [Venkata Ramana Gollamudi] Added a HiveQuerySuite testcase
193fa81 [Venkata Ramana Gollamudi] corrected testcase
3624471 [Venkata Ramana Gollamudi] modified testcase
e112c09 [Venkata Ramana Gollamudi] corrected the testcase
513d0e0 [Venkata Ramana Gollamudi] modified to add modulus support to fractional types float,double,decimal
296d253 [Venkata Ramana Gollamudi] modified to add modulus support to fractional types float,double,decimal
2014-09-23 12:17:47 -07:00
Daoyuan Wang 66bc0f2d67 [SPARK-3598][SQL]cast to timestamp should be the same as hive
this patch fixes timestamp smaller than 0 and cast int as timestamp

select cast(1000 as timestamp) from src limit 1;

should return 1970-01-01 00:00:01, but we now take it as 1000 seconds.
also, current implementation has bug when the time is before 1970-01-01 00:00:00.
rxin marmbrus chenghao-intel

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #2458 from adrian-wang/timestamp and squashes the following commits:

4274b1d [Daoyuan Wang] set test not related to timezone
1234f66 [Daoyuan Wang] fix timestamp smaller than 0 and cast int as timestamp
2014-09-23 11:45:44 -07:00
Michael Armbrust 293ce85145 [SPARK-3414][SQL] Replace LowerCaseSchema with Resolver
**This PR introduces a subtle change in semantics for HiveContext when using the results in Python or Scala.  Specifically, while resolution remains case insensitive, it is now case preserving.**

_This PR is a follow up to #2293 (and to a lesser extent #2262 #2334)._

In #2293 the catalog was changed to store analyzed logical plans instead of unresolved ones.  While this change fixed the reported bug (which was caused by yet another instance of us forgetting to put in a `LowerCaseSchema` operator) it had the consequence of breaking assumptions made by `MultiInstanceRelation`.  Specifically, we can't replace swap out leaf operators in a tree without rewriting changed expression ids (which happens when you self join the same RDD that has been registered as a temp table).

In this PR, I instead remove the need to insert `LowerCaseSchema` operators at all, by moving the concern of matching up identifiers completely into analysis.  Doing so allows the test cases from both #2293 and #2262 to pass at the same time (and likely fixes a slew of other "unknown unknown" bugs).

While it is rolled back in this PR, storing the analyzed plan might actually be a good idea.  For instance, it is kind of confusing if you register a temporary table, change the case sensitivity of resolution and now you can't query that table anymore.  This can be addressed in a follow up PR.

Follow-ups:
 - Configurable case sensitivity
 - Consider storing analyzed plans for temp tables

Author: Michael Armbrust <michael@databricks.com>

Closes #2382 from marmbrus/lowercase and squashes the following commits:

c21171e [Michael Armbrust] Ensure the resolver is used for field lookups and ensure that case insensitive resolution is still case preserving.
d4320f1 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into lowercase
2de881e [Michael Armbrust] Address comments.
219805a [Michael Armbrust] style
5b93711 [Michael Armbrust] Replace LowerCaseSchema with Resolver.
2014-09-20 16:41:14 -07:00
Cheng Lian 7f54580c45 [SPARK-3609][SQL] Adds sizeInBytes statistics for Limit operator when all output attributes are of native data types
This helps to replace shuffled hash joins with broadcast hash joins in some cases.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2468 from liancheng/more-stats and squashes the following commits:

32687dc [Cheng Lian] Moved the test case to PlannerSuite
5595a91 [Cheng Lian] Removes debugging code
73faf69 [Cheng Lian] Test case for auto choosing broadcast hash join
f30fe1d [Cheng Lian] Adds sizeInBytes estimation for Limit when all output types are native types
2014-09-20 16:30:49 -07:00
ravipesala 5522151eb1 [SPARK-2594][SQL] Support CACHE TABLE <name> AS SELECT ...
This feature allows user to add cache table from the select query.
Example : ```CACHE TABLE testCacheTable AS SELECT * FROM TEST_TABLE```
Spark takes this type of SQL as command and it does lazy caching just like ```SQLContext.cacheTable```, ```CACHE TABLE <name>``` does.
It can be executed from both SQLContext and HiveContext.

Recreated the pull request after rebasing with master.And fixed all the comments raised in previous pull requests.
https://github.com/apache/spark/pull/2381
https://github.com/apache/spark/pull/2390

Author : ravipesala ravindra.pesalahuawei.com

Author: ravipesala <ravindra.pesala@huawei.com>

Closes #2397 from ravipesala/SPARK-2594 and squashes the following commits:

a5f0beb [ravipesala] Simplified the code as per Admin comment.
8059cd2 [ravipesala] Changed the behaviour from eager caching to lazy caching.
d6e469d [ravipesala] Code review comments by Admin are handled.
c18aa38 [ravipesala] Merge remote-tracking branch 'remotes/ravipesala/Add-Cache-table-as' into SPARK-2594
394d5ca [ravipesala] Changed style
fb1759b [ravipesala] Updated as per Admin comments
8c9993c [ravipesala] Changed the style
d8b37b2 [ravipesala] Updated as per the comments by Admin
bc0bffc [ravipesala] Merge remote-tracking branch 'ravipesala/Add-Cache-table-as' into Add-Cache-table-as
e3265d0 [ravipesala] Updated the code as per the comments by Admin in pull request.
724b9db [ravipesala] Changed style
aaf5b59 [ravipesala] Added comment
dc33895 [ravipesala] Updated parser to support add cache table command
b5276b2 [ravipesala] Updated parser to support add cache table command
eebc0c1 [ravipesala] Add CACHE TABLE <name> AS SELECT ...
6758f80 [ravipesala] Changed style
7459ce3 [ravipesala] Added comment
13c8e27 [ravipesala] Updated parser to support add cache table command
4e858d8 [ravipesala] Updated parser to support add cache table command
b803fc8 [ravipesala] Add CACHE TABLE <name> AS SELECT ...
2014-09-19 15:31:57 -07:00
Cheng Hao 2c3cc7641d [SPARK-3501] [SQL] Fix the bug of Hive SimpleUDF creates unnecessary type cast
When do the query like:
```
select datediff(cast(value as timestamp), cast('2002-03-21 00:00:00' as timestamp)) from src;
```
SparkSQL will raise exception:
```
[info] scala.MatchError: TimestampType (of class org.apache.spark.sql.catalyst.types.TimestampType$)
[info] at org.apache.spark.sql.catalyst.expressions.Cast.castToTimestamp(Cast.scala:77)
[info] at org.apache.spark.sql.catalyst.expressions.Cast.cast$lzycompute(Cast.scala:251)
[info] at org.apache.spark.sql.catalyst.expressions.Cast.cast(Cast.scala:247)
[info] at org.apache.spark.sql.catalyst.expressions.Cast.eval(Cast.scala:263)
[info] at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$5$$anonfun$applyOrElse$2.applyOrElse(Optimizer.scala:217)
[info] at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$5$$anonfun$applyOrElse$2.applyOrElse(Optimizer.scala:210)
[info] at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:144)
[info] at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4$$anonfun$apply$2.apply(TreeNode.scala:180)
[info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
[info] at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
```

Author: Cheng Hao <hao.cheng@intel.com>

Closes #2368 from chenghao-intel/cast_exception and squashes the following commits:

5c9c3a5 [Cheng Hao] make more clear code
49dfc50 [Cheng Hao] Add no-op for Cast and revert the position of SimplifyCasts
b804abd [Cheng Hao] Add unit test to show the failure in identical data type casting
330a5c8 [Cheng Hao] Update Code based on comments
b834ed4 [Cheng Hao] Fix bug of HiveSimpleUDF with unnecessary type cast which cause exception in constant folding
2014-09-19 15:29:22 -07:00
Michael Armbrust 30f288ae34 [SPARK-2890][SQL] Allow reading of data when case insensitive resolution could cause possible ambiguity.
Throwing an error in the constructor makes it possible to run queries, even when there is no actual ambiguity.  Remove this check in favor of throwing an error in analysis when they query is actually is ambiguous.

Also took the opportunity to add test cases that would have caught a subtle bug in my first attempt at fixing this and refactor some other test code.

Author: Michael Armbrust <michael@databricks.com>

Closes #2209 from marmbrus/sameNameStruct and squashes the following commits:

729cca4 [Michael Armbrust] Better tests.
a003aeb [Michael Armbrust] Remove error (it'll be caught in analysis).
2014-09-16 11:42:26 -07:00
Cheng Lian 74049249ab [SPARK-3294][SQL] Eliminates boxing costs from in-memory columnar storage
This is a major refactoring of the in-memory columnar storage implementation, aims to eliminate boxing costs from critical paths (building/accessing column buffers) as much as possible. The basic idea is to refactor all major interfaces into a row-based form and use them together with `SpecificMutableRow`. The difficult part is how to adapt all compression schemes, esp. `RunLengthEncoding` and `DictionaryEncoding`, to this design. Since in-memory compression is disabled by default for now, and this PR should be strictly better than before no matter in-memory compression is enabled or not, maybe I'll finish that part in another PR.

**UPDATE** This PR also took the chance to optimize `HiveTableScan` by

1. leveraging `SpecificMutableRow` to avoid boxing cost, and
1. building specific `Writable` unwrapper functions a head of time to avoid per row pattern matching and branching costs.

TODO

- [x] Benchmark
- [ ] ~~Eliminate boxing costs in `RunLengthEncoding`~~ (left to future PRs)
- [ ] ~~Eliminate boxing costs in `DictionaryEncoding` (seems not easy to do without specializing `DictionaryEncoding` for every supported column type)~~  (left to future PRs)

## Micro benchmark

The benchmark uses a 10 million line CSV table consists of bytes, shorts, integers, longs, floats and doubles, measures the time to build the in-memory version of this table, and the time to scan the whole in-memory table.

Benchmark code can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-hivetablescanbenchmark-scala). Script used to generate the input table can be found [here](https://gist.github.com/liancheng/fe70a148de82e77bd2c8#file-tablegen-scala).

Speedup:

- Hive table scanning + column buffer building: **18.74%**

  The original benchmark uses 1K as in-memory batch size, when increased to 10K, it can be 28.32% faster.

- In-memory table scanning: **7.95%**

Before:

        | Building | Scanning
------- | -------- | --------
1       | 16472    | 525
2       | 16168    | 530
3       | 16386    | 529
4       | 16184    | 538
5       | 16209    | 521
Average | 16283.8  | 528.6

After:

        | Building | Scanning
------- | -------- | --------
1       | 13124    | 458
2       | 13260    | 529
3       | 12981    | 463
4       | 13214    | 483
5       | 13583    | 500
Average | 13232.4  | 486.6

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2327 from liancheng/prevent-boxing/unboxing and squashes the following commits:

4419fe4 [Cheng Lian] Addressing comments
e5d2cf2 [Cheng Lian] Bug fix: should call setNullAt when field value is null to avoid NPE
8b8552b [Cheng Lian] Only checks for partition batch pruning flag once
489f97b [Cheng Lian] Bug fix: TableReader.fillObject uses wrong ordinals
97bbc4e [Cheng Lian] Optimizes hive.TableReader by by providing specific Writable unwrappers a head of time
3dc1f94 [Cheng Lian] Minor changes to eliminate row object creation
5b39cb9 [Cheng Lian] Lowers log level of compression scheme details
f2a7890 [Cheng Lian] Use SpecificMutableRow in InMemoryColumnarTableScan to avoid boxing
9cf30b0 [Cheng Lian] Added row based ColumnType.append/extract
456c366 [Cheng Lian] Made compression decoder row based
edac3cd [Cheng Lian] Makes ColumnAccessor.extractSingle row based
8216936 [Cheng Lian] Removes boxing cost in IntDelta and LongDelta by providing specialized implementations
b70d519 [Cheng Lian] Made some in-memory columnar storage interfaces row-based
2014-09-13 15:08:30 -07:00
Cody Koeninger f858f46686 SPARK-3462 push down filters and projections into Unions
Author: Cody Koeninger <cody.koeninger@mediacrossing.com>

Closes #2345 from koeninger/SPARK-3462 and squashes the following commits:

5c8d24d [Cody Koeninger] SPARK-3462 remove now-unused parameter
0788691 [Cody Koeninger] SPARK-3462 add tests, handle compatible schema with different aliases, per marmbrus feedback
ef47b3b [Cody Koeninger] SPARK-3462 push down filters and projections into Unions
2014-09-11 17:49:36 -07:00
Cheng Hao ca83f1e2c4 [SPARK-2917] [SQL] Avoid table creation in logical plan analyzing for CTAS
Author: Cheng Hao <hao.cheng@intel.com>

Closes #1846 from chenghao-intel/ctas and squashes the following commits:

56a0578 [Cheng Hao] remove the unused imports
9a57abc [Cheng Hao] Avoid table creation in logical plan analyzing
2014-09-11 11:57:01 -07:00
Aaron Staple c27718f376 [SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer.
LogicalPlan contains a ‘resolved’ attribute indicating that all of its execution requirements have been resolved. This attribute is not checked before query execution. The analyzer contains a step to check that all Expressions are resolved, but this is not equivalent to checking all LogicalPlans. In particular, the Union plan’s implementation of ‘resolved’ verifies that the types of its children’s columns are compatible. Because the analyzer does not check that a Union plan is resolved, it is possible to execute a Union plan that outputs different types in the same column.  See SPARK-2781 for an example.

This patch adds two checks to the analyzer’s CheckResolution rule. First, each logical plan is checked to see if it is not resolved despite its children being resolved. This allows the ‘problem’ unresolved plan to be included in the TreeNodeException for reporting. Then as a backstop the root plan is checked to see if it is resolved, which recursively checks that the entire plan tree is resolved. Note that the resolved attribute is implemented recursively, and this patch also explicitly checks the resolved attribute on each logical plan in the tree. I assume the query plan trees will not be large enough for this redundant checking to meaningfully impact performance.

Because this patch starts validating that LogicalPlans are resolved before execution, I had to fix some cases where unresolved plans were passing through the analyzer as part of the implementation of the hive query system. In particular, HiveContext applies the CreateTables and PreInsertionCasts, and ExtractPythonUdfs rules manually after the analyzer runs. I moved these rules to the analyzer stage (for hive queries only), in the process completing a code TODO indicating the rules should be moved to the analyzer.

It’s worth noting that moving the CreateTables rule means introducing an analyzer rule with a significant side effect - in this case the side effect is creating a hive table. The rule will only attempt to create a table once even if its batch is executed multiple times, because it converts the InsertIntoCreatedTable plan it matches against into an InsertIntoTable. Additionally, these hive rules must be added to the Resolution batch rather than as a separate batch because hive rules rules may be needed to resolve non-root nodes, leaving the root to be resolved on a subsequent batch iteration. For example, the hive compatibility test auto_smb_mapjoin_14, and others, make use of a query plan where the root is a Union and its children are each a hive InsertIntoTable.

Mixing the custom hive rules with standard analyzer rules initially resulted in an additional failure because of policy differences between spark sql and hive when casting a boolean to a string. Hive casts booleans to strings as “true” / “false” while spark sql casts booleans to strings as “1” / “0” (causing the cast1.q test to fail). This behavior is a result of the BooleanCasts rule in HiveTypeCoercion.scala, and from looking at the implementation of BooleanCasts I think converting to to “1”/“0” is potentially a programming mistake. (If the BooleanCasts rule is disabled, casting produces “true”/“false” instead.) I believe “true” / “false” should be the behavior for spark sql - I changed the behavior so bools are converted to “true”/“false” to be consistent with hive, and none of the existing spark tests failed.

Finally, in some initial testing with hive it appears that an implicit type coercion of boolean to string results in a lowercase string, e.g. CONCAT( TRUE, “” ) -> “true” while an explicit cast produces an all caps string, e.g. CAST( TRUE AS STRING ) -> “TRUE”.  The change I’ve made just converts to lowercase strings in all cases.  I believe it is at least more correct than the existing spark sql implementation where all Cast expressions become “1” / “0”.

Author: Aaron Staple <aaron.staple@gmail.com>

Closes #1706 from staple/SPARK-2781 and squashes the following commits:

32683c4 [Aaron Staple] Fix compilation failure due to merge.
7c77fda [Aaron Staple] Move ExtractPythonUdfs to Analyzer's extendedRules in HiveContext.
d49bfb3 [Aaron Staple] Address review comments.
915b690 [Aaron Staple] Fix merge issue causing compilation failure.
701dcd2 [Aaron Staple] [SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer.
2014-09-10 21:01:53 -07:00
Wenchen Fan e4f4886d71 [SPARK-2096][SQL] Correctly parse dot notations
First let me write down the current `projections` grammar of spark sql:

    expression                : orExpression
    orExpression              : andExpression {"or" andExpression}
    andExpression             : comparisonExpression {"and" comparisonExpression}
    comparisonExpression      : termExpression | termExpression "=" termExpression | termExpression ">" termExpression | ...
    termExpression            : productExpression {"+"|"-" productExpression}
    productExpression         : baseExpression {"*"|"/"|"%" baseExpression}
    baseExpression            : expression "[" expression "]" | ... | ident | ...
    ident                     : identChar {identChar | digit} | delimiters | ...
    identChar                 : letter | "_" | "."
    delimiters                : "," | ";" | "(" | ")" | "[" | "]" | ...
    projection                : expression [["AS"] ident]
    projections               : projection { "," projection}

For something like `a.b.c[1]`, it will be parsed as:
<img src="http://img51.imgspice.com/i/03008/4iltjsnqgmtt_t.jpg" border=0>
But for something like `a[1].b`, the current grammar can't parse it correctly.
A simple solution is written in `ParquetQuerySuite#NestedSqlParser`, changed grammars are:

    delimiters                : "." | "," | ";" | "(" | ")" | "[" | "]" | ...
    identChar                 : letter | "_"
    baseExpression            : expression "[" expression "]" | expression "." ident | ... | ident | ...
This works well, but can't cover some corner case like `select t.a.b from table as t`:
<img src="http://img51.imgspice.com/i/03008/v2iau3hoxoxg_t.jpg" border=0>
`t.a.b` parsed as `GetField(GetField(UnResolved("t"), "a"), "b")` instead of `GetField(UnResolved("t.a"), "b")` using this new grammar.
However, we can't resolve `t` as it's not a filed, but the whole table.(if we could do this, then `select t from table as t` is legal, which is unexpected)
My solution is:

    dotExpressionHeader       : ident "." ident
    baseExpression            : expression "[" expression "]" | expression "." ident | ... | dotExpressionHeader  | ident | ...
I passed all test cases under sql locally and add a more complex case.
"arrayOfStruct.field1 to access all values of field1" is not supported yet. Since this PR has changed a lot of code, I will open another PR for it.
I'm not familiar with the latter optimize phase, please correct me if I missed something.

Author: Wenchen Fan <cloud0fan@163.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #2230 from cloud-fan/dot and squashes the following commits:

e1a8898 [Wenchen Fan] remove support for arbitrary nested arrays
ee8a724 [Wenchen Fan] rollback LogicalPlan, support dot operation on nested array type
a58df40 [Michael Armbrust] add regression test for doubly nested data
16bc4c6 [Wenchen Fan] some enhance
95d733f [Wenchen Fan] split long line
dc31698 [Wenchen Fan] SPARK-2096 Correctly parse dot notations
2014-09-10 12:56:59 -07:00
Daoyuan Wang f0c87dc86a [SPARK-3363][SQL] Type Coercion should promote null to all other types.
Type Coercion should support every type to have null value

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #2246 from adrian-wang/spark3363-0 and squashes the following commits:

c6241de [Daoyuan Wang] minor code clean
595b417 [Daoyuan Wang] Merge pull request #2 from marmbrus/pr/2246
832e640 [Michael Armbrust] reduce code duplication
ef6f986 [Daoyuan Wang] make double boolean miss in jsonRDD compatibleType
c619f0a [Daoyuan Wang] Type Coercion should support every type to have null value
2014-09-10 10:48:36 -07:00
Daoyuan Wang a0283300c4 [SPARK-3362][SQL] Fix resolution for casewhen with nulls.
Current implementation will ignore else val type.

Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #2245 from adrian-wang/casewhenbug and squashes the following commits:

3332f6e [Daoyuan Wang] remove wrong comment
83b536c [Daoyuan Wang] a comment to trigger retest
d7315b3 [Daoyuan Wang] code improve
eed35fc [Daoyuan Wang] bug in casewhen resolve
2014-09-10 10:45:24 -07:00
Cheng Lian c110614b33 [SPARK-3448][SQL] Check for null in SpecificMutableRow.update
`SpecificMutableRow.update` doesn't check for null, and breaks existing `MutableRow` contract.

The tricky part here is that for performance considerations, the `update` method of all subclasses of `MutableValue` doesn't check for null and sets the null bit to false.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2325 from liancheng/check-for-null and squashes the following commits:

9366c44 [Cheng Lian] Check for null in SpecificMutableRow.update
2014-09-09 18:39:33 -07:00
xinyunh 07ee4a28c3 [SPARK-3176] Implement 'ABS and 'LAST' for sql
Add support for the mathematical function"ABS" and the analytic function "last" to return a subset of the rows satisfying a query within spark sql. Test-cases included.

Author: xinyunh <xinyun.huang@huawei.com>
Author: bomeng <golf8lover>

Closes #2099 from xinyunh/sqlTest and squashes the following commits:

71d15e7 [xinyunh] remove POWER part
8843643 [xinyunh] fix the code style issue
39f0309 [bomeng] Modify the code of POWER and ABS. Move them to the file arithmetic
ff8e51e [bomeng] add abs() function support
7f6980a [xinyunh] fix the bug in 'Last' component
b3df91b [xinyunh] add 'Last' component
2014-09-09 16:55:39 -07:00
William Benton ca0348e682 SPARK-3423: [SQL] Implement BETWEEN for SQLParser
This patch improves the SQLParser by adding support for BETWEEN conditions

Author: William Benton <willb@redhat.com>

Closes #2295 from willb/sql-between and squashes the following commits:

0016d30 [William Benton] Implement BETWEEN for SQLParser
2014-09-08 19:05:02 -07:00
GuoQiang Li 607ae39c22 [SPARK-3397] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT
Author: GuoQiang Li <witgo@qq.com>

Closes #2268 from witgo/SPARK-3397 and squashes the following commits:

eaf913f [GuoQiang Li] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT
2014-09-06 15:04:50 -07:00
Cheng Lian 248067adbe [SPARK-2961][SQL] Use statistics to prune batches within cached partitions
This PR is based on #1883 authored by marmbrus. Key differences:

1. Batch pruning instead of partition pruning

   When #1883 was authored, batched column buffer building (#1880) hadn't been introduced. This PR combines these two and provide partition batch level pruning, which leads to smaller memory footprints and can generally skip more elements. The cost is that the pruning predicates are evaluated more frequently (partition number multiplies batch number per partition).

1. More filters are supported

   Filter predicates consist of `=`, `<`, `<=`, `>`, `>=` and their conjunctions and disjunctions are supported.

Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2188 from liancheng/in-mem-batch-pruning and squashes the following commits:

68cf019 [Cheng Lian] Marked sqlContext as @transient
4254f6c [Cheng Lian] Enables in-memory partition pruning in PartitionBatchPruningSuite
3784105 [Cheng Lian] Overrides InMemoryColumnarTableScan.sqlContext
d2a1d66 [Cheng Lian] Disables in-memory partition pruning by default
062c315 [Cheng Lian] HiveCompatibilitySuite code cleanup
16b77bf [Cheng Lian] Fixed pruning predication conjunctions and disjunctions
16195c5 [Cheng Lian] Enabled both disjunction and conjunction
89950d0 [Cheng Lian] Worked around Scala style check
9c167f6 [Cheng Lian] Minor code cleanup
3c4d5c7 [Cheng Lian] Minor code cleanup
ea59ee5 [Cheng Lian] Renamed PartitionSkippingSuite to PartitionBatchPruningSuite
fc517d0 [Cheng Lian] More test cases
1868c18 [Cheng Lian] Code cleanup, bugfix, and adding tests
cb76da4 [Cheng Lian] Added more predicate filters, fixed table scan stats for testing purposes
385474a [Cheng Lian] Merge branch 'inMemStats' into in-mem-batch-pruning
2014-09-03 18:59:26 -07:00
Takuya UESHIN 0cd91f666d [SPARK-3341][SQL] The dataType of Sqrt expression should be DoubleType.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #2233 from ueshin/issues/SPARK-3341 and squashes the following commits:

e497320 [Takuya UESHIN] Fix data type of Sqrt expression.
2014-09-02 20:31:15 -07:00
Zdenek Farana 98ddbe6cdb [SPARK-3173][SQL] Timestamp support in the parser
If you have a table with TIMESTAMP column, that column can't be used in WHERE clause properly - it is not evaluated properly. [More](https://issues.apache.org/jira/browse/SPARK-3173)

Motivation: http://www.aproint.com/aggregation-with-spark-sql/

- [x] modify SqlParser so it supports casting to TIMESTAMP (workaround for item 2)
- [x] the string literal should be converted into Timestamp if the column is Timestamp.

Author: Zdenek Farana <zdenek.farana@gmail.com>
Author: Zdenek Farana <zdenek.farana@aproint.com>

Closes #2084 from byF/SPARK-3173 and squashes the following commits:

442b59d [Zdenek Farana] Fixed test merge conflict
2dbf4f6 [Zdenek Farana] Merge remote-tracking branch 'origin/SPARK-3173' into SPARK-3173
65b6215 [Zdenek Farana] Fixed timezone sensitivity in the test
47b27b4 [Zdenek Farana] Now works in the case of "StringLiteral=TimestampColumn"
96a661b [Zdenek Farana] Code style change
491dfcf [Zdenek Farana] Added test cases for SPARK-3173
4446b1e [Zdenek Farana] A string literal is casted into Timestamp when the column is Timestamp.
59af397 [Zdenek Farana] Added a new TIMESTAMP keyword; CAST to TIMESTAMP now can be used in SQL expression.
2014-08-29 15:39:15 -07:00
Cheng Hao dc4d577c65 [SPARK-3198] [SQL] Remove the TreeNode.id
Thus id property of the TreeNode API does save time in a faster way to compare 2 TreeNodes, it is kind of performance bottleneck during the expression object creation in a multi-threading env (because of the memory barrier).
Fortunately, the tree node comparison only happen once in master, so even we remove it, the entire performance will not be affected.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #2155 from chenghao-intel/treenode and squashes the following commits:

7cf2cd2 [Cheng Hao] Remove the implicit keyword for TreeNodeRef and some other small issues
5873415 [Cheng Hao] Remove the TreeNode.id
2014-08-29 15:32:26 -07:00
William Benton 2f1519defa SPARK-2813: [SQL] Implement SQRT() directly in Spark SQL
This PR adds a native implementation for SQL SQRT() and thus avoids delegating this function to Hive.

Author: William Benton <willb@redhat.com>

Closes #1750 from willb/spark-2813 and squashes the following commits:

22c8a79 [William Benton] Fixed missed newline from rebase
d673861 [William Benton] Added string coercions for SQRT and associated test case
e125df4 [William Benton] Added ExpressionEvaluationSuite test cases for SQRT
7b84bcd [William Benton] SQL SQRT now properly returns NULL for NULL inputs
8256971 [William Benton] added SQRT test to SqlQuerySuite
504d2e5 [William Benton] Added native SQRT implementation
2014-08-29 15:26:59 -07:00
Michael Armbrust 76e3ba4264 [SPARK-3230][SQL] Fix udfs that return structs
We need to convert the case classes into Rows.

Author: Michael Armbrust <michael@databricks.com>

Closes #2133 from marmbrus/structUdfs and squashes the following commits:

189722f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into structUdfs
8e29b1c [Michael Armbrust] Use existing function
d8d0b76 [Michael Armbrust] Fix udfs that return structs
2014-08-28 00:15:23 -07:00
Michael Armbrust 024178c574 [HOTFIX][SQL] Remove cleaning of UDFs
It is not safe to run the closure cleaner on slaves.  #2153 introduced this which broke all UDF execution on slaves.  Will re-add cleaning of UDF closures in a follow-up PR.

Author: Michael Armbrust <michael@databricks.com>

Closes #2174 from marmbrus/fixUdfs and squashes the following commits:

55406de [Michael Armbrust] [HOTFIX] Remove cleaning of UDFs
2014-08-27 23:05:34 -07:00
Michael Armbrust 7d2a7a91f2 [SPARK-3235][SQL] Ensure in-memory tables don't always broadcast.
Author: Michael Armbrust <michael@databricks.com>

Closes #2147 from marmbrus/inMemDefaultSize and squashes the following commits:

5390360 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into inMemDefaultSize
14204d3 [Michael Armbrust] Set the context before creating SparkLogicalPlans.
8da4414 [Michael Armbrust] Make sure we throw errors when leaf nodes fail to provide statistcs
18ce029 [Michael Armbrust] Ensure in-memory tables don't always broadcast.
2014-08-27 15:14:08 -07:00
Cheng Hao 4238c17dc9 [SPARK-3197] [SQL] Reduce the Expression tree object creations for aggregation function (min/max)
Aggregation function min/max in catalyst will create expression tree for each single row, however, the expression tree creation is quite expensive in a multithreading env currently. Hence we got a very bad performance for the min/max.
Here is the benchmark that I've done in my local.

Master | Previous Result (ms) | Current Result (ms)
------------ | ------------- | -------------
local | 3645 | 3416
local[6] | 3602 | 1002

The Benchmark source code.
```
case class Record(key: Int, value: Int)

object TestHive2 extends HiveContext(new SparkContext("local[6]", "TestSQLContext", new SparkConf()))

object DataPrepare extends App {
  import TestHive2._

  val rdd = sparkContext.parallelize((1 to 10000000).map(i => Record(i % 3000, i)), 12)

  runSqlHive("SHOW TABLES")
  runSqlHive("DROP TABLE if exists a")
  runSqlHive("DROP TABLE if exists result")
  rdd.registerAsTable("records")

  runSqlHive("""CREATE TABLE a (key INT, value INT)
                 | ROW FORMAT SERDE
                 | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'
                 | STORED AS RCFILE
               """.stripMargin)
  runSqlHive("""CREATE TABLE result (key INT, value INT)
                 | ROW FORMAT SERDE
                 | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'
                 | STORED AS RCFILE
               """.stripMargin)

  hql(s"""from records
             | insert into table a
             | select key, value
           """.stripMargin)
}

object PerformanceTest extends App {
  import TestHive2._

  hql("SHOW TABLES")
  hql("set spark.sql.shuffle.partitions=12")

  val cmd = "select min(value), max(value) from a group by key"

  val results = ("Result1", benchmark(cmd)) ::
                ("Result2", benchmark(cmd)) ::
                ("Result3", benchmark(cmd)) :: Nil
  results.foreach { case (prompt, result) => {
      println(s"$prompt: took ${result._1} ms (${result._2} records)")
    }
  }

  def benchmark(cmd: String) = {
    val begin = System.currentTimeMillis()
    val count = hql(cmd).count
    val end = System.currentTimeMillis()
    ((end - begin), count)
  }
}
```

Author: Cheng Hao <hao.cheng@intel.com>

Closes #2113 from chenghao-intel/aggregation_expression_optimization and squashes the following commits:

db40395 [Cheng Hao] remove the transient and add val for the expression property
d56167d [Cheng Hao] Reduce the Expressions creation
2014-08-27 12:50:47 -07:00
Michael Armbrust e1139dd60e [SPARK-3237][SQL] Fix parquet filters with UDFs
Author: Michael Armbrust <michael@databricks.com>

Closes #2153 from marmbrus/parquetFilters and squashes the following commits:

712731a [Michael Armbrust] Use closure serializer for sending filters.
1e83f80 [Michael Armbrust] Clean udf functions.
2014-08-27 00:59:23 -07:00
Michael Armbrust c4787a3690 [SPARK-3194][SQL] Add AttributeSet to fix bugs with invalid comparisons of AttributeReferences
It is common to want to describe sets of attributes that are in various parts of a query plan.  However, the semantics of putting `AttributeReference` objects into a standard Scala `Set` result in subtle bugs when references differ cosmetically.  For example, with case insensitive resolution it is possible to have two references to the same attribute whose names are not equal.

In this PR I introduce a new abstraction, an `AttributeSet`, which performs all comparisons using the globally unique `ExpressionId` instead of case class equality.  (There is already a related class, [`AttributeMap`](https://github.com/marmbrus/spark/blob/inMemStats/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala#L32))  This new type of set is used to fix a bug in the optimizer where needed attributes were getting projected away underneath join operators.

I also took this opportunity to refactor the expression and query plan base classes.  In all but one instance the logic for computing the `references` of an `Expression` were the same.  Thus, I moved this logic into the base class.

For query plans the semantics of  the `references` method were ill defined (is it the references output? or is it those used by expression evaluation? or what?).  As a result, this method wasn't really used very much.  So, I removed it.

TODO:
 - [x] Finish scala doc for `AttributeSet`
 - [x] Scan the code for other instances of `Set[Attribute]` and refactor them.
 - [x] Finish removing `references` from `QueryPlan`

Author: Michael Armbrust <michael@databricks.com>

Closes #2109 from marmbrus/attributeSets and squashes the following commits:

1c0dae5 [Michael Armbrust] work on serialization bug.
9ba868d [Michael Armbrust] Merge remote-tracking branch 'origin/master' into attributeSets
3ae5288 [Michael Armbrust] review comments
40ce7f6 [Michael Armbrust] style
d577cc7 [Michael Armbrust] Scaladoc
cae5d22 [Michael Armbrust] remove more references implementations
d6e16be [Michael Armbrust] Remove more instances of "def references" and normal sets of attributes.
fc26b49 [Michael Armbrust] Add AttributeSet class, remove references from Expression.
2014-08-26 16:29:14 -07:00
Takuya UESHIN 98c2bb0bbd [SPARK-2969][SQL] Make ScalaReflection be able to handle ArrayType.containsNull and MapType.valueContainsNull.
Make `ScalaReflection` be able to handle like:

- `Seq[Int]` as `ArrayType(IntegerType, containsNull = false)`
- `Seq[java.lang.Integer]` as `ArrayType(IntegerType, containsNull = true)`
- `Map[Int, Long]` as `MapType(IntegerType, LongType, valueContainsNull = false)`
- `Map[Int, java.lang.Long]` as `MapType(IntegerType, LongType, valueContainsNull = true)`

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #1889 from ueshin/issues/SPARK-2969 and squashes the following commits:

24f1c5c [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Python API.
79f5b65 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Java API.
7cd1a7a [Takuya UESHIN] Fix json test failures.
2cfb862 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true.
2f38e61 [Takuya UESHIN] Revert the default value of MapTypes.valueContainsNull.
9fa02f5 [Takuya UESHIN] Fix a test failure.
1a9a96b [Takuya UESHIN] Modify ScalaReflection to handle ArrayType.containsNull and MapType.valueContainsNull.
2014-08-26 13:22:55 -07:00
Cheng Hao 156eb39661 [SPARK-3058] [SQL] Support EXTENDED for EXPLAIN
Provide `extended` keyword support for `explain` command in SQL. e.g.
```
explain extended select key as a1, value as a2 from src where key=1;
== Parsed Logical Plan ==
Project ['key AS a1#3,'value AS a2#4]
 Filter ('key = 1)
  UnresolvedRelation None, src, None

== Analyzed Logical Plan ==
Project [key#8 AS a1#3,value#9 AS a2#4]
 Filter (CAST(key#8, DoubleType) = CAST(1, DoubleType))
  MetastoreRelation default, src, None

== Optimized Logical Plan ==
Project [key#8 AS a1#3,value#9 AS a2#4]
 Filter (CAST(key#8, DoubleType) = 1.0)
  MetastoreRelation default, src, None

== Physical Plan ==
Project [key#8 AS a1#3,value#9 AS a2#4]
 Filter (CAST(key#8, DoubleType) = 1.0)
  HiveTableScan [key#8,value#9], (MetastoreRelation default, src, None), None

Code Generation: false
== RDD ==
(2) MappedRDD[14] at map at HiveContext.scala:350
  MapPartitionsRDD[13] at mapPartitions at basicOperators.scala:42
  MapPartitionsRDD[12] at mapPartitions at basicOperators.scala:57
  MapPartitionsRDD[11] at mapPartitions at TableReader.scala:112
  MappedRDD[10] at map at TableReader.scala:240
  HadoopRDD[9] at HadoopRDD at TableReader.scala:230
```

It's the sub task of #1847. But can go without any dependency.

Author: Cheng Hao <hao.cheng@intel.com>

Closes #1962 from chenghao-intel/explain_extended and squashes the following commits:

295db74 [Cheng Hao] Fix bug in printing the simple execution plan
48bc989 [Cheng Hao] Support EXTENDED for EXPLAIN
2014-08-25 17:43:56 -07:00
Takuya UESHIN d299e2bf2f [SPARK-3204][SQL] MaxOf would be foldable if both left and right are foldable.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #2116 from ueshin/issues/SPARK-3204 and squashes the following commits:

7d9b107 [Takuya UESHIN] Make MaxOf foldable if both left and right are foldable.
2014-08-25 16:27:00 -07:00
Michael Armbrust 7e191fe29b [SPARK-2554][SQL] CountDistinct partial aggregation and object allocation improvements
Author: Michael Armbrust <michael@databricks.com>
Author: Gregory Owen <greowen@gmail.com>

Closes #1935 from marmbrus/countDistinctPartial and squashes the following commits:

5c7848d [Michael Armbrust] turn off caching in the constructor
8074a80 [Michael Armbrust] fix tests
32d216f [Michael Armbrust] reynolds comments
c122cca [Michael Armbrust] Address comments, add tests
b2e8ef3 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial
fae38f4 [Michael Armbrust] Fix style
fdca896 [Michael Armbrust] cleanup
93d0f64 [Michael Armbrust] metastore concurrency fix.
db44a30 [Michael Armbrust] JIT hax.
3868f6c [Michael Armbrust] Merge pull request #9 from GregOwen/countDistinctPartial
c9e67de [Gregory Owen] Made SpecificRow and types serializable by Kryo
2b46c4b [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial
8ff6402 [Michael Armbrust] Add specific row.
58d15f1 [Michael Armbrust] disable codegen logging
87d101d [Michael Armbrust] Fix isNullAt bug
abee26d [Michael Armbrust] WIP
27984d0 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into countDistinctPartial
57ae3b1 [Michael Armbrust] Fix order dependent test
b3d0f64 [Michael Armbrust] Add golden files.
c1f7114 [Michael Armbrust] Improve tests / fix serialization.
f31b8ad [Michael Armbrust] more fixes
38c7449 [Michael Armbrust] comments and style
9153652 [Michael Armbrust] better toString
d494598 [Michael Armbrust] Fix tests now that the planner is better
41fbd1d [Michael Armbrust] Never try and create an empty hash set.
050bb97 [Michael Armbrust] Skip no-arg constructors for kryo,
bd08239 [Michael Armbrust] WIP
213ada8 [Michael Armbrust] First draft of partially aggregated and code generated count distinct / max
2014-08-23 16:19:10 -07:00
Michael Armbrust bfa09b01d7 [SQL] Improve debug logging and toStrings.
Author: Michael Armbrust <michael@databricks.com>

Closes #2004 from marmbrus/codgenDebugging and squashes the following commits:

b7a7e41 [Michael Armbrust] Improve debug logging and toStrings.
2014-08-17 19:00:38 -07:00
Sean Owen e1b85f3102 SPARK-2955 [BUILD] Test code fails to compile with "mvn compile" without "install"
(This is the corrected follow-up to https://issues.apache.org/jira/browse/SPARK-2903)

Right now, `mvn compile test-compile` fails to compile Spark. (Don't worry; `mvn package` works, so this is not major.) The issue stems from test code in some modules depending on test code in other modules. That is perfectly fine and supported by Maven.

It takes extra work to get this to work with scalatest, and this has been attempted: https://github.com/apache/spark/blob/master/sql/catalyst/pom.xml#L86

This formulation is not quite enough, since the SQL Core module's tests fail to compile for lack of finding test classes in SQL Catalyst, and likewise for most Streaming integration modules depending on core Streaming test code. Example:

```
[error] /Users/srowen/Documents/spark/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala:23: not found: type PlanTest
[error] class QueryTest extends PlanTest {
[error]                         ^
[error] /Users/srowen/Documents/spark/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala:28: package org.apache.spark.sql.test is not a value
[error]   test("SPARK-1669: cacheTable should be idempotent") {
[error]   ^
...
```

The issue I believe is that generation of a `test-jar` is bound here to the `compile` phase, but the test classes are not being compiled in this phase. It should bind to the `test-compile` phase.

It works when executing `mvn package` or `mvn install` since test-jar artifacts are actually generated available through normal Maven mechanisms as each module is built. They are then found normally, regardless of scalatest configuration.

It would be nice for a simple `mvn compile test-compile` to work since the test code is perfectly compilable given the Maven declarations.

On the plus side, this change is low-risk as it only affects tests.
yhuai made the original scalatest change and has glanced at this and thinks it makes sense.

Author: Sean Owen <srowen@gmail.com>

Closes #1879 from srowen/SPARK-2955 and squashes the following commits:

ad8242f [Sean Owen] Generate test-jar on test-compile for modules whose tests are needed by others' tests
2014-08-14 22:08:44 -07:00
Takuya UESHIN c686b7dd46 [SPARK-2968][SQL] Fix nullabilities of Explode.
Output nullabilities of `Explode` could be detemined by `ArrayType.containsNull` or `MapType.valueContainsNull`.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #1888 from ueshin/issues/SPARK-2968 and squashes the following commits:

d128c95 [Takuya UESHIN] Fix nullability of Explode.
2014-08-11 20:18:03 -07:00
Michael Armbrust 5a826c00c3 [SQL] Fix logging warn -> debug
Author: Michael Armbrust <michael@databricks.com>

Closes #1800 from marmbrus/warning and squashes the following commits:

8ea9cf1 [Michael Armbrust] [SQL] Fix logging warn -> debug.
2014-08-05 22:30:32 -07:00
Michael Armbrust 1d70c4f66d [SPARK-2866][SQL] Support attributes in ORDER BY that aren't in SELECT
Minor refactoring to allow resolution either using a nodes input or output.

Author: Michael Armbrust <michael@databricks.com>

Closes #1795 from marmbrus/ordering and squashes the following commits:

237f580 [Michael Armbrust] style
74d833b [Michael Armbrust] newline
705d963 [Michael Armbrust] Add a rule for resolving ORDER BY expressions that reference attributes not present in the SELECT clause.
82cabda [Michael Armbrust] Generalize attribute resolution.
2014-08-05 20:55:02 -07:00
Michael Armbrust 6e821e3d1a [SPARK-2860][SQL] Fix coercion of CASE WHEN.
Author: Michael Armbrust <michael@databricks.com>

Closes #1785 from marmbrus/caseNull and squashes the following commits:

126006d [Michael Armbrust] better error message
2fe357f [Michael Armbrust] Fix coercion of CASE WHEN.
2014-08-05 11:17:50 -07:00
Michael Armbrust 158ad0bba9 [SPARK-2097][SQL] UDF Support
This patch adds the ability to register lambda functions written in Python, Java or Scala as UDFs for use in SQL or HiveQL.

Scala:
```scala
registerFunction("strLenScala", (_: String).length)
sql("SELECT strLenScala('test')")
```
Python:
```python
sqlCtx.registerFunction("strLenPython", lambda x: len(x), IntegerType())
sqlCtx.sql("SELECT strLenPython('test')")
```
Java:
```java
sqlContext.registerFunction("stringLengthJava", new UDF1<String, Integer>() {
  Override
  public Integer call(String str) throws Exception {
    return str.length();
  }
}, DataType.IntegerType);

sqlContext.sql("SELECT stringLengthJava('test')");
```

Author: Michael Armbrust <michael@databricks.com>

Closes #1063 from marmbrus/udfs and squashes the following commits:

9eda0fe [Michael Armbrust] newline
747c05e [Michael Armbrust] Add some scala UDF tests.
d92727d [Michael Armbrust] Merge remote-tracking branch 'apache/master' into udfs
005d684 [Michael Armbrust] Fix naming and formatting.
d14dac8 [Michael Armbrust] Fix last line of autogened java files.
8135c48 [Michael Armbrust] Move UDF unit tests to pyspark.
40b0ffd [Michael Armbrust] Merge remote-tracking branch 'apache/master' into udfs
6a36890 [Michael Armbrust] Switch logging so that SQLContext can be serializable.
7a83101 [Michael Armbrust] Drop toString
795fd15 [Michael Armbrust] Try to avoid capturing SQLContext.
e54fb45 [Michael Armbrust] Docs and tests.
437cbe3 [Michael Armbrust] Update use of dataTypes, fix some python tests, address review comments.
01517d6 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udfs
8e6c932 [Michael Armbrust] WIP
3f96a52 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udfs
6237c8d [Michael Armbrust] WIP
2766f0b [Michael Armbrust] Move udfs support to SQL from hive. Add support for Java UDFs.
0f7d50c [Michael Armbrust] Draft of native Spark SQL UDFs for Scala and Python.
2014-08-02 16:33:48 -07:00
GuoQiang Li 4c477117bb SPARK-2804: Remove scalalogging-slf4j dependency
This also Closes #1701.

Author: GuoQiang Li <witgo@qq.com>

Closes #1208 from witgo/SPARK-1470 and squashes the following commits:

422646b [GuoQiang Li] Remove scalalogging-slf4j dependency
2014-08-02 13:59:58 -07:00
Anand Avati 08c095b664 [SPARK-1812] sql/catalyst - Provide explicit type information
For Scala 2.11 compatibility.

Without the explicit type specification, withNullability
return type is inferred to be Attribute, and thus calling
at() on the returned object fails in these tests:

[ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:370: value at is not a
[ERROR]     val c4_notNull = 'a.boolean.notNull.at(3)
[ERROR]                                         ^
[ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:371: value at is not a
[ERROR]     val c5_notNull = 'a.boolean.notNull.at(4)
[ERROR]                                         ^
[ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:372: value at is not a
[ERROR]     val c6_notNull = 'a.boolean.notNull.at(5)
[ERROR]                                         ^
[ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:558: value at is not a
[ERROR]     val s_notNull = 'a.string.notNull.at(0)

Signed-off-by: Anand Avati <avatiredhat.com>

Author: Anand Avati <avati@redhat.com>

Closes #1709 from avati/SPARK-1812-notnull and squashes the following commits:

0470eb3 [Anand Avati] SPARK-1812: sql/catalyst - Provide explicit type information
2014-08-02 00:48:17 -07:00
Patrick Wendell dab37966b0 Revert "[SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api"
This reverts commit adc8303294.
2014-08-01 23:55:30 -07:00
GuoQiang Li adc8303294 [SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the directly sfl4j api
Author: GuoQiang Li <witgo@qq.com>

Closes #1369 from witgo/SPARK-1470_new and squashes the following commits:

66a1641 [GuoQiang Li] IncompatibleResultTypeProblem
73a89ba [GuoQiang Li] Use the scala-logging wrapper instead of the directly sfl4j api.
2014-08-01 23:55:11 -07:00
GuoQiang Li 9998efab96 SPARK-2766: ScalaReflectionSuite throw an llegalArgumentException in JDK 6
Author: GuoQiang Li <witgo@qq.com>

Closes #1683 from witgo/SPARK-2766 and squashes the following commits:

d0db00c [GuoQiang Li] ScalaReflectionSuite  throw an llegalArgumentException in JDK 6
2014-07-31 21:06:57 -07:00
Yin Huai 9632719c9e [SPARK-2779] [SQL] asInstanceOf[Map[...]] should use scala.collection.Map instead of scala.collection.immutable.Map
Since we let users create Rows. It makes sense to accept mutable Maps as values of MapType columns.

JIRA: https://issues.apache.org/jira/browse/SPARK-2779

Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #1705 from yhuai/SPARK-2779 and squashes the following commits:

00d72fd [Yin Huai] Use scala.collection.Map.
2014-07-31 21:02:11 -07:00
Yin Huai 7003c163db [SPARK-2179][SQL] Public API for DataTypes and Schema
The current PR contains the following changes:
* Expose `DataType`s in the sql package (internal details are private to sql).
* Users can create Rows.
* Introduce `applySchema` to create a `SchemaRDD` by applying a `schema: StructType` to an `RDD[Row]`.
* Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`.
* `ScalaReflection.typeOfObject` provides a way to infer the Catalyst data type based on an object. Also, we can compose `typeOfObject` with some custom logics to form a new function to infer the data type (for different use cases).
* `JsonRDD` has been refactored to use changes introduced by this PR.
* Add a field `containsNull` to `ArrayType`. So, we can explicitly mark if an `ArrayType` can contain null values. The default value of `containsNull` is `false`.

New APIs are introduced in the sql package object and SQLContext. You can find the scaladoc at
[sql package object](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.package) and [SQLContext](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.SQLContext).

An example of using `applySchema` is shown below.
```scala
import org.apache.spark.sql._
val sqlContext = new org.apache.spark.sql.SQLContext(sc)

val schema =
  StructType(
    StructField("name", StringType, false) ::
    StructField("age", IntegerType, true) :: Nil)

val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Row(p(0), p(1).trim.toInt))
val peopleSchemaRDD = sqlContext. applySchema(people, schema)
peopleSchemaRDD.printSchema
// root
// |-- name: string (nullable = false)
// |-- age: integer (nullable = true)

peopleSchemaRDD.registerAsTable("people")
sqlContext.sql("select name from people").collect.foreach(println)
```

I will add new contents to the SQL programming guide later.

JIRA: https://issues.apache.org/jira/browse/SPARK-2179

Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #1346 from yhuai/dataTypeAndSchema and squashes the following commits:

1d45977 [Yin Huai] Clean up.
a6e08b4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
c712fbf [Yin Huai] Converts types of values based on defined schema.
4ceeb66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
e5f8df5 [Yin Huai] Scaladoc.
122d1e7 [Yin Huai] Address comments.
03bfd95 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
2476ed0 [Yin Huai] Minor updates.
ab71f21 [Yin Huai] Format.
fc2bed1 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
bd40a33 [Yin Huai] Address comments.
991f860 [Yin Huai] Move "asJavaDataType" and "asScalaDataType" to DataTypeConversions.scala.
1cb35fe [Yin Huai] Add "valueContainsNull" to MapType.
3edb3ae [Yin Huai] Python doc.
692c0b9 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
1d93395 [Yin Huai] Python APIs.
246da96 [Yin Huai] Add java data type APIs to javadoc index.
1db9531 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
d48fc7b [Yin Huai] Minor updates.
33c4fec [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
b9f3071 [Yin Huai] Java API for applySchema.
1c9f33c [Yin Huai] Java APIs for DataTypes and Row.
624765c [Yin Huai] Tests for applySchema.
aa92e84 [Yin Huai] Update data type tests.
8da1a17 [Yin Huai] Add Row.fromSeq.
9c99bc0 [Yin Huai] Several minor updates.
1d9c13a [Yin Huai] Update applySchema API.
85e9b51 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
e495e4e [Yin Huai] More comments.
42d47a3 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
c3f4a02 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
2e58dbd [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
b8b7db4 [Yin Huai] 1. Move sql package object and package-info to sql-core. 2. Minor updates on APIs. 3. Update scala doc.
68525a2 [Yin Huai] Update JSON unit test.
3209108 [Yin Huai] Add unit tests.
dcaf22f [Yin Huai] Add a field containsNull to ArrayType to indicate if an array can contain null values or not. If an ArrayType is constructed by "ArrayType(elementType)" (the existing constructor), the value of containsNull is false.
9168b83 [Yin Huai] Update comments.
fc649d7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
eca7d04 [Yin Huai] Add two apply methods which will be used to extract StructField(s) from a StructType.
949d6bb [Yin Huai] When creating a SchemaRDD for a JSON dataset, users can apply an existing schema.
7a6a7e5 [Yin Huai] Fix bug introduced by the change made on SQLContext.inferSchema.
43a45e1 [Yin Huai] Remove sql.util.package introduced in a previous commit.
0266761 [Yin Huai] Format
03eec4c [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema
90460ac [Yin Huai] Infer the Catalyst data type from an object and cast a data value to the expected type.
3fa0df5 [Yin Huai] Provide easier ways to construct a StructType.
16be3e5 [Yin Huai] This commit contains three changes: * Expose `DataType`s in the sql package (internal details are private to sql). * Introduce `createSchemaRDD` to create a `SchemaRDD` from an `RDD` with a provided schema (represented by a `StructType`) and a provided function to construct `Row`, * Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`.
2014-07-30 00:15:31 -07:00
Michael Armbrust 84467468d4 [SPARK-2054][SQL] Code Generation for Expression Evaluation
Adds a new method for evaluating expressions using code that is generated though Scala reflection.  This functionality is configured by the SQLConf option `spark.sql.codegen` and is currently turned off by default.

Evaluation can be done in several specialized ways:
 - *Projection* - Given an input row, produce a new row from a set of expressions that define each column in terms of the input row.  This can either produce a new Row object or perform the projection in-place on an existing Row (MutableProjection).
 - *Ordering* - Compares two rows based on a list of `SortOrder` expressions
 - *Condition* - Returns `true` or `false` given an input row.

For each of the above operations there is both a Generated and Interpreted version.  When generation for a given expression type is undefined, the code generator falls back on calling the `eval` function of the expression class.  Even without custom code, there is still a potential speed up, as loops are unrolled and code can still be inlined by JIT.

This PR also contains a new type of Aggregation operator, `GeneratedAggregate`, that performs aggregation by using generated `Projection` code.  Currently the required expression rewriting only works for simple aggregations like `SUM` and `COUNT`.  This functionality will be extended in a future PR.

This PR also performs several clean ups that simplified the implementation:
 - The notion of `Binding` all expressions in a tree automatically before query execution has been removed.  Instead it is the responsibly of an operator to provide the input schema when creating one of the specialized evaluators defined above.  In cases when the standard eval method is going to be called, binding can still be done manually using `BindReferences`.  There are a few reasons for this change:  First, there were many operators where it just didn't work before.  For example, operators with more than one child, and operators like aggregation that do significant rewriting of the expression. Second, the semantics of equality with `BoundReferences` are broken.  Specifically, we have had a few bugs where partitioning breaks because of the binding.
 - A copy of the current `SQLContext` is automatically propagated to all `SparkPlan` nodes by the query planner.  Before this was done ad-hoc for the nodes that needed this.  However, this required a lot of boilerplate as one had to always remember to make it `transient` and also had to modify the `otherCopyArgs`.

Author: Michael Armbrust <michael@databricks.com>

Closes #993 from marmbrus/newCodeGen and squashes the following commits:

96ef82c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into newCodeGen
f34122d [Michael Armbrust] Merge remote-tracking branch 'apache/master' into newCodeGen
67b1c48 [Michael Armbrust] Use conf variable in SQLConf object
4bdc42c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen
41a40c9 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen
de22aac [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen
fed3634 [Michael Armbrust] Inspectors are not serializable.
ef8d42b [Michael Armbrust] comments
533fdfd [Michael Armbrust] More logging of expression rewriting for GeneratedAggregate.
3cd773e [Michael Armbrust] Allow codegen for Generate.
64b2ee1 [Michael Armbrust] Implement copy
3587460 [Michael Armbrust] Drop unused string builder function.
9cce346 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen
1a61293 [Michael Armbrust] Address review comments.
0672e8a [Michael Armbrust] Address comments.
1ec2d6e [Michael Armbrust] Address comments
033abc6 [Michael Armbrust] off by default
4771fab [Michael Armbrust] Docs, more test coverage.
d30fee2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen
d2ad5c5 [Michael Armbrust] Refactor putting SQLContext into SparkPlan. Fix ordering, other test cases.
be2cd6b [Michael Armbrust] WIP: Remove old method for reference binding, more work on configuration.
bc88ecd [Michael Armbrust] Style
6cc97ca [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen
4220f1e [Michael Armbrust] Better config, docs, etc.
ca6cc6b [Michael Armbrust] WIP
9d67d85 [Michael Armbrust] Fix hive planner
fc522d5 [Michael Armbrust] Hook generated aggregation in to the planner.
e742640 [Michael Armbrust] Remove unneeded changes and code.
675e679 [Michael Armbrust] Upgrade paradise.
0093376 [Michael Armbrust] Comment / indenting cleanup.
d81f998 [Michael Armbrust] include schema for binding.
0e889e8 [Michael Armbrust] Use typeOf instead tq
f623ffd [Michael Armbrust] Quiet logging from test suite.
efad14f [Michael Armbrust] Remove some half finished functions.
92e74a4 [Michael Armbrust] add overrides
a2b5408 [Michael Armbrust] WIP: Code generation with scala reflection.
2014-07-29 20:58:05 -07:00
Michael Armbrust 39b8193102 [SPARK-2716][SQL] Don't check resolved for having filters.
For queries like `... HAVING COUNT(*) > 9` the expression is always resolved since it contains no attributes.  This was causing us to avoid doing the Having clause aggregation rewrite.

Author: Michael Armbrust <michael@databricks.com>

Closes #1640 from marmbrus/havingNoRef and squashes the following commits:

92d3901 [Michael Armbrust] Don't check resolved for having filters.
2014-07-29 18:14:20 -07:00
Zongheng Yang c7db274be7 [SPARK-2393][SQL] Cost estimation optimization framework for Catalyst logical plans & sample usage.
The idea is that every Catalyst logical plan gets hold of a Statistics class, the usage of which provides useful estimations on various statistics. See the implementations of `MetastoreRelation`.

This patch also includes several usages of the estimation interface in the planner. For instance, we now use physical table sizes from the estimate interface to convert an equi-join to a broadcast join (when doing so is beneficial, as determined by a size threshold).

Finally, there are a couple minor accompanying changes including:
- Remove the not-in-use `BaseRelation`.
- Make SparkLogicalPlan take a `SQLContext` in the second param list.

Author: Zongheng Yang <zongheng.y@gmail.com>

Closes #1238 from concretevitamin/estimates and squashes the following commits:

329071d [Zongheng Yang] Address review comments; turn config name from string to field in SQLConf.
8663e84 [Zongheng Yang] Use BigInt for stat; for logical leaves, by default throw an exception.
2f2fb89 [Zongheng Yang] Fix statistics for SparkLogicalPlan.
9951305 [Zongheng Yang] Remove childrenStats.
16fc60a [Zongheng Yang] Avoid calling statistics on plans if auto join conversion is disabled.
8bd2816 [Zongheng Yang] Add a note on performance of statistics.
6e594b8 [Zongheng Yang] Get size info from metastore for MetastoreRelation.
01b7a3e [Zongheng Yang] Update scaladoc for a field and move it to @param section.
549061c [Zongheng Yang] Remove numTuples in Statistics for now.
729a8e2 [Zongheng Yang] Update docs to be more explicit.
573e644 [Zongheng Yang] Remove singleton SQLConf and move back `settings` to the trait.
2d99eb5 [Zongheng Yang] {Cleanup, use synchronized in, enrich} StatisticsSuite.
ca5b825 [Zongheng Yang] Inject SQLContext into SparkLogicalPlan, removing SQLConf mixin from it.
43d38a6 [Zongheng Yang] Revert optimization for BroadcastNestedLoopJoin (this fixes tests).
0ef9e5b [Zongheng Yang] Use multiplication instead of sum for default estimates.
4ef0d26 [Zongheng Yang] Make Statistics a case class.
3ba8f3e [Zongheng Yang] Add comment.
e5bcf5b [Zongheng Yang] Fix optimization conditions & update scala docs to explain.
7d9216a [Zongheng Yang] Apply estimation to planning ShuffleHashJoin & BroadcastNestedLoopJoin.
73cde01 [Zongheng Yang] Move SQLConf back. Assign default sizeInBytes to SparkLogicalPlan.
73412be [Zongheng Yang] Move SQLConf to Catalyst & add default val for sizeInBytes.
7a60ab7 [Zongheng Yang] s/Estimates/Statistics, s/cardinality/numTuples.
de3ae13 [Zongheng Yang] Add parquetAfter() properly in test.
dcff9bd [Zongheng Yang] Cleanups.
84301a4 [Zongheng Yang] Refactors.
5bf5586 [Zongheng Yang] Typo.
56a8e6e [Zongheng Yang] Prototype impl of estimations for Catalyst logical plans.
2014-07-29 15:32:50 -07:00
Yin Huai e3643485de [SPARK-2730][SQL] When retrieving a value from a Map, GetItem evaluates key twice
JIRA: https://issues.apache.org/jira/browse/SPARK-2730

Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #1637 from yhuai/SPARK-2730 and squashes the following commits:

1a9f24e [Yin Huai] Remove unnecessary key evaluation.
2014-07-29 12:23:34 -07:00
Aaron Staple fc4d057000 Minor indentation and comment typo fixes.
Author: Aaron Staple <astaple@gmail.com>

Closes #1630 from staple/minor and squashes the following commits:

6f295a2 [Aaron Staple] Fix typos in comment about ExprId.
8566467 [Aaron Staple] Fix off by one column indentation in SqlParser.
2014-07-29 01:35:26 -07:00