Commit graph

579 commits

Author SHA1 Message Date
Marcelo Vanzin 82c8c37c09 [MINOR] [HIVE] Fix QueryPartitionSuite.
At least in the version of Hive I tested on, the test was deleting
a temp directory generated by Hive instead of one containing partition
data. So fix the filter to only consider partition directories when
deciding what to delete.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5854 from vanzin/hive-test-fix and squashes the following commits:

7594ae9 [Marcelo Vanzin] Fix typo.
729fa80 [Marcelo Vanzin] [minor] [hive] Fix QueryPartitionSuite.
2015-05-02 23:10:35 +01:00
Burak Yavuz b5347a4664 [SPARK-7248] implemented random number generators for DataFrames
Adds the functions `rand` (Uniform Dist) and `randn` (Normal Dist.) as expressions to DataFrames.

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5819 from brkyvz/df-rng and squashes the following commits:

50d69d4 [Burak Yavuz] add seed for test that failed
4234c3a [Burak Yavuz] fix Rand expression
13cad5c [Burak Yavuz] couple fixes
7d53953 [Burak Yavuz] waiting for hive tests
b453716 [Burak Yavuz] move radn with seed down
03637f0 [Burak Yavuz] fix broken hive func
c5909eb [Burak Yavuz] deleted old implementation of Rand
6d43895 [Burak Yavuz] implemented random generators
2015-04-30 21:56:03 -07:00
Patrick Wendell beeafcfd6e Revert "[SPARK-5213] [SQL] Pluggable SQL Parser Support"
This reverts commit 3ba5aaab82.
2015-04-30 20:33:36 -07:00
Cheng Hao 3ba5aaab82 [SPARK-5213] [SQL] Pluggable SQL Parser Support
This PR aims to make the SQL Parser Pluggable, and user can register it's own parser via Spark SQL CLI.

```
# add the jar into the classpath
$hchengmydesktop:spark>bin/spark-sql --jars sql99.jar

-- switch to "hiveql" dialect
   spark-sql>SET spark.sql.dialect=hiveql;
   spark-sql>SELECT * FROM src LIMIT 1;

-- switch to "sql" dialect
   spark-sql>SET spark.sql.dialect=sql;
   spark-sql>SELECT * FROM src LIMIT 1;

-- switch to a custom dialect
   spark-sql>SET spark.sql.dialect=com.xxx.xxx.SQL99Dialect;
   spark-sql>SELECT * FROM src LIMIT 1;

-- register the non-exist SQL dialect
   spark-sql> SET spark.sql.dialect=NotExistedClass;
   spark-sql> SELECT * FROM src LIMIT 1;
-- Exception will be thrown and switch to default sql dialect ("sql" for SQLContext and "hiveql" for HiveContext)
```

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

Closes #4015 from chenghao-intel/sqlparser and squashes the following commits:

493775c [Cheng Hao] update the code as feedback
81a731f [Cheng Hao] remove the unecessary comment
aab0b0b [Cheng Hao] polish the code a little bit
49b9d81 [Cheng Hao] shrink the comment for rebasing
2015-04-30 18:49:06 -07:00
Burak Yavuz d7dbce8f7d [SPARK-7156][SQL] support RandomSplit in DataFrames
This is built on top of kaka1992 's PR #5711 using Logical plans.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5761 from brkyvz/random-sample and squashes the following commits:

a1fb0aa [Burak Yavuz] remove unrelated file
69669c3 [Burak Yavuz] fix broken test
1ddb3da [Burak Yavuz] copy base
6000328 [Burak Yavuz] added python api and fixed test
3c11d1b [Burak Yavuz] fixed broken test
f400ade [Burak Yavuz] fix build errors
2384266 [Burak Yavuz] addressed comments v0.1
e98ebac [Burak Yavuz] [SPARK-7156][SQL] support RandomSplit in DataFrames
2015-04-29 15:34:05 -07:00
Burak Yavuz 271c4c621d [SPARK-7215] made coalesce and repartition a part of the query plan
Coalesce and repartition now show up as part of the query plan, rather than resulting in a new `DataFrame`.

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5762 from brkyvz/df-repartition and squashes the following commits:

b1e76dd [Burak Yavuz] added documentation on repartitions
5807e35 [Burak Yavuz] renamed coalescepartitions
fa4509f [Burak Yavuz] rename coalesce
2c349b5 [Burak Yavuz] address comments
f2e6af1 [Burak Yavuz] add ticks
686c90b [Burak Yavuz] made coalesce and repartition a part of the query plan
2015-04-28 22:48:04 -07:00
Sean Owen 7f3b3b7eb7 [SPARK-7168] [BUILD] Update plugin versions in Maven build and centralize versions
Update Maven build plugin versions and centralize plugin version management

Author: Sean Owen <sowen@cloudera.com>

Closes #5720 from srowen/SPARK-7168 and squashes the following commits:

98a8947 [Sean Owen] Make install, deploy plugin versions explicit
4ecf3b2 [Sean Owen] Update Maven build plugin versions and centralize plugin version management
2015-04-28 07:48:34 -04:00
Sean Owen ab5adb7a97 [SPARK-7145] [CORE] commons-lang (2.x) classes used instead of commons-lang3 (3.x); commons-io used without dependency
Remove use of commons-lang in favor of commons-lang3 classes; remove commons-io use in favor of Guava

Author: Sean Owen <sowen@cloudera.com>

Closes #5703 from srowen/SPARK-7145 and squashes the following commits:

21fbe03 [Sean Owen] Remove use of commons-lang in favor of commons-lang3 classes; remove commons-io use in favor of Guava
2015-04-27 19:50:55 -04:00
baishuo 82bb7fd41a [SPARK-6505] [SQL] Remove the reflection call in HiveFunctionWrapper
according liancheng‘s  comment in https://issues.apache.org/jira/browse/SPARK-6505,  this patch remove the  reflection call in HiveFunctionWrapper, and implement the functions named "deserializeObjectByKryo" and "serializeObjectByKryo" according the functions with the save name in
org.apache.hadoop.hive.ql.exec.Utilities.java

Author: baishuo <vc_java@hotmail.com>

Closes #5660 from baishuo/SPARK-6505-20150423 and squashes the following commits:

ae61ec4 [baishuo] modify code style
78d9fa3 [baishuo] modify code style
0b522a7 [baishuo] modify code style
a5ff9c7 [baishuo] Remove the reflection call in HiveFunctionWrapper
2015-04-27 14:08:05 +08:00
wangfei d188b8bad8 [SQL][Minor] rename DataTypeParser.apply to DataTypeParser.parse
rename DataTypeParser.apply to DataTypeParser.parse to make it more clear and readable.
/cc rxin

Author: wangfei <wangfei1@huawei.com>

Closes #5710 from scwf/apply and squashes the following commits:

c319977 [wangfei] rename apply to parse
2015-04-26 21:08:47 -07:00
Calvin Jia 438859eb7c [SPARK-6122] [CORE] Upgrade tachyon-client version to 0.6.3
This is a reopening of #4867.
A short summary of the issues resolved from the previous PR:

1. HTTPClient version mismatch: Selenium (used for UI tests) requires version 4.3.x, and Tachyon included 4.2.5 through a transitive dependency of its shaded thrift jar. To address this, Tachyon 0.6.3 will promote the transitive dependencies of the shaded jar so they can be excluded in spark.

2. Jackson-Mapper-ASL version mismatch: In lower versions of hadoop-client (ie. 1.0.4), version 1.0.1 is included. The parquet library used in spark sql requires version 1.8+. Its unclear to me why upgrading tachyon-client would cause this dependency to break. The solution was to exclude jackson-mapper-asl from hadoop-client.

It seems that the dependency management in spark-parent will not work on transitive dependencies, one way to make sure jackson-mapper-asl is included with the correct version is to add it as a top level dependency. The best solution would be to exclude the dependency in the modules which require a higher version, but that did not fix the unit tests. Any suggestions on the best way to solve this would be appreciated!

Author: Calvin Jia <jia.calvin@gmail.com>

Closes #5354 from calvinjia/upgrade_tachyon_0.6.3 and squashes the following commits:

0eefe4d [Calvin Jia] Handle httpclient version in maven dependency management. Remove httpclient version setting from profiles.
7c00dfa [Calvin Jia] Set httpclient version to 4.3.2 for selenium. Specify version of httpclient for sql/hive (previously 4.2.5 transitive dependency of libthrift).
9263097 [Calvin Jia] Merge master to test latest changes
dbfc1bd [Calvin Jia] Use Tachyon 0.6.4 for cleaner dependencies.
e2ff80a [Calvin Jia] Exclude the jetty and curator promoted dependencies from tachyon-client.
a3a29da [Calvin Jia] Update tachyon-client exclusions.
0ae6c97 [Calvin Jia] Change tachyon version to 0.6.3
a204df9 [Calvin Jia] Update make distribution tachyon version.
a93c94f [Calvin Jia] Exclude jackson-mapper-asl from hadoop client since it has a lower version than spark's expected version.
a8a923c [Calvin Jia] Exclude httpcomponents from Tachyon
910fabd [Calvin Jia] Update to master
eed9230 [Calvin Jia] Update tachyon version to 0.6.1.
11907b3 [Calvin Jia] Use TachyonURI for tachyon paths instead of strings.
71bf441 [Calvin Jia] Upgrade Tachyon client version to 0.6.0.
2015-04-24 17:57:41 -04:00
Cheng Hao cc48e6387a [SPARK-7044] [SQL] Fix the deadlock in script transformation
Author: Cheng Hao <hao.cheng@intel.com>

Closes #5625 from chenghao-intel/transform and squashes the following commits:

5ec1dd2 [Cheng Hao] fix the deadlock issue in ScriptTransform
2015-04-23 10:35:22 -07:00
Reynold Xin cdf0328684 [SQL] Rename some apply functions.
I was looking at the code gen code and got confused by a few of use cases of apply, in particular apply on objects. So I went ahead and changed a few of them. Hopefully slightly more clear with a proper verb.

Author: Reynold Xin <rxin@databricks.com>

Closes #5624 from rxin/apply-rename and squashes the following commits:

ee45034 [Reynold Xin] [SQL] Rename some apply functions.
2015-04-22 11:18:01 -07:00
Cheng Hao 7662ec23bb [SPARK-5817] [SQL] Fix bug of udtf with column names
It's a bug while do query like:
```sql
select d from (select explode(array(1,1)) d from src limit 1) t
```
And it will throws exception like:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'd' given input columns _c0; line 1 pos 7
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:48)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:45)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:50)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:249)
at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$transformExpressionUp$1(QueryPlan.scala:103)
at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2$$anonfun$apply$2.apply(QueryPlan.scala:117)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
at scala.collection.AbstractTraversable.map(Traversable.scala:105)
at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2.apply(QueryPlan.scala:116)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
```

To solve the bug, it requires code refactoring for UDTF
The major changes are about:
* Simplifying the UDTF development, UDTF will manage the output attribute names any more, instead, the `logical.Generate` will handle that properly.
* UDTF will be asked for the output schema (data types) during the logical plan analyzing.

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

Closes #4602 from chenghao-intel/explode_bug and squashes the following commits:

c2a5132 [Cheng Hao] add back resolved for Alias
556e982 [Cheng Hao] revert the unncessary change
002c361 [Cheng Hao] change the rule of resolved for Generate
04ae500 [Cheng Hao] add qualifier only for generator output
5ee5d2c [Cheng Hao] prepend the new qualifier
d2e8b43 [Cheng Hao] Update the code as feedback
ca5e7f4 [Cheng Hao] shrink the commits
2015-04-21 15:11:15 -07:00
Yin Huai 6265cba00f [SPARK-6969][SQL] Refresh the cached table when REFRESH TABLE is used
https://issues.apache.org/jira/browse/SPARK-6969

Author: Yin Huai <yhuai@databricks.com>

Closes #5583 from yhuai/refreshTableRefreshDataCache and squashes the following commits:

1e5142b [Yin Huai] Add todo.
92b2498 [Yin Huai] Minor updates.
367df92 [Yin Huai] Recache data in the command of REFRESH TABLE.
2015-04-21 14:48:42 -07:00
Daoyuan Wang 585638e81c [SPARK-2213] [SQL] sort merge join for spark sql
Thanks for the initial work from Ishiihara in #3173

This PR introduce a new join method of sort merge join, which firstly ensure that keys of same value are in the same partition, and inside each partition the Rows are sorted by key. Then we can run down both sides together, find matched rows using [sort merge join](http://en.wikipedia.org/wiki/Sort-merge_join). In this way, we don't have to store the whole hash table of one side as hash join, thus we have less memory usage. Also, this PR would benefit from #3438 , making the sorting phrase much more efficient.

We introduced a new configuration of "spark.sql.planner.sortMergeJoin" to switch between this(`true`) and ShuffledHashJoin(`false`), probably we want the default value of it be `false` at first.

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

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>

Closes #5208 from adrian-wang/smj and squashes the following commits:

2493b9f [Daoyuan Wang] fix style
5049d88 [Daoyuan Wang] propagate rowOrdering for RangePartitioning
f91a2ae [Daoyuan Wang] yin's comment: use external sort if option is enabled, add comments
f515cd2 [Daoyuan Wang] yin's comment: outputOrdering, join suite refine
ec8061b [Daoyuan Wang] minor change
413fd24 [Daoyuan Wang] Merge pull request #3 from marmbrus/pr/5208
952168a [Michael Armbrust] add type
5492884 [Michael Armbrust] copy when ordering
7ddd656 [Michael Armbrust] Cleanup addition of ordering requirements
b198278 [Daoyuan Wang] inherit ordering in project
c8e82a3 [Daoyuan Wang] fix style
6e897dd [Daoyuan Wang] hide boundReference from manually construct RowOrdering for key compare in smj
8681d73 [Daoyuan Wang] refactor Exchange and fix copy for sorting
2875ef2 [Daoyuan Wang] fix changed configuration
61d7f49 [Daoyuan Wang] add omitted comment
00a4430 [Daoyuan Wang] fix bug
078d69b [Daoyuan Wang] address comments: add comments, do sort in shuffle, and others
3af6ba5 [Daoyuan Wang] use buffer for only one side
171001f [Daoyuan Wang] change default outputordering
47455c9 [Daoyuan Wang] add apache license ...
a28277f [Daoyuan Wang] fix style
645c70b [Daoyuan Wang] address comments using sort
068c35d [Daoyuan Wang] fix new style and add some tests
925203b [Daoyuan Wang] address comments
07ce92f [Daoyuan Wang] fix ArrayIndexOutOfBound
42fca0e [Daoyuan Wang] code clean
e3ec096 [Daoyuan Wang] fix comment style..
2edd235 [Daoyuan Wang] fix outputpartitioning
57baa40 [Daoyuan Wang] fix sort eval bug
303b6da [Daoyuan Wang] fix several errors
95db7ad [Daoyuan Wang] fix brackets for if-statement
4464f16 [Daoyuan Wang] fix error
880d8e9 [Daoyuan Wang] sort merge join for spark sql
2015-04-15 14:06:10 -07:00
Wenchen Fan 4754e16f47 [SPARK-6898][SQL] completely support special chars in column names
Even if we wrap column names in backticks like `` `a#$b.c` ``,  we still handle the "." inside column name specially. I think it's fragile to use a special char to split name parts, why not put name parts in `UnresolvedAttribute` directly?

Author: Wenchen Fan <cloud0fan@outlook.com>

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>

Closes #5511 from cloud-fan/6898 and squashes the following commits:

48e3e57 [Wenchen Fan] more style fix
820dc45 [Wenchen Fan] do not ignore newName in UnresolvedAttribute
d81ad43 [Wenchen Fan] fix style
11699d6 [Wenchen Fan] completely support special chars in column names
2015-04-15 13:39:12 -07:00
Davies Liu 85842760dc [SPARK-6638] [SQL] Improve performance of StringType in SQL
This PR change the internal representation for StringType from java.lang.String to UTF8String, which is implemented use ArrayByte.

This PR should not break any public API, Row.getString() will still return java.lang.String.

This is the first step of improve the performance of String in SQL.

cc rxin

Author: Davies Liu <davies@databricks.com>

Closes #5350 from davies/string and squashes the following commits:

3b7bfa8 [Davies Liu] fix schema of AddJar
2772f0d [Davies Liu] fix new test failure
6d776a9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
59025c8 [Davies Liu] address comments from @marmbrus
341ec2c [Davies Liu] turn off scala style check in UTF8StringSuite
744788f [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
b04a19c [Davies Liu] add comment for getString/setString
08d897b [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
5116b43 [Davies Liu] rollback unrelated changes
1314a37 [Davies Liu] address comments from Yin
867bf50 [Davies Liu] fix String filter push down
13d9d42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
2089d24 [Davies Liu] add hashcode check back
ac18ae6 [Davies Liu] address comment
fd11364 [Davies Liu] optimize UTF8String
8d17f21 [Davies Liu] fix hive compatibility tests
e5fa5b8 [Davies Liu] remove clone in UTF8String
28f3d81 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
28d6f32 [Davies Liu] refactor
537631c [Davies Liu] some comment about Date
9f4c194 [Davies Liu] convert data type for data source
956b0a4 [Davies Liu] fix hive tests
73e4363 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string
9dc32d1 [Davies Liu] fix some hive tests
23a766c [Davies Liu] refactor
8b45864 [Davies Liu] fix codegen with UTF8String
bb52e44 [Davies Liu] fix scala style
c7dd4d2 [Davies Liu] fix some catalyst tests
38c303e [Davies Liu] fix python sql tests
5f9e120 [Davies Liu] fix sql tests
6b499ac [Davies Liu] fix style
a85fb27 [Davies Liu] refactor
d32abd1 [Davies Liu] fix utf8 for python api
4699c3a [Davies Liu] use Array[Byte] in UTF8String
21f67c6 [Davies Liu] cleanup
685fd07 [Davies Liu] use UTF8String instead of String for StringType
2015-04-15 13:06:38 -07:00
Daoyuan Wang b45059d0d7 [SPARK-5794] [SQL] fix add jar
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4586 from adrian-wang/addjar and squashes the following commits:

efdd602 [Daoyuan Wang] move jar to another place
6c707e8 [Daoyuan Wang] restrict hive version for test
32c4fb8 [Daoyuan Wang] fix style and add a test
9957d87 [Daoyuan Wang] use sessionstate classloader in makeRDDforTable
0810e71 [Daoyuan Wang] remove variable substitution
1898309 [Daoyuan Wang] fix classnotfound
95a40da [Daoyuan Wang] support env argus in add jar, and set add jar ret to 0
2015-04-13 18:26:00 -07:00
Cheng Hao c5602bdc31 [SPARK-5941] [SQL] Unit Test loads the table src twice for leftsemijoin.q
In `leftsemijoin.q`, there is a data loading command for table `sales` already, but in `TestHive`, it also created the table `sales`, which causes duplicated records inserted into the `sales`.

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

Closes #4506 from chenghao-intel/df_table and squashes the following commits:

0be05f7 [Cheng Hao] Remove the table `sales` creating from TestHive
2015-04-13 16:02:18 -07:00
Daoyuan Wang 85ee0cabe8 [SPARK-6130] [SQL] support if not exists for insert overwrite into partition in hiveQl
Standard syntax:
INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 FROM from_statement;
INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 FROM from_statement;
 
Hive extension (multiple inserts):
FROM from_statement
INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1
[INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2]
[INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] ...;
FROM from_statement
INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1
[INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2]
[INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] ...;
 
Hive extension (dynamic partition inserts):
INSERT OVERWRITE TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement;
INSERT INTO TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement;

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

Closes #4865 from adrian-wang/insertoverwrite and squashes the following commits:

2fce94f [Daoyuan Wang] add assert
10ea6f3 [Daoyuan Wang] add name for boolean parameter
0bbe9b9 [Daoyuan Wang] fix failure
4391154 [Daoyuan Wang] support if not exists for insert overwrite into partition in hiveQl
2015-04-13 14:29:07 -07:00
Reynold Xin c5b0b296b8 [SPARK-6765] Enable scalastyle on test code.
Turn scalastyle on for all test code. Most of the violations have been resolved in my previous pull requests:

Core: https://github.com/apache/spark/pull/5484
SQL: https://github.com/apache/spark/pull/5412
MLlib: https://github.com/apache/spark/pull/5411
GraphX: https://github.com/apache/spark/pull/5410
Streaming: https://github.com/apache/spark/pull/5409

Author: Reynold Xin <rxin@databricks.com>

Closes #5486 from rxin/test-style-enable and squashes the following commits:

01683de [Reynold Xin] Fixed new code.
a4ab46e [Reynold Xin] Fixed tests.
20adbc8 [Reynold Xin] Missed one violation.
5e36521 [Reynold Xin] [SPARK-6765] Enable scalastyle on test code.
2015-04-13 09:29:04 -07:00
DoingDone9 48cc840021 [SPARK-6179][SQL] Add token for "SHOW PRINCIPALS role_name" and "SHOW TRANSACTIONS" and "SHOW COMPACTIONS"
[SHOW PRINCIPALS role_name]
Lists all roles and users who belong to this role.
Only the admin role has privilege for this.

[SHOW COMPACTIONS]
It returns a list of all tables and partitions currently being compacted or scheduled for compaction when Hive transactions are being used.

[SHOW TRANSACTIONS]
It is for use by administrators when Hive transactions are being used. It returns a list of all currently open and aborted transactions in the system.

Author: DoingDone9 <799203320@qq.com>
Author: Zhongshuai Pei <799203320@qq.com>
Author: Xu Tingjun <xutingjun@huawei.com>

Closes #4902 from DoingDone9/SHOW_PRINCIPALS and squashes the following commits:

4add42f [Zhongshuai Pei] for test
311f806 [Zhongshuai Pei] for test
0c7550a [DoingDone9] Update HiveQl.scala
c8aeb1c [Xu Tingjun] aa
802261c [DoingDone9] Merge pull request #7 from apache/master
d00303b [DoingDone9] Merge pull request #6 from apache/master
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-04-11 18:34:17 -07:00
lazymam500 1f39a61118 [Spark-5068][SQL]Fix bug query data when path doesn't exist for HiveContext
This PR follow up PR #3907 & #3891 & #4356.
According to  marmbrus  liancheng 's comments, I try to use fs.globStatus to retrieve all FileStatus objects under path(s), and then do the filtering locally.

[1]. get pathPattern by path, and put it into pathPatternSet. (hdfs://cluster/user/demo/2016/08/12 -> hdfs://cluster/user/demo/*/*/*)
[2]. retrieve all FileStatus objects ,and cache them by undating existPathSet.
[3]. do the filtering locally
[4]. if we have new pathPattern,do 1,2 step again. (external table maybe have more than one partition pathPattern)

chenghao-intel jeanlyn

Author: lazymam500 <lazyman500@gmail.com>
Author: lazyman <lazyman500@gmail.com>

Closes #5059 from lazyman500/SPARK-5068 and squashes the following commits:

5bfcbfd [lazyman] move spark.sql.hive.verifyPartitionPath to SQLConf,fix scala style
e1d6386 [lazymam500] fix scala style
f23133f [lazymam500] bug fix
47e0023 [lazymam500] fix scala style,add config flag,break the chaining
04c443c [lazyman] SPARK-5068: fix bug when partition path doesn't exists #2
41f60ce [lazymam500] Merge pull request #1 from apache/master
2015-04-11 18:33:14 -07:00
haiyang 2f53588738 [SPARK-6199] [SQL] Support CTE in HiveContext and SQLContext
Author: haiyang <huhaiyang@huawei.com>

Closes #4929 from haiyangsea/cte and squashes the following commits:

220b67d [haiyang] add golden files for cte test
d3c7681 [haiyang] Merge branch 'master' into cte-repair
0ba2070 [haiyang] modify code style
9ce6b58 [haiyang] fix conflict
ff74741 [haiyang] add comment for With plan
0d56af4 [haiyang] code indention
776a440 [haiyang] add comments for resolve relation strategy
2fccd7e [haiyang] add comments for resolve relation strategy
241bbe2 [haiyang] fix cte problem of view
e9e1237 [haiyang] fix test case problem
614182f [haiyang] add test cases for CTE feature
32e415b [haiyang] add comment
1cc8c15 [haiyang] support with
03f1097 [haiyang] support with
e960099 [haiyang] support with
9aaa874 [haiyang] support with
0566978 [haiyang] support with
a99ecd2 [haiyang] support with
c3fa4c2 [haiyang] support with
3b6077f [haiyang] support with
5f8abe3 [haiyang] support with
4572b05 [haiyang] support with
f801f54 [haiyang] support with
2015-04-11 18:30:17 -07:00
Cheng Hao 3ceb810aa8 [SPARK-6835] [SQL] Fix bug of Hive UDTF in Lateral View (ClassNotFound)
```SQL
select key, v from src lateral view stack(3, 1+1, 2+2, 3) d as v;
```
Will cause exception
```
java.lang.ClassNotFoundException: stack
at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
at java.security.AccessController.doPrivileged(Native Method)
at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308)
at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
at org.apache.spark.sql.hive.HiveFunctionWrapper.createFunction(Shim13.scala:148)
at org.apache.spark.sql.hive.HiveGenericUdtf.function$lzycompute(hiveUdfs.scala:274)
at org.apache.spark.sql.hive.HiveGenericUdtf.function(hiveUdfs.scala:274)
at org.apache.spark.sql.hive.HiveGenericUdtf.outputInspector$lzycompute(hiveUdfs.scala:280)
at org.apache.spark.sql.hive.HiveGenericUdtf.outputInspector(hiveUdfs.scala:280)
at org.apache.spark.sql.hive.HiveGenericUdtf.outputDataTypes$lzycompute(hiveUdfs.scala:285)
at org.apache.spark.sql.hive.HiveGenericUdtf.outputDataTypes(hiveUdfs.scala:285)
at org.apache.spark.sql.hive.HiveGenericUdtf.makeOutput(hiveUdfs.scala:291)
at org.apache.spark.sql.catalyst.expressions.Generator.output(generators.scala:60)
at org.apache.spark.sql.catalyst.plans.logical.Generate$$anonfun$2.apply(basicOperators.scala:60)
at org.apache.spark.sql.catalyst.plans.logical.Generate$$anonfun$2.apply(basicOperators.scala:60)
at scala.Option.map(Option.scala:145)
at org.apache.spark.sql.catalyst.plans.logical.Generate.generatorOutput(basicOperators.scala:60)
at org.apache.spark.sql.catalyst.plans.logical.Generate.output(basicOperators.scala:70)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveChildren$1.apply(LogicalPlan.scala:117)
at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveChildren$1.apply(LogicalPlan.scala:117)
```

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

Closes #5444 from chenghao-intel/hive_udtf and squashes the following commits:

065a98c [Cheng Hao] fix bug of Hive UDTF in Lateral View (ClassNotFound)
2015-04-11 22:11:03 +08:00
Michael Armbrust 23d5f8864f [SPARK-6851][SQL] Create new instance for each converted parquet relation
Otherwise we end up rewriting predicates to be trivially equal (i.e. `a#1 = a#2` -> `a#3 = a#3`), at which point the query is no longer valid.

Author: Michael Armbrust <michael@databricks.com>

Closes #5458 from marmbrus/selfJoinParquet and squashes the following commits:

22df77c [Michael Armbrust] [SPARK-6851][SQL] Create new instance for each converted parquet relation
2015-04-10 16:05:14 -07:00
Reynold Xin 1b2aab8d5b [SPARK-6765] Fix test code style for SQL
So we can turn style checker on for test code.

Author: Reynold Xin <rxin@databricks.com>

Closes #5412 from rxin/test-style-sql and squashes the following commits:

9098a31 [Reynold Xin] One more compilation error ...
8c7250a [Reynold Xin] Fix compilation.
82d0944 [Reynold Xin] Indentation.
0b03fbb [Reynold Xin] code review.
f2f4348 [Reynold Xin] oops.
ef4ec48 [Reynold Xin] Hive module.
7e0db5e [Reynold Xin] sql module
04ec7ac [Reynold Xin] catalyst module
2015-04-08 20:35:29 -07:00
Liang-Chi Hsieh 7bca62f790 [SPARK-6607][SQL] Check invalid characters for Parquet schema and show error messages
'(' and ')' are special characters used in Parquet schema for type annotation. When we run an aggregation query, we will obtain attribute name such as "MAX(a)".

If we directly store the generated DataFrame as Parquet file, it causes failure when reading and parsing the stored schema string.

Several methods can be adopted to solve this. This pr uses a simplest one to just replace attribute names before generating Parquet schema based on these attributes.

Another possible method might be modifying all aggregation expression names from "func(column)" to "func[column]".

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

Closes #5263 from viirya/parquet_aggregation_name and squashes the following commits:

2d70542 [Liang-Chi Hsieh] Address comment.
463dff4 [Liang-Chi Hsieh] Instead of replacing special chars, showing error message to user to suggest using Alias.
1de001d [Liang-Chi Hsieh] Replace special characters '(' and ')' of Parquet schema.
2015-04-05 00:20:43 +08:00
guowei2 c23ba81b8c [SPARK-5203][SQL] fix union with different decimal type
When union non-decimal types with decimals, we use the following rules:
      - FIRST `intTypeToFixed`, then fixed union decimals with precision/scale p1/s2 and p2/s2  will be promoted to
      DecimalType(max(p1, p2), max(s1, s2))
      - FLOAT and DOUBLE cause fixed-length decimals to turn into DOUBLE (this is the same as Hive,
      but note that unlimited decimals are considered bigger than doubles in WidenTypes)

Author: guowei2 <guowei2@asiainfo.com>

Closes #4004 from guowei2/SPARK-5203 and squashes the following commits:

ff50f5f [guowei2] fix code style
11df1bf [guowei2] fix decimal union with double, double->Decimal(15,15)
0f345f9 [guowei2] fix structType merge with decimal
101ed4d [guowei2] fix build error after rebase
0b196e4 [guowei2] code style
fe2c2ca [guowei2] handle union decimal precision in 'DecimalPrecision'
421d840 [guowei2] fix union types for decimal precision
ef2c661 [guowei2] fix union with different decimal type
2015-04-04 02:02:30 +08:00
Liang-Chi Hsieh dc6dff248d [Minor][SQL] Fix typo
Just fix a typo.

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

Closes #5352 from viirya/fix_a_typo and squashes the following commits:

303b2d2 [Liang-Chi Hsieh] Fix typo.
2015-04-03 18:31:48 +01:00
Reynold Xin 82701ee25f [SPARK-6428] Turn on explicit type checking for public methods.
This builds on my earlier pull requests and turns on the explicit type checking in scalastyle.

Author: Reynold Xin <rxin@databricks.com>

Closes #5342 from rxin/SPARK-6428 and squashes the following commits:

7b531ab [Reynold Xin] import ordering
2d9a8a5 [Reynold Xin] jl
e668b1c [Reynold Xin] override
9b9e119 [Reynold Xin] Parenthesis.
82e0cf5 [Reynold Xin] [SPARK-6428] Turn on explicit type checking for public methods.
2015-04-03 01:25:02 -07:00
Yin Huai c42c3fc7f7 [SPARK-6575][SQL] Converted Parquet Metastore tables no longer cache metadata
https://issues.apache.org/jira/browse/SPARK-6575

Author: Yin Huai <yhuai@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Cheng Lian <lian@databricks.com>

Closes #5339 from yhuai/parquetRelationCache and squashes the following commits:

b0e1a42 [Yin Huai] Address comments.
83d9846 [Yin Huai] Remove unnecessary change.
c0dc7a4 [Yin Huai] Cache converted parquet relations.
2015-04-03 14:40:36 +08:00
Yin Huai 4b82bd730a [SPARK-6575][SQL] Converted Parquet Metastore tables no longer cache metadata
https://issues.apache.org/jira/browse/SPARK-6575

Author: Yin Huai <yhuai@databricks.com>

Closes #5339 from yhuai/parquetRelationCache and squashes the following commits:

83d9846 [Yin Huai] Remove unnecessary change.
c0dc7a4 [Yin Huai] Cache converted parquet relations.
2015-04-02 20:23:08 -07:00
Cheng Hao dfd2982bc7 [SQL][Minor] Use analyzed logical instead of unresolved in HiveComparisonTest
Some internal unit test failed due to the logical plan node in pattern matching in `HiveComparisonTest`,  e.g.
https://github.com/apache/spark/blob/master/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala#L137

Which will may call the `output` function on an unresolved logical plan.

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

Closes #4946 from chenghao-intel/logical and squashes the following commits:

432ecb3 [Cheng Hao] Use analyzed instead of logical in HiveComparisonTest
2015-04-02 17:20:31 -07:00
Yin Huai 5db89127e7 [SPARK-6618][SPARK-6669][SQL] Lock Hive metastore client correctly.
Author: Yin Huai <yhuai@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #5333 from yhuai/lookupRelationLock and squashes the following commits:

59c884f [Michael Armbrust] [SQL] Lock metastore client in analyzeTable
7667030 [Yin Huai] Merge pull request #2 from marmbrus/pr/5333
e4a9b0b [Michael Armbrust] Correctly lock on MetastoreCatalog
d6fc32f [Yin Huai] Missing `)`.
1e241af [Yin Huai] Protect InsertIntoHive.
fee7e9c [Yin Huai] A test?
5416b0f [Yin Huai] Just protect client.
2015-04-02 16:46:50 -07:00
Yin Huai 251698fb73 [SPARK-6655][SQL] We need to read the schema of a data source table stored in spark.sql.sources.schema property
https://issues.apache.org/jira/browse/SPARK-6655

Author: Yin Huai <yhuai@databricks.com>

Closes #5313 from yhuai/SPARK-6655 and squashes the following commits:

1e00c03 [Yin Huai] Unnecessary change.
f131bd9 [Yin Huai] Fix.
f1218c1 [Yin Huai] Failed test.
2015-04-02 16:02:31 -07:00
Michael Armbrust 4214e50fc3 [SQL] Throw UnsupportedOperationException instead of NotImplementedError
NotImplementedError in scala 2.10 is a fatal exception, which is not very nice to throw when not actually fatal.

Author: Michael Armbrust <michael@databricks.com>

Closes #5315 from marmbrus/throwUnsupported and squashes the following commits:

c29e03b [Michael Armbrust] [SQL] Throw UnsupportedOperationException instead of NotImplementedError
052e05b [Michael Armbrust] [SQL] Throw UnsupportedOperationException instead of NotImplementedError
2015-04-02 16:01:03 -07:00
Davies Liu 40df5d49bb [SPARK-6663] [SQL] use Literal.create instread of constructor
In order to do inbound checking and type conversion, we should use Literal.create() instead of  constructor.

Author: Davies Liu <davies@databricks.com>

Closes #5320 from davies/literal and squashes the following commits:

1667604 [Davies Liu] fix style and add comment
5f8c0fd [Davies Liu] use Literal.create instread of constructor
2015-04-01 23:11:38 -07:00
Cheng Lian 2bc7fe7f7e Revert "[SPARK-6618][SQL] HiveMetastoreCatalog.lookupRelation should use fine-grained lock"
This reverts commit 314afd0e2f.
2015-04-02 12:56:34 +08:00
Steve Loughran ee11be2582 SPARK-6433 hive tests to import spark-sql test JAR for QueryTest access
1. Test JARs are built & published
1. log4j.resources is explicitly excluded. Without this, downstream test run logging depends on the order the JARs are listed/loaded
1. sql/hive pulls in spark-sql &...spark-catalyst for its test runs
1. The copied in test classes were rm'd, and a test edited to remove its now duplicate assert method
1. Spark streaming is now build with the same plugin/phase as the rest, but its shade plugin declaration is kept in (so different from the rest of the test plugins). Due to (#2), this means the test JAR no longer includes its log4j file.

Outstanding issues:
* should the JARs be shaded? `spark-streaming-test.jar` does, but given these are test jars for developers only, especially in the same spark source tree, it's hard to justify.
* `maven-jar-plugin` v 2.6 was explicitly selected; without this the apache-1.4 parent template JAR version (2.4) chosen.
* Are there any other resources to exclude?

Author: Steve Loughran <stevel@hortonworks.com>

Closes #5119 from steveloughran/stevel/patches/SPARK-6433-test-jars and squashes the following commits:

81ceb01 [Steve Loughran] SPARK-6433 add a clearer comment explaining what the plugin is doing & why
a6dca33 [Steve Loughran] SPARK-6433 : pull configuration section form archive plugin
c2b5f89 [Steve Loughran] SPARK-6433 omit "jar" goal from jar plugin
fdac51b [Steve Loughran] SPARK-6433 -002; indentation & delegate plugin version to parent
650f442 [Steve Loughran] SPARK-6433 patch 001: test JARs are built; sql/hive pulls in spark-sql & spark-catalyst for its test runs
2015-04-01 16:26:54 +01:00
Michael Armbrust beebb7ffc2 [SPARK-5371][SQL] Propagate types after function conversion, before futher resolution
Before it was possible for a query to flip back and forth from a resolved state, allowing resolution to propagate up before coercion had stabilized.  The issue was that `ResolvedReferences` would run after `FunctionArgumentConversion`, but before `PropagateTypes` had run.  This PR ensures we correctly `PropagateTypes` after any coercion has applied.

Author: Michael Armbrust <michael@databricks.com>

Closes #5278 from marmbrus/unionNull and squashes the following commits:

dc3581a [Michael Armbrust] [SPARK-5371][SQL] Propogate types after function conversion / before futher resolution
2015-03-31 11:34:52 -07:00
Cheng Lian 8102014470 [SPARK-6575] [SQL] Adds configuration to disable schema merging while converting metastore Parquet tables
Consider a metastore Parquet table that

1. doesn't have schema evolution issue
2. has lots of data files and/or partitions

In this case, driver schema merging can be both slow and unnecessary. Would be good to have a configuration to let the use disable schema merging when converting such a metastore Parquet table.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5231 from liancheng/spark-6575 and squashes the following commits:

cd96159 [Cheng Lian] Adds configuration to disable schema merging while converting metastore Parquet tables
2015-03-31 11:21:15 -07:00
Cheng Lian a7992ffaf1 [SPARK-6555] [SQL] Overrides equals() and hashCode() for MetastoreRelation
Also removes temporary workarounds made in #5183 and #5251.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5289 from liancheng/spark-6555 and squashes the following commits:

d0095ac [Cheng Lian] Removes unused imports
cfafeeb [Cheng Lian] Removes outdated comment
75a2746 [Cheng Lian] Overrides equals() and hashCode() for MetastoreRelation
2015-03-31 11:18:25 -07:00
Yin Huai 314afd0e2f [SPARK-6618][SQL] HiveMetastoreCatalog.lookupRelation should use fine-grained lock
JIRA: https://issues.apache.org/jira/browse/SPARK-6618

Author: Yin Huai <yhuai@databricks.com>

Closes #5281 from yhuai/lookupRelationLock and squashes the following commits:

591b4be [Yin Huai] A test?
b3a9625 [Yin Huai] Just protect client.
2015-03-31 16:28:40 +08:00
Cheng Lian fde6945417 [SPARK-6369] [SQL] Uses commit coordinator to help committing Hive and Parquet tables
This PR leverages the output commit coordinator introduced in #4066 to help committing Hive and Parquet tables.

This PR extracts output commit code in `SparkHadoopWriter.commit` to `SparkHadoopMapRedUtil.commitTask`, and reuses it for committing Parquet and Hive tables on executor side.

TODO

- [ ] Add tests

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

Author: Cheng Lian <lian@databricks.com>

Closes #5139 from liancheng/spark-6369 and squashes the following commits:

72eb628 [Cheng Lian] Fixes typo in javadoc
9a4b82b [Cheng Lian] Adds javadoc and addresses @aarondav's comments
dfdf3ef [Cheng Lian] Uses commit coordinator to help committing Hive and Parquet tables
2015-03-31 07:48:37 +08:00
Michael Armbrust fe81f6c779 [SPARK-6595][SQL] MetastoreRelation should be a MultiInstanceRelation
Now that we have `DataFrame`s it is possible to have multiple copies in a single query plan.  As such, it needs to inherit from `MultiInstanceRelation` or self joins will break.  I also add better debugging errors when our self join handling fails in case there are future bugs.

Author: Michael Armbrust <michael@databricks.com>

Closes #5251 from marmbrus/multiMetaStore and squashes the following commits:

4272f6d [Michael Armbrust] [SPARK-6595][SQL] MetastoreRelation should be MuliInstanceRelation
2015-03-30 22:24:12 +08:00
Reynold Xin 3af7334304 [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 row, not 1 row
Author: Reynold Xin <rxin@databricks.com>

Closes #5226 from rxin/empty-df and squashes the following commits:

1306d88 [Reynold Xin] Proper fix.
e135bb9 [Reynold Xin] [SPARK-6564][SQL] SQLContext.emptyDataFrame should contain 0 rows, not 1 row.
2015-03-27 14:56:57 -07:00
DoingDone9 855cba8fe5 [SPARK-6546][Build] Using the wrong code that will make spark compile failed!!
wrong code : val tmpDir = Files.createTempDir()
not Files should Utils

Author: DoingDone9 <799203320@qq.com>

Closes #5198 from DoingDone9/FilesBug and squashes the following commits:

6e0140d [DoingDone9] Update InsertIntoHiveTableSuite.scala
e57d23f [DoingDone9] Update InsertIntoHiveTableSuite.scala
802261c [DoingDone9] Merge pull request #7 from apache/master
d00303b [DoingDone9] Merge pull request #6 from apache/master
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-03-26 17:04:19 +08:00
KaiXinXiaoLei e87bf3713e The UT test of spark is failed. Because there is a test in SQLQuerySuite about creating table “test”
If the tests in "sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala" are  running before CachedTableSuite.scala, the test("Drop cached table") will failed. Because the table test is created in SQLQuerySuite.scala  ,and this table not droped. So when running "drop cached table", table test already exists.

There is error info:
01:18:35.738 ERROR hive.ql.exec.DDLTask: org.apache.hadoop.hive.ql.metadata.HiveException: AlreadyExistsException(message:Table test already exists)
at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:616)
at org.apache.hadoop.hive.ql.exec.DDLTask.createTable(DDLTask.java:4189)
at org.apache.hadoop.hive.ql.exec.DDLTask.execute(DDLTask.java:281)
at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:153)
at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:85)
at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:1503)
at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1270)
at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1088)
at org.apache.hadoop.hive.ql.Driver.run(Driver.java:911)
at org.apache.hadoop.hive.ql.Driver.run(Driver.java:901)test”

And the test about "create table test" in "sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala,is:

  test("SPARK-4825 save join to table") {
    val testData = sparkContext.parallelize(1 to 10).map(i => TestData(i, i.toString)).toDF()
    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("CREATE TABLE test AS SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key =   b.key")
    checkAnswer(
      table("test"),
      sql("SELECT COUNT(a.value) FROM test1 a JOIN test2 b ON a.key = b.key").collect().toSeq)
  }

Author: KaiXinXiaoLei <huleilei1@huawei.com>

Closes #5150 from KaiXinXiaoLei/testFailed and squashes the following commits:

7534b02 [KaiXinXiaoLei] The UT test of spark is failed.
2015-03-25 19:15:30 -07:00
Daoyuan Wang 5ab6e9f0c0 [SPARK-6202] [SQL] enable variable substitution on test framework
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4930 from adrian-wang/testvs and squashes the following commits:

2ce590f [Daoyuan Wang] add explicit function types
b1d68bf [Daoyuan Wang] only substitute for parseSql
9c4a950 [Daoyuan Wang] add a comment explaining
18fb481 [Daoyuan Wang] enable variable substitute on test framework
2015-03-25 18:43:26 -07:00
DoingDone9 328daf65f8 [SPARK-6271][SQL] Sort these tokens in alphabetic order to avoid further duplicate in HiveQl
Author: DoingDone9 <799203320@qq.com>

Closes #4973 from DoingDone9/sort_token and squashes the following commits:

855fa10 [DoingDone9] Update HiveQl.scala
c7080b3 [DoingDone9] Sort these tokens in alphabetic order to avoid further duplicate in HiveQl
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-03-25 18:41:59 -07:00
jeanlyn e6d1406abd [SPARK-5498][SQL]fix query exception when partition schema does not match table schema
In hive,the schema of partition may be difference from  the table schema.When we use spark-sql to query the data of partition which schema is difference from the table schema,we will get the exceptions as the description of the [jira](https://issues.apache.org/jira/browse/SPARK-5498) .For example:
* We take a look of the schema for the partition and the table

```sql
DESCRIBE partition_test PARTITION (dt='1');
id                  	int              	None
name                	string              	None
dt                  	string              	None

# Partition Information
# col_name            	data_type           	comment

dt                  	string              	None
```
```
DESCRIBE partition_test;
OK
id                  	bigint              	None
name                	string              	None
dt                  	string              	None

# Partition Information
# col_name            	data_type           	comment

dt                  	string              	None
```
*  run the sql
```sql
SELECT * FROM partition_test where dt='1';
```
we will get the cast exception `java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.MutableLong cannot be cast to org.apache.spark.sql.catalyst.expressions.MutableInt`

Author: jeanlyn <jeanlyn92@gmail.com>

Closes #4289 from jeanlyn/schema and squashes the following commits:

9c8da74 [jeanlyn] fix style
b41d6b9 [jeanlyn] fix compile errors
07d84b6 [jeanlyn] Merge branch 'master' into schema
535b0b6 [jeanlyn] reduce conflicts
d6c93c5 [jeanlyn] fix bug
1e8b30c [jeanlyn] fix code style
0549759 [jeanlyn] fix code style
c879aa1 [jeanlyn] clean the code
2a91a87 [jeanlyn] add more test case and clean the code
12d800d [jeanlyn] fix code style
63d170a [jeanlyn] fix compile problem
7470901 [jeanlyn] reduce conflicts
afc7da5 [jeanlyn] make getConvertedOI compatible between 0.12.0 and 0.13.1
b1527d5 [jeanlyn] fix type mismatch
10744ca [jeanlyn] Insert a space after the start of the comment
3b27af3 [jeanlyn] SPARK-5498:fix bug when query the data when partition schema does not match table schema
2015-03-25 17:47:45 -07:00
Cheng Lian 8c3b0052f4 [SPARK-6450] [SQL] Fixes metastore Parquet table conversion
The `ParquetConversions` analysis rule generates a hash map, which maps from the original `MetastoreRelation` instances to the newly created `ParquetRelation2` instances. However, `MetastoreRelation.equals` doesn't compare output attributes. Thus, if a single metastore Parquet table appears multiple times in a query, only a single entry ends up in the hash map, and the conversion is not correctly performed.

Proper fix for this issue should be overriding `equals` and `hashCode` for MetastoreRelation. Unfortunately, this breaks more tests than expected. It's possible that these tests are ill-formed from the very beginning. As 1.3.1 release is approaching, we'd like to make the change more surgical to avoid potential regressions. The proposed fix here is to make both the metastore relations and their output attributes as keys in the hash map used in ParquetConversions.

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

Author: Cheng Lian <lian@databricks.com>

Closes #5183 from liancheng/spark-6450 and squashes the following commits:

3536780 [Cheng Lian] Fixes metastore Parquet table conversion
2015-03-25 17:40:19 -07:00
DoingDone9 968408b345 [SPARK-6409][SQL] It is not necessary that avoid old inteface of hive, because this will make some UDAF can not work.
spark avoid old inteface of hive, then some udaf can not work like "org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage"

Author: DoingDone9 <799203320@qq.com>

Closes #5131 from DoingDone9/udaf and squashes the following commits:

9de08d0 [DoingDone9] Update HiveUdfSuite.scala
49c62dc [DoingDone9] Update hiveUdfs.scala
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-03-25 11:11:52 -07:00
Michael Armbrust cbeaf9ebab [SPARK-6376][SQL] Avoid eliminating subqueries until optimization
Previously it was okay to throw away subqueries after analysis, as we would never try to use that tree for resolution again.  However, with eager analysis in `DataFrame`s this can cause errors for queries such as:

```scala
val df = Seq(1,2,3).map(i => (i, i.toString)).toDF("int", "str")
df.as('x).join(df.as('y), $"x.str" === $"y.str").groupBy("x.str").count()
```

As a result, in this PR we defer the elimination of subqueries until the optimization phase.

Author: Michael Armbrust <michael@databricks.com>

Closes #5160 from marmbrus/subqueriesInDfs and squashes the following commits:

a9bb262 [Michael Armbrust] Update Optimizer.scala
27d25bf [Michael Armbrust] fix hive tests
9137e03 [Michael Armbrust] add type
81cd597 [Michael Armbrust] Avoid eliminating subqueries until optimization
2015-03-24 14:08:20 -07:00
Michael Armbrust 046c1e2aa4 [SPARK-6375][SQL] Fix formatting of error messages.
Author: Michael Armbrust <michael@databricks.com>

Closes #5155 from marmbrus/errorMessages and squashes the following commits:

b898188 [Michael Armbrust] Fix formatting of error messages.
2015-03-24 13:22:46 -07:00
Reynold Xin b6090f902e [SPARK-6428][SQL] Added explicit type for all public methods for Hive module
Author: Reynold Xin <rxin@databricks.com>

Closes #5108 from rxin/hive-public-type and squashes the following commits:

a320328 [Reynold Xin] [SPARK-6428][SQL] Added explicit type for all public methods for Hive module.
2015-03-21 14:30:04 -07:00
Yin Huai 94a102acb8 [SPARK-6250][SPARK-6146][SPARK-5911][SQL] Types are now reserved words in DDL parser.
This PR creates a trait `DataTypeParser` used to parse data types. This trait aims to be single place to provide the functionality of parsing data types' string representation. It is currently mixed in with `DDLParser` and `SqlParser`. It is also used to parse the data type for `DataFrame.cast` and to convert Hive metastore's data type string back to a `DataType`.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #5078 from yhuai/ddlKeywords and squashes the following commits:

0e66097 [Yin Huai] Special handle struct<>.
fea6012 [Yin Huai] Style.
c9733fb [Yin Huai] Create a trait to parse data types.
2015-03-21 13:27:53 -07:00
Venkata Ramana Gollamudi ee569a0c71 [SPARK-5680][SQL] Sum function on all null values, should return zero
SELECT sum('a'), avg('a'), variance('a'), std('a') FROM src;
Should give output as
0.0	NULL	NULL	NULL
This fixes hive udaf_number_format.q

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

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

Closes #4466 from gvramana/sum_fix and squashes the following commits:

42e14d1 [Venkata Ramana Gollamudi] Added comments
39415c0 [Venkata Ramana Gollamudi] Handled the partitioned Sum expression scenario
df66515 [Venkata Ramana Gollamudi] code style fix
4be2606 [Venkata Ramana Gollamudi] Add udaf_number_format to whitelist and golden answer
330fd64 [Venkata Ramana Gollamudi] fix sum function for all null data
2015-03-21 13:24:24 -07:00
Marcelo Vanzin a74564591f [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5056 from vanzin/SPARK-6371 and squashes the following commits:

63220df [Marcelo Vanzin] Merge branch 'master' into SPARK-6371
6506f75 [Marcelo Vanzin] Use more fine-grained exclusion.
178ba71 [Marcelo Vanzin] Oops.
75b2375 [Marcelo Vanzin] Exclude VertexRDD in MiMA.
a45a62c [Marcelo Vanzin] Work around MIMA warning.
1d8a670 [Marcelo Vanzin] Re-group jetty exclusion.
0e8e909 [Marcelo Vanzin] Ignore ml, don't ignore graphx.
cef4603 [Marcelo Vanzin] Indentation.
296cf82 [Marcelo Vanzin] [SPARK-6371] [build] Update version to 1.4.0-SNAPSHOT.
2015-03-20 18:43:57 +00:00
Sean Owen 6f80c3e888 SPARK-6338 [CORE] Use standard temp dir mechanisms in tests to avoid orphaned temp files
Use `Utils.createTempDir()` to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify

Author: Sean Owen <sowen@cloudera.com>

Closes #5029 from srowen/SPARK-6338 and squashes the following commits:

27b740a [Sean Owen] Fix hive-thriftserver tests that don't expect an existing dir
4a212fa [Sean Owen] Standardize a bit more temp dir management
9004081 [Sean Owen] Revert some added recursive-delete calls
57609e4 [Sean Owen] Use Utils.createTempDir() to replace other temp file mechanisms used in some tests, to further ensure they are cleaned up, and simplify
2015-03-20 14:16:21 +00:00
Michael Armbrust 3579003115 [SPARK-6247][SQL] Fix resolution of ambiguous joins caused by new aliases
We need to handle ambiguous `exprId`s that are produced by new aliases as well as those caused by leaf nodes (`MultiInstanceRelation`).

Attempting to fix this revealed a bug in `equals` for `Alias` as these objects were comparing equal even when the expression ids did not match. Additionally, `LocalRelation` did not correctly provide statistics, and some tests in `catalyst` and `hive` were not using the helper functions for comparing plans.

Based on #4991 by chenghao-intel

Author: Michael Armbrust <michael@databricks.com>

Closes #5062 from marmbrus/selfJoins and squashes the following commits:

8e9b84b [Michael Armbrust] check qualifier too
8038a36 [Michael Armbrust] handle aggs too
0b9c687 [Michael Armbrust] fix more tests
c3c574b [Michael Armbrust] revert change.
725f1ab [Michael Armbrust] add statistics
a925d08 [Michael Armbrust] check for conflicting attributes in join resolution
b022ef7 [Michael Armbrust] Handle project aliases.
d8caa40 [Michael Armbrust] test case: SPARK-6247
f9c67c2 [Michael Armbrust] Check for duplicate attributes in join resolution.
898af73 [Michael Armbrust] Fix Alias equality.
2015-03-17 19:47:51 -07:00
watermen a6ee2f7940 [SPARK-5651][SQL] Add input64 in blacklist and add test suit for create table within backticks
Now spark version is only support
```create table table_in_database_creation.test1 as select * from src limit 1;``` in HiveContext.

This patch is used to support
```create table `table_in_database_creation.test2` as select * from src limit 1;``` in HiveContext.

Author: watermen <qiyadong2010@gmail.com>
Author: q00251598 <qiyadong@huawei.com>

Closes #4427 from watermen/SPARK-5651 and squashes the following commits:

c5c8ed1 [watermen] add the generated golden files
1f0e42e [q00251598] add input64 in blacklist and add test suit
2015-03-17 19:35:18 -07:00
Liang-Chi Hsieh 5c80643d13 [SPARK-5908][SQL] Resolve UdtfsAlias when only single Alias is used
`ResolveUdtfsAlias` in `hiveUdfs` only considers the `HiveGenericUdtf` with multiple alias. When only single alias is used with `HiveGenericUdtf`, the alias is not working.

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

Closes #4692 from viirya/udft_alias and squashes the following commits:

8a3bae4 [Liang-Chi Hsieh] No need to test selected column from DataFrame since DataFrame API is updated.
160a379 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into udft_alias
e6531cc [Liang-Chi Hsieh] Selected column from DataFrame should not re-analyze logical plan.
a45cc2a [Liang-Chi Hsieh] Resolve UdtfsAlias when only single Alias is used.
2015-03-17 18:58:52 -07:00
Daoyuan Wang 9667b9f9c3 [SPARK-5712] [SQL] fix comment with semicolon at end
---- comment;

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

Closes #4500 from adrian-wang/semicolon and squashes the following commits:

70b8abb [Daoyuan Wang] use mkstring instead of reduce
2d49738 [Daoyuan Wang] remove outdated golden file
317346e [Daoyuan Wang] only skip comment with semicolon at end of line, to avoid golden file outdated
d3ae01e [Daoyuan Wang] fix error
a11602d [Daoyuan Wang] fix comment with semicolon at end
2015-03-17 12:29:15 +08:00
Cheng Hao 12a345adcb [SPARK-2087] [SQL] Multiple thriftserver sessions with single HiveContext instance
Still, we keep only a single HiveContext within ThriftServer, and we also create a object called `SQLSession` for isolating the different user states.

Developers can obtain/release a new user session via `openSession` and `closeSession`, and `SQLContext` and `HiveContext` will also provide a default session if no `openSession` called, for backward-compatibility.

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

Closes #4885 from chenghao-intel/multisessions_singlecontext and squashes the following commits:

1c47b2a [Cheng Hao] rename the tss => tlSession
815b27a [Cheng Hao] code style issue
57e3fa0 [Cheng Hao] openSession is not compatible between Hive0.12 & 0.13.1
4665b0d [Cheng Hao] thriftservice with single context
2015-03-17 01:09:27 +08:00
Sean Owen 6e94c4eadf SPARK-6225 [CORE] [SQL] [STREAMING] Resolve most build warnings, 1.3.0 edition
Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.

Author: Sean Owen <sowen@cloudera.com>

Closes #4950 from srowen/SPARK-6225 and squashes the following commits:

3080972 [Sean Owen] Ordered imports: Java, Scala, 3rd party, Spark
c67985b [Sean Owen] Resolve javac, scalac warnings of various types -- deprecations, Scala lang, unchecked cast, etc.
2015-03-11 13:15:19 +00:00
Sean Owen c9cfba0ceb SPARK-6182 [BUILD] spark-parent pom needs to be published for both 2.10 and 2.11
Option 1 of 2: Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11

Author: Sean Owen <sowen@cloudera.com>

Closes #4912 from srowen/SPARK-6182.1 and squashes the following commits:

eff60de [Sean Owen] Convert spark-parent module name to spark-parent_2.10 / spark-parent_2.11
2015-03-05 11:31:48 -08:00
Reynold Xin 54d19689ff [SPARK-5310][SQL] Fixes to Docs and Datasources API
- Various Fixes to docs
 - Make data source traits actually interfaces

Based on #4862 but with fixed conflicts.

Author: Reynold Xin <rxin@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #4868 from marmbrus/pr/4862 and squashes the following commits:

fe091ea [Michael Armbrust] Merge remote-tracking branch 'origin/master' into pr/4862
0208497 [Reynold Xin] Test fixes.
34e0a28 [Reynold Xin] [SPARK-5310][SQL] Various fixes to Spark SQL docs.
2015-03-02 22:14:08 -08:00
Yin Huai 12599942e6 [SPARK-5950][SQL]Insert array into a metastore table saved as parquet should work when using datasource api
This PR contains the following changes:
1. Add a new method, `DataType.equalsIgnoreCompatibleNullability`, which is the middle ground between DataType's equality check and `DataType.equalsIgnoreNullability`. For two data types `from` and `to`, it does `equalsIgnoreNullability` as well as if the nullability of `from` is compatible with that of `to`. For example, the nullability of `ArrayType(IntegerType, containsNull = false)` is compatible with that of `ArrayType(IntegerType, containsNull = true)` (for an array without null values, we can always say it may contain null values). However,  the nullability of `ArrayType(IntegerType, containsNull = true)` is incompatible with that of `ArrayType(IntegerType, containsNull = false)` (for an array that may have null values, we cannot say it does not have null values).
2. For the `resolved` field of `InsertIntoTable`, use `equalsIgnoreCompatibleNullability` to replace the equality check of the data types.
3. For our data source write path, when appending data, we always use the schema of existing table to write the data. This is important for parquet, since nullability direct impacts the way to encode/decode values. If we do not do this, we may see corrupted values when reading values from a set of parquet files generated with different nullability settings.
4. When generating a new parquet table, we always set nullable/containsNull/valueContainsNull to true. So, we will not face situations that we cannot append data because containsNull/valueContainsNull in an Array/Map column of the existing table has already been set to `false`. This change makes the whole data pipeline more robust.
5. Update the equality check of JSON relation. Since JSON does not really cares nullability,  `equalsIgnoreNullability` seems a better choice to compare schemata from to JSON tables.

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

Thanks viirya for the initial work in #4729.

cc marmbrus liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #4826 from yhuai/insertNullabilityCheck and squashes the following commits:

3b61a04 [Yin Huai] Revert change on equals.
80e487e [Yin Huai] asNullable in UDT.
587d88b [Yin Huai] Make methods private.
0cb7ea2 [Yin Huai] marmbrus's comments.
3cec464 [Yin Huai] Cheng's comments.
486ed08 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
d3747d1 [Yin Huai] Remove unnecessary change.
8360817 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertNullabilityCheck
8a3f237 [Yin Huai] Use equalsIgnoreNullability instead of equality check.
0eb5578 [Yin Huai] Fix tests.
f6ed813 [Yin Huai] Update old parquet path.
e4f397c [Yin Huai] Unit tests.
b2c06f8 [Yin Huai] Ignore nullability in JSON relation's equality check.
8bd008b [Yin Huai] nullable, containsNull, and valueContainsNull will be always true for parquet data.
bf50d73 [Yin Huai] When appending data, we use the schema of the existing table instead of the schema of the new data.
0a703e7 [Yin Huai] Test failed again since we cannot read correct content.
9a26611 [Yin Huai] Make InsertIntoTable happy.
8f19fe5 [Yin Huai] equalsIgnoreCompatibleNullability
4ec17fd [Yin Huai] Failed test.
2015-03-02 19:31:55 -08:00
Michael Armbrust 8223ce6a81 [SPARK-6114][SQL] Avoid metastore conversions before plan is resolved
Author: Michael Armbrust <michael@databricks.com>

Closes #4855 from marmbrus/explodeBug and squashes the following commits:

a712249 [Michael Armbrust] [SPARK-6114][SQL] Avoid metastore conversions before plan is resolved
2015-03-02 16:10:54 -08:00
q00251598 582e5a24c5 [SPARK-6040][SQL] Fix the percent bug in tablesample
HiveQL expression like `select count(1) from src tablesample(1 percent);` means take 1% sample to select. But it means 100% in the current version of the Spark.

Author: q00251598 <qiyadong@huawei.com>

Closes #4789 from watermen/SPARK-6040 and squashes the following commits:

2453ebe [q00251598] check and adjust the fraction.
2015-03-02 13:16:29 -08:00
q00251598 9ce12aaf28 [SPARK-5741][SQL] Support the path contains comma in HiveContext
When run ```select * from nzhang_part where hr = 'file,';```, it throws exception ```java.lang.IllegalArgumentException: Can not create a Path from an empty string```
. Because the path of hdfs contains comma, and FileInputFormat.setInputPaths will split path by comma.

### SQL
```
set hive.merge.mapfiles=true;
set hive.merge.mapredfiles=true;
set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat;
set hive.exec.dynamic.partition=true;
set hive.exec.dynamic.partition.mode=nonstrict;

create table nzhang_part like srcpart;

insert overwrite table nzhang_part partition (ds='2010-08-15', hr) select key, value, hr from srcpart where ds='2008-04-08';

insert overwrite table nzhang_part partition (ds='2010-08-15', hr=11) select key, value from srcpart where ds='2008-04-08';

insert overwrite table nzhang_part partition (ds='2010-08-15', hr)
select * from (
select key, value, hr from srcpart where ds='2008-04-08'
union all
select '1' as key, '1' as value, 'file,' as hr from src limit 1) s;

select * from nzhang_part where hr = 'file,';
```

### Error Log
```
15/02/10 14:33:16 ERROR SparkSQLDriver: Failed in [select * from nzhang_part where hr = 'file,']
java.lang.IllegalArgumentException: Can not create a Path from an empty string
at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
at org.apache.hadoop.fs.Path.<init>(Path.java:135)
at org.apache.hadoop.util.StringUtils.stringToPath(StringUtils.java:241)
at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:400)
at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:251)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$11.apply(TableReader.scala:229)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:172)
at scala.Option.map(Option.scala:145)
at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:172)
at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:196)

Author: q00251598 <qiyadong@huawei.com>

Closes #4532 from watermen/SPARK-5741 and squashes the following commits:

9758ab1 [q00251598] fix bug
1db1a1c [q00251598] use setInputPaths(Job job, Path... inputPaths)
b788a72 [q00251598] change FileInputFormat.setInputPaths to jobConf.set and add test suite
2015-03-02 10:13:11 -08:00
Yin Huai 39a54b40af [SPARK-6073][SQL] Need to refresh metastore cache after append data in CreateMetastoreDataSourceAsSelect
JIRA: https://issues.apache.org/jira/browse/SPARK-6073

liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #4824 from yhuai/refreshCache and squashes the following commits:

b9542ef [Yin Huai] Refresh metadata cache in the Catalog in CreateMetastoreDataSourceAsSelect.
2015-03-02 22:42:18 +08:00
Cheng Lian e6003f0a57 [SPARK-5775] [SQL] BugFix: GenericRow cannot be cast to SpecificMutableRow when nested data and partitioned table
This PR adapts anselmevignon's #4697 to master and branch-1.3. Please refer to PR description of #4697 for details.

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

Author: Cheng Lian <lian@databricks.com>
Author: Cheng Lian <liancheng@users.noreply.github.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #4792 from liancheng/spark-5775 and squashes the following commits:

538f506 [Cheng Lian] Addresses comments
cee55cf [Cheng Lian] Merge pull request #4 from yhuai/spark-5775-yin
b0b74fb [Yin Huai] Remove runtime pattern matching.
ca6e038 [Cheng Lian] Fixes SPARK-5775
2015-02-28 21:15:43 +08:00
Yin Huai 5e5ad6558d [SPARK-6024][SQL] When a data source table has too many columns, it's schema cannot be stored in metastore.
JIRA: https://issues.apache.org/jira/browse/SPARK-6024

Author: Yin Huai <yhuai@databricks.com>

Closes #4795 from yhuai/wideSchema and squashes the following commits:

4882e6f [Yin Huai] Address comments.
73e71b4 [Yin Huai] Address comments.
143927a [Yin Huai] Simplify code.
cc1d472 [Yin Huai] Make the schema wider.
12bacae [Yin Huai] If the JSON string of a schema is too large, split it before storing it in metastore.
e9b4f70 [Yin Huai] Failed test.
2015-02-26 20:46:05 -08:00
Yin Huai 192e42a293 [SPARK-6016][SQL] Cannot read the parquet table after overwriting the existing table when spark.sql.parquet.cacheMetadata=true
Please see JIRA (https://issues.apache.org/jira/browse/SPARK-6016) for details of the bug.

Author: Yin Huai <yhuai@databricks.com>

Closes #4775 from yhuai/parquetFooterCache and squashes the following commits:

78787b1 [Yin Huai] Remove footerCache in FilteringParquetRowInputFormat.
dff6fba [Yin Huai] Failed unit test.
2015-02-27 01:01:32 +08:00
Yin Huai f02394d064 [SPARK-6023][SQL] ParquetConversions fails to replace the destination MetastoreRelation of an InsertIntoTable node to ParquetRelation2
JIRA: https://issues.apache.org/jira/browse/SPARK-6023

Author: Yin Huai <yhuai@databricks.com>

Closes #4782 from yhuai/parquetInsertInto and squashes the following commits:

ae7e806 [Yin Huai] Convert MetastoreRelation in InsertIntoTable and InsertIntoHiveTable.
ba543cd [Yin Huai] More tests.
50b6d0f [Yin Huai] Update error messages.
346780c [Yin Huai] Failed test.
2015-02-26 22:39:49 +08:00
Yin Huai 769e092bdc [SPARK-5286][SQL] SPARK-5286 followup
https://issues.apache.org/jira/browse/SPARK-5286

Author: Yin Huai <yhuai@databricks.com>

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

4c0c450 [Yin Huai] Catch Throwable instead of Exception.
2015-02-24 19:51:36 -08:00
Michael Armbrust a2b9137923 [SPARK-5952][SQL] Lock when using hive metastore client
Author: Michael Armbrust <michael@databricks.com>

Closes #4746 from marmbrus/hiveLock and squashes the following commits:

8b871cf [Michael Armbrust] [SPARK-5952][SQL] Lock when using hive metastore client
2015-02-24 13:39:29 -08:00
Michael Armbrust 1ed57086d4 [SPARK-5873][SQL] Allow viewing of partially analyzed plans in queryExecution
Author: Michael Armbrust <michael@databricks.com>

Closes #4684 from marmbrus/explainAnalysis and squashes the following commits:

afbaa19 [Michael Armbrust] fix python
d93278c [Michael Armbrust] fix hive
e5fa0a4 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis
52119f2 [Michael Armbrust] more tests
82a5431 [Michael Armbrust] fix tests
25753d2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explainAnalysis
aee1e6a [Michael Armbrust] fix hive
b23a844 [Michael Armbrust] newline
de8dc51 [Michael Armbrust] more comments
acf620a [Michael Armbrust] [SPARK-5873][SQL] Show partially analyzed plans in query execution
2015-02-23 17:34:54 -08:00
Reynold Xin f0e3b71077 [SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction
Also added test cases for checking the serializability of HiveContext and SQLContext.

Author: Reynold Xin <rxin@databricks.com>

Closes #4628 from rxin/SPARK-5840 and squashes the following commits:

ecb3bcd [Reynold Xin] test cases and reviews.
55eb822 [Reynold Xin] [SPARK-5840][SQL] HiveContext cannot be serialized due to tuple extraction.
2015-02-18 14:02:32 -08:00
Cheng Lian 61ab08549c [Minor] [SQL] Cleans up DataFrame variable names and toDF() calls
Although we've migrated to the DataFrame API, lots of code still uses `rdd` or `srdd` as local variable names. This PR tries to address these naming inconsistencies and some other minor DataFrame related style issues.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4670 from liancheng/df-cleanup and squashes the following commits:

3e14448 [Cheng Lian] Cleans up DataFrame variable names and toDF() calls
2015-02-17 23:36:20 -08:00
Yin Huai e50934f11e [SPARK-5723][SQL]Change the default file format to Parquet for CTAS statements.
JIRA: https://issues.apache.org/jira/browse/SPARK-5723

Author: Yin Huai <yhuai@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>

Closes #4639 from yhuai/defaultCTASFileFormat and squashes the following commits:

a568137 [Yin Huai] Merge remote-tracking branch 'upstream/master' into defaultCTASFileFormat
ad2b07d [Yin Huai] Update tests and error messages.
8af5b2a [Yin Huai] Update conf key and unit test.
5a67903 [Yin Huai] Use data source write path for Hive's CTAS statements when no storage format/handler is specified.
2015-02-17 18:14:33 -08:00
Yin Huai d5f12bfe8f [SPARK-5875][SQL]logical.Project should not be resolved if it contains aggregates or generators
https://issues.apache.org/jira/browse/SPARK-5875 has a case to reproduce the bug and explain the root cause.

Author: Yin Huai <yhuai@databricks.com>

Closes #4663 from yhuai/projectResolved and squashes the following commits:

472f7b6 [Yin Huai] If a logical.Project has any AggregateExpression or Generator, it's resolved field should be false.
2015-02-17 17:50:39 -08:00
Yin Huai 117121a4ec [SPARK-5852][SQL]Fail to convert a newly created empty metastore parquet table to a data source parquet table.
The problem is that after we create an empty hive metastore parquet table (e.g. `CREATE TABLE test (a int) STORED AS PARQUET`), Hive will create an empty dir for us, which cause our data source `ParquetRelation2` fail to get the schema of the table. See JIRA for the case to reproduce the bug and the exception.

This PR is based on #4562 from chenghao-intel.

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

Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Hao <hao.cheng@intel.com>

Closes #4655 from yhuai/CTASParquet and squashes the following commits:

b8b3450 [Yin Huai] Update tests.
2ac94f7 [Yin Huai] Update tests.
3db3d20 [Yin Huai] Minor update.
d7e2308 [Yin Huai] Revert changes in HiveMetastoreCatalog.scala.
36978d1 [Cheng Hao] Update the code as feedback
a04930b [Cheng Hao] fix bug of scan an empty parquet based table
442ffe0 [Cheng Hao] passdown the schema for Parquet File in HiveContext
2015-02-17 15:47:59 -08:00
Cheng Hao 9d281fa560 [SQL] [Minor] Update the HiveContext Unittest
In unit test, the table src(key INT, value STRING) is not the same as HIVE src(key STRING, value STRING)
https://github.com/apache/hive/blob/branch-0.13/data/scripts/q_test_init.sql

And in the reflect.q, test failed for expression `reflect("java.lang.Integer", "valueOf", key, 16)`, which expect the argument `key` as STRING not INT.

This PR doesn't aim to change the `src` schema, we can do that after 1.3 released, however, we probably need to re-generate all the golden files.

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

Closes #4584 from chenghao-intel/reflect and squashes the following commits:

e5bdc3a [Cheng Hao] Move the test case reflect into blacklist
184abfd [Cheng Hao] revert the change to table src1
d9bcf92 [Cheng Hao] Update the HiveContext Unittest
2015-02-17 12:25:35 -08:00
Liang-Chi Hsieh ac506b7c28 [Minor][SQL] Use same function to check path parameter in JSONRelation
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #4649 from viirya/use_checkpath and squashes the following commits:

0f9a1a1 [Liang-Chi Hsieh] Use same function to check path parameter.
2015-02-17 12:24:13 -08:00
Liang-Chi Hsieh 4611de1cef [SPARK-5862][SQL] Only transformUp the given plan once in HiveMetastoreCatalog
Current `ParquetConversions` in `HiveMetastoreCatalog` will transformUp the given plan multiple times if there are many Metastore Parquet tables. Since the transformUp operation is recursive, it should be better to only perform it once.

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

Closes #4651 from viirya/parquet_atonce and squashes the following commits:

c1ed29d [Liang-Chi Hsieh] Fix bug.
e0f919b [Liang-Chi Hsieh] Only transformUp the given plan once.
2015-02-17 12:23:18 -08:00
Michael Armbrust c74b07fa94 [SPARK-5166][SPARK-5247][SPARK-5258][SQL] API Cleanup / Documentation
Author: Michael Armbrust <michael@databricks.com>

Closes #4642 from marmbrus/docs and squashes the following commits:

d291c34 [Michael Armbrust] python tests
9be66e3 [Michael Armbrust] comments
d56afc2 [Michael Armbrust] fix style
f004747 [Michael Armbrust] fix build
c4a907b [Michael Armbrust] fix tests
42e2b73 [Michael Armbrust] [SQL] Documentation / API Clean-up.
2015-02-17 10:21:17 -08:00
Reynold Xin 0e180bfc3c [SQL] Various DataFrame doc changes.
Added a bunch of tags.

Also changed parquetFile to take varargs rather than a string followed by varargs.

Author: Reynold Xin <rxin@databricks.com>

Closes #4636 from rxin/df-doc and squashes the following commits:

651f80c [Reynold Xin] Fixed parquetFile in PySpark.
8dc3024 [Reynold Xin] [SQL] Various DataFrame doc changes.
2015-02-16 19:00:30 -08:00
Yin Huai e189cbb052 [SPARK-4865][SQL]Include temporary tables in SHOW TABLES
This PR adds a `ShowTablesCommand` to support `SHOW TABLES [IN databaseName]` SQL command. The result of `SHOW TABLE` has two columns, `tableName` and `isTemporary`. For temporary tables, the value of `isTemporary` column will be `false`.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #4618 from yhuai/showTablesCommand and squashes the following commits:

0c09791 [Yin Huai] Use ShowTablesCommand.
85ee76d [Yin Huai] Since SHOW TABLES is not a Hive native command any more and we will not see "OK" (originally generated by Hive's driver), use SHOW DATABASES in the test.
94bacac [Yin Huai] Add SHOW TABLES to the list of noExplainCommands.
d71ed09 [Yin Huai] Fix test.
a4a6ec3 [Yin Huai] Add SHOW TABLE command.
2015-02-16 15:59:23 -08:00
Yin Huai f3ff1eb298 [SPARK-5839][SQL]HiveMetastoreCatalog does not recognize table names and aliases of data source tables.
JIRA: https://issues.apache.org/jira/browse/SPARK-5839

Author: Yin Huai <yhuai@databricks.com>

Closes #4626 from yhuai/SPARK-5839 and squashes the following commits:

f779d85 [Yin Huai] Use subqeury to wrap replaced ParquetRelation.
2695f13 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-5839
f1ba6ca [Yin Huai] Address comment.
2c7fa08 [Yin Huai] Use Subqueries to wrap a data source table.
2015-02-16 15:54:01 -08:00
Yin Huai 5b6cd65cd6 [SPARK-5746][SQL] Check invalid cases for the write path of data source API
JIRA: https://issues.apache.org/jira/browse/SPARK-5746

liancheng marmbrus

Author: Yin Huai <yhuai@databricks.com>

Closes #4617 from yhuai/insertOverwrite and squashes the following commits:

8e3019d [Yin Huai] Fix compilation error.
499e8e7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite
e76e85a [Yin Huai] Address comments.
ac31b3c [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite
f30bdad [Yin Huai] Use toDF.
99da57e [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertOverwrite
6b7545c [Yin Huai] Add a pre write check to the data source API.
a88c516 [Yin Huai] DDLParser will take a parsering function to take care CTAS statements.
2015-02-16 15:51:59 -08:00
Cheng Lian c51ab37fad [SPARK-5833] [SQL] Adds REFRESH TABLE command
Lifts `HiveMetastoreCatalog.refreshTable` to `Catalog`. Adds `RefreshTable` command to refresh (possibly cached) metadata in external data sources tables.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4624 from liancheng/refresh-table and squashes the following commits:

8d1aa4c [Cheng Lian] Adds REFRESH TABLE command
2015-02-16 12:52:05 -08:00
Michael Armbrust 104b2c4580 [SQL] Initial support for reporting location of error in sql string
Author: Michael Armbrust <michael@databricks.com>

Closes #4587 from marmbrus/position and squashes the following commits:

0810052 [Michael Armbrust] fix tests
395c019 [Michael Armbrust] Merge remote-tracking branch 'marmbrus/position' into position
e155dce [Michael Armbrust] more errors
f3efa51 [Michael Armbrust] Update AnalysisException.scala
d45ff60 [Michael Armbrust] [SQL] Initial support for reporting location of error in sql string
2015-02-16 12:32:56 -08:00
Daoyuan Wang 275a0c0813 [SPARK-5824] [SQL] add null format in ctas and set default col comment to null
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4609 from adrian-wang/ctas and squashes the following commits:

0a75d5a [Daoyuan Wang] reorder import
93d1863 [Daoyuan Wang] add null format in ctas and set default col comment to null
2015-02-16 12:31:36 -08:00
Cheng Lian 3ce58cf9c0 [SPARK-4553] [SPARK-5767] [SQL] Wires Parquet data source with the newly introduced write support for data source API
This PR migrates the Parquet data source to the new data source write support API.  Now users can also overwriting and appending to existing tables. Notice that inserting into partitioned tables is not supported yet.

When Parquet data source is enabled, insertion to Hive Metastore Parquet tables is also fullfilled by the Parquet data source. This is done by the newly introduced `HiveMetastoreCatalog.ParquetConversions` rule, which is a "proper" implementation of the original hacky `HiveStrategies.ParquetConversion`. The latter is still preserved, and can be removed together with the old Parquet support in the future.

TODO:

- [x] Update outdated comments in `newParquet.scala`.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4563 from liancheng/parquet-refining and squashes the following commits:

fa98d27 [Cheng Lian] Fixes test cases which should disable off Parquet data source
2476e82 [Cheng Lian] Fixes compilation error introduced during rebasing
a83d290 [Cheng Lian] Passes Hive Metastore partitioning information to ParquetRelation2
2015-02-16 01:38:31 -08:00
Reynold Xin e98dfe627c [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
- The old implicit would convert RDDs directly to DataFrames, and that added too many methods.
- toDataFrame -> toDF
- Dsl -> functions
- implicits moved into SQLContext.implicits
- addColumn -> withColumn
- renameColumn -> withColumnRenamed

Python changes:
- toDataFrame -> toDF
- Dsl -> functions package
- addColumn -> withColumn
- renameColumn -> withColumnRenamed
- add toDF functions to RDD on SQLContext init
- add flatMap to DataFrame

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

Closes #4556 from rxin/SPARK-5752 and squashes the following commits:

5ef9910 [Reynold Xin] More fix
61d3fca [Reynold Xin] Merge branch 'df5' of github.com:davies/spark into SPARK-5752
ff5832c [Reynold Xin] Fix python
749c675 [Reynold Xin] count(*) fixes.
5806df0 [Reynold Xin] Fix build break again.
d941f3d [Reynold Xin] Fixed explode compilation break.
fe1267a [Davies Liu] flatMap
c4afb8e [Reynold Xin] style
d9de47f [Davies Liu] add comment
b783994 [Davies Liu] add comment for toDF
e2154e5 [Davies Liu] schema() -> schema
3a1004f [Davies Liu] Dsl -> functions, toDF()
fb256af [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed
0dd74eb [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
97dd47c [Davies Liu] fix mistake
6168f74 [Davies Liu] fix test
1fc0199 [Davies Liu] fix test
a075cd5 [Davies Liu] clean up, toPandas
663d314 [Davies Liu] add test for agg('*')
9e214d5 [Reynold Xin] count(*) fixes.
1ed7136 [Reynold Xin] Fix build break again.
921b2e3 [Reynold Xin] Fixed explode compilation break.
14698d4 [Davies Liu] flatMap
ba3e12d [Reynold Xin] style
d08c92d [Davies Liu] add comment
5c8b524 [Davies Liu] add comment for toDF
a4e5e66 [Davies Liu] schema() -> schema
d377fc9 [Davies Liu] Dsl -> functions, toDF()
6b3086c [Reynold Xin] - toDataFrame -> toDF - Dsl -> functions - implicits moved into SQLContext.implicits - addColumn -> withColumn - renameColumn -> withColumnRenamed
807e8b1 [Reynold Xin] [SPARK-5752][SQL] Don't implicitly convert RDDs directly to DataFrames
2015-02-13 23:03:22 -08:00
Yin Huai 1d0596a16e [SPARK-3299][SQL]Public API in SQLContext to list tables
https://issues.apache.org/jira/browse/SPARK-3299

Author: Yin Huai <yhuai@databricks.com>

Closes #4547 from yhuai/tables and squashes the following commits:

6c8f92e [Yin Huai] Add tableNames.
acbb281 [Yin Huai] Update Python test.
7793dcb [Yin Huai] Fix scala test.
572870d [Yin Huai] Address comments.
aba2e88 [Yin Huai] Format.
12c86df [Yin Huai] Add tables() to SQLContext to return a DataFrame containing existing tables.
2015-02-12 18:08:01 -08:00
Yin Huai c025a46882 [SQL] Move SaveMode to SQL package.
Author: Yin Huai <yhuai@databricks.com>

Closes #4542 from yhuai/moveSaveMode and squashes the following commits:

65a4425 [Yin Huai] Move SaveMode to sql package.
2015-02-12 15:32:17 -08:00
Daoyuan Wang d5fc514918 [SPARK-5755] [SQL] remove unnecessary Add
explain extended select +key from src;
before:
== Parsed Logical Plan ==
'Project [(0 + 'key) AS _c0#8]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [(0 + key#10) AS _c0#8]
 MetastoreRelation test, src, None

== Physical Plan ==
Project [(0 + key#10) AS _c0#8]
 HiveTableScan [key#10], (MetastoreRelation test, src, None), None

after this patch:
== Parsed Logical Plan ==
'Project ['key]
 'UnresolvedRelation [src], None

== Analyzed Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Optimized Logical Plan ==
Project [key#10]
 MetastoreRelation test, src, None

== Physical Plan ==
HiveTableScan [key#10], (MetastoreRelation test, src, None), None

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

Closes #4551 from adrian-wang/positive and squashes the following commits:

0821ae4 [Daoyuan Wang] remove unnecessary Add
2015-02-12 15:22:07 -08:00
Michael Armbrust aa4ca8b873 [SQL] Improve error messages
Author: Michael Armbrust <michael@databricks.com>
Author: wangfei <wangfei1@huawei.com>

Closes #4558 from marmbrus/errorMessages and squashes the following commits:

5e5ab50 [Michael Armbrust] Merge pull request #15 from scwf/errorMessages
fa38881 [wangfei] fix for grouping__id
f279a71 [wangfei] make right references for ScriptTransformation
d29fbde [Michael Armbrust] extra case
1a797b4 [Michael Armbrust] comments
d4e9015 [Michael Armbrust] add comment
af9e668 [Michael Armbrust] no braces
34eb3a4 [Michael Armbrust] more work
6197cd5 [Michael Armbrust] [SQL] Better error messages for analysis failures
2015-02-12 13:11:28 -08:00
tianyi 44b2311d94 [SPARK-3688][SQL]LogicalPlan can't resolve column correctlly
This PR fixed the resolving problem described in https://issues.apache.org/jira/browse/SPARK-3688
```
CREATE TABLE t1(x INT);
CREATE TABLE t2(a STRUCT<x: INT>, k INT);
SELECT a.x FROM t1 a JOIN t2 b ON a.x = b.k;
```

Author: tianyi <tianyi.asiainfo@gmail.com>

Closes #4524 from tianyi/SPARK-3688 and squashes the following commits:

237a256 [tianyi] resolve a name with table.column pattern first.
2015-02-11 12:50:17 -08:00
Michael Armbrust a60d2b70ad [SPARK-5454] More robust handling of self joins
Also I fix a bunch of bad output in test cases.

Author: Michael Armbrust <michael@databricks.com>

Closes #4520 from marmbrus/selfJoin and squashes the following commits:

4f4a85c [Michael Armbrust] comments
49c8e26 [Michael Armbrust] fix tests
6fc38de [Michael Armbrust] fix style
55d64b3 [Michael Armbrust] fix dataframe selfjoins
2015-02-11 12:31:56 -08:00
Patrick Wendell c2131c0cdc HOTFIX: Adding Junit to Hive tests for Maven build 2015-02-10 23:39:21 -08:00
Patrick Wendell 7e2f8821e0 HOTFIX: Java 6 compilation error in Spark SQL 2015-02-10 22:43:32 -08:00
Cheng Hao 45df77b841 [SPARK-5709] [SQL] Add EXPLAIN support in DataFrame API for debugging purpose
Author: Cheng Hao <hao.cheng@intel.com>

Closes #4496 from chenghao-intel/df_explain and squashes the following commits:

552aa58 [Cheng Hao] Add explain support for DF
2015-02-10 19:40:51 -08:00
Davies Liu ea60284095 [SPARK-5704] [SQL] [PySpark] createDataFrame from RDD with columns
Deprecate inferSchema() and applySchema(), use createDataFrame() instead, which could take an optional `schema` to create an DataFrame from an RDD. The `schema` could be StructType or list of names of columns.

Author: Davies Liu <davies@databricks.com>

Closes #4498 from davies/create and squashes the following commits:

08469c1 [Davies Liu] remove Scala/Java API for now
c80a7a9 [Davies Liu] fix hive test
d1bd8f2 [Davies Liu] cleanup applySchema
9526e97 [Davies Liu] createDataFrame from RDD with columns
2015-02-10 19:40:12 -08:00
Michael Armbrust 6195e2473b [SQL] Add an exception for analysis errors.
Also start from the bottom so we show the first error instead of the top error.

Author: Michael Armbrust <michael@databricks.com>

Closes #4439 from marmbrus/analysisException and squashes the following commits:

45862a0 [Michael Armbrust] fix hive test
a773bba [Michael Armbrust] Merge remote-tracking branch 'origin/master' into analysisException
f88079f [Michael Armbrust] update more cases
fede90a [Michael Armbrust] newline
fbf4bc3 [Michael Armbrust] move to sql
6235db4 [Michael Armbrust] [SQL] Add an exception for analysis errors.
2015-02-10 17:32:42 -08:00
Yin Huai aaf50d05c7 [SPARK-5658][SQL] Finalize DDL and write support APIs
https://issues.apache.org/jira/browse/SPARK-5658

Author: Yin Huai <yhuai@databricks.com>

This patch had conflicts when merged, resolved by
Committer: Michael Armbrust <michael@databricks.com>

Closes #4446 from yhuai/writeSupportFollowup and squashes the following commits:

f3a96f7 [Yin Huai] davies's comments.
225ff71 [Yin Huai] Use Scala TestHiveContext to initialize the Python HiveContext in Python tests.
2306f93 [Yin Huai] Style.
2091fcd [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
537e28f [Yin Huai] Correctly clean up temp data.
ae4649e [Yin Huai] Fix Python test.
609129c [Yin Huai] Doc format.
92b6659 [Yin Huai] Python doc and other minor updates.
cbc717f [Yin Huai] Rename dataSourceName to source.
d1c12d3 [Yin Huai] No need to delete the duplicate rule since it has been removed in master.
22cfa70 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
d91ecb8 [Yin Huai] Fix test.
4c76d78 [Yin Huai] Simplify APIs.
3abc215 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
0832ce4 [Yin Huai] Fix test.
98e7cdb [Yin Huai] Python style.
2bf44ef [Yin Huai] Python APIs.
c204967 [Yin Huai] Format
a10223d [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
9ff97d8 [Yin Huai] Add SaveMode to saveAsTable.
9b6e570 [Yin Huai] Update doc.
c2be775 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
99950a2 [Yin Huai] Use Java enum for SaveMode.
4679665 [Yin Huai] Remove duplicate rule.
77d89dc [Yin Huai] Update doc.
e04d908 [Yin Huai] Move import and add (Scala-specific) to scala APIs.
cf5703d [Yin Huai] Add checkAnswer to Java tests.
7db95ff [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
6dfd386 [Yin Huai] Add java test.
f2f33ef [Yin Huai] Fix test.
e702386 [Yin Huai] Apache header.
b1e9b1b [Yin Huai] Format.
ed4e1b4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into writeSupportFollowup
af9e9b3 [Yin Huai] DDL and write support API followup.
2a6213a [Yin Huai] Update API names.
e6a0b77 [Yin Huai] Update test.
43bae01 [Yin Huai] Remove createTable from HiveContext.
5ffc372 [Yin Huai] Add more load APIs to SQLContext.
5390743 [Yin Huai] Add more save APIs to DataFrame.
2015-02-10 17:29:52 -08:00
Yin Huai e28b6bdbb5 [SQL] Make Options in the data source API CREATE TABLE statements optional.
Users will not need to put `Options()` in a CREATE TABLE statement when there is not option provided.

Author: Yin Huai <yhuai@databricks.com>

Closes #4515 from yhuai/makeOptionsOptional and squashes the following commits:

1a898d3 [Yin Huai] Make options optional.
2015-02-10 17:06:12 -08:00
OopsOutOfMemory f98707c043 [SPARK-5686][SQL] Add show current roles command in HiveQl
show current roles

Author: OopsOutOfMemory <victorshengli@126.com>

Closes #4471 from OopsOutOfMemory/show_current_role and squashes the following commits:

1c6b210 [OopsOutOfMemory] add show current roles
2015-02-10 13:20:15 -08:00
wangfei 59272dad77 [SPARK-5592][SQL] java.net.URISyntaxException when insert data to a partitioned table
flowing sql get URISyntaxException:
```
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;
create table sc_part (key string) partitioned by (ts string) stored as rcfile;
set hive.exec.dynamic.partition=true;
set hive.exec.dynamic.partition.mode=nonstrict;
insert overwrite table sc_part partition(ts) select * from sc;
```
java.net.URISyntaxException: Relative path in absolute URI: ts=2011-01-11+15:18:26
at org.apache.hadoop.fs.Path.initialize(Path.java:206)
at org.apache.hadoop.fs.Path.<init>(Path.java:172)
at org.apache.hadoop.fs.Path.<init>(Path.java:94)
at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.org$apache$spark$sql$hive$SparkHiveDynamicPartitionWriterContainer$$newWriter$1(hiveWriterContainers.scala:230)
at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer$$anonfun$getLocalFileWriter$1.apply(hiveWriterContainers.scala:243)
at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer$$anonfun$getLocalFileWriter$1.apply(hiveWriterContainers.scala:243)
at scala.collection.mutable.MapLike$class.getOrElseUpdate(MapLike.scala:189)
at scala.collection.mutable.AbstractMap.getOrElseUpdate(Map.scala:91)
at org.apache.spark.sql.hive.SparkHiveDynamicPartitionWriterContainer.getLocalFileWriter(hiveWriterContainers.scala:243)
at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$org$apache$spark$sql$hive$execution$InsertIntoHiveTable$$writeToFile$1$1.apply(InsertIntoHiveTable.scala:113)
at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$org$apache$spark$sql$hive$execution$InsertIntoHiveTable$$writeToFile$1$1.apply(InsertIntoHiveTable.scala:105)
at scala.collection.Iterator$class.foreach(Iterator.scala:727)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.org$apache$spark$sql$hive$execution$InsertIntoHiveTable$$writeToFile$1(InsertIntoHiveTable.scala:105)
at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:87)
at org.apache.spark.sql.hive.execution.InsertIntoHiveTable$$anonfun$saveAsHiveFile$3.apply(InsertIntoHiveTable.scala:87)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)
at org.apache.spark.scheduler.Task.run(Task.scala:64)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:194)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1110)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:603)
at java.lang.Thread.run(Thread.java:722)
Caused by: java.net.URISyntaxException: Relative path in absolute URI: ts=2011-01-11+15:18:26
at java.net.URI.checkPath(URI.java:1804)
at java.net.URI.<init>(URI.java:752)
at org.apache.hadoop.fs.Path.initialize(Path.java:203)

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

Closes #4368 from scwf/SPARK-5592 and squashes the following commits:

aa55ef4 [Fei Wang] comments addressed
f8f8bb1 [wangfei] added test case
f24624f [wangfei] Merge branch 'master' of https://github.com/apache/spark into SPARK-5592
9998177 [wangfei] added test case
ea81daf [wangfei] fix URISyntaxException
2015-02-10 11:54:30 -08:00
Daoyuan Wang c7ad80ae42 [SPARK-5716] [SQL] Support TOK_CHARSETLITERAL in HiveQl
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #4502 from adrian-wang/utf8 and squashes the following commits:

4d7b0ee [Daoyuan Wang] remove useless import
606f981 [Daoyuan Wang] support TOK_CHARSETLITERAL in HiveQl
2015-02-10 11:08:21 -08:00
DoingDone9 d08e7c2b49 [SPARK-5648][SQL] support "alter ... unset tblproperties("key")"
make hivecontext support "alter ... unset tblproperties("key")"
like :
alter view viewName unset tblproperties("k")
alter table tableName unset tblproperties("k")

Author: DoingDone9 <799203320@qq.com>

Closes #4424 from DoingDone9/unset and squashes the following commits:

6dd8bee [DoingDone9] support "alter ... unset tblproperties("key")"
2015-02-09 16:40:26 -08:00
Yin Huai 5f0b30e59c [SQL] Code cleanup.
I added an unnecessary line of code in 13531dd97c.

My bad. Let's delete it.

Author: Yin Huai <yhuai@databricks.com>

Closes #4482 from yhuai/unnecessaryCode and squashes the following commits:

3645af0 [Yin Huai] Code cleanup.
2015-02-09 16:20:42 -08:00
Yin Huai 804949d519 [SQL] Set sessionState in QueryExecution.
This PR sets the SessionState in HiveContext's QueryExecution. So, we can make sure that SessionState.get can return the SessionState every time.

Author: Yin Huai <yhuai@databricks.com>

Closes #4445 from yhuai/setSessionState and squashes the following commits:

769c9f1 [Yin Huai] Remove unused import.
439f329 [Yin Huai] Try again.
427a0c9 [Yin Huai] Set SessionState everytime when we create a QueryExecution in HiveContext.
a3b7793 [Yin Huai] Set sessionState when dealing with CreateTableAsSelect.
2015-02-08 14:55:07 -08:00
Cheng Lian c4021401e3 [SQL] [Minor] HiveParquetSuite was disabled by mistake, re-enable them
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/4440)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #4440 from liancheng/parquet-oops and squashes the following commits:

f21ede4 [Cheng Lian] HiveParquetSuite was disabled by mistake, re-enable them.
2015-02-06 15:23:42 -08:00
Wenchen Fan 4793c8402a [SPARK-5278][SQL] Introduce UnresolvedGetField and complete the check of ambiguous reference to fields
When the `GetField` chain(`a.b.c.d.....`) is interrupted by `GetItem` like `a.b[0].c.d....`, then the check of ambiguous reference to fields is broken.
The reason is that: for something like `a.b[0].c.d`, we first parse it to `GetField(GetField(GetItem(Unresolved("a.b"), 0), "c"), "d")`. Then in `LogicalPlan#resolve`, we resolve `"a.b"` and build a `GetField` chain from bottom(the relation). But for the 2 outer `GetFiled`, we have to resolve them in `Analyzer` or do it in `GetField` lazily, check data type of child, search needed field, etc. which is similar to what we have done in `LogicalPlan#resolve`.
So in this PR, the fix is just copy the same logic in `LogicalPlan#resolve` to `Analyzer`, which is simple and quick, but I do suggest introduce `UnresolvedGetFiled` like I explained in https://github.com/apache/spark/pull/2405.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #4068 from cloud-fan/simple and squashes the following commits:

a6857b5 [Wenchen Fan] fix import order
8411c40 [Wenchen Fan] use UnresolvedGetField
2015-02-06 13:08:09 -08:00
Yin Huai 3eccf29ce0 [SPARK-5595][SPARK-5603][SQL] Add a rule to do PreInsert type casting and field renaming and invalidating in memory cache after INSERT
This PR adds a rule to Analyzer that will add preinsert data type casting and field renaming to the select clause in an `INSERT INTO/OVERWRITE` statement. Also, with the change of this PR, we always invalidate our in memory data cache after inserting into a BaseRelation.

cc marmbrus liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #4373 from yhuai/insertFollowUp and squashes the following commits:

08237a7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into insertFollowUp
316542e [Yin Huai] Doc update.
c9ccfeb [Yin Huai] Revert a unnecessary change.
84aecc4 [Yin Huai] Address comments.
1951fe1 [Yin Huai] Merge remote-tracking branch 'upstream/master'
c18da34 [Yin Huai] Invalidate cache after insert.
727f21a [Yin Huai] Preinsert casting and renaming.
2015-02-06 12:38:07 -08:00
OopsOutOfMemory 0b7eb3f3b7 [SPARK-5324][SQL] Results of describe can't be queried
Make below code works.
```
sql("DESCRIBE test").registerTempTable("describeTest")
sql("SELECT * FROM describeTest").collect()
```

Author: OopsOutOfMemory <victorshengli@126.com>
Author: Sheng, Li <OopsOutOfMemory@users.noreply.github.com>

Closes #4249 from OopsOutOfMemory/desc_query and squashes the following commits:

6fee13d [OopsOutOfMemory] up-to-date
e71430a [Sheng, Li] Update HiveOperatorQueryableSuite.scala
3ba1058 [OopsOutOfMemory] change to default argument
aac7226 [OopsOutOfMemory] Merge branch 'master' into desc_query
68eb6dd [OopsOutOfMemory] Merge branch 'desc_query' of github.com:OopsOutOfMemory/spark into desc_query
354ad71 [OopsOutOfMemory] query describe command
d541a35 [OopsOutOfMemory] refine test suite
e1da481 [OopsOutOfMemory] refine test suite
a780539 [OopsOutOfMemory] Merge branch 'desc_query' of github.com:OopsOutOfMemory/spark into desc_query
0015f82 [OopsOutOfMemory] code style
dd0aaef [OopsOutOfMemory] code style
c7d606d [OopsOutOfMemory] rename test suite
75f2342 [OopsOutOfMemory] refine code and test suite
f942c9b [OopsOutOfMemory] initial
11559ae [OopsOutOfMemory] code style
c5fdecf [OopsOutOfMemory] code style
aeaea5f [OopsOutOfMemory] rename test suite
ac2c3bb [OopsOutOfMemory] refine code and test suite
544573e [OopsOutOfMemory] initial
2015-02-06 12:33:20 -08:00
q00251598 a958d60975 [SPARK-5619][SQL] Support 'show roles' in HiveContext
Author: q00251598 <qiyadong@huawei.com>

Closes #4397 from watermen/SPARK-5619 and squashes the following commits:

f819b6c [q00251598] Support show roles in HiveContext.
2015-02-06 12:29:26 -08:00
Liang-Chi Hsieh d433816157 [SPARK-5650][SQL] Support optional 'FROM' clause
In Hive, 'FROM' clause is optional. This pr supports it.

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

Closes #4426 from viirya/optional_from and squashes the following commits:

fe81f31 [Liang-Chi Hsieh] Support optional 'FROM' clause.
2015-02-06 12:13:44 -08:00
Cheng Lian 7c0a648fb5 [HOTFIX] [SQL] Disables Metastore Parquet table conversion for "SQLQuerySuite.CTAS with serde"
Ideally we should convert Metastore Parquet tables with our own Parquet implementation on both read path and write path. However, the write path is not well covered, and causes this test failure. This PR is a hotfix to bring back Jenkins PR builder. A proper fix will be delivered in a follow-up PR.

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

Author: Cheng Lian <lian@databricks.com>

Closes #4413 from liancheng/hotfix-parquet-ctas and squashes the following commits:

5291289 [Cheng Lian] Hot fix for "SQLQuerySuite.CTAS with serde"
2015-02-05 18:09:18 -08:00
Cheng Lian a9ed51178c [SPARK-5182] [SPARK-5528] [SPARK-5509] [SPARK-3575] [SQL] Parquet data source improvements
This PR adds three major improvements to Parquet data source:

1.  Partition discovery

    While reading Parquet files resides in Hive style partition directories, `ParquetRelation2` automatically discovers partitioning information and infers partition column types.

    This is also a partial work for [SPARK-5182] [1], which aims to provide first class partitioning support for the data source API.  Related code in this PR can be easily extracted to the data source API level in future versions.

1.  Schema merging

    When enabled, Parquet data source collects schema information from all Parquet part-files and tries to merge them.  Exceptions are thrown when incompatible schemas are detected.  This feature is controlled by data source option `parquet.mergeSchema`, and is enabled by default.

1.  Metastore Parquet table conversion moved to analysis phase

    This greatly simplifies the conversion logic.  `ParquetConversion` strategy can be removed once the old Parquet implementation is removed in the future.

This version of Parquet data source aims to entirely replace the old Parquet implementation.  However, the old version hasn't been removed yet.  Users can fall back to the old version by turning off SQL configuration `spark.sql.parquet.useDataSourceApi`.

Other JIRA tickets fixed as side effects in this PR:

- [SPARK-5509] [3]: `EqualTo` now uses a proper `Ordering` to compare binary types.

- [SPARK-3575] [4]: Metastore schema is now preserved and passed to `ParquetRelation2` via data source option `parquet.metastoreSchema`.

TODO:

- [ ] More test cases for partition discovery
- [x] Fix write path after data source write support (#4294) is merged

      It turned out to be non-trivial to fall back to old Parquet implementation on the write path when Parquet data source is enabled.  Since we're planning to include data source write support in 1.3.0, I simply ignored two test cases involving Parquet insertion for now.

- [ ] Fix outdated comments and documentations

PS: This PR looks big, but more than a half of the changed lines in this PR are trivial changes to test cases. To test Parquet with and without the new data source, almost all Parquet test cases are moved into wrapper driver functions. This introduces hundreds of lines of changes.

[1]: https://issues.apache.org/jira/browse/SPARK-5182
[2]: https://issues.apache.org/jira/browse/SPARK-5528
[3]: https://issues.apache.org/jira/browse/SPARK-5509
[4]: https://issues.apache.org/jira/browse/SPARK-3575

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

Author: Cheng Lian <lian@databricks.com>

Closes #4308 from liancheng/parquet-partition-discovery and squashes the following commits:

b6946e6 [Cheng Lian] Fixes MiMA issues, addresses comments
8232e17 [Cheng Lian] Write support for Parquet data source
a49bd28 [Cheng Lian] Fixes spelling typo in trait name "CreateableRelationProvider"
808380f [Cheng Lian] Fixes issues introduced while rebasing
50dd8d1 [Cheng Lian] Addresses @rxin's comment, fixes UDT schema merging
adf2aae [Cheng Lian] Fixes compilation error introduced while rebasing
4e0175f [Cheng Lian] Fixes Python Parquet API, we need Py4J array to call varargs method
0d8ec1d [Cheng Lian] Adds more test cases
b35c8c6 [Cheng Lian] Fixes some typos and outdated comments
dd704fd [Cheng Lian] Fixes Python Parquet API
596c312 [Cheng Lian] Uses switch to control whether use Parquet data source or not
7d0f7a2 [Cheng Lian] Fixes Metastore Parquet table conversion
a1896c7 [Cheng Lian] Fixes all existing Parquet test suites except for ParquetMetastoreSuite
5654c9d [Cheng Lian] Draft version of Parquet partition discovery and schema merging
2015-02-05 15:29:56 -08:00
OopsOutOfMemory 4d8d070c4f [SPARK-5135][SQL] Add support for describe table to DDL in SQLContext
Hi, rxin marmbrus
I considered your suggestion (in #4127) and now re-write it. This is now up-to-date.
Could u please review it ?

Author: OopsOutOfMemory <victorshengli@126.com>

Closes #4227 from OopsOutOfMemory/describe and squashes the following commits:

053826f [OopsOutOfMemory] describe
2015-02-05 13:07:48 -08:00
Reynold Xin 7d789e117d [SPARK-5612][SQL] Move DataFrame implicit functions into SQLContext.implicits.
Author: Reynold Xin <rxin@databricks.com>

Closes #4386 from rxin/df-implicits and squashes the following commits:

9d96606 [Reynold Xin] style fix
edd296b [Reynold Xin] ReplSuite
1c946ab [Reynold Xin] [SPARK-5612][SQL] Move DataFrame implicit functions into SQLContext.implicits.
2015-02-04 23:44:34 -08:00
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