Commit graph

500 commits

Author SHA1 Message Date
q00251598 9d3a75ef80 [SPARK-5606][SQL] Support plus sign in HiveContext
Now spark version is only support ```SELECT -key FROM DECIMAL_UDF;``` in HiveContext.
This patch is used to support ```SELECT +key FROM DECIMAL_UDF;``` in HiveContext.

Author: q00251598 <qiyadong@huawei.com>

Closes #4378 from watermen/SPARK-5606 and squashes the following commits:

777f132 [q00251598] sql-case22
74dd368 [q00251598] sql-case22
1a67410 [q00251598] sql-case22
c5cd5bc [q00251598] sql-case22
2015-02-04 23:16:01 -08:00
guowei2 e0490e271d [SPARK-5118][SQL] Fix: create table test stored as parquet as select ..
Author: guowei2 <guowei2@asiainfo.com>

Closes #3921 from guowei2/SPARK-5118 and squashes the following commits:

b1ba3be [guowei2] add table file check in test case
9da56f8 [guowei2] test case only run in Shim13
112a0b6 [guowei2] add test case
187c7d8 [guowei2] Fix: create table test stored as parquet as select ..
2015-02-04 15:26:10 -08:00
Yin Huai 548c9c2b2a [SQL] Use HiveContext's sessionState in HiveMetastoreCatalog.hiveDefaultTableFilePath
`client.getDatabaseCurrent` uses SessionState's local variable which can be an issue.

Author: Yin Huai <yhuai@databricks.com>

Closes #4355 from yhuai/defaultTablePath and squashes the following commits:

84a29e5 [Yin Huai] Use HiveContext's sessionState instead of using SessionState's thread local variable.
2015-02-04 15:22:40 -08:00
wangfei 417d1118cd [SPARK-5367][SQL] Support star expression in udfs
A follow up for #4163: support  `select array(key, *) from src`

Since  array(key, *)  will not go into this case
```
case Alias(f  UnresolvedFunction(_, args), name) if containsStar(args) =>
              val expandedArgs = args.flatMap {
                case s: Star => s.expand(child.output, resolver)
                case o => o :: Nil
              }
```
here added a case to cover the corner case of array.

/cc liancheng

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #4353 from scwf/udf-star1 and squashes the following commits:

4350d17 [wangfei] minor fix
a7cd191 [wangfei] minor fix
0942fb1 [wangfei] follow up: support select array(key, *) from src
6ae00db [wangfei] also fix problem with array
da1da09 [scwf] minor fix
f87b5f9 [scwf] added test case
587bf7e [wangfei] compile fix
eb93c16 [wangfei] fix star resolve issue in udf
2015-02-04 15:12:07 -08:00
wangfei b90dd39793 [SPARK-5587][SQL] Support change database owner
Support change database owner, here i do not add the golden files since the golden answer is related to the tmp dir path (see 6331e4ac0f)

Author: wangfei <wangfei1@huawei.com>

Closes #4357 from scwf/db_owner and squashes the following commits:

f761533 [wangfei] remove the alter_db_owner which have added to whitelist
79413c6 [wangfei] Revert "added golden files"
6331e4a [wangfei] added golden files
6f7cacd [wangfei] support change database owner
2015-02-04 14:35:12 -08:00
wangfei a9f0db1fb4 [SPARK-5591][SQL] Fix NoSuchObjectException for CTAS
Now CTAS runs successfully but will throw a NoSuchObjectException.
```
create table sc as select *
from (select '2011-01-11', '2011-01-11+14:18:26' from src tablesample (1 rows)
union all
select '2011-01-11', '2011-01-11+15:18:26' from src tablesample (1 rows)
union all
select '2011-01-11', '2011-01-11+16:18:26' from src tablesample (1 rows) ) s;
```
Get this exception:
ERROR Hive: NoSuchObjectException(message:default.sc table not found)
at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.get_table(HiveMetaStore.java:1560)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:601)
at org.apache.hadoop.hive.metastore.RetryingHMSHandler.invoke(RetryingHMSHandler.java:105)
at $Proxy8.get_table(Unknown Source)
at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.getTable(HiveMetaStoreClient.java:997)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:601)
at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:89)
at $Proxy9.getTable(Unknown Source)
at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:976)
at org.apache.hadoop.hive.ql.metadata.Hive.getTable(Hive.java:950)
at org.apache.spark.sql.hive.HiveMetastoreCatalog.tableExists(HiveMetastoreCatalog.scala:152)
at org.apache.spark.sql.hive.HiveContext$$anon$2.org$apache$spark$sql$catalyst$analysis$OverrideCatalog$$super$tableExists(HiveContext.scala:309)
at org.apache.spark.sql.catalyst.analysis.OverrideCatalog$class.tableExists(Catalog.scala:121)
at org.apache.spark.sql.hive.HiveContext$$anon$2.tableExists(HiveContext.scala:309)
at org.apache.spark.sql.hive.execution.CreateTableAsSelect.run(CreateTableAsSelect.scala:63)
at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:53)

Author: wangfei <wangfei1@huawei.com>

Closes #4365 from scwf/ctas-exception and squashes the following commits:

c7c67bc [wangfei] no used imports
f54eb2a [wangfei] fix exception for CTAS
2015-02-04 14:33:07 -08:00
Daoyuan Wang db821ed2ed [SPARK-4508] [SQL] build native date type to conform behavior to Hive
The previous #3732 is reverted due to some test failure.
Have fixed that.

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

Closes #4325 from adrian-wang/datenative and squashes the following commits:

096e20d [Daoyuan Wang] fix for mixed timezone
0ed0fdc [Daoyuan Wang] fix test data
a2fdd4e [Daoyuan Wang] getDate
c37832b [Daoyuan Wang] row to catalyst
f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion
024c9a6 [Daoyuan Wang] clean some import order
d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally
374abd5 [Daoyuan Wang] spark native date type support
2015-02-03 12:21:45 -08:00
wangfei 5adbb39482 [SPARK-5383][SQL] Support alias for udtfs
Add support for alias of udtfs, such as
```
select stack(2, key, value, key, value) as (a, b) from src limit 5;

select a, b from (select stack(2, key, value, key, value) as (a, b) from src) t limit 5

```

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>
Author: Fei Wang <wangfei1@huawei.com>

Closes #4186 from scwf/multi-alias-names and squashes the following commits:

c35e922 [wangfei] fix conflicts
adc8311 [wangfei] minor format fix
2783aed [wangfei] convert it to a Generate instead of leaving it inside of a Project clause
a87668a [wangfei] minor improvement
b25d9b3 [wangfei] resolve conflicts
d38f041 [wangfei] style fix
8cfcebf [wangfei] minor improvement
12a239e [wangfei] fix test case
050177f [wangfei] added extendedCheckRules
3d69329 [wangfei] added CheckMultiAlias to analyzer
324150d [wangfei] added multi alias node
74f5a81 [Fei Wang] imports order fix
5bc3f59 [scwf] style fix
3daec28 [scwf] support alias for udfs with multi output columns
2015-02-03 12:16:31 -08:00
Cheng Hao ca7a6cdff0 [SPARK-5550] [SQL] Support the case insensitive for UDF
SQL in HiveContext, should be case insensitive, however, the following query will fail.

```scala
udf.register("random0", ()  => { Math.random()})
assert(sql("SELECT RANDOM0() FROM src LIMIT 1").head().getDouble(0) >= 0.0)
```

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

Closes #4326 from chenghao-intel/udf_case_sensitive and squashes the following commits:

485cf66 [Cheng Hao] Support the case insensitive for UDF
2015-02-03 12:12:26 -08:00
Yin Huai 13531dd97c [SPARK-5501][SPARK-5420][SQL] Write support for the data source API
This PR aims to support `INSERT INTO/OVERWRITE TABLE tableName` and `CREATE TABLE tableName AS SELECT` for the data source API (partitioned tables are not supported).

In this PR, I am also adding the support of `IF NOT EXISTS` for our ddl parser. The current semantic of `IF NOT EXISTS` is explained as follows.
* For a `CREATE TEMPORARY TABLE` statement, it does not `IF NOT EXISTS` for now.
* For a `CREATE TABLE` statement (we are creating a metastore table), if there is an existing table having the same name ...
  * when `IF NOT EXISTS` clause is used, we will do nothing.
  * when `IF NOT EXISTS` clause is not used, the user will see an exception saying the table already exists.

TODOs:
- [x] CTAS support
- [x] Programmatic APIs
- [ ] Python API (another PR)
- [x] More unit tests
- [ ] Documents (another PR)

marmbrus liancheng rxin

Author: Yin Huai <yhuai@databricks.com>

Closes #4294 from yhuai/writeSupport and squashes the following commits:

3db1539 [Yin Huai] save does not take overwrite.
1c98881 [Yin Huai] Fix test.
142372a [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupport
34e1bfb [Yin Huai] Address comments.
1682ca6 [Yin Huai] Better support for CTAS statements.
e789d64 [Yin Huai] For the Scala API, let users to use tuples to provide options.
0128065 [Yin Huai] Short hand versions of save and load.
66ebd74 [Yin Huai] Formatting.
9203ec2 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupport
e5d29f2 [Yin Huai] Programmatic APIs.
1a719a5 [Yin Huai] CREATE TEMPORARY TABLE with IF NOT EXISTS is not allowed for now.
909924f [Yin Huai] Add saveAsTable for the data source API to DataFrame.
95a7c71 [Yin Huai] Fix bug when handling IF NOT EXISTS clause in a CREATE TEMPORARY TABLE statement.
d37b19c [Yin Huai] Cheng's comments.
fd6758c [Yin Huai] Use BeforeAndAfterAll.
7880891 [Yin Huai] Support CREATE TABLE AS SELECT STATEMENT and the IF NOT EXISTS clause.
cb85b05 [Yin Huai] Initial write support.
2f91354 [Yin Huai] Make INSERT OVERWRITE/INTO statements consistent between HiveQL and SqlParser.
2015-02-02 23:30:44 -08:00
Reynold Xin 554403fd91 [SQL] Improve DataFrame API error reporting
1. Throw UnsupportedOperationException if a Column is not computable.
2. Perform eager analysis on DataFrame so we can catch errors when they happen (not when an action is run).

Author: Reynold Xin <rxin@databricks.com>
Author: Davies Liu <davies@databricks.com>

Closes #4296 from rxin/col-computability and squashes the following commits:

6527b86 [Reynold Xin] Merge pull request #8 from davies/col-computability
fd92bc7 [Reynold Xin] Merge branch 'master' into col-computability
f79034c [Davies Liu] fix python tests
5afe1ff [Reynold Xin] Fix scala test.
17f6bae [Reynold Xin] Various fixes.
b932e86 [Reynold Xin] Added eager analysis for error reporting.
e6f00b8 [Reynold Xin] [SQL][API] ComputableColumn vs IncomputableColumn
2015-02-02 19:01:47 -08:00
Patrick Wendell eccb9fbb2d Revert "[SPARK-4508] [SQL] build native date type to conform behavior to Hive"
This reverts commit 1646f89d96.
2015-02-02 17:52:17 -08:00
seayi dca6faa29a [SPARK-5195][sql]Update HiveMetastoreCatalog.scala(override the MetastoreRelation's sameresult method only compare databasename and table name)
override  the MetastoreRelation's  sameresult method only compare databasename and table name

because in previous :
cache table t1;
select count(*) from t1;
it will read data from memory  but the sql below will not,instead it read from hdfs:
select count(*) from t1 t;

because cache data is keyed by logical plan and compare with sameResult ,so  when table with alias  the same table 's logicalplan is not the same logical plan with out alias  so modify  the sameresult method only compare databasename and table name

Author: seayi <405078363@qq.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #3898 from seayi/branch-1.2 and squashes the following commits:

8f0c7d2 [seayi] Update CachedTableSuite.scala
a277120 [seayi] Update HiveMetastoreCatalog.scala
8d910aa [seayi] Update HiveMetastoreCatalog.scala
2015-02-02 16:18:55 -08:00
Daoyuan Wang 1646f89d96 [SPARK-4508] [SQL] build native date type to conform behavior to Hive
Store daysSinceEpoch as an Int value(4 bytes) to represent DateType, instead of using java.sql.Date(8 bytes as Long) in catalyst row. This ensures the same comparison behavior of Hive and Catalyst.
Subsumes #3381
I thinks there are already some tests in JavaSQLSuite, and for python it will not affect python's datetime class.

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

Closes #3732 from adrian-wang/datenative and squashes the following commits:

0ed0fdc [Daoyuan Wang] fix test data
a2fdd4e [Daoyuan Wang] getDate
c37832b [Daoyuan Wang] row to catalyst
f0005b1 [Daoyuan Wang] add date in sql parser and java type conversion
024c9a6 [Daoyuan Wang] clean some import order
d6715fc [Daoyuan Wang] refactoring Date as Primitive Int internally
374abd5 [Daoyuan Wang] spark native date type support
2015-02-02 15:49:22 -08:00
Liang-Chi Hsieh 683e938242 [SPARK-5212][SQL] Add support of schema-less, custom field delimiter and SerDe for HiveQL transform
This pr adds the support of schema-less syntax, custom field delimiter and SerDe for HiveQL's transform.

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

Closes #4014 from viirya/schema_less_trans and squashes the following commits:

ac2d1fe [Liang-Chi Hsieh] Refactor codes for comments.
a137933 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into schema_less_trans
aa10fbd [Liang-Chi Hsieh] Add Hive golden answer files again.
575f695 [Liang-Chi Hsieh] Add Hive golden answer files for new unit tests.
a422562 [Liang-Chi Hsieh] Use createQueryTest for unit tests and remove unnecessary imports.
ccb71e3 [Liang-Chi Hsieh] Refactor codes for comments.
37bd391 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into schema_less_trans
6000889 [Liang-Chi Hsieh] Wrap input and output schema into ScriptInputOutputSchema.
21727f7 [Liang-Chi Hsieh] Move schema-less output to proper place. Use multilines instead of a long line SQL.
9a6dc04 [Liang-Chi Hsieh] setRecordReaderID is introduced in 0.13.1, use reflection API to call it.
7a14f31 [Liang-Chi Hsieh] Fix bug.
799b5e1 [Liang-Chi Hsieh] Call getSerializedClass instead of using Text.
be2c3fc [Liang-Chi Hsieh] Fix style.
32d3046 [Liang-Chi Hsieh] Add SerDe support.
ab22f7b [Liang-Chi Hsieh] Fix style.
7a48e42 [Liang-Chi Hsieh] Add support of custom field delimiter.
b1729d9 [Liang-Chi Hsieh] Fix style.
ccee49e [Liang-Chi Hsieh] Add unit test.
f561c37 [Liang-Chi Hsieh] Add support of schema-less script transformation.
2015-02-02 13:53:55 -08:00
Daoyuan Wang 8cf4a1f02e [SPARK-5262] [SPARK-5244] [SQL] add coalesce in SQLParser and widen types for parameters of coalesce
I'll add test case in #4040

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

Closes #4057 from adrian-wang/coal and squashes the following commits:

4d0111a [Daoyuan Wang] address Yin's comments
c393e18 [Daoyuan Wang] fix rebase conflicts
e47c03a [Daoyuan Wang] add coalesce in parser
c74828d [Daoyuan Wang] cast types for coalesce
2015-02-01 18:51:38 -08:00
Reynold Xin 80def9deb3 [SQL] Support df("*") to select all columns in a data frame.
This PR makes Star a trait, and provides two implementations: UnresolvedStar (used for *, tblName.*) and ResolvedStar (used for df("*")).

Author: Reynold Xin <rxin@databricks.com>

Closes #4283 from rxin/df-star and squashes the following commits:

c9cba3e [Reynold Xin] Removed mapFunction in UnresolvedStar.
1a3a1d7 [Reynold Xin] [SQL] Support df("*") to select all columns in a data frame.
2015-01-29 19:09:08 -08:00
Yin Huai c00d517d66 [SPARK-4296][SQL] Trims aliases when resolving and checking aggregate expressions
I believe that SPARK-4296 has been fixed by 3684fd21e1. I am adding tests based #3910 (change the udf to HiveUDF instead).

Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Lian <lian@databricks.com>

Closes #4010 from yhuai/SPARK-4296-yin and squashes the following commits:

6343800 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-4296-yin
6cfadd2 [Yin Huai] Actually, this issue has been fixed by 3684fd21e1.
d42b707 [Yin Huai] Update comment.
8b3a274 [Yin Huai] Since expressions in grouping expressions can have aliases, which can be used by the outer query block,     revert this change.
443538d [Cheng Lian] Trims aliases when resolving and checking aggregate expressions
2015-01-29 15:49:34 -08:00
wangfei fbaf9e0896 [SPARK-5367][SQL] Support star expression in udf
now spark sql does not support star expression in udf, run the following sql by spark-sql will get error
```
select concat(*) from src
```

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #4163 from scwf/udf-star and squashes the following commits:

9db7b39 [wangfei] addressed comments
da1da09 [scwf] minor fix
f87b5f9 [scwf] added test case
587bf7e [wangfei] compile fix
eb93c16 [wangfei] fix star resolve issue in udf
2015-01-29 15:44:53 -08:00
Liang-Chi Hsieh bce0ba1fbd [SPARK-5429][SQL] Use javaXML plan serialization for Hive golden answers on Hive 0.13.1
I found that running `HiveComparisonTest.createQueryTest` to generate Hive golden answer files on Hive 0.13.1 would throw KryoException. I am not sure if this can be reproduced by others. Since Hive 0.13.0, Kryo plan serialization is introduced to replace javaXML as default plan serialization format. This is a quick fix to set hive configuration to use javaXML serialization.

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

Closes #4223 from viirya/fix_hivetest and squashes the following commits:

97a8760 [Liang-Chi Hsieh] Use javaXML plan serialization.
2015-01-29 15:28:22 -08:00
Reynold Xin 715632232d [SPARK-5445][SQL] Consolidate Java and Scala DSL static methods.
Turns out Scala does generate static methods for ones defined in a companion object. Finally no need to separate api.java.dsl and api.scala.dsl.

Author: Reynold Xin <rxin@databricks.com>

Closes #4276 from rxin/dsl and squashes the following commits:

30aa611 [Reynold Xin] Add all files.
1a9d215 [Reynold Xin] [SPARK-5445][SQL] Consolidate Java and Scala DSL static methods.
2015-01-29 15:13:09 -08:00
Reynold Xin 5b9760de8d [SPARK-5445][SQL] Made DataFrame dsl usable in Java
Also removed the literal implicit transformation since it is pretty scary for API design. Instead, created a new lit method for creating literals. This doesn't break anything from a compatibility perspective because Literal was added two days ago.

Author: Reynold Xin <rxin@databricks.com>

Closes #4241 from rxin/df-docupdate and squashes the following commits:

c0f4810 [Reynold Xin] Fix Python merge conflict.
094c7d7 [Reynold Xin] Minor style fix. Reset Python tests.
3c89f4a [Reynold Xin] Package.
dfe6962 [Reynold Xin] Updated Python aggregate.
5dd4265 [Reynold Xin] Made dsl Java callable.
14b3c27 [Reynold Xin] Fix literal expression for symbols.
68b31cb [Reynold Xin] Literal.
4cfeb78 [Reynold Xin] [SPARK-5097][SQL] Address DataFrame code review feedback.
2015-01-28 19:10:32 -08:00
Reynold Xin c8e934ef3c [SPARK-5447][SQL] Replaced reference to SchemaRDD with DataFrame.
and

[SPARK-5448][SQL] Make CacheManager a concrete class and field in SQLContext

Author: Reynold Xin <rxin@databricks.com>

Closes #4242 from rxin/sqlCleanup and squashes the following commits:

e351cb2 [Reynold Xin] Fixed toDataFrame.
6545c42 [Reynold Xin] More changes.
728c017 [Reynold Xin] [SPARK-5447][SQL] Replaced reference to SchemaRDD with DataFrame.
2015-01-28 12:10:01 -08:00
Reynold Xin 119f45d61d [SPARK-5097][SQL] DataFrame
This pull request redesigns the existing Spark SQL dsl, which already provides data frame like functionalities.

TODOs:
With the exception of Python support, other tasks can be done in separate, follow-up PRs.
- [ ] Audit of the API
- [ ] Documentation
- [ ] More test cases to cover the new API
- [x] Python support
- [ ] Type alias SchemaRDD

Author: Reynold Xin <rxin@databricks.com>
Author: Davies Liu <davies@databricks.com>

Closes #4173 from rxin/df1 and squashes the following commits:

0a1a73b [Reynold Xin] Merge branch 'df1' of github.com:rxin/spark into df1
23b4427 [Reynold Xin] Mima.
828f70d [Reynold Xin] Merge pull request #7 from davies/df
257b9e6 [Davies Liu] add repartition
6bf2b73 [Davies Liu] fix collect with UDT and tests
e971078 [Reynold Xin] Missing quotes.
b9306b4 [Reynold Xin] Remove removeColumn/updateColumn for now.
a728bf2 [Reynold Xin] Example rename.
e8aa3d3 [Reynold Xin] groupby -> groupBy.
9662c9e [Davies Liu] improve DataFrame Python API
4ae51ea [Davies Liu] python API for dataframe
1e5e454 [Reynold Xin] Fixed a bug with symbol conversion.
2ca74db [Reynold Xin] Couple minor fixes.
ea98ea1 [Reynold Xin] Documentation & literal expressions.
2b22684 [Reynold Xin] Got rid of IntelliJ problems.
02bbfbc [Reynold Xin] Tightening imports.
ffbce66 [Reynold Xin] Fixed compilation error.
59b6d8b [Reynold Xin] Style violation.
b85edfb [Reynold Xin] ALS.
8c37f0a [Reynold Xin] Made MLlib and examples compile
6d53134 [Reynold Xin] Hive module.
d35efd5 [Reynold Xin] Fixed compilation error.
ce4a5d2 [Reynold Xin] Fixed test cases in SQL except ParquetIOSuite.
66d5ef1 [Reynold Xin] SQLContext minor patch.
c9bcdc0 [Reynold Xin] Checkpoint: SQL module compiles!
2015-01-27 16:08:24 -08:00
Cheng Hao 27bccc5ea9 [SPARK-5202] [SQL] Add hql variable substitution support
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+VariableSubstitution

This is a block issue for the CLI user, it impacts the existed hql scripts from Hive.

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

Closes #4003 from chenghao-intel/substitution and squashes the following commits:

bb41fd6 [Cheng Hao] revert the removed the implicit conversion
af7c31a [Cheng Hao] add hql variable substitution support
2015-01-21 17:34:18 -08:00
Cheng Hao 8361078efa [SPARK-5009] [SQL] Long keyword support in SQL Parsers
* The `SqlLexical.allCaseVersions` will cause `StackOverflowException` if the key word is too long, the patch will fix that by normalizing all of the keywords in `SqlLexical`.
* And make a unified SparkSQLParser for sharing the common code.

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

Closes #3926 from chenghao-intel/long_keyword and squashes the following commits:

686660f [Cheng Hao] Support Long Keyword and Refactor the SQLParsers
2015-01-21 13:05:56 -08:00
Reynold Xin d181c2a1fc [SPARK-5323][SQL] Remove Row's Seq inheritance.
Author: Reynold Xin <rxin@databricks.com>

Closes #4115 from rxin/row-seq and squashes the following commits:

e33abd8 [Reynold Xin] Fixed compilation error.
cceb650 [Reynold Xin] Python test fixes, and removal of WrapDynamic.
0334a52 [Reynold Xin] mkString.
9cdeb7d [Reynold Xin] Hive tests.
15681c2 [Reynold Xin] Fix more test cases.
ea9023a [Reynold Xin] Fixed a catalyst test.
c5e2cb5 [Reynold Xin] Minor patch up.
b9cab7c [Reynold Xin] [SPARK-5323][SQL] Remove Row's Seq inheritance.
2015-01-20 15:16:14 -08:00
Yin Huai 2604bc35d7 [SPARK-5286][SQL] Fail to drop an invalid table when using the data source API
JIRA: https://issues.apache.org/jira/browse/SPARK-5286

Author: Yin Huai <yhuai@databricks.com>

Closes #4076 from yhuai/SPARK-5286 and squashes the following commits:

6b69ed1 [Yin Huai] Catch all exception when we try to uncache a query.
2015-01-19 10:45:29 -08:00
Yin Huai cd5da42853 [SPARK-5284][SQL] Insert into Hive throws NPE when a inner complex type field has a null value
JIRA: https://issues.apache.org/jira/browse/SPARK-5284

Author: Yin Huai <yhuai@databricks.com>

Closes #4077 from yhuai/SPARK-5284 and squashes the following commits:

fceacd6 [Yin Huai] Check if a value is null when the field has a complex type.
2015-01-19 10:44:12 -08:00
Reynold Xin 1727e0841c [SPARK-5279][SQL] Use java.math.BigDecimal as the exposed Decimal type.
Author: Reynold Xin <rxin@databricks.com>

Closes #4092 from rxin/bigdecimal and squashes the following commits:

27b08c9 [Reynold Xin] Fixed test.
10cb496 [Reynold Xin] [SPARK-5279][SQL] Use java.math.BigDecimal as the exposed Decimal type.
2015-01-18 11:01:42 -08:00
Reynold Xin 61b427d4b1 [SPARK-5193][SQL] Remove Spark SQL Java-specific API.
After the following patches, the main (Scala) API is now usable for Java users directly.

https://github.com/apache/spark/pull/4056
https://github.com/apache/spark/pull/4054
https://github.com/apache/spark/pull/4049
https://github.com/apache/spark/pull/4030
https://github.com/apache/spark/pull/3965
https://github.com/apache/spark/pull/3958

Author: Reynold Xin <rxin@databricks.com>

Closes #4065 from rxin/sql-java-api and squashes the following commits:

b1fd860 [Reynold Xin] Fix Mima
6d86578 [Reynold Xin] Ok one more attempt in fixing Python...
e8f1455 [Reynold Xin] Fix Python again...
3e53f91 [Reynold Xin] Fixed Python.
83735da [Reynold Xin] Fix BigDecimal test.
e9f1de3 [Reynold Xin] Use scala BigDecimal.
500d2c4 [Reynold Xin] Fix Decimal.
ba3bfa2 [Reynold Xin] Updated javadoc for RowFactory.
c4ae1c5 [Reynold Xin] [SPARK-5193][SQL] Remove Spark SQL Java-specific API.
2015-01-16 21:09:06 -08:00
Reynold Xin 1881431dd5 [SPARK-5274][SQL] Reconcile Java and Scala UDFRegistration.
As part of SPARK-5193:

1. Removed UDFRegistration as a mixin in SQLContext and made it a field ("udf").
2. For Java UDFs, renamed dataType to returnType.
3. For Scala UDFs, added type tags.
4. Added all Java UDF registration methods to Scala's UDFRegistration.
5. Documentation

Author: Reynold Xin <rxin@databricks.com>

Closes #4056 from rxin/udf-registration and squashes the following commits:

ae9c556 [Reynold Xin] Updated example.
675a3c9 [Reynold Xin] Style fix
47c24ff [Reynold Xin] Python fix.
5f00c45 [Reynold Xin] Restore data type position in java udf and added typetags.
032f006 [Reynold Xin] [SPARK-5193][SQL] Reconcile Java and Scala UDFRegistration.
2015-01-15 16:15:12 -08:00
Reynold Xin 4b325c77a2 [SPARK-5193][SQL] Tighten up HiveContext API
1. Removed the deprecated LocalHiveContext
2. Made private[sql] fields protected[sql] so they don't show up in javadoc.
3. Added javadoc to refreshTable.
4. Added Experimental tag to analyze command.

Author: Reynold Xin <rxin@databricks.com>

Closes #4054 from rxin/hivecontext-api and squashes the following commits:

25cc00a [Reynold Xin] Add implicit conversion back.
cbca886 [Reynold Xin] [SPARK-5193][SQL] Tighten up HiveContext API
2015-01-14 20:31:02 -08:00
Reynold Xin cfa397c126 [SPARK-5193][SQL] Tighten up SQLContext API
1. Removed 2 implicits (logicalPlanToSparkQuery and baseRelationToSchemaRDD)
2. Moved extraStrategies into ExperimentalMethods.
3. Made private methods protected[sql] so they don't show up in javadocs.
4. Removed createParquetFile.
5. Added Java version of applySchema to SQLContext.

Author: Reynold Xin <rxin@databricks.com>

Closes #4049 from rxin/sqlContext-refactor and squashes the following commits:

a326a1a [Reynold Xin] Remove createParquetFile and add applySchema for Java to SQLContext.
ecd6685 [Reynold Xin] Added baseRelationToSchemaRDD back.
4a38c9b [Reynold Xin] [SPARK-5193][SQL] Tighten up SQLContext API
2015-01-14 18:36:15 -08:00
Josh Rosen 259936be71 [SPARK-4014] Add TaskContext.attemptNumber and deprecate TaskContext.attemptId
`TaskContext.attemptId` is misleadingly-named, since it currently returns a taskId, which uniquely identifies a particular task attempt within a particular SparkContext, instead of an attempt number, which conveys how many times a task has been attempted.

This patch deprecates `TaskContext.attemptId` and add `TaskContext.taskId` and `TaskContext.attemptNumber` fields.  Prior to this change, it was impossible to determine whether a task was being re-attempted (or was a speculative copy), which made it difficult to write unit tests for tasks that fail on early attempts or speculative tasks that complete faster than original tasks.

Earlier versions of the TaskContext docs suggest that `attemptId` behaves like `attemptNumber`, so there's an argument to be made in favor of changing this method's implementation.  Since we've decided against making that change in maintenance branches, I think it's simpler to add better-named methods and retain the old behavior for `attemptId`; if `attemptId` behaved differently in different branches, then this would cause confusing build-breaks when backporting regression tests that rely on the new `attemptId` behavior.

Most of this patch is fairly straightforward, but there is a bit of trickiness related to Mesos tasks: since there's no field in MesosTaskInfo to encode the attemptId, I packed it into the `data` field alongside the task binary.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #3849 from JoshRosen/SPARK-4014 and squashes the following commits:

89d03e0 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
5cfff05 [Josh Rosen] Introduce wrapper for serializing Mesos task launch data.
38574d4 [Josh Rosen] attemptId -> taskAttemptId in PairRDDFunctions
a180b88 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
1d43aa6 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
eee6a45 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-4014
0b10526 [Josh Rosen] Use putInt instead of putLong (silly mistake)
8c387ce [Josh Rosen] Use local with maxRetries instead of local-cluster.
cbe4d76 [Josh Rosen] Preserve attemptId behavior and deprecate it:
b2dffa3 [Josh Rosen] Address some of Reynold's minor comments
9d8d4d1 [Josh Rosen] Doc typo
1e7a933 [Josh Rosen] [SPARK-4014] Change TaskContext.attemptId to return attempt number instead of task ID.
fd515a5 [Josh Rosen] Add failing test for SPARK-4014
2015-01-14 11:45:40 -08:00
Yin Huai 81f72a0df2 [SPARK-5211][SQL]Restore HiveMetastoreTypes.toDataType
jira: https://issues.apache.org/jira/browse/SPARK-5211

Author: Yin Huai <yhuai@databricks.com>

Closes #4026 from yhuai/SPARK-5211 and squashes the following commits:

15ee32b [Yin Huai] Remove extra line.
c6c1651 [Yin Huai] Get back HiveMetastoreTypes.toDataType.
2015-01-14 09:47:30 -08:00
Daoyuan Wang a3f7421b42 [SPARK-5248] [SQL] move sql.types.decimal.Decimal to sql.types.Decimal
rxin follow up of #3732

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

Closes #4041 from adrian-wang/decimal and squashes the following commits:

aa3d738 [Daoyuan Wang] fix auto refactor
7777a58 [Daoyuan Wang] move sql.types.decimal.Decimal to sql.types.Decimal
2015-01-14 09:36:59 -08:00
Reynold Xin f9969098c8 [SPARK-5123][SQL] Reconcile Java/Scala API for data types.
Having two versions of the data type APIs (one for Java, one for Scala) requires downstream libraries to also have two versions of the APIs if the library wants to support both Java and Scala. I took a look at the Scala version of the data type APIs - it can actually work out pretty well for Java out of the box.

As part of the PR, I created a sql.types package and moved all type definitions there. I then removed the Java specific data type API along with a lot of the conversion code.

This subsumes https://github.com/apache/spark/pull/3925

Author: Reynold Xin <rxin@databricks.com>

Closes #3958 from rxin/SPARK-5123-datatype-2 and squashes the following commits:

66505cc [Reynold Xin] [SPARK-5123] Expose only one version of the data type APIs (i.e. remove the Java-specific API).
2015-01-13 17:16:41 -08:00
Reynold Xin 14e3f114ef [SPARK-5168] Make SQLConf a field rather than mixin in SQLContext
This change should be binary and source backward compatible since we didn't change any user facing APIs.

Author: Reynold Xin <rxin@databricks.com>

Closes #3965 from rxin/SPARK-5168-sqlconf and squashes the following commits:

42eec09 [Reynold Xin] Fix default conf value.
0ef86cc [Reynold Xin] Fix constructor ordering.
4d7f910 [Reynold Xin] Properly override config.
ccc8e6a [Reynold Xin] [SPARK-5168] Make SQLConf a field rather than mixin in SQLContext
2015-01-13 13:30:35 -08:00
Yin Huai 6463e0b9e8 [SPARK-4912][SQL] Persistent tables for the Spark SQL data sources api
With changes in this PR, users can persist metadata of tables created based on the data source API in metastore through DDLs.

Author: Yin Huai <yhuai@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #3960 from yhuai/persistantTablesWithSchema2 and squashes the following commits:

069c235 [Yin Huai] Make exception messages user friendly.
c07cbc6 [Yin Huai] Get the location of test file in a correct way.
4456e98 [Yin Huai] Test data.
5315dfc [Yin Huai] rxin's comments.
7fc4b56 [Yin Huai] Add DDLStrategy and HiveDDLStrategy to plan DDLs based on the data source API.
aeaf4b3 [Yin Huai] Add comments.
06f9b0c [Yin Huai] Revert unnecessary changes.
feb88aa [Yin Huai] Merge remote-tracking branch 'apache/master' into persistantTablesWithSchema2
172db80 [Yin Huai] Fix unit test.
49bf1ac [Yin Huai] Unit tests.
8f8f1a1 [Yin Huai] [SPARK-4574][SQL] Adding support for defining schema in foreign DDL commands. #3431
f47fda1 [Yin Huai] Unit tests.
2b59723 [Michael Armbrust] Set external when creating tables
c00bb1b [Michael Armbrust] Don't use reflection to read options
1ea6e7b [Michael Armbrust] Don't fail when trying to uncache a table that doesn't exist
6edc710 [Michael Armbrust] Add tests.
d7da491 [Michael Armbrust] First draft of persistent tables.
2015-01-13 13:01:27 -08:00
Michael Armbrust 5d9fa55082 [SPARK-5049][SQL] Fix ordering of partition columns in ParquetTableScan
Followup to #3870.  Props to rahulaggarwalguavus for identifying the issue.

Author: Michael Armbrust <michael@databricks.com>

Closes #3990 from marmbrus/SPARK-5049 and squashes the following commits:

dd03e4e [Michael Armbrust] Fill in the partition values of parquet scans instead of using JoinedRow
2015-01-12 15:19:09 -08:00
YanTangZhai 0ca51cc31d [SPARK-4692] [SQL] Support ! boolean logic operator like NOT
Support ! boolean logic operator like NOT in sql as follows
select * from for_test where !(col1 > col2)

Author: YanTangZhai <hakeemzhai@tencent.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #3555 from YanTangZhai/SPARK-4692 and squashes the following commits:

1a9f605 [YanTangZhai] Update HiveQuerySuite.scala
7c03c68 [YanTangZhai] Merge pull request #23 from apache/master
992046e [YanTangZhai] Update HiveQuerySuite.scala
ea618f4 [YanTangZhai] Update HiveQuerySuite.scala
192411d [YanTangZhai] Merge pull request #17 from YanTangZhai/master
e4c2c0a [YanTangZhai] Merge pull request #15 from apache/master
1e1ebb4 [YanTangZhai] Update HiveQuerySuite.scala
efc4210 [YanTangZhai] Update HiveQuerySuite.scala
bd2c444 [YanTangZhai] Update HiveQuerySuite.scala
1893956 [YanTangZhai] Merge pull request #14 from marmbrus/pr/3555
59e4de9 [Michael Armbrust] make hive test
718afeb [YanTangZhai] Merge pull request #12 from apache/master
950b21e [YanTangZhai] Update HiveQuerySuite.scala
74175b4 [YanTangZhai] Update HiveQuerySuite.scala
92242c7 [YanTangZhai] Update HiveQl.scala
6e643f8 [YanTangZhai] Merge pull request #11 from apache/master
e249846 [YanTangZhai] Merge pull request #10 from apache/master
d26d982 [YanTangZhai] Merge pull request #9 from apache/master
76d4027 [YanTangZhai] Merge pull request #8 from apache/master
03b62b0 [YanTangZhai] Merge pull request #7 from apache/master
8a00106 [YanTangZhai] Merge pull request #6 from apache/master
cbcba66 [YanTangZhai] Merge pull request #3 from apache/master
cdef539 [YanTangZhai] Merge pull request #1 from apache/master
2015-01-10 15:05:23 -08:00
Michael Armbrust 3684fd21e1 [SPARK-5187][SQL] Fix caching of tables with HiveUDFs in the WHERE clause
Author: Michael Armbrust <michael@databricks.com>

Closes #3987 from marmbrus/hiveUdfCaching and squashes the following commits:

8bca2fa [Michael Armbrust] [SPARK-5187][SQL] Fix caching of tables with HiveUDFs in the WHERE clause
2015-01-10 14:25:45 -08:00
Yanbo Liang 77106df691 SPARK-4963 [SQL] Add copy to SQL's Sample operator
https://issues.apache.org/jira/browse/SPARK-4963
SchemaRDD.sample() return wrong results due to GapSamplingIterator operating on mutable row.
HiveTableScan make RDD with SpecificMutableRow and SchemaRDD.sample() will return GapSamplingIterator for iterating.

override def next(): T = {
    val r = data.next()
    advance
    r
  }

GapSamplingIterator.next() return the current underlying element and assigned it to r.
However if the underlying iterator is mutable row just like what HiveTableScan returned, underlying iterator and r will point to the same object.
After advance operation, we drop some underlying elments and it also changed r which is not expected. Then we return the wrong value different from initial r.

To fix this issue, the most direct way is to make HiveTableScan return mutable row with copy just like the initial commit that I have made. This solution will make HiveTableScan can not get the full advantage of reusable MutableRow, but it can make sample operation return correct result.
Further more, we need to investigate  GapSamplingIterator.next() and make it can implement copy operation inside it. To achieve this, we should define every elements that RDD can store implement the function like cloneable and it will make huge change.

Author: Yanbo Liang <yanbohappy@gmail.com>

Closes #3827 from yanbohappy/spark-4963 and squashes the following commits:

0912ca0 [Yanbo Liang] code format keep
65c4e7c [Yanbo Liang] import file and clear annotation
55c7c56 [Yanbo Liang] better output of test case
cea7e2e [Yanbo Liang] SchemaRDD add copy operation before Sample operator
e840829 [Yanbo Liang] HiveTableScan return mutable row with copy
2015-01-10 14:19:32 -08:00
scwf b3e86dc624 [SPARK-4861][SQL] Refactory command in spark sql
Follow up for #3712.
This PR finally remove ```CommandStrategy``` and make all commands follow ```RunnableCommand``` so they can go with ```case r: RunnableCommand => ExecutedCommand(r) :: Nil```.

One exception is the ```DescribeCommand``` of hive, which is a special case and need to distinguish hive table and temporary table, so still keep ```HiveCommandStrategy``` here.

Author: scwf <wangfei1@huawei.com>

Closes #3948 from scwf/followup-SPARK-4861 and squashes the following commits:

6b48e64 [scwf] minor style fix
2c62e9d [scwf] fix for hive module
5a7a819 [scwf] Refactory command in spark sql
2015-01-10 14:08:04 -08:00
scwf 693a323a70 [SPARK-4574][SQL] Adding support for defining schema in foreign DDL commands.
Adding support for defining schema in foreign DDL commands. Now foreign DDL support commands like:
```
CREATE TEMPORARY TABLE avroTable
USING org.apache.spark.sql.avro
OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")
```
With this PR user can define schema instead of infer from file, so  support ddl command as follows:
```
CREATE TEMPORARY TABLE avroTable(a int, b string)
USING org.apache.spark.sql.avro
OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro")
```

Author: scwf <wangfei1@huawei.com>
Author: Yin Huai <yhuai@databricks.com>
Author: Fei Wang <wangfei1@huawei.com>
Author: wangfei <wangfei1@huawei.com>

Closes #3431 from scwf/ddl and squashes the following commits:

7e79ce5 [Fei Wang] Merge pull request #22 from yhuai/pr3431yin
38f634e [Yin Huai] Remove Option from createRelation.
65e9c73 [Yin Huai] Revert all changes since applying a given schema has not been testd.
a852b10 [scwf] remove cleanIdentifier
f336a16 [Fei Wang] Merge pull request #21 from yhuai/pr3431yin
baf79b5 [Yin Huai] Test special characters quoted by backticks.
50a03b0 [Yin Huai] Use JsonRDD.nullTypeToStringType to convert NullType to StringType.
1eeb769 [Fei Wang] Merge pull request #20 from yhuai/pr3431yin
f5c22b0 [Yin Huai] Refactor code and update test cases.
f1cffe4 [Yin Huai] Revert "minor refactory"
b621c8f [scwf] minor refactory
d02547f [scwf] fix HiveCompatibilitySuite test failure
8dfbf7a [scwf] more tests for complex data type
ddab984 [Fei Wang] Merge pull request #19 from yhuai/pr3431yin
91ad91b [Yin Huai] Parse data types in DDLParser.
cf982d2 [scwf] fixed test failure
445b57b [scwf] address comments
02a662c [scwf] style issue
44eb70c [scwf] fix decimal parser issue
83b6fc3 [scwf] minor fix
9bf12f8 [wangfei] adding test case
7787ec7 [wangfei] added SchemaRelationProvider
0ba70df [wangfei] draft version
2015-01-10 13:53:21 -08:00
Alex Liu 4b39fd1e63 [SPARK-4943][SQL] Allow table name having dot for db/catalog
The pull only fixes the parsing error and changes API to use tableIdentifier. Joining different catalog datasource related change is not done in this pull.

Author: Alex Liu <alex_liu68@yahoo.com>

Closes #3941 from alexliu68/SPARK-SQL-4943-3 and squashes the following commits:

343ae27 [Alex Liu] [SPARK-4943][SQL] refactoring according to review
29e5e55 [Alex Liu] [SPARK-4943][SQL] fix failed Hive CTAS tests
6ae77ce [Alex Liu] [SPARK-4943][SQL] fix TestHive matching error
3652997 [Alex Liu] [SPARK-4943][SQL] Allow table name having dot to support db/catalog ...
2015-01-10 13:23:09 -08:00
Marcelo Vanzin 48cecf673c [SPARK-4048] Enhance and extend hadoop-provided profile.
This change does a few things to make the hadoop-provided profile more useful:

- Create new profiles for other libraries / services that might be provided by the infrastructure
- Simplify and fix the poms so that the profiles are only activated while building assemblies.
- Fix tests so that they're able to run when the profiles are activated
- Add a new env variable to be used by distributions that use these profiles to provide the runtime
  classpath for Spark jobs and daemons.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #2982 from vanzin/SPARK-4048 and squashes the following commits:

82eb688 [Marcelo Vanzin] Add a comment.
eb228c0 [Marcelo Vanzin] Fix borked merge.
4e38f4e [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
9ef79a3 [Marcelo Vanzin] Alternative way to propagate test classpath to child processes.
371ebee [Marcelo Vanzin] Review feedback.
52f366d [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
83099fc [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
7377e7b [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
322f882 [Marcelo Vanzin] Fix merge fail.
f24e9e7 [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
8b00b6a [Marcelo Vanzin] Merge branch 'master' into SPARK-4048
9640503 [Marcelo Vanzin] Cleanup child process log message.
115fde5 [Marcelo Vanzin] Simplify a comment (and make it consistent with another pom).
e3ab2da [Marcelo Vanzin] Fix hive-thriftserver profile.
7820d58 [Marcelo Vanzin] Fix CliSuite with provided profiles.
1be73d4 [Marcelo Vanzin] Restore flume-provided profile.
d1399ed [Marcelo Vanzin] Restore jetty dependency.
82a54b9 [Marcelo Vanzin] Remove unused profile.
5c54a25 [Marcelo Vanzin] Fix HiveThriftServer2Suite with *-provided profiles.
1fc4d0b [Marcelo Vanzin] Update dependencies for hive-thriftserver.
f7b3bbe [Marcelo Vanzin] Add snappy to hadoop-provided list.
9e4e001 [Marcelo Vanzin] Remove duplicate hive profile.
d928d62 [Marcelo Vanzin] Redirect child stderr to parent's log.
4d67469 [Marcelo Vanzin] Propagate SPARK_DIST_CLASSPATH on Yarn.
417d90e [Marcelo Vanzin] Introduce "SPARK_DIST_CLASSPATH".
2f95f0d [Marcelo Vanzin] Propagate classpath to child processes during testing.
1adf91c [Marcelo Vanzin] Re-enable maven-install-plugin for a few projects.
284dda6 [Marcelo Vanzin] Rework the "hadoop-provided" profile, add new ones.
2015-01-08 17:15:13 -08:00
Sean Owen 4cba6eb420 SPARK-4159 [CORE] Maven build doesn't run JUnit test suites
This PR:

- Reenables `surefire`, and copies config from `scalatest` (which is itself an old fork of `surefire`, so similar)
- Tells `surefire` to test only Java tests
- Enables `surefire` and `scalatest` for all children, and in turn eliminates some duplication.

For me this causes the Scala and Java tests to be run once each, it seems, as desired. It doesn't affect the SBT build but works for Maven. I still need to verify that all of the Scala tests and Java tests are being run.

Author: Sean Owen <sowen@cloudera.com>

Closes #3651 from srowen/SPARK-4159 and squashes the following commits:

2e8a0af [Sean Owen] Remove specialized SPARK_HOME setting for REPL, YARN tests as it appears to be obsolete
12e4558 [Sean Owen] Append to unit-test.log instead of overwriting, so that both surefire and scalatest output is preserved. Also standardize/correct comments a bit.
e6f8601 [Sean Owen] Reenable Java tests by reenabling surefire with config cloned from scalatest; centralize test config in the parent
2015-01-06 12:02:08 -08:00
wangxiaojing 07fa1910d9 [SPARK-4570][SQL]add BroadcastLeftSemiJoinHash
JIRA issue: [SPARK-4570](https://issues.apache.org/jira/browse/SPARK-4570)
We are planning to create a `BroadcastLeftSemiJoinHash` to implement the broadcast join for `left semijoin`
In left semijoin :
If the size of data from right side is smaller than the user-settable threshold `AUTO_BROADCASTJOIN_THRESHOLD`,
the planner would mark it as the `broadcast` relation and mark the other relation as the stream side. The broadcast table will be broadcasted to all of the executors involved in the join, as a `org.apache.spark.broadcast.Broadcast` object. It will use `joins.BroadcastLeftSemiJoinHash`.,else it will use `joins.LeftSemiJoinHash`.

The benchmark suggests these  made the optimized version 4x faster  when `left semijoin`
<pre><code>
Original:
left semi join : 9288 ms
Optimized:
left semi join : 1963 ms
</code></pre>
The micro benchmark load `data1/kv3.txt` into a normal Hive table.
Benchmark code:
<pre><code>
 def benchmark(f: => Unit) = {
    val begin = System.currentTimeMillis()
    f
    val end = System.currentTimeMillis()
    end - begin
  }
  val sc = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))
  val hiveContext = new HiveContext(sc)
  import hiveContext._
  sql("drop table if exists left_table")
  sql("drop table if exists right_table")
  sql( """create table left_table (key int, value string)
       """.stripMargin)
  sql( s"""load data local inpath "/data1/kv3.txt" into table left_table""")
  sql( """create table right_table (key int, value string)
       """.stripMargin)
  sql(
    """
      |from left_table
      |insert overwrite table right_table
      |select left_table.key, left_table.value
    """.stripMargin)

  val leftSimeJoin = sql(
    """select a.key from left_table a
      |left semi join right_table b on a.key = b.key""".stripMargin)
  val leftSemiJoinDuration = benchmark(leftSimeJoin.count())
  println(s"left semi join : $leftSemiJoinDuration ms ")
</code></pre>

Author: wangxiaojing <u9jing@gmail.com>

Closes #3442 from wangxiaojing/SPARK-4570 and squashes the following commits:

a4a43c9 [wangxiaojing] rebase
f103983 [wangxiaojing] change style
fbe4887 [wangxiaojing] change style
ff2e618 [wangxiaojing] add testsuite
1a8da2a [wangxiaojing] add BroadcastLeftSemiJoinHash
2014-12-30 13:54:12 -08:00
Cheng Hao 53f0a00b60 [Spark-4512] [SQL] Unresolved Attribute Exception in Sort By
It will cause exception while do query like:
SELECT key+key FROM src sort by value;

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

Closes #3386 from chenghao-intel/sort and squashes the following commits:

38c78cc [Cheng Hao] revert the SortPartition in SparkStrategies
7e9dd15 [Cheng Hao] update the typo
fcd1d64 [Cheng Hao] rebase the latest master and update the SortBy unit test
2014-12-30 12:11:44 -08:00
Cheng Hao 63b84b7d67 [SPARK-4904] [SQL] Remove the unnecessary code change in Generic UDF
Since #3429 has been merged, the bug of wrapping to Writable for HiveGenericUDF is resolved, we can safely remove the foldable checking in `HiveGenericUdf.eval`, which discussed in #2802.

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

Closes #3745 from chenghao-intel/generic_udf and squashes the following commits:

622ad03 [Cheng Hao] Remove the unnecessary code change in Generic UDF
2014-12-30 11:47:08 -08:00
Cheng Hao 5595eaa74f [SPARK-4959] [SQL] Attributes are case sensitive when using a select query from a projection
Author: Cheng Hao <hao.cheng@intel.com>

Closes #3796 from chenghao-intel/spark_4959 and squashes the following commits:

3ec08f8 [Cheng Hao] Replace the attribute in comparing its exprId other than itself
2014-12-30 11:33:47 -08:00
scwf 65357f11c2 [SPARK-4975][SQL] Fix HiveInspectorSuite test failure
HiveInspectorSuite test failure:
[info] - wrap / unwrap null, constant null and writables *** FAILED *** (21 milliseconds)
[info] 1 did not equal 0 (HiveInspectorSuite.scala:136)
this is because the origin date(is 3914-10-23) not equals the date returned by ```unwrap```(is 3914-10-22).

Setting TimeZone and Locale fix this.
Another minor change here is rename ```def checkValues(v1: Any, v2: Any): Unit```  to  ```def checkValue(v1: Any, v2: Any): Unit ``` to make the code more clear

Author: scwf <wangfei1@huawei.com>
Author: Fei Wang <wangfei1@huawei.com>

Closes #3814 from scwf/fix-inspectorsuite and squashes the following commits:

d8531ef [Fei Wang] Delete test.log
72b19a9 [scwf] fix HiveInspectorSuite test error
2014-12-30 11:30:47 -08:00
Daoyuan Wang 94d60b7021 [SQL] enable view test
This is a follow up of #3396 , just add a test to white list.

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

Closes #3826 from adrian-wang/viewtest and squashes the following commits:

f105f68 [Daoyuan Wang] enable view test
2014-12-30 11:29:13 -08:00
Michael Armbrust 480bd1d2ed [SPARK-4908][SQL] Prevent multiple concurrent hive native commands
This is just a quick fix that locks when calling `runHive`.  If we can find a way to avoid the error without a global lock that would be better.

Author: Michael Armbrust <michael@databricks.com>

Closes #3834 from marmbrus/hiveConcurrency and squashes the following commits:

bf25300 [Michael Armbrust] prevent multiple concurrent hive native commands
2014-12-30 11:24:46 -08:00
Sean Owen 29fabb1b52 SPARK-4297 [BUILD] Build warning fixes omnibus
There are a number of warnings generated in a normal, successful build right now. They're mostly Java unchecked cast warnings, which can be suppressed. But there's a grab bag of other Scala language warnings and so on that can all be easily fixed. The forthcoming PR fixes about 90% of the build warnings I see now.

Author: Sean Owen <sowen@cloudera.com>

Closes #3157 from srowen/SPARK-4297 and squashes the following commits:

8c9e469 [Sean Owen] Suppress unchecked cast warnings, and several other build warning fixes
2014-12-24 13:32:51 -08:00
Liang-Chi Hsieh a61aa669af [Minor] Fix scala doc
Minor fix for an obvious scala doc error.

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

Closes #3751 from viirya/fix_scaladoc and squashes the following commits:

03fddaa [Liang-Chi Hsieh] Fix scala doc.
2014-12-22 14:13:31 -08:00
Cheng Hao 5479450c45 [SPARK-4901] [SQL] Hot fix for ByteWritables.copyBytes
HiveInspectors.scala failed in compiling with Hadoop 1, as the BytesWritable.copyBytes is not available in Hadoop 1.

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

Closes #3742 from chenghao-intel/settable_oi_hotfix and squashes the following commits:

bb04d1f [Cheng Hao] hot fix for ByteWritables.copyBytes
2014-12-19 08:04:41 -08:00
wangfei c3d91da5ea [SPARK-4861][SQL] Refactory command in spark sql
Remove ```Command``` and use ```RunnableCommand``` instead.

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #3712 from scwf/cmd and squashes the following commits:

51a82f2 [wangfei] fix test failure
0e03be8 [wangfei] address comments
4033bed [scwf] remove CreateTableAsSelect in hivestrategy
5d20010 [wangfei] address comments
125f542 [scwf] factory command in spark sql
2014-12-18 20:24:56 -08:00
Cheng Hao ae9f128608 [SPARK-4573] [SQL] Add SettableStructObjectInspector support in "wrap" function
Hive UDAF may create an customized object constructed by SettableStructObjectInspector, this is critical when integrate Hive UDAF with the refactor-ed UDAF interface.

Performance issue in `wrap/unwrap` since more match cases added, will do it in another PR.

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

Closes #3429 from chenghao-intel/settable_oi and squashes the following commits:

9f0aff3 [Cheng Hao] update code style issues as feedbacks
2b0561d [Cheng Hao] Add more scala doc
f5a40e8 [Cheng Hao] add scala doc
2977e9b [Cheng Hao] remove the timezone setting for test suite
3ed284c [Cheng Hao] fix the date type comparison
f1b6749 [Cheng Hao] Update the comment
932940d [Cheng Hao] Add more unit test
72e4332 [Cheng Hao] Add settable StructObjectInspector support
2014-12-18 20:21:52 -08:00
ravipesala 7687415c25 [SPARK-2554][SQL] Supporting SumDistinct partial aggregation
Adding support to the partial aggregation of SumDistinct

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

Closes #3348 from ravipesala/SPARK-2554 and squashes the following commits:

fd28e4d [ravipesala] Fixed review comments
e60e67f [ravipesala] Fixed test cases and made it as nullable
32fe234 [ravipesala] Supporting SumDistinct partial aggregation Conflicts: 	sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
2014-12-18 20:19:10 -08:00
YanTangZhai e7de7e5f46 [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references
The sql "select * from spark_test::for_test where abs(20141202) is not null" has predicates=List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202)) and
partitionKeyIds=AttributeSet(). PruningPredicates is List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202)). Then the exception "java.lang.IllegalArgumentException: requirement failed: Partition pruning predicates only supported for partitioned tables." is thrown.
The sql "select * from spark_test::for_test_partitioned_table where abs(20141202) is not null and type_id=11 and platform = 3" with partitioned key insert_date has predicates=List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202), (type_id#12 = 11), (platform#8 = 3)) and partitionKeyIds=AttributeSet(insert_date#24). PruningPredicates is List(IS NOT NULL HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFAbs(20141202)).

Author: YanTangZhai <hakeemzhai@tencent.com>
Author: yantangzhai <tyz0303@163.com>

Closes #3556 from YanTangZhai/SPARK-4693 and squashes the following commits:

620ebe3 [yantangzhai] [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references
37cfdf5 [yantangzhai] [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references
70a3544 [yantangzhai] [SPARK-4693] [SQL] PruningPredicates may be wrong if predicates contains an empty AttributeSet() references
efa9b03 [YanTangZhai] Update HiveQuerySuite.scala
72accf1 [YanTangZhai] Update HiveQuerySuite.scala
e572b9a [YanTangZhai] Update HiveStrategies.scala
6e643f8 [YanTangZhai] Merge pull request #11 from apache/master
e249846 [YanTangZhai] Merge pull request #10 from apache/master
d26d982 [YanTangZhai] Merge pull request #9 from apache/master
76d4027 [YanTangZhai] Merge pull request #8 from apache/master
03b62b0 [YanTangZhai] Merge pull request #7 from apache/master
8a00106 [YanTangZhai] Merge pull request #6 from apache/master
cbcba66 [YanTangZhai] Merge pull request #3 from apache/master
cdef539 [YanTangZhai] Merge pull request #1 from apache/master
2014-12-18 20:13:46 -08:00
Cheng Hao f728e0fe7e [SPARK-2663] [SQL] Support the Grouping Set
Add support for `GROUPING SETS`, `ROLLUP`, `CUBE` and the the virtual column `GROUPING__ID`.

More details on how to use the `GROUPING SETS" can be found at: https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation,+Cube,+Grouping+and+Rollup
https://issues.apache.org/jira/secure/attachment/12676811/grouping_set.pdf

The generic idea of the implementations are :
1 Replace the `ROLLUP`, `CUBE` with `GROUPING SETS`
2 Explode each of the input row, and then feed them to `Aggregate`
  * Each grouping set are represented as the bit mask for the `GroupBy Expression List`, for each bit, `1` means the expression is selected, otherwise `0` (left is the lower bit, and right is the higher bit in the `GroupBy Expression List`)
  * Several of projections are constructed according to the grouping sets, and within each projection(Seq[Expression), we replace those expressions with `Literal(null)` if it's not selected in the grouping set (based on the bit mask)
  * Output Schema of `Explode` is `child.output :+ grouping__id`
  * GroupBy Expressions of `Aggregate` is `GroupBy Expression List :+ grouping__id`
  * Keep the `Aggregation expressions` the same for the `Aggregate`

The expressions substitutions happen in Logic Plan analyzing, so we will benefit from the Logical Plan optimization (e.g. expression constant folding, and map side aggregation etc.), Only an `Explosive` operator added for Physical Plan, which will explode the rows according the pre-set projections.

A known issue will be done in the follow up PR:
* Optimization `ColumnPruning` is not supported yet for `Explosive` node.

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

Closes #1567 from chenghao-intel/grouping_sets and squashes the following commits:

fe65fcc [Cheng Hao] Remove the extra space
3547056 [Cheng Hao] Add more doc and Simplify the Expand
a7c869d [Cheng Hao] update code as feedbacks
d23c672 [Cheng Hao] Add GroupingExpression to replace the Seq[Expression]
414b165 [Cheng Hao] revert the unnecessary changes
ec276c6 [Cheng Hao] Support Rollup/Cube/GroupingSets
2014-12-18 18:58:29 -08:00
Venkata Ramana Gollamudi f33d550464 [SPARK-3891][SQL] Add array support to percentile, percentile_approx and constant inspectors support
Supported passing array to percentile and percentile_approx UDAFs
To support percentile_approx,  constant inspectors are supported for GenericUDAF
Constant folding support added to CreateArray expression
Avoided constant udf expression re-evaluation

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

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

Closes #2802 from gvramana/percentile_array_support and squashes the following commits:

a0182e5 [Venkata Ramana Gollamudi] fixed review comment
a18f917 [Venkata Ramana Gollamudi] avoid constant udf expression re-evaluation - fixes failure due to return iterator and value type mismatch
c46db0f [Venkata Ramana Gollamudi] Removed TestHive reset
4d39105 [Venkata Ramana Gollamudi] Unified inspector creation, style check fixes
f37fd69 [Venkata Ramana Gollamudi] Fixed review comments
47f6365 [Venkata Ramana Gollamudi] fixed test
cb7c61e [Venkata Ramana Gollamudi] Supported ConstantInspector for UDAF Fixed HiveUdaf wrap object issue.
7f94aff [Venkata Ramana Gollamudi] Added foldable support to CreateArray
2014-12-17 15:41:35 -08:00
Cheng Hao 636d9fc450 [SPARK-3739] [SQL] Update the split num base on block size for table scanning
In local mode, Hadoop/Hive will ignore the "mapred.map.tasks", hence for small table file, it's always a single input split, however, SparkSQL doesn't honor that in table scanning, and we will get different result when do the Hive Compatibility test. This PR will fix that.

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

Closes #2589 from chenghao-intel/source_split and squashes the following commits:

dff38e7 [Cheng Hao] Remove the extra blank line
160a2b6 [Cheng Hao] fix the compiling bug
04d67f7 [Cheng Hao] Keep 1 split for small file in table scanning
2014-12-17 13:39:36 -08:00
Michael Armbrust 7ad579ee97 [SPARK-3698][SQL] Fix case insensitive resolution of GetField.
Based on #2543.

Author: Michael Armbrust <michael@databricks.com>

Closes #3724 from marmbrus/resolveGetField and squashes the following commits:

0a47aae [Michael Armbrust] Fix case insensitive resolution of GetField.
2014-12-17 12:43:51 -08:00
Cheng Lian 3b395e1051 [SPARK-4798][SQL] A new set of Parquet testing API and test suites
This PR provides a set Parquet testing API (see trait `ParquetTest`) that enables developers to write more concise test cases. A new set of Parquet test suites built upon this API  are added and aim to replace the old `ParquetQuerySuite`. To avoid potential merge conflicts, old testing code are not removed yet. The following classes can be safely removed after most Parquet related PRs are handled:

- `ParquetQuerySuite`
- `ParquetTestData`

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/3644)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #3644 from liancheng/parquet-tests and squashes the following commits:

800e745 [Cheng Lian] Enforces ordering of test output
3bb8731 [Cheng Lian] Refactors HiveParquetSuite
aa2cb2e [Cheng Lian] Decouples ParquetTest and TestSQLContext
7b43a68 [Cheng Lian] Updates ParquetTest Scaladoc
7f07af0 [Cheng Lian] Adds a new set of Parquet test suites
2014-12-16 21:16:03 -08:00
jerryshao dc8280dcca [SPARK-4847][SQL]Fix "extraStrategies cannot take effect in SQLContext" issue
Author: jerryshao <saisai.shao@intel.com>

Closes #3698 from jerryshao/SPARK-4847 and squashes the following commits:

4741130 [jerryshao] Make later added extraStrategies effect when calling strategies
2014-12-16 14:08:28 -08:00
Sean Owen 81112e4b57 SPARK-4814 [CORE] Enable assertions in SBT, Maven tests / AssertionError from Hive's LazyBinaryInteger
This enables assertions for the Maven and SBT build, but overrides the Hive module to not enable assertions.

Author: Sean Owen <sowen@cloudera.com>

Closes #3692 from srowen/SPARK-4814 and squashes the following commits:

caca704 [Sean Owen] Disable assertions just for Hive
f71e783 [Sean Owen] Enable assertions for SBT and Maven build
2014-12-15 17:12:05 -08:00
Cheng Hao 0abbff2862 [SPARK-4825] [SQL] CTAS fails to resolve when created using saveAsTable
Fix bug when query like:
```
  test("save join to table") {
    val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString))
    sql("CREATE TABLE test1 (key INT, value STRING)")
    testData.insertInto("test1")
    sql("CREATE TABLE test2 (key INT, value STRING)")
    testData.insertInto("test2")
    testData.insertInto("test2")
    sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").saveAsTable("test")
    checkAnswer(
      table("test"),
      sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq)
  }
```

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

Closes #3673 from chenghao-intel/spark_4825 and squashes the following commits:

e8cbd56 [Cheng Hao] alternate the pattern matching order for logical plan:CTAS
e004895 [Cheng Hao] fix bug
2014-12-11 22:51:49 -08:00
Daoyuan Wang cbb634ae69 [SQL] enable empty aggr test case
This is fixed by SPARK-4318 #3184

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

Closes #3445 from adrian-wang/emptyaggr and squashes the following commits:

982575e [Daoyuan Wang] enable empty aggr test case
2014-12-11 22:50:18 -08:00
scwf d8cf678589 [SQL] Remove unnecessary case in HiveContext.toHiveString
a follow up of #3547
/cc marmbrus

Author: scwf <wangfei1@huawei.com>

Closes #3563 from scwf/rnc and squashes the following commits:

9395661 [scwf] remove unnecessary condition
2014-12-11 22:48:03 -08:00
Cheng Hao a7f07f511c [SPARK-4662] [SQL] Whitelist more unittest
Whitelist more hive unit test:

"create_like_tbl_props"
"udf5"
"udf_java_method"
"decimal_1"
"udf_pmod"
"udf_to_double"
"udf_to_float"
"udf7" (this will fail in Hive 0.12)

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

Closes #3522 from chenghao-intel/unittest and squashes the following commits:

f54e4c7 [Cheng Hao] work around to clean up the hive.table.parameters.default in reset
16fee22 [Cheng Hao] Whitelist more unittest
2014-12-11 22:43:02 -08:00
Cheng Hao 383c5555c9 [SPARK-4785][SQL] Initilize Hive UDFs on the driver and serialize them with a wrapper
Different from Hive 0.12.0, in Hive 0.13.1 UDF/UDAF/UDTF (aka Hive function) objects should only be initialized once on the driver side and then serialized to executors. However, not all function objects are serializable (e.g. GenericUDF doesn't implement Serializable). Hive 0.13.1 solves this issue with Kryo or XML serializer. Several utility ser/de methods are provided in class o.a.h.h.q.e.Utilities for this purpose. In this PR we chose Kryo for efficiency. The Kryo serializer used here is created in Hive. Spark Kryo serializer wasn't used because there's no available SparkConf instance.

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

Closes #3640 from chenghao-intel/udf_serde and squashes the following commits:

8e13756 [Cheng Hao] Update the comment
74466a3 [Cheng Hao] refactor as feedbacks
396c0e1 [Cheng Hao] avoid Simple UDF to be serialized
e9c3212 [Cheng Hao] update the comment
19cbd46 [Cheng Hao] support udf instance ser/de after initialization
2014-12-09 10:28:33 -08:00
Cheng Hao 51b1fe1426 [SPARK-4769] [SQL] CTAS does not work when reading from temporary tables
This is the code refactor and follow ups for #2570

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

Closes #3336 from chenghao-intel/createtbl and squashes the following commits:

3563142 [Cheng Hao] remove the unused variable
e215187 [Cheng Hao] eliminate the compiling warning
4f97f14 [Cheng Hao] fix bug in unittest
5d58812 [Cheng Hao] revert the API changes
b85b620 [Cheng Hao] fix the regression of temp tabl not found in CTAS
2014-12-08 17:39:12 -08:00
Michael Armbrust 513ef82e85 [SPARK-4552][SQL] Avoid exception when reading empty parquet data through Hive
This is a very small fix that catches one specific exception and returns an empty table.  #3441 will address this in a more principled way.

Author: Michael Armbrust <michael@databricks.com>

Closes #3586 from marmbrus/fixEmptyParquet and squashes the following commits:

2781d9f [Michael Armbrust] Handle empty lists for newParquet
04dd376 [Michael Armbrust] Avoid exception when reading empty parquet data through Hive
2014-12-03 14:13:35 -08:00
wangfei 3ae0cda83c [SPARK-4695][SQL] Get result using executeCollect
Using ```executeCollect``` to collect the result, because executeCollect is a custom implementation of collect in spark sql which better than rdd's collect

Author: wangfei <wangfei1@huawei.com>

Closes #3547 from scwf/executeCollect and squashes the following commits:

a5ab68e [wangfei] Revert "adding debug info"
a60d680 [wangfei] fix test failure
0db7ce8 [wangfei] adding debug info
184c594 [wangfei] using executeCollect instead collect
2014-12-02 14:30:44 -08:00
Daoyuan Wang 4df60a8cbc [SPARK-4529] [SQL] support view with column alias
Support view definition like

CREATE VIEW view3(valoo)
TBLPROPERTIES ("fear" = "factor")
AS SELECT upper(value) FROM src WHERE key=86;

[valoo as the alias of upper(value)]. This is missing part of SPARK-4239, for a fully view support.

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

Closes #3396 from adrian-wang/viewcolumn and squashes the following commits:

4d001d0 [Daoyuan Wang] support view with column alias
2014-12-01 16:08:51 -08:00
zsxwing 30a86acdef [SPARK-4661][Core] Minor code and docs cleanup
Author: zsxwing <zsxwing@gmail.com>

Closes #3521 from zsxwing/SPARK-4661 and squashes the following commits:

03cbe3f [zsxwing] Minor code and docs cleanup
2014-12-01 00:35:01 -08:00
Michael Armbrust 02ec058efe [SPARK-4413][SQL] Parquet support through datasource API
Goals:
 - Support for accessing parquet using SQL but not requiring Hive (thus allowing support of parquet tables with decimal columns)
 - Support for folder based partitioning with automatic discovery of available partitions
 - Caching of file metadata

See scaladoc of `ParquetRelation2` for more details.

Author: Michael Armbrust <michael@databricks.com>

Closes #3269 from marmbrus/newParquet and squashes the following commits:

1dd75f1 [Michael Armbrust] Pass all paths for FileInputFormat at once.
645768b [Michael Armbrust] Review comments.
abd8e2f [Michael Armbrust] Alternative implementation of parquet based on the datasources API.
938019e [Michael Armbrust] Add an experimental interface to data sources that exposes catalyst expressions.
e9d2641 [Michael Armbrust] logging / formatting improvements.
2014-11-20 18:31:02 -08:00
Cheng Hao 84d79ee9ec [SPARK-4244] [SQL] Support Hive Generic UDFs with constant object inspector parameters
Query `SELECT named_struct(lower("AA"), "12", lower("Bb"), "13") FROM src LIMIT 1` will throw exception, some of the Hive Generic UDF/UDAF requires the input object inspector is `ConstantObjectInspector`, however, we won't get that before the expression optimization executed. (Constant Folding).

This PR is a work around to fix this. (As ideally, the `output` of LogicalPlan should be identical before and after Optimization).

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

Closes #3109 from chenghao-intel/optimized and squashes the following commits:

487ff79 [Cheng Hao] rebase to the latest master & update the unittest
2014-11-20 16:50:59 -08:00
Cheng Hao 6aa0fc9f4d [SPARK-2918] [SQL] Support the CTAS in EXPLAIN command
Hive supports the `explain` the CTAS, which was supported by Spark SQL previously, however, seems it was reverted after the code refactoring in HiveQL.

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

Closes #3357 from chenghao-intel/explain and squashes the following commits:

7aace63 [Cheng Hao] Support the CTAS in EXPLAIN command
2014-11-20 15:46:00 -08:00
Marcelo Vanzin 397d3aae5b Bumping version to 1.3.0-SNAPSHOT.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #3277 from vanzin/version-1.3 and squashes the following commits:

7c3c396 [Marcelo Vanzin] Added temp repo to sbt build.
5f404ff [Marcelo Vanzin] Add another exclusion.
19457e7 [Marcelo Vanzin] Update old version to 1.2, add temporary 1.2 repo.
3c8d705 [Marcelo Vanzin] Workaround for MIMA checks.
e940810 [Marcelo Vanzin] Bumping version to 1.3.0-SNAPSHOT.
2014-11-18 21:24:18 -08:00
Michael Armbrust 90d72ec850 [SQL] Support partitioned parquet tables that have the key in both the directory and the file
Author: Michael Armbrust <michael@databricks.com>

Closes #3272 from marmbrus/keyInPartitionedTable and squashes the following commits:

447f08c [Michael Armbrust] Support partitioned parquet tables that have the key in both the directory and the file
2014-11-18 12:13:23 -08:00
Cheng Hao ef7c464eff [SPARK-4448] [SQL] unwrap for the ConstantObjectInspector
Author: Cheng Hao <hao.cheng@intel.com>

Closes #3308 from chenghao-intel/unwrap_constant_oi and squashes the following commits:

156b500 [Cheng Hao] rebase the master
c5b20ab [Cheng Hao] unwrap for the ConstantObjectInspector
2014-11-17 16:35:49 -08:00
w00228970 42389b1780 [SPARK-4443][SQL] Fix statistics for external table in spark sql hive
The `totalSize` of external table  is always zero, which will influence join strategy(always use broadcast join for external table).

Author: w00228970 <wangfei1@huawei.com>

Closes #3304 from scwf/statistics and squashes the following commits:

568f321 [w00228970] fix statistics for external table
2014-11-17 16:33:50 -08:00
Cheng Lian 6b7f2f753d [SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types
This PR is exactly the same as #3178 except it reverts the `FileStatus.isDir` to `FileStatus.isDirectory` change, since it doesn't compile with Hadoop 1.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/3298)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #3298 from liancheng/date-for-thriftserver and squashes the following commits:

866037e [Cheng Lian] Revers isDirectory to isDir (it breaks Hadoop 1 profile)
6f71d0b [Cheng Lian] Makes toHiveString static
26fa955 [Cheng Lian] Fixes complex type support in Hive 0.13.1 shim
a92882a [Cheng Lian] Updates HiveShim for 0.13.1
73f442b [Cheng Lian] Adds Date support for HiveThriftServer2 (Hive 0.12.0)
2014-11-17 16:31:05 -08:00
Michael Armbrust 45ce3273cb Revert "[SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types"
Author: Michael Armbrust <michael@databricks.com>

Closes #3292 from marmbrus/revert4309 and squashes the following commits:

808e96e [Michael Armbrust] Revert "[SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types"
2014-11-16 15:05:08 -08:00
Cheng Lian cb6bd83a91 [SPARK-4309][SPARK-4407][SQL] Date type support for Thrift server, and fixes for complex types
SPARK-4407 was detected while working on SPARK-4309. Merged these two into a single PR since 1.2.0 RC is approaching.

<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/3178)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #3178 from liancheng/date-for-thriftserver and squashes the following commits:

6f71d0b [Cheng Lian] Makes toHiveString static
26fa955 [Cheng Lian] Fixes complex type support in Hive 0.13.1 shim
a92882a [Cheng Lian] Updates HiveShim for 0.13.1
73f442b [Cheng Lian] Adds Date support for HiveThriftServer2 (Hive 0.12.0)
2014-11-16 14:26:41 -08:00
Michael Armbrust a0300ea32a [SPARK-4390][SQL] Handle NaN cast to decimal correctly
Author: Michael Armbrust <michael@databricks.com>

Closes #3256 from marmbrus/NanDecimal and squashes the following commits:

4c3ba46 [Michael Armbrust] fix style
d360f83 [Michael Armbrust] Handle NaN cast to decimal
2014-11-14 14:56:57 -08:00
Sandy Ryza f5f757e4ed SPARK-4375. no longer require -Pscala-2.10
It seems like the winds might have moved away from this approach, but wanted to post the PR anyway because I got it working and to show what it would look like.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #3239 from sryza/sandy-spark-4375 and squashes the following commits:

0ffbe95 [Sandy Ryza] Enable -Dscala-2.11 in sbt
cd42d94 [Sandy Ryza] Update doc
f6644c3 [Sandy Ryza] SPARK-4375 take 2
2014-11-14 14:21:57 -08:00
Takuya UESHIN bbd8f5bee8 [SPARK-4245][SQL] Fix containsNull of the result ArrayType of CreateArray expression.
The `containsNull` of the result `ArrayType` of `CreateArray` should be `true` only if the children is empty or there exists nullable child.

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

Closes #3110 from ueshin/issues/SPARK-4245 and squashes the following commits:

6f64746 [Takuya UESHIN] Move equalsIgnoreNullability method into DataType.
5a90e02 [Takuya UESHIN] Refine InsertIntoHiveType and add some comments.
cbecba8 [Takuya UESHIN] Fix a test title.
884ec37 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4245
3c5274b [Takuya UESHIN] Add tests to insert data of types ArrayType / MapType / StructType with nullability is false into Hive table.
41a94a9 [Takuya UESHIN] Replace InsertIntoTable with InsertIntoHiveTable if data types ignoring nullability are same.
43e6ef5 [Takuya UESHIN] Fix containsNull for empty array.
778e997 [Takuya UESHIN] Fix containsNull of the result ArrayType of CreateArray expression.
2014-11-14 14:21:16 -08:00
Daoyuan Wang ade72c4362 [SPARK-4239] [SQL] support view in HiveQl
Currently still not support view like

CREATE VIEW view3(valoo)
TBLPROPERTIES ("fear" = "factor")
AS SELECT upper(value) FROM src WHERE key=86;

because the text in metastore for this view is like

select \`_c0\` as \`valoo\` from (select upper(\`src\`.\`value\`) from \`default\`.\`src\` where ...) \`view3\`

while catalyst cannot resolve \`_c0\` for this query.
For view without colname definition in parentheses, it works fine.

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

Closes #3131 from adrian-wang/view and squashes the following commits:

8a56fd6 [Daoyuan Wang] michael's comments
e46c056 [Daoyuan Wang] add some golden file
079290a [Daoyuan Wang] remove useless import
88afcad [Daoyuan Wang] support view in HiveQl
2014-11-14 13:51:20 -08:00
Cheng Hao fa777833b5 [SPARK-4250] [SQL] Fix bug of constant null value mapping to ConstantObjectInspector
Author: Cheng Hao <hao.cheng@intel.com>

Closes #3114 from chenghao-intel/constant_null_oi and squashes the following commits:

e603bda [Cheng Hao] fix the bug of null value for primitive types
50a13ba [Cheng Hao] fix the timezone issue
f54f369 [Cheng Hao] fix bug of constant null value for ObjectInspector
2014-11-10 17:22:57 -08:00
Xiangrui Meng 894a7245c3 [SQL] support udt to hive types conversion (hive->udt is not supported)
marmbrus

Author: Xiangrui Meng <meng@databricks.com>

Closes #3164 from mengxr/hive-udt and squashes the following commits:

57c7519 [Xiangrui Meng] support udt->hive types (hive->udt is not supported)
2014-11-10 11:04:12 -08:00
Sean Owen f8e5732307 SPARK-1209 [CORE] (Take 2) SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop
andrewor14 Another try at SPARK-1209, to address https://github.com/apache/spark/pull/2814#issuecomment-61197619

I successfully tested with `mvn -Dhadoop.version=1.0.4 -DskipTests clean package; mvn -Dhadoop.version=1.0.4 test` I assume that is what failed Jenkins last time. I also tried `-Dhadoop.version1.2.1` and `-Phadoop-2.4 -Pyarn -Phive` for more coverage.

So this is why the class was put in `org.apache.hadoop` to begin with, I assume. One option is to leave this as-is for now and move it only when Hadoop 1.0.x support goes away.

This is the other option, which adds a call to force the constructor to be public at run-time. It's probably less surprising than putting Spark code in `org.apache.hadoop`, but, does involve reflection. A `SecurityManager` might forbid this, but it would forbid a lot of stuff Spark does. This would also only affect Hadoop 1.0.x it seems.

Author: Sean Owen <sowen@cloudera.com>

Closes #3048 from srowen/SPARK-1209 and squashes the following commits:

0d48f4b [Sean Owen] For Hadoop 1.0.x, make certain constructors public, which were public in later versions
466e179 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though?
eb61820 [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark
2014-11-09 22:11:20 -08:00
Matthew Taylor ac70c972a5 [SPARK-4203][SQL] Partition directories in random order when inserting into hive table
When doing an insert into hive table with partitions the folders written to the file system are in a random order instead of the order defined in table creation. Seems that the loadPartition method in Hive.java has a Map<String,String> parameter but expects to be called with a map that has a defined ordering such as LinkedHashMap. Working on a test but having intillij problems

Author: Matthew Taylor <matthew.t@tbfe.net>

Closes #3076 from tbfenet/partition_dir_order_problem and squashes the following commits:

f1b9a52 [Matthew Taylor] Comment format fix
bca709f [Matthew Taylor] review changes
0e50f6b [Matthew Taylor] test fix
99f1a31 [Matthew Taylor] partition ordering fix
369e618 [Matthew Taylor] partition ordering fix
2014-11-07 12:53:08 -08:00
Cheng Hao 60ab80f501 [SPARK-4272] [SQL] Add more unwrapper functions for primitive type in TableReader
Currently, the data "unwrap" only support couple of primitive types, not all, it will not cause exception, but may get some performance in table scanning for the type like binary, date, timestamp, decimal etc.

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

Closes #3136 from chenghao-intel/table_reader and squashes the following commits:

fffb729 [Cheng Hao] fix bug for retrieving the timestamp object
e9c97a4 [Cheng Hao] Add more unwrapper functions for primitive type in TableReader
2014-11-07 12:15:53 -08:00
Michael Armbrust 25bef7e695 [SQL] More aggressive defaults
- Turns on compression for in-memory cached data by default
 - Changes the default parquet compression format back to gzip (we have seen more OOMs with production workloads due to the way Snappy allocates memory)
 - Ups the batch size to 10,000 rows
 - Increases the broadcast threshold to 10mb.
 - Uses our parquet implementation instead of the hive one by default.
 - Cache parquet metadata by default.

Author: Michael Armbrust <michael@databricks.com>

Closes #3064 from marmbrus/fasterDefaults and squashes the following commits:

97ee9f8 [Michael Armbrust] parquet codec docs
e641694 [Michael Armbrust] Remote also
a12866a [Michael Armbrust] Cache metadata.
2d73acc [Michael Armbrust] Update docs defaults.
d63d2d5 [Michael Armbrust] document parquet option
da373f9 [Michael Armbrust] More aggressive defaults
2014-11-03 14:08:27 -08:00
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
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
wangfei e749f5dedb [SPARK-4191][SQL]move wrapperFor to HiveInspectors to reuse it
Move wrapperFor in InsertIntoHiveTable to HiveInspectors to reuse them, this method can be reused when writing date with ObjectInspector(such as orc support)

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #3057 from scwf/reuse-wraperfor and squashes the following commits:

7ccf932 [scwf] fix conflicts
d44f4da [wangfei] fix imports
9bf1b50 [wangfei] revert no related change
9a5276a [wangfei] move wrapfor to hiveinspector to reuse them
2014-11-02 15:45:55 -08:00
Cheng Lian c9f840046f [SPARK-3791][SQL] Provides Spark version and Hive version in HiveThriftServer2
This PR overrides the `GetInfo` Hive Thrift API to provide correct version information. Another property `spark.sql.hive.version` is added to reveal the underlying Hive version. These are generally useful for Spark SQL ODBC driver providers. The Spark version information is extracted from the jar manifest. Also took the chance to remove the `SET -v` hack, which was a workaround for Simba ODBC driver connectivity.

TODO

- [x] Find a general way to figure out Hive (or even any dependency) version.

  This [blog post](http://blog.soebes.de/blog/2014/01/02/version-information-into-your-appas-with-maven/) suggests several methods to inspect application version. In the case of Spark, this can be tricky because the chosen method:

  1. must applies to both Maven build and SBT build

    For Maven builds, we can retrieve the version information from the META-INF/maven directory within the assembly jar. But this doesn't work for SBT builds.

  2. must not rely on the original jars of dependencies to extract specific dependency version, because Spark uses assembly jar.

    This implies we can't read Hive version from Hive jar files since standard Spark distribution doesn't include them.

  3. should play well with `SPARK_PREPEND_CLASSES` to ease local testing during development.

     `SPARK_PREPEND_CLASSES` prevents classes to be loaded from the assembly jar, thus we can't locate the jar file and read its manifest.

  Given these, maybe the only reliable method is to generate a source file containing version information at build time. pwendell Do you have any suggestions from the perspective of the build process?

**Update** Hive version is now retrieved from the newly introduced `HiveShim` object.

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

Closes #2843 from liancheng/get-info and squashes the following commits:

a873d0f [Cheng Lian] Updates test case
53f43cd [Cheng Lian] Retrieves underlying Hive verson via HiveShim
1d282b8 [Cheng Lian] Removes the Simba ODBC "SET -v" hack
f857fce [Cheng Lian] Overrides Hive GetInfo Thrift API and adds Hive version property
2014-11-02 15:18:29 -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
wangfei f0a4b630ab [HOTFIX][SQL] hive test missing some golden files
cc marmbrus

Author: wangfei <wangfei1@huawei.com>

Closes #3055 from scwf/hotfix and squashes the following commits:

d881bd7 [wangfei] miss golden files
2014-11-02 14:59:41 -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
Cheng Lian ad0fde10b2 [SPARK-4037][SQL] Removes the SessionState instance created in HiveThriftServer2
`HiveThriftServer2` creates a global singleton `SessionState` instance and overrides `HiveContext` to inject the `SessionState` object. This messes up `SessionState` initialization and causes problems.

This PR replaces the global `SessionState` with `HiveContext.sessionState` to avoid the initialization conflict. Also `HiveContext` reuses existing started `SessionState` if any (this is required by `SparkSQLCLIDriver`, which uses specialized `CliSessionState`).

Author: Cheng Lian <lian@databricks.com>

Closes #2887 from liancheng/spark-4037 and squashes the following commits:

8446675 [Cheng Lian] Removes redundant Driver initialization
a28fef5 [Cheng Lian] Avoid starting HiveContext.sessionState multiple times
49b1c5b [Cheng Lian] Reuses existing started SessionState if any
3cd6fab [Cheng Lian] Fixes SPARK-4037
2014-11-01 15:03:11 -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
Venkata Ramana Gollamudi fa712b309c [SPARK-4077][SQL] Spark SQL return wrong values for valid string timestamp values
In org.apache.hadoop.hive.serde2.io.TimestampWritable.set , if the next entry is null then current time stamp object is being reset.
However because of this hiveinspectors:unwrap cannot use the same timestamp object without creating a copy.

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

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

Closes #3019 from gvramana/spark_4077 and squashes the following commits:

32d818f [Venkata Ramana Gollamudi] fixed check style
fa01e71 [Venkata Ramana Gollamudi] cloned timestamp object as org.apache.hadoop.hive.serde2.io.TimestampWritable.set will reset current time object
2014-10-31 11:30:28 -07:00
wangfei 7c41d13570 [SPARK-3826][SQL]enable hive-thriftserver to support hive-0.13.1
In #2241 hive-thriftserver is not enabled. This patch enable hive-thriftserver to support hive-0.13.1 by using a shim layer refer to #2241.

 1 A light shim layer(code in sql/hive-thriftserver/hive-version) for each different hive version to handle api compatibility

 2 New pom profiles "hive-default" and "hive-versions"(copy from #2241) to activate different hive version

 3 SBT cmd for different version as follows:
   hive-0.12.0 --- sbt/sbt -Phive,hadoop-2.3 -Phive-0.12.0 assembly
   hive-0.13.1 --- sbt/sbt -Phive,hadoop-2.3 -Phive-0.13.1 assembly

 4 Since hive-thriftserver depend on hive subproject, this patch should be merged with #2241 to enable hive-0.13.1 for hive-thriftserver

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #2685 from scwf/shim-thriftserver1 and squashes the following commits:

f26f3be [wangfei] remove clean to save time
f5cac74 [wangfei] remove local hivecontext test
578234d [wangfei] use new shaded hive
18fb1ff [wangfei] exclude kryo in hive pom
fa21d09 [wangfei] clean package assembly/assembly
8a4daf2 [wangfei] minor fix
0d7f6cf [wangfei] address comments
f7c93ae [wangfei] adding build with hive 0.13 before running tests
bcf943f [wangfei] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver1
c359822 [wangfei] reuse getCommandProcessor in hiveshim
52674a4 [scwf] sql/hive included since examples depend on it
3529e98 [scwf] move hive module to hive profile
f51ff4e [wangfei] update and fix conflicts
f48d3a5 [scwf] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver1
41f727b [scwf] revert pom changes
13afde0 [scwf] fix small bug
4b681f4 [scwf] enable thriftserver in profile hive-0.13.1
0bc53aa [scwf] fixed when result filed is null
dfd1c63 [scwf] update run-tests to run hive-0.12.0 default now
c6da3ce [scwf] Merge branch 'master' of https://github.com/apache/spark into shim-thriftserver
7c66b8e [scwf] update pom according spark-2706
ae47489 [scwf] update and fix conflicts
2014-10-31 11:27:59 -07:00
Cheng Hao 58a6077e56 [SPARK-4143] [SQL] Move inner class DeferredObjectAdapter to top level
The class DeferredObjectAdapter is the inner class of HiveGenericUdf, which may cause some overhead in closure ser/de-ser. Move it to top level.

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

Closes #3007 from chenghao-intel/move_deferred and squashes the following commits:

3a139b1 [Cheng Hao] Move inner class DeferredObjectAdapter to top level
2014-10-30 23:59:46 -07:00
Andrew Or 26d31d15fd Revert "SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop"
This reverts commit 68cb69daf3.
2014-10-30 17:56:10 -07:00
Sean Owen 68cb69daf3 SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop
(This is just a look at what completely moving the classes would look like. I know Patrick flagged that as maybe not OK, although, it's private?)

Author: Sean Owen <sowen@cloudera.com>

Closes #2814 from srowen/SPARK-1209 and squashes the following commits:

ead1115 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though?
2d42c1d [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark
2014-10-30 15:54:53 -07:00
Cheng Hao b5e79bf889 [SPARK-3904] [SQL] add constant objectinspector support for udfs
In HQL, we convert all of the data type into normal `ObjectInspector`s for UDFs, most of cases it works, however, some of the UDF actually requires its children `ObjectInspector` to be the `ConstantObjectInspector`, which will cause exception.
e.g.
select named_struct("x", "str") from src limit 1;

I updated the method `wrap` by adding the one more parameter `ObjectInspector`(to describe what it expects to wrap to, for example: java.lang.Integer or IntWritable).

As well as the `unwrap` method by providing the input `ObjectInspector`.

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

Closes #2762 from chenghao-intel/udf_coi and squashes the following commits:

bcacfd7 [Cheng Hao] Shim for both Hive 0.12 & 0.13.1
2416e5d [Cheng Hao] revert to hive 0.12
5793c01 [Cheng Hao] add space before while
4e56e1b [Cheng Hao] style issue
683d3fd [Cheng Hao] Add golden files
fe591e4 [Cheng Hao] update HiveGenericUdf for set the ObjectInspector while constructing the DeferredObject
f6740fe [Cheng Hao] Support Constant ObjectInspector for Map & List
8814c3a [Cheng Hao] Passing ContantObjectInspector(when necessary) for UDF initializing
2014-10-28 19:11:57 -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
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
wangxiaojing 0c34fa5b4b [SPARK-3907][SQL] Add truncate table support
JIRA issue: [SPARK-3907]https://issues.apache.org/jira/browse/SPARK-3907

Add turncate table support
TRUNCATE TABLE table_name [PARTITION partition_spec];
partition_spec:
  : (partition_col = partition_col_value, partition_col = partiton_col_value, ...)
Removes all rows from a table or partition(s). Currently target table should be native/managed table or exception will be thrown. User can specify partial partition_spec for truncating multiple partitions at once and omitting partition_spec will truncate all partitions in the table.

Author: wangxiaojing <u9jing@gmail.com>

Closes #2770 from wangxiaojing/spark-3907 and squashes the following commits:

63dbd81 [wangxiaojing] change hive scalastyle
7a03707 [wangxiaojing] add comment
f6e710e [wangxiaojing] change truncate table
a1f692c [wangxiaojing] Correct spelling mistakes
3b20007 [wangxiaojing] add truncate can not support column err message
e483547 [wangxiaojing] add golden file
77b1f20 [wangxiaojing]  add truncate table support
2014-10-27 22:02:52 -07:00
wangfei 89af6dfc3a [SPARK-4041][SQL] Attributes names in table scan should converted to lowercase when compare with relation attributes
In ```MetastoreRelation``` the attributes name is lowercase because of hive using lowercase for fields name, so we should convert attributes name in table scan lowercase in ```indexWhere(_.name == a.name)```.
```neededColumnIDs``` may be not correct if not convert to lowercase.

Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #2884 from scwf/fixColumnIds and squashes the following commits:

6174046 [scwf] use AttributeMap for this issue
dc74a24 [wangfei] use lowerName and add a test case for this issue
3ff3a80 [wangfei] more safer change
294fcb7 [scwf] attributes names in table scan should convert lowercase in neededColumnsIDs
2014-10-27 20:46:26 -07:00
Alex Liu 698a7eab77 [SPARK-3816][SQL] Add table properties from storage handler to output jobConf
...ob conf in SparkHadoopWriter class

Author: Alex Liu <alex_liu68@yahoo.com>

Closes #2677 from alexliu68/SPARK-SQL-3816 and squashes the following commits:

79c269b [Alex Liu] [SPARK-3816][SQL] Add table properties from storage handler to job conf
2014-10-27 20:43:29 -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
scwf f4e8c289d8 [SPARK-4042][SQL] Append columns ids and names before broadcast
Append columns ids and names before broadcast ```hiveExtraConf```  in ```HadoopTableReader```.

Author: scwf <wangfei1@huawei.com>

Closes #2885 from scwf/HadoopTableReader and squashes the following commits:

a8c498c [scwf] append columns ids and names before broadcast
2014-10-26 16:56: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
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 3a906c6631 [HOTFIX][SQL] Remove sleep on reset() failure.
Author: Michael Armbrust <michael@databricks.com>

Closes #2934 from marmbrus/patch-2 and squashes the following commits:

a96dab2 [Michael Armbrust] Remove sleep on reset() failure.
2014-10-24 14:03:03 -07:00
Zhan Zhang 7c89a8f0c8 [SPARK-2706][SQL] Enable Spark to support Hive 0.13
Given that a lot of users are trying to use hive 0.13 in spark, and the incompatibility between hive-0.12 and hive-0.13 on the API level I want to propose following approach, which has no or minimum impact on existing hive-0.12 support, but be able to jumpstart the development of hive-0.13 and future version support.

Approach: Introduce “hive-version” property,  and manipulate pom.xml files to support different hive version at compiling time through shim layer, e.g., hive-0.12.0 and hive-0.13.1. More specifically,

1. For each different hive version, there is a very light layer of shim code to handle API differences, sitting in sql/hive/hive-version, e.g., sql/hive/v0.12.0 or sql/hive/v0.13.1

2. Add a new profile hive-default active by default, which picks up all existing configuration and hive-0.12.0 shim (v0.12.0)  if no hive.version is specified.

3. If user specifies different version (currently only 0.13.1 by -Dhive.version = 0.13.1), hive-versions profile will be activated, which pick up hive-version specific shim layer and configuration, mainly the hive jars and hive-version shim, e.g., v0.13.1.

4. With this approach, nothing is changed with current hive-0.12 support.

No change by default: sbt/sbt -Phive
For example: sbt/sbt -Phive -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly

To enable hive-0.13: sbt/sbt -Dhive.version=0.13.1
For example: sbt/sbt -Dhive.version=0.13.1 -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 assembly

Note that in hive-0.13, hive-thriftserver is not enabled, which should be fixed by other Jira, and we don’t need -Phive with -Dhive.version in building (probably we should use -Phive -Dhive.version=xxx instead after thrift server is also supported in hive-0.13.1).

Author: Zhan Zhang <zhazhan@gmail.com>
Author: zhzhan <zhazhan@gmail.com>
Author: Patrick Wendell <pwendell@gmail.com>

Closes #2241 from zhzhan/spark-2706 and squashes the following commits:

3ece905 [Zhan Zhang] minor fix
410b668 [Zhan Zhang] solve review comments
cbb4691 [Zhan Zhang] change run-test for new options
0d4d2ed [Zhan Zhang] rebase
497b0f4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
8fad1cf [Zhan Zhang] change the pom file and make hive-0.13.1 as the default
ab028d1 [Zhan Zhang] rebase
4a2e36d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
4cb1b93 [zhzhan] Merge pull request #1 from pwendell/pr-2241
b0478c0 [Patrick Wendell] Changes to simplify the build of SPARK-2706
2b50502 [Zhan Zhang] rebase
a72c0d4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cb22863 [Zhan Zhang] correct the typo
20f6cf7 [Zhan Zhang] solve compatability issue
f7912a9 [Zhan Zhang] rebase and solve review feedback
301eb4a [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
10c3565 [Zhan Zhang] address review comments
6bc9204 [Zhan Zhang] rebase and remove temparory repo
d3aa3f2 [Zhan Zhang] Merge branch 'master' into spark-2706
cedcc6f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3ced0d7 [Zhan Zhang] rebase
d9b981d [Zhan Zhang] rebase and fix error due to rollback
adf4924 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3dd50e8 [Zhan Zhang] solve conflicts and remove unnecessary implicts
d10bf00 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
dc7bdb3 [Zhan Zhang] solve conflicts
7e0cc36 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
d7c3e1e [Zhan Zhang] Merge branch 'master' into spark-2706
68deb11 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
d48bd18 [Zhan Zhang] address review comments
3ee3b2b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
57ea52e [Zhan Zhang] Merge branch 'master' into spark-2706
2b0d513 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
9412d24 [Zhan Zhang] address review comments
f4af934 [Zhan Zhang] rebase
1ccd7cc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
128b60b [Zhan Zhang] ignore 0.12.0 test cases for the time being
af9feb9 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
5f5619f [Zhan Zhang] restructure the directory and different hive version support
05d3683 [Zhan Zhang] solve conflicts
e4c1982 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
94b4fdc [Zhan Zhang] Spark-2706: hive-0.13.1 support on spark
87ebf3b [Zhan Zhang] Merge branch 'master' into spark-2706
921e914 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
f896b2a [Zhan Zhang] Merge branch 'master' into spark-2706
789ea21 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cb53a2c [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
f6a8a40 [Zhan Zhang] revert
ba14f28 [Zhan Zhang] test
dbedff3 [Zhan Zhang] Merge remote-tracking branch 'upstream/master'
70964fe [Zhan Zhang] revert
fe0f379 [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark
70ffd93 [Zhan Zhang] revert
42585ec [Zhan Zhang] test
7d5fce2 [Zhan Zhang] test
2014-10-24 11:03:17 -07:00
wangfei 856b081729 [SQL]redundant methods for broadcast
redundant methods for broadcast in ```TableReader```

Author: wangfei <wangfei1@huawei.com>

Closes #2862 from scwf/TableReader and squashes the following commits:

414cc24 [wangfei] unnecessary methods for broadcast
2014-10-21 16:20:58 -07:00
chirag e6e37701f1 SPARK-3807: SparkSql does not work for tables created using custom serde
SparkSql crashes on selecting tables using custom serde.

Example:
----------------

CREATE EXTERNAL TABLE table_name PARTITIONED BY ( a int) ROW FORMAT 'SERDE "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer" with serdeproperties("serialization.format"="org.apache.thrift.protocol.TBinaryProtocol","serialization.class"="ser_class") STORED AS SEQUENCEFILE;

The following exception is seen on running a query like 'select * from table_name limit 1':

ERROR CliDriver: org.apache.hadoop.hive.serde2.SerDeException: java.lang.NullPointerException
at org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer.initialize(ThriftDeserializer.java:68)
at org.apache.hadoop.hive.ql.plan.TableDesc.getDeserializer(TableDesc.java:80)
at org.apache.spark.sql.hive.execution.HiveTableScan.addColumnMetadataToConf(HiveTableScan.scala:86)
at org.apache.spark.sql.hive.execution.HiveTableScan.<init>(HiveTableScan.scala:100)
at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188)
at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$$anonfun$14.apply(HiveStrategies.scala:188)
at org.apache.spark.sql.SQLContext$SparkPlanner.pruneFilterProject(SQLContext.scala:364)
at org.apache.spark.sql.hive.HiveStrategies$HiveTableScans$.apply(HiveStrategies.scala:184)
at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59)
at org.apache.spark.sql.catalyst.planning.QueryPlanner.planLater(QueryPlanner.scala:54)
at org.apache.spark.sql.execution.SparkStrategies$BasicOperators$.apply(SparkStrategies.scala:280)
at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59)
at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:402)
at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:400)
at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:406)
at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:406)
at org.apache.spark.sql.hive.HiveContext$QueryExecution.stringResult(HiveContext.scala:406)
at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:59)
at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:291)
at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413)
at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:226)
at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at sun.reflect.NativeMethodAccessorImpl.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(Unknown Source)
at java.lang.reflect.Method.invoke(Unknown Source)
at org.apache.spark.deploy.SparkSubmit$.launch(SparkSubmit.scala:328)
at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:75)
at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.lang.NullPointerException

Author: chirag <chirag.aggarwal@guavus.com>

Closes #2674 from chiragaggarwal/branch-1.1 and squashes the following commits:

370c31b [chirag] SPARK-3807: Add a test case to validate the fix.
1f26805 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde (Incorporated Review Comments)
ba4bc0c [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde
5c73b72 [chirag] SPARK-3807: SparkSql does not work for tables created using custom serde

(cherry picked from commit 925e22d313)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2014-10-13 13:47:51 -07:00
Venkata Ramana Gollamudi e10d71e7e5 [SPARK-3559][SQL] Remove unnecessary columns from List of needed Column Ids in Hive Conf
Author: Venkata Ramana G <ramana.gollamudihuawei.com>

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

Closes #2713 from gvramana/remove_unnecessary_columns and squashes the following commits:

b7ba768 [Venkata Ramana Gollamudi] Added comment and checkstyle fix
6a93459 [Venkata Ramana Gollamudi] cloned hiveconf for each TableScanOperators so that only required columns are added
2014-10-13 13:45:34 -07:00
Cheng Hao d3cdf9128a [SPARK-3529] [SQL] Delete the temp files after test exit
There are lots of temporal files created by TestHive under the /tmp by default, which may cause potential performance issue for testing. This PR will automatically delete them after test exit.

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

Closes #2393 from chenghao-intel/delete_temp_on_exit and squashes the following commits:

3a6511f [Cheng Hao] Remove the temp dir after text exit
2014-10-13 13:40:20 -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
ravipesala 6f98902a3d [SPARK-3834][SQL] Backticks not correctly handled in subquery aliases
The queries like SELECT a.key FROM (SELECT key FROM src) \`a\` does not work as backticks in subquery aliases are not handled properly. This PR fixes that.

Author : ravipesala ravindra.pesalahuawei.com

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

Closes #2737 from ravipesala/SPARK-3834 and squashes the following commits:

0e0ab98 [ravipesala] Fixing issue in backtick handling for subquery aliases
2014-10-09 18:41:36 -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
Vida Ha b77a02f41c [SPARK-3752][SQL]: Add tests for different UDF's
Author: Vida Ha <vida@databricks.com>

Closes #2621 from vidaha/vida/SPARK-3752 and squashes the following commits:

d7fdbbc [Vida Ha] Add tests for different UDF's
2014-10-09 13:13:31 -07:00
Reynold Xin bcb1ae049b [SPARK-3857] Create joins package for various join operators.
Author: Reynold Xin <rxin@apache.org>

Closes #2719 from rxin/sql-join-break and squashes the following commits:

0c0082b [Reynold Xin] Fix line length.
cbc664c [Reynold Xin] Rename join -> joins package.
a070d44 [Reynold Xin] Fix line length in HashJoin
a39be8c [Reynold Xin] [SPARK-3857] Create a join package for various join operators.
2014-10-08 18:17:01 -07:00
Cheng Lian 3e4f09d2fc [SQL] Prevents per row dynamic dispatching and pattern matching when inserting Hive values
Builds all wrappers at first according to object inspector types to avoid per row costs.

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

Closes #2592 from liancheng/hive-value-wrapper and squashes the following commits:

9696559 [Cheng Lian] Passes all tests
4998666 [Cheng Lian] Prevents per row dynamic dispatching and pattern matching when inserting Hive values
2014-10-08 18:13:22 -07:00
Cheng Lian e703357233 [SPARK-3810][SQL] Makes PreInsertionCasts handle partitions properly
Includes partition keys into account when applying `PreInsertionCasts` rule.

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

Closes #2672 from liancheng/fix-pre-insert-casts and squashes the following commits:

def1a1a [Cheng Lian] Makes PreInsertionCasts handle partitions properly
2014-10-08 18:11:18 -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
scwf 58f5361caa [SPARK-3792][SQL] Enable JavaHiveQLSuite
Do not use TestSQLContext in JavaHiveQLSuite, that may lead to two SparkContexts in one jvm and enable JavaHiveQLSuite

Author: scwf <wangfei1@huawei.com>

Closes #2652 from scwf/fix-JavaHiveQLSuite and squashes the following commits:

be35c91 [scwf] enable JavaHiveQLSuite
2014-10-05 17:49:41 -07:00
Liang-Chi Hsieh 79b2108de3 [Minor] Trivial fix to make codes more readable
It should just use `maxResults` there.

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

Closes #2654 from viirya/trivial_fix and squashes the following commits:

1362289 [Liang-Chi Hsieh] Trivial fix to make codes more readable.
2014-10-05 17:44:38 -07:00
Cheng Lian 1b97a941a0 [SPARK-3007][SQL] Fixes dynamic partitioning support for lower Hadoop versions
This is a follow up of #2226 and #2616 to fix Jenkins master SBT build failures for lower Hadoop versions (1.0.x and 2.0.x).

The root cause is the semantics difference of `FileSystem.globStatus()` between different versions of Hadoop, as illustrated by the following test code:

```scala
object GlobExperiments extends App {
  val conf = new Configuration()
  val fs = FileSystem.getLocal(conf)
  fs.globStatus(new Path("/tmp/wh/*/*/*")).foreach { status =>
    println(status.getPath)
  }
}
```

Target directory structure:

```
/tmp/wh
├── dir0
│   ├── dir1
│   │   └── level2
│   └── level1
└── level0
```

Hadoop 2.4.1 result:

```
file:/tmp/wh/dir0/dir1/level2
```

Hadoop 1.0.4 resuet:

```
file:/tmp/wh/dir0/dir1/level2
file:/tmp/wh/dir0/level1
file:/tmp/wh/level0
```

In #2226 and #2616, we call `FileOutputCommitter.commitJob()` at the end of the job, and the `_SUCCESS` mark file is written. When working with lower Hadoop versions, due to the `globStatus()` semantics issue, `_SUCCESS` is included as a separate partition data file by `Hive.loadDynamicPartitions()`, and fails partition spec checking.  The fix introduced in this PR is kind of a hack: when inserting data with dynamic partitioning, we intentionally avoid writing the `_SUCCESS` marker to workaround this issue.

Hive doesn't suffer this issue because `FileSinkOperator` doesn't call `FileOutputCommitter.commitJob()`, instead, it calls `Utilities.mvFileToFinalPath()` to cleanup the output directory and then loads it into Hive warehouse by with `loadDynamicPartitions()`/`loadPartition()`/`loadTable()`. This approach is better because it handles failed job and speculative tasks properly. We should add this step to `InsertIntoHiveTable` in another PR.

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

Closes #2663 from liancheng/dp-hadoop-1-fix and squashes the following commits:

0177dae [Cheng Lian] Fixes dynamic partitioning support for lower Hadoop versions
2014-10-05 11:19:17 -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
Cheng Lian bec0d0eaa3 [SPARK-3007][SQL] Adds dynamic partitioning support
PR #2226 was reverted because it broke Jenkins builds for unknown reason. This debugging PR aims to fix the Jenkins build.

This PR also fixes two bugs:

1. Compression configurations in `InsertIntoHiveTable` are disabled by mistake

   The `FileSinkDesc` object passed to the writer container doesn't have compression related configurations. These configurations are not taken care of until `saveAsHiveFile` is called. This PR moves compression code forward, right after instantiation of the `FileSinkDesc` object.

1. `PreInsertionCasts` doesn't take table partitions into account

   In `castChildOutput`, `table.attributes` only contains non-partition columns, thus for partitioned table `childOutputDataTypes` never equals to `tableOutputDataTypes`. This results funny analyzed plan like this:

   ```
   == Analyzed Logical Plan ==
   InsertIntoTable Map(partcol1 -> None, partcol2 -> None), false
    MetastoreRelation default, dynamic_part_table, None
    Project [c_0#1164,c_1#1165,c_2#1166]
     Project [c_0#1164,c_1#1165,c_2#1166]
      Project [c_0#1164,c_1#1165,c_2#1166]
       ... (repeats 99 times) ...
        Project [c_0#1164,c_1#1165,c_2#1166]
         Project [c_0#1164,c_1#1165,c_2#1166]
          Project [1 AS c_0#1164,1 AS c_1#1165,1 AS c_2#1166]
           Filter (key#1170 = 150)
            MetastoreRelation default, src, None
   ```

   Awful though this logical plan looks, it's harmless because all projects will be eliminated by optimizer. Guess that's why this issue hasn't been caught before.

Author: Cheng Lian <lian.cs.zju@gmail.com>
Author: baishuo(白硕) <vc_java@hotmail.com>
Author: baishuo <vc_java@hotmail.com>

Closes #2616 from liancheng/dp-fix and squashes the following commits:

21935b6 [Cheng Lian] Adds back deleted trailing space
f471c4b [Cheng Lian] PreInsertionCasts should take table partitions into account
a132c80 [Cheng Lian] Fixes output compression
9c6eb2d [Cheng Lian] Adds tests to verify dynamic partitioning folder layout
0eed349 [Cheng Lian] Addresses @yhuai's comments
26632c3 [Cheng Lian] Adds more tests
9227181 [Cheng Lian] Minor refactoring
c47470e [Cheng Lian] Refactors InsertIntoHiveTable to a Command
6fb16d7 [Cheng Lian] Fixes typo in test name, regenerated golden answer files
d53daa5 [Cheng Lian] Refactors dynamic partitioning support
b821611 [baishuo] pass check style
997c990 [baishuo] use HiveConf.DEFAULTPARTITIONNAME to replace hive.exec.default.partition.name
761ecf2 [baishuo] modify according micheal's advice
207c6ac [baishuo] modify for some bad indentation
caea6fb [baishuo] modify code to pass scala style checks
b660e74 [baishuo] delete a empty else branch
cd822f0 [baishuo] do a little modify
8e7268c [baishuo] update file after test
3f91665 [baishuo(白硕)] Update Cast.scala
8ad173c [baishuo(白硕)] Update InsertIntoHiveTable.scala
051ba91 [baishuo(白硕)] Update Cast.scala
d452eb3 [baishuo(白硕)] Update HiveQuerySuite.scala
37c603b [baishuo(白硕)] Update InsertIntoHiveTable.scala
98cfb1f [baishuo(白硕)] Update HiveCompatibilitySuite.scala
6af73f4 [baishuo(白硕)] Update InsertIntoHiveTable.scala
adf02f1 [baishuo(白硕)] Update InsertIntoHiveTable.scala
1867e23 [baishuo(白硕)] Update SparkHadoopWriter.scala
6bb5880 [baishuo(白硕)] Update HiveQl.scala
2014-10-03 12:26:02 -07:00
ravipesala 22f8e1ee7c [SPARK-2693][SQL] Supported for UDAF Hive Aggregates like PERCENTILE
Implemented UDAF Hive aggregates by adding wrapper to Spark Hive.

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

Closes #2620 from ravipesala/SPARK-2693 and squashes the following commits:

a8df326 [ravipesala] Removed resolver from constructor arguments
caf25c6 [ravipesala] Fixed style issues
5786200 [ravipesala] Supported for UDAF Hive Aggregates like PERCENTILE
2014-10-03 11:25:18 -07:00
ravipesala 1c90347a4b [SPARK-3654][SQL] Implement all extended HiveQL statements/commands with a separate parser combinator
Created separate parser for hql. It preparses the commands like cache,uncache,add jar etc.. and then parses with HiveQl

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

Closes #2590 from ravipesala/SPARK-3654 and squashes the following commits:

bbca7dd [ravipesala] Fixed code as per admin comments.
ae9290a [ravipesala] Fixed style issues as per Admin comments
898ed81 [ravipesala] Removed spaces
fb24edf [ravipesala] Updated the code as per admin comments
8947d37 [ravipesala] Removed duplicate code
ba26cd1 [ravipesala] Created seperate parser for hql.It pre parses the commands like cache,uncache,add jar etc.. and then parses with HiveQl
2014-10-02 20:04:33 -07:00
Michael Armbrust 7de4e50a01 [SQL] Initilize session state before creating CommandProcessor
With the old ordering it was possible for commands in the HiveDriver to NPE due to the lack of configuration in the threadlocal session state.

Author: Michael Armbrust <michael@databricks.com>

Closes #2635 from marmbrus/initOrder and squashes the following commits:

9749850 [Michael Armbrust] Initilize session state before creating CommandProcessor
2014-10-02 19:32:21 -07:00
Michael Armbrust 45e058ca4b [SPARK-3729][SQL] Do all hive session state initialization in lazy val
This change avoids a NPE during context initialization when settings are present.

Author: Michael Armbrust <michael@databricks.com>

Closes #2583 from marmbrus/configNPE and squashes the following commits:

da2ec57 [Michael Armbrust] Do all hive session state initilialization in lazy val
2014-10-01 16:30:28 -07:00
Cheng Lian a31f4ff22f [SQL] Made Command.sideEffectResult protected
Considering `Command.executeCollect()` simply delegates to `Command.sideEffectResult`, we no longer need to leave the latter `protected[sql]`.

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

Closes #2431 from liancheng/narrow-scope and squashes the following commits:

1bfc16a [Cheng Lian] Made Command.sideEffectResult protected
2014-10-01 16:00:29 -07:00
scwf f315fb7efc [SPARK-3705][SQL] Add case for VoidObjectInspector to cover NullType
add case for VoidObjectInspector in ```inspectorToDataType```

Author: scwf <wangfei1@huawei.com>

Closes #2552 from scwf/inspectorToDataType and squashes the following commits:

453d892 [scwf] add case for VoidObjectInspector
2014-10-01 15:55:13 -07:00
ravipesala 3508ce8a5a [SPARK-3708][SQL] Backticks aren't handled correctly is aliases
The below query gives error
sql("SELECT k FROM (SELECT \`key\` AS \`k\` FROM src) a")
It gives error because the aliases are not cleaned so it could not be resolved in further processing.

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

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

d55db54 [ravipesala] Fixed SPARK-3708 (Backticks aren't handled correctly is aliases)
2014-10-01 15:44:41 -07:00
Michael Armbrust fcad3fae61 [SPARK-3746][SQL] Lock hive client when creating tables
Author: Michael Armbrust <michael@databricks.com>

Closes #2598 from marmbrus/hiveClientLock and squashes the following commits:

ca89fe8 [Michael Armbrust] Lock hive client when creating tables
2014-10-01 14:37:27 -07:00
Cheng Lian 8cc70e7e15 [SQL] Kill dangerous trailing space in query string
MD5 of query strings in `createQueryTest` calls are used to generate golden files, leaving trailing spaces there can be really dangerous. Got bitten by this while working on #2616: my "smart" IDE automatically removed a trailing space and makes Jenkins fail.

(Really should add "no trailing space" to our coding style guidelines!)

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

Closes #2619 from liancheng/kill-trailing-space and squashes the following commits:

034f119 [Cheng Lian] Kill dangerous trailing space in query string
2014-10-01 12:40:37 -07:00
Reynold Xin 3888ee2f38 [SPARK-3748] Log thread name in unit test logs
Thread names are useful for correlating failures.

Author: Reynold Xin <rxin@apache.org>

Closes #2600 from rxin/log4j and squashes the following commits:

83ffe88 [Reynold Xin] [SPARK-3748] Log thread name in unit test logs
2014-10-01 01:03:49 -07:00
Patrick Wendell b64fcbd2dc Revert "[SPARK-3007][SQL]Add Dynamic Partition support to Spark Sql hive"
This reverts commit 0bbe7faeff.
2014-09-30 09:43:46 -07:00
baishuo(白硕) 0bbe7faeff [SPARK-3007][SQL]Add Dynamic Partition support to Spark Sql hive
a new PR base on new master.  changes are the same as https://github.com/apache/spark/pull/1919

Author: baishuo(白硕) <vc_java@hotmail.com>
Author: baishuo <vc_java@hotmail.com>
Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2226 from baishuo/patch-3007 and squashes the following commits:

e69ce88 [Cheng Lian] Adds tests to verify dynamic partitioning folder layout
b20a3dc [Cheng Lian] Addresses @yhuai's comments
096bbbc [baishuo(白硕)] Merge pull request #1 from liancheng/refactor-dp
1093c20 [Cheng Lian] Adds more tests
5004542 [Cheng Lian] Minor refactoring
fae9eff [Cheng Lian] Refactors InsertIntoHiveTable to a Command
528e84c [Cheng Lian] Fixes typo in test name, regenerated golden answer files
c464b26 [Cheng Lian] Refactors dynamic partitioning support
5033928 [baishuo] pass check style
2201c75 [baishuo] use HiveConf.DEFAULTPARTITIONNAME to replace hive.exec.default.partition.name
b47c9bf [baishuo] modify according micheal's advice
c3ab36d [baishuo] modify for some bad indentation
7ce2d9f [baishuo] modify code to pass scala style checks
37c1c43 [baishuo] delete a empty else branch
66e33fc [baishuo] do a little modify
88d0110 [baishuo] update file after test
a3961d9 [baishuo(白硕)] Update Cast.scala
f7467d0 [baishuo(白硕)] Update InsertIntoHiveTable.scala
c1a59dd [baishuo(白硕)] Update Cast.scala
0e18496 [baishuo(白硕)] Update HiveQuerySuite.scala
60f70aa [baishuo(白硕)] Update InsertIntoHiveTable.scala
0a50db9 [baishuo(白硕)] Update HiveCompatibilitySuite.scala
491c7d0 [baishuo(白硕)] Update InsertIntoHiveTable.scala
a2374a8 [baishuo(白硕)] Update InsertIntoHiveTable.scala
701a814 [baishuo(白硕)] Update SparkHadoopWriter.scala
dc24c41 [baishuo(白硕)] Update HiveQl.scala
2014-09-29 15:51:55 -07:00
Michael Armbrust f0c7e19550 [SPARK-3680][SQL] Fix bug caused by eager typing of HiveGenericUDFs
Typing of UDFs should be lazy as it is often not valid to call `dataType` on an expression until after all of its children are `resolved`.

Author: Michael Armbrust <michael@databricks.com>

Closes #2525 from marmbrus/concatBug and squashes the following commits:

5b8efe7 [Michael Armbrust] fix bug with eager typing of udfs
2014-09-27 12:10:16 -07:00
w00228970 0800881051 [SPARK-3676][SQL] Fix hive test suite failure due to diffs in JDK 1.6/1.7
This is a bug in JDK6: http://bugs.java.com/bugdatabase/view_bug.do?bug_id=4428022

this is because jdk get different result to operate ```double```,
```System.out.println(1/500d)``` in different jdk get different result
jdk 1.6.0(_31) ---- 0.0020
jdk 1.7.0(_05) ---- 0.002
this leads to HiveQuerySuite failed when generate golden answer in jdk 1.7 and run tests in jdk 1.6, result did not match

Author: w00228970 <wangfei1@huawei.com>

Closes #2517 from scwf/HiveQuerySuite and squashes the following commits:

0cb5e8d [w00228970] delete golden answer of division-0 and timestamp cast #1
1df3964 [w00228970] Jdk version leads to different query output for Double, this make HiveQuerySuite failed
2014-09-27 12:06:16 -07:00
Daoyuan Wang 0ec2d2e8f0 [SPARK-3531][SQL]select null from table would throw a MatchError
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #2396 from adrian-wang/selectnull and squashes the following commits:

2458229 [Daoyuan Wang] rebase solution
2014-09-26 12:04:37 -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
wangfei ae60f8fb2d [SPARK-3481][SQL] removes the evil MINOR HACK
a follow up of https://github.com/apache/spark/pull/2377 and https://github.com/apache/spark/pull/2352, see detail there.

Author: wangfei <wangfei1@huawei.com>

Closes #2505 from scwf/patch-6 and squashes the following commits:

4874ec8 [wangfei] removes the evil MINOR HACK
2014-09-23 11:59:44 -07:00
Daoyuan Wang 116016b481 [SPARK-3582][SQL] not limit argument type for hive simple udf
Since we have moved to `ConventionHelper`, it is quite easy to avoid call `javaClassToDataType` in hive simple udf. This will solve SPARK-3582.

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

Closes #2506 from adrian-wang/spark3582 and squashes the following commits:

450c28e [Daoyuan Wang] not limit argument type for hive simple udf
2014-09-23 11:47:53 -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
Daoyuan Wang ba68a51c40 [SPARK-3485][SQL] Use GenericUDFUtils.ConversionHelper for Simple UDF type conversions
This is just another solution to SPARK-3485, in addition to PR #2355
In this patch, we will use ConventionHelper and FunctionRegistry to invoke a simple udf evaluation, which rely more on hive, but much cleaner and safer.
We can discuss which one is better.

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

Closes #2407 from adrian-wang/simpleudf and squashes the following commits:

15762d2 [Daoyuan Wang] add posmod test which would fail the test but now ok
0d69eb4 [Daoyuan Wang] another way to pass to hive simple udf
2014-09-19 15:39:31 -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
Bertrand Bossy c243b21a8b SPARK-3039: Allow spark to be built using avro-mapred for hadoop2
SPARK-3039: Adds the maven property "avro.mapred.classifier" to build spark-assembly with avro-mapred with support for the new Hadoop API. Sets this property to hadoop2 for Hadoop 2 profiles.

I am not very familiar with maven, nor do I know whether this potentially breaks something in the hive part of spark. There might be a more elegant way of doing this.

Author: Bertrand Bossy <bertrandbossy@gmail.com>

Closes #1945 from bbossy/SPARK-3039 and squashes the following commits:

c32ce59 [Bertrand Bossy] SPARK-3039: Allow spark to be built using avro-mapred for hadoop2
2014-09-14 21:10:17 -07:00
Michael Armbrust 0f8c4edf4e [SQL] Decrease partitions when testing
Author: Michael Armbrust <michael@databricks.com>

Closes #2164 from marmbrus/shufflePartitions and squashes the following commits:

0da1e8c [Michael Armbrust] test hax
ef2d985 [Michael Armbrust] more test hacks.
2dabae3 [Michael Armbrust] more test fixes
0bdbf21 [Michael Armbrust] Make parquet tests less order dependent
b42eeab [Michael Armbrust] increase test parallelism
80453d5 [Michael Armbrust] Decrease partitions when testing
2014-09-13 16:08:04 -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
Cheng Lian 184cd51c42 [SPARK-3481][SQL] Removes the evil MINOR HACK
This is a follow up of #2352. Now we can finally remove the evil "MINOR HACK", which covered up the eldest bug in the history of Spark SQL (see details [here](https://github.com/apache/spark/pull/2352#issuecomment-55440621)).

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

Closes #2377 from liancheng/remove-evil-minor-hack and squashes the following commits:

0869c78 [Cheng Lian] Removes the evil MINOR HACK
2014-09-13 12:35:40 -07:00
Cheng Lian 6d887db789 [SPARK-3515][SQL] Moves test suite setup code to beforeAll rather than in constructor
Please refer to the JIRA ticket for details.

**NOTE** We should check all test suites that do similar initialization-like side effects in their constructors. This PR only fixes `ParquetMetastoreSuite` because it breaks our Jenkins Maven build.

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

Closes #2375 from liancheng/say-no-to-constructor and squashes the following commits:

0ceb75b [Cheng Lian] Moves test suite setup code to beforeAll rather than in constructor
2014-09-12 20:14:09 -07:00
Cheng Hao 8194fc662c [SPARK-3481] [SQL] Eliminate the error log in local Hive comparison test
Logically, we should remove the Hive Table/Database first and then reset the Hive configuration, repoint to the new data warehouse directory etc.
Otherwise it raised exceptions like "Database doesn't not exists: default" in the local testing.

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

Closes #2352 from chenghao-intel/test_hive and squashes the following commits:

74fd76b [Cheng Hao] eliminate the error log
2014-09-12 11:29:30 -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
Michael Armbrust 84e2c8bfe4 [SQL] Add test case with workaround for reading partitioned Avro files
In order to read from partitioned Avro files we need to also set the `SERDEPROPERTIES` since `TBLPROPERTIES` are not passed to the initialization.  This PR simply adds a test to make sure we don't break this workaround.

Author: Michael Armbrust <michael@databricks.com>

Closes #2340 from marmbrus/avroPartitioned and squashes the following commits:

6b969d6 [Michael Armbrust] fix style
fea2124 [Michael Armbrust] Add test case with workaround for reading partitioned avro files.
2014-09-10 20:57:38 -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 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
William Benton 2b7ab814f9 [SPARK-3329][SQL] Don't depend on Hive SET pair ordering in tests.
This fixes some possible spurious test failures in `HiveQuerySuite` by comparing sets of key-value pairs as sets, rather than as lists.

Author: William Benton <willb@redhat.com>
Author: Aaron Davidson <aaron@databricks.com>

Closes #2220 from willb/spark-3329 and squashes the following commits:

3b3e205 [William Benton] Collapse collectResults case match in HiveQuerySuite
6525d8e [William Benton] Handle cases where SET returns Rows of (single) strings
cf11b0e [Aaron Davidson] Fix flakey HiveQuerySuite test
2014-09-08 19:29:23 -07:00
Cheng Lian dc1dbf206e [SPARK-3414][SQL] Stores analyzed logical plan when registering a temp table
Case insensitivity breaks when unresolved relation contains attributes with uppercase letters in their names, because we store unanalyzed logical plan when registering temp tables while the `CaseInsensitivityAttributeReferences` batch runs before the `Resolution` batch. To fix this issue, we need to store analyzed logical plan.

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

Closes #2293 from liancheng/spark-3414 and squashes the following commits:

d9fa1d6 [Cheng Lian] Stores analyzed logical plan when registering a temp table
2014-09-08 19:08:05 -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 ee575f12f2 [SPARK-2219][SQL] Added support for the "add jar" command
Adds logical and physical command classes for the "add jar" command.

Note that this PR conflicts with and should be merged after #2215.

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

Closes #2242 from liancheng/add-jar and squashes the following commits:

e43a2f1 [Cheng Lian] Updates AddJar according to conventions introduced in #2215
b99107f [Cheng Lian] Added test case for ADD JAR command
095b2c7 [Cheng Lian] Also forward ADD JAR command to Hive
9be031b [Cheng Lian] Trims Jar path string
8195056 [Cheng Lian] Added support for the "add jar" command
2014-09-04 18:47:45 -07:00
Kousuke Saruta dc1ba9e9fc [SPARK-3378] [DOCS] Replace the word "SparkSQL" with right word "Spark SQL"
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #2251 from sarutak/SPARK-3378 and squashes the following commits:

0bfe234 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3378
bb5938f [Kousuke Saruta] Replaced rest of "SparkSQL" with "Spark SQL"
6df66de [Kousuke Saruta] Replaced "SparkSQL" with "Spark SQL"
2014-09-04 15:06:08 -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
Cheng Lian f48420fde5 [SPARK-2973][SQL] Lightweight SQL commands without distributed jobs when calling .collect()
By overriding `executeCollect()` in physical plan classes of all commands, we can avoid to kick off a distributed job when collecting result of a SQL command, e.g. `sql("SET").collect()`.

Previously, `Command.sideEffectResult` returns a `Seq[Any]`, and the `execute()` method in sub-classes of `Command` typically convert that to a `Seq[Row]` then parallelize it to an RDD. Now with this PR, `sideEffectResult` is required to return a `Seq[Row]` directly, so that `executeCollect()` can directly leverage that and be factored to the `Command` parent class.

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

Closes #2215 from liancheng/lightweight-commands and squashes the following commits:

3fbef60 [Cheng Lian] Factored execute() method of physical commands to parent class Command
5a0e16c [Cheng Lian] Passes test suites
e0e12e9 [Cheng Lian] Refactored Command.sideEffectResult and Command.executeCollect
995bdd8 [Cheng Lian] Cleaned up DescribeHiveTableCommand
542977c [Cheng Lian] Avoids confusion between logical and physical plan by adding package prefixes
55b2aa5 [Cheng Lian] Avoids distributed jobs when execution SQL commands
2014-09-03 18:57:20 -07:00
qiping.lqp 634d04b87c [SPARK-3291][SQL]TestcaseName in createQueryTest should not contain ":"
":" is not allowed to appear in a file name of Windows system. If file name contains ":", this file can't be checked out in a Windows system and developers using Windows must be careful to not commit the deletion of such files, Which is very inconvenient.

Author: qiping.lqp <qiping.lqp@alibaba-inc.com>

Closes #2191 from chouqin/querytest and squashes the following commits:

0e943a1 [qiping.lqp] rename golden file
60a863f [qiping.lqp] TestcaseName in createQueryTest should not contain ":"
2014-08-29 15:37:43 -07:00
Cheng Lian b1eccfc88a [SQL] Turns on in-memory columnar compression in HiveCompatibilitySuite
`HiveCompatibilitySuite` already turns on in-memory columnar caching, it would be good to also enable compression to improve test coverage.

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

Closes #2190 from liancheng/compression-on and squashes the following commits:

88b536c [Cheng Lian] Code cleanup, narrowed field visibility
d13efd2 [Cheng Lian] Turns on in-memory columnar compression in HiveCompatibilitySuite
2014-08-29 15:34:59 -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
luogankun 65253502b9 [SPARK-3065][SQL] Add locale setting to fix results do not match for udf_unix_timestamp format "yyyy MMM dd h:mm:ss a" run with not "America/Los_Angeles" TimeZone in HiveCompatibilitySuite
When run the udf_unix_timestamp of org.apache.spark.sql.hive.execution.HiveCompatibilitySuite testcase
with not "America/Los_Angeles" TimeZone throws error. [https://issues.apache.org/jira/browse/SPARK-3065]
add locale setting on beforeAll and afterAll method to fix the bug of HiveCompatibilitySuite testcase

Author: luogankun <luogankun@gmail.com>

Closes #1968 from luogankun/SPARK-3065 and squashes the following commits:

c167832 [luogankun] [SPARK-3065][SQL] Add Locale setting to HiveCompatibilitySuite
0a25e3a [luogankun] [SPARK-3065][SQL] Add Locale setting to HiveCompatibilitySuite
2014-08-27 15:08:22 -07:00
Aaron Davidson cc275f4b79 [SQL] [SPARK-3236] Reading Parquet tables from Metastore mangles location
Currently we do `relation.hiveQlTable.getDataLocation.getPath`, which returns the path-part of the URI (e.g., "s3n://my-bucket/my-path" => "/my-path"). We should do `relation.hiveQlTable.getDataLocation.toString` instead, as a URI's toString returns a faithful representation of the full URI, which can later be passed into a Hadoop Path.

Author: Aaron Davidson <aaron@databricks.com>

Closes #2150 from aarondav/parquet-location and squashes the following commits:

459f72c [Aaron Davidson] [SQL] [SPARK-3236] Reading Parquet tables from Metastore mangles location
2014-08-27 15:05:47 -07:00
viirya 28d41d6279 [SPARK-3252][SQL] Add missing condition for test
According to the text message, both relations should be tested. So add the missing condition.

Author: viirya <viirya@gmail.com>

Closes #2159 from viirya/fix_test and squashes the following commits:

b1c0f52 [viirya] add missing condition.
2014-08-27 14:55:05 -07:00
u0jing 3b5eb7083d [SPARK-3118][SQL]add "SHOW TBLPROPERTIES tblname;" and "SHOW COLUMNS (FROM|IN) table_name [(FROM|IN) db_name]" support
JIRA issue: [SPARK-3118] https://issues.apache.org/jira/browse/SPARK-3118

eg:
> SHOW TBLPROPERTIES test;
SHOW TBLPROPERTIES test;
numPartitions	0
numFiles	1
transient_lastDdlTime	1407923642
numRows	0
totalSize	82
rawDataSize	0

eg:
> SHOW COLUMNS  in test;
SHOW COLUMNS  in test;
OK
Time taken: 0.304 seconds
id
stid
bo

Author: u0jing <u9jing@gmail.com>

Closes #2034 from u0jing/spark-3118 and squashes the following commits:

b231d87 [u0jing] add golden answer files
35f4885 [u0jing] add 'show columns' and 'show tblproperties' support
2014-08-27 12:47:14 -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
Daoyuan Wang 52fbdc2ded [Spark-3222] [SQL] Cross join support in HiveQL
We can simple treat cross join as inner join without join conditions.

Author: Daoyuan Wang <daoyuan.wang@intel.com>
Author: adrian-wang <daoyuanwong@gmail.com>

Closes #2124 from adrian-wang/crossjoin and squashes the following commits:

8c9b7c5 [Daoyuan Wang] add a test
7d47bbb [adrian-wang] add cross join support for hql
2014-08-25 22:56:35 -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
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
Yin Huai 2fb1c72ea2 [SQL] Make functionRegistry in HiveContext transient.
Seems we missed `transient` for the `functionRegistry` in `HiveContext`.

cc: marmbrus

Author: Yin Huai <huaiyin.thu@gmail.com>

Closes #2074 from yhuai/makeFunctionRegistryTransient and squashes the following commits:

6534e7d [Yin Huai] Make functionRegistry transient.
2014-08-23 12:46:41 -07:00
Alex Liu d9e94146a6 [SPARK-2846][SQL] Add configureInputJobPropertiesForStorageHandler to initialization of job conf
...al job conf

Author: Alex Liu <alex_liu68@yahoo.com>

Closes #1927 from alexliu68/SPARK-SQL-2846 and squashes the following commits:

e4bdc4c [Alex Liu] SPARK-SQL-2846 add configureInputJobPropertiesForStorageHandler to initial job conf
2014-08-20 16:14:06 -07:00
Michael Armbrust 3abd0c1cda [SPARK-2406][SQL] Initial support for using ParquetTableScan to read HiveMetaStore tables.
This PR adds an experimental flag `spark.sql.hive.convertMetastoreParquet` that when true causes the planner to detects tables that use Hive's Parquet SerDe and instead plans them using Spark SQL's native `ParquetTableScan`.

Author: Michael Armbrust <michael@databricks.com>
Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #1819 from marmbrus/parquetMetastore and squashes the following commits:

1620079 [Michael Armbrust] Revert "remove hive parquet bundle"
cc30430 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore
4f3d54f [Michael Armbrust] fix style
41ebc5f [Michael Armbrust] remove hive parquet bundle
a43e0da [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore
4c4dc19 [Michael Armbrust] Fix bug with tree splicing.
ebb267e [Michael Armbrust] include parquet hive to tests pass (Remove this later).
c0d9b72 [Michael Armbrust] Avoid creating a HadoopRDD per partition.  Add dirty hacks to retrieve partition values from the InputSplit.
8cdc93c [Michael Armbrust] Merge pull request #8 from yhuai/parquetMetastore
a0baec7 [Yin Huai] Partitioning columns can be resolved.
1161338 [Michael Armbrust] Add a test to make sure conversion is actually happening
212d5cd [Michael Armbrust] Initial support for using ParquetTableScan to read HiveMetaStore tables.
2014-08-18 13:17:10 -07:00