Commit graph

724 commits

Author SHA1 Message Date
ravipesala 9b6ebe33db [SPARK-4120][SQL] Join of multiple tables with syntax like SELECT .. FROM T1,T2,T3.. does not work in SparkSQL
Right now it works for only 2 tables like below query.
sql("SELECT * FROM records1 as a,records2 as b where a.key=b.key ")

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

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

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

429b005 [ravipesala] Support multiple joins
2014-10-30 17:15:45 -07:00
Sean Owen 68cb69daf3 SPARK-1209 [CORE] SparkHadoop{MapRed,MapReduce}Util should not use package org.apache.hadoop
(This is just a look at what completely moving the classes would look like. I know Patrick flagged that as maybe not OK, although, it's private?)

Author: Sean Owen <sowen@cloudera.com>

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

ead1115 [Sean Owen] Disable MIMA warnings resulting from moving the class -- this was also part of the PairRDDFunctions type hierarchy though?
2d42c1d [Sean Owen] Move SparkHadoopMapRedUtil / SparkHadoopMapReduceUtil from org.apache.hadoop to org.apache.spark
2014-10-30 15:54:53 -07:00
Daoyuan Wang 3535467663 [SPARK-4003] [SQL] add 3 types for java SQL context
In JavaSqlContext, we need to let java program use big decimal, timestamp, date types.

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

Closes #2850 from adrian-wang/javacontext and squashes the following commits:

4c4292c [Daoyuan Wang] change underlying type of JavaSchemaRDD as scala
bb0508f [Daoyuan Wang] add test cases
3c58b0d [Daoyuan Wang] add 3 types for java SQL context
2014-10-29 12:10:58 -07:00
Davies Liu 8c0bfd08fc [SPARK-4133] [SQL] [PySpark] type conversionfor python udf
Call Python UDF on ArrayType/MapType/PrimitiveType, the returnType can also be ArrayType/MapType/PrimitiveType.

For StructType, it will act as tuple (without attributes). If returnType is StructType, it also should be tuple.

Author: Davies Liu <davies@databricks.com>

Closes #2973 from davies/udf_array and squashes the following commits:

306956e [Davies Liu] Merge branch 'master' of github.com:apache/spark into udf_array
2c00e43 [Davies Liu] fix merge
11395fa [Davies Liu] Merge branch 'master' of github.com:apache/spark into udf_array
9df50a2 [Davies Liu] address comments
79afb4e [Davies Liu] type conversionfor python udf
2014-10-28 19:38:16 -07:00
Cheng Hao b5e79bf889 [SPARK-3904] [SQL] add constant objectinspector support for udfs
In HQL, we convert all of the data type into normal `ObjectInspector`s for UDFs, most of cases it works, however, some of the UDF actually requires its children `ObjectInspector` to be the `ConstantObjectInspector`, which will cause exception.
e.g.
select named_struct("x", "str") from src limit 1;

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

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

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

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

bcacfd7 [Cheng Hao] Shim for both Hive 0.12 & 0.13.1
2416e5d [Cheng Hao] revert to hive 0.12
5793c01 [Cheng Hao] add space before while
4e56e1b [Cheng Hao] style issue
683d3fd [Cheng Hao] Add golden files
fe591e4 [Cheng Hao] update HiveGenericUdf for set the ObjectInspector while constructing the DeferredObject
f6740fe [Cheng Hao] Support Constant ObjectInspector for Map & List
8814c3a [Cheng Hao] Passing ContantObjectInspector(when necessary) for UDF initializing
2014-10-28 19:11:57 -07:00
Cheng Hao 4b55482abf [SPARK-3343] [SQL] Add serde support for CTAS
Currently, `CTAS` (Create Table As Select) doesn't support specifying the `SerDe` in HQL. This PR will pass down the `ASTNode` into the physical operator `execution.CreateTableAsSelect`, which will extract the `CreateTableDesc` object via Hive `SemanticAnalyzer`. In the meantime, I also update the `HiveMetastoreCatalog.createTable` to optionally support the `CreateTableDesc` for table creation.

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

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

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

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

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

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

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

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

a391c7a [ravipesala] Rebase with master
2014-10-28 13:36:06 -07:00
wangxiaojing 0c34fa5b4b [SPARK-3907][SQL] Add truncate table support
JIRA issue: [SPARK-3907]https://issues.apache.org/jira/browse/SPARK-3907

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

Author: wangxiaojing <u9jing@gmail.com>

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

63dbd81 [wangxiaojing] change hive scalastyle
7a03707 [wangxiaojing] add comment
f6e710e [wangxiaojing] change truncate table
a1f692c [wangxiaojing] Correct spelling mistakes
3b20007 [wangxiaojing] add truncate can not support column err message
e483547 [wangxiaojing] add golden file
77b1f20 [wangxiaojing]  add truncate table support
2014-10-27 22:02:52 -07:00
Yin Huai 27470d3406 [SQL] Correct a variable name in JavaApplySchemaSuite.applySchemaToJSON
`schemaRDD2` is not tested because `schemaRDD1` is registered again.

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

Closes #2869 from yhuai/JavaApplySchemaSuite and squashes the following commits:

95fe894 [Yin Huai] Correct variable name.
2014-10-27 20:50:09 -07:00
wangfei 89af6dfc3a [SPARK-4041][SQL] Attributes names in table scan should converted to lowercase when compare with relation attributes
In ```MetastoreRelation``` the attributes name is lowercase because of hive using lowercase for fields name, so we should convert attributes name in table scan lowercase in ```indexWhere(_.name == a.name)```.
```neededColumnIDs``` may be not correct if not convert to lowercase.

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

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

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

Author: Alex Liu <alex_liu68@yahoo.com>

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

79c269b [Alex Liu] [SPARK-3816][SQL] Add table properties from storage handler to job conf
2014-10-27 20:43:29 -07:00
Cheng Hao 418ad83fe1 [SPARK-3911] [SQL] HiveSimpleUdf can not be optimized in constant folding
```
explain extended select cos(null) from src limit 1;
```
outputs:
```
 Project [HiveSimpleUdf#org.apache.hadoop.hive.ql.udf.UDFCos(null) AS c_0#5]
  MetastoreRelation default, src, None

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

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

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

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

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

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

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

1379c73 [Cheng Hao] duplicate the PlanTest with catalyst/plans/PlanTest
1e52dda [Cheng Hao] add unit test for hive simple udf constant folding
01609ff [Cheng Hao] support constant folding for HiveSimpleUdf
2014-10-27 20:42:05 -07:00
Cheng Lian 1d7bcc8840 [SQL] Fixes caching related JoinSuite failure
PR #2860 refines in-memory table statistics and enables broader broadcasted hash join optimization for in-memory tables. This makes `JoinSuite` fail when some test suite caches test table `testData` and gets executed before `JoinSuite`. Because expected `ShuffledHashJoin`s are optimized to `BroadcastedHashJoin` according to collected in-memory table statistics.

This PR fixes this issue by clearing the cache before testing join operator selection. A separate test case is also added to test broadcasted hash join operator selection.

Author: Cheng Lian <lian@databricks.com>

Closes #2960 from liancheng/fix-join-suite and squashes the following commits:

715b2de [Cheng Lian] Fixes caching related JoinSuite failure
2014-10-27 10:06:09 -07:00
scwf f4e8c289d8 [SPARK-4042][SQL] Append columns ids and names before broadcast
Append columns ids and names before broadcast ```hiveExtraConf```  in ```HadoopTableReader```.

Author: scwf <wangfei1@huawei.com>

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

a8c498c [scwf] append columns ids and names before broadcast
2014-10-26 16:56:11 -07:00
Kousuke Saruta 3a9d66cf59 [SPARK-4061][SQL] We cannot use EOL character in the operand of LIKE predicate.
We cannot use EOL character like \n or \r in the operand of LIKE predicate.
So following condition is never true.

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

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

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

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

    SELECT value FROM someTable WHERE value > -9223372036854775808

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

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

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

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

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

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

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

f6329f3 [ravipesala] Rebased with master
2014-10-26 16:36:11 -07:00
Yin Huai 0481aaa8d7 [SPARK-4068][SQL] NPE in jsonRDD schema inference
Please refer to added tests for cases that can trigger the bug.

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

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

Closes #2918 from yhuai/SPARK-4068 and squashes the following commits:

d360eae [Yin Huai] Handle nulls when building key paths from elements of an array.
2014-10-26 16:32:02 -07:00
Yin Huai 05308426f0 [SPARK-4052][SQL] Use scala.collection.Map for pattern matching instead of using Predef.Map (it is scala.collection.immutable.Map)
Please check https://issues.apache.org/jira/browse/SPARK-4052 for cases triggering this bug.

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

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

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

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

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

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

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

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

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

   Statistics are collected to driver side with an accumulator.

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

Author: Cheng Lian <lian@databricks.com>

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

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

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

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

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

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

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

I also upgraded ScalaTest to 2.2.1.

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

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

fcc9e83 [Josh Rosen] scalautils -> scalactic package rename
510e54a [Josh Rosen] [SPARK-3616] Add basic Selenium tests to WebUISuite.
2014-10-26 11:29:27 -07:00
Sean Owen df7974b8e5 SPARK-3359 [DOCS] sbt/sbt unidoc doesn't work with Java 8
This follows https://github.com/apache/spark/pull/2893 , but does not completely fix SPARK-3359 either. This fixes minor scaladoc/javadoc issues that Javadoc 8 will treat as errors.

Author: Sean Owen <sowen@cloudera.com>

Closes #2909 from srowen/SPARK-3359 and squashes the following commits:

f62c347 [Sean Owen] Fix some javadoc issues that javadoc 8 considers errors. This is not all of the errors turned up when javadoc 8 runs on output of genjavadoc.
2014-10-25 23:18:02 -07:00
Michael Armbrust 3a845d3c04 [SQL] Update Hive test harness for Hive 12 and 13
As part of the upgrade I also copy the newest version of the query tests, and whitelist a bunch of new ones that are now passing.

Author: Michael Armbrust <michael@databricks.com>

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

d9cbdab [Michael Armbrust] Remove user specific tests
65801cd [Michael Armbrust] style and rat
8f6b09a [Michael Armbrust] Update test harness to work with both Hive 12 and 13.
f044843 [Michael Armbrust] Update Hive query tests and golden files to 0.13
2014-10-24 18:36:35 -07:00
Michael Armbrust 3a906c6631 [HOTFIX][SQL] Remove sleep on reset() failure.
Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

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

3ece905 [Zhan Zhang] minor fix
410b668 [Zhan Zhang] solve review comments
cbb4691 [Zhan Zhang] change run-test for new options
0d4d2ed [Zhan Zhang] rebase
497b0f4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
8fad1cf [Zhan Zhang] change the pom file and make hive-0.13.1 as the default
ab028d1 [Zhan Zhang] rebase
4a2e36d [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
4cb1b93 [zhzhan] Merge pull request #1 from pwendell/pr-2241
b0478c0 [Patrick Wendell] Changes to simplify the build of SPARK-2706
2b50502 [Zhan Zhang] rebase
a72c0d4 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cb22863 [Zhan Zhang] correct the typo
20f6cf7 [Zhan Zhang] solve compatability issue
f7912a9 [Zhan Zhang] rebase and solve review feedback
301eb4a [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
10c3565 [Zhan Zhang] address review comments
6bc9204 [Zhan Zhang] rebase and remove temparory repo
d3aa3f2 [Zhan Zhang] Merge branch 'master' into spark-2706
cedcc6f [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3ced0d7 [Zhan Zhang] rebase
d9b981d [Zhan Zhang] rebase and fix error due to rollback
adf4924 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
3dd50e8 [Zhan Zhang] solve conflicts and remove unnecessary implicts
d10bf00 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
dc7bdb3 [Zhan Zhang] solve conflicts
7e0cc36 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
d7c3e1e [Zhan Zhang] Merge branch 'master' into spark-2706
68deb11 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
d48bd18 [Zhan Zhang] address review comments
3ee3b2b [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
57ea52e [Zhan Zhang] Merge branch 'master' into spark-2706
2b0d513 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
9412d24 [Zhan Zhang] address review comments
f4af934 [Zhan Zhang] rebase
1ccd7cc [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
128b60b [Zhan Zhang] ignore 0.12.0 test cases for the time being
af9feb9 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
5f5619f [Zhan Zhang] restructure the directory and different hive version support
05d3683 [Zhan Zhang] solve conflicts
e4c1982 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
94b4fdc [Zhan Zhang] Spark-2706: hive-0.13.1 support on spark
87ebf3b [Zhan Zhang] Merge branch 'master' into spark-2706
921e914 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
f896b2a [Zhan Zhang] Merge branch 'master' into spark-2706
789ea21 [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
cb53a2c [Zhan Zhang] Merge branch 'master' of https://github.com/apache/spark
f6a8a40 [Zhan Zhang] revert
ba14f28 [Zhan Zhang] test
dbedff3 [Zhan Zhang] Merge remote-tracking branch 'upstream/master'
70964fe [Zhan Zhang] revert
fe0f379 [Zhan Zhang] Merge branch 'master' of https://github.com/zhzhan/spark
70ffd93 [Zhan Zhang] revert
42585ec [Zhan Zhang] test
7d5fce2 [Zhan Zhang] test
2014-10-24 11:03:17 -07:00
Michael Armbrust 0e886610ee [SPARK-4050][SQL] Fix caching of temporary tables with projections.
Previously cached data was found by `sameResult` plan matching on optimized plans.  This technique however fails to locate the cached data when a temporary table with a projection is queried with a further reduced projection.  The failure is due to the fact that optimization will collapse the projections, producing a plan that no longer produces the sameResult as the cached data (though the cached data still subsumes the desired data).  For example consider the following previously failing test case.

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

9c822d4 [Michael Armbrust] remove commented out code
5c72fb7 [Michael Armbrust] Add a test case / question about uncaching semantics.
63a23e4 [Michael Armbrust] Perform caching on analyzed instead of optimized plan.
03f1cfe [Michael Armbrust] Clean-up / add tests to SameResult suite.
2014-10-24 10:52:25 -07:00
wangfei 856b081729 [SQL]redundant methods for broadcast
redundant methods for broadcast in ```TableReader```

Author: wangfei <wangfei1@huawei.com>

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

414cc24 [wangfei] unnecessary methods for broadcast
2014-10-21 16:20:58 -07:00
wangxiaojing 0fe1c09369 [SPARK-3940][SQL] Avoid console printing error messages three times
If  wrong sql,the console print error one times。
eg:
<pre>
spark-sql> show tabless;
show tabless;
14/10/13 21:03:48 INFO ParseDriver: Parsing command: show tabless
............
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:274)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:413)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:209)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
Caused by: org.apache.hadoop.hive.ql.parse.ParseException: line 1:5 cannot recognize input near 'show' 'tabless' '<EOF>' in ddl statement

	at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:193)
	at org.apache.hadoop.hive.ql.parse.ParseDriver.parse(ParseDriver.java:161)
	at org.apache.spark.sql.hive.HiveQl$.getAst(HiveQl.scala:218)
	at org.apache.spark.sql.hive.HiveQl$.createPlan(HiveQl.scala:226)
	... 47 more
Time taken: 4.35 seconds
14/10/13 21:03:51 INFO CliDriver: Time taken: 4.35 seconds
</pre>

Author: wangxiaojing <u9jing@gmail.com>

Closes #2790 from wangxiaojing/spark-3940 and squashes the following commits:

e2e5c14 [wangxiaojing] sql Print the error code three times
2014-10-20 17:15:28 -07:00
Takuya UESHIN 7586e2e67a [SPARK-3969][SQL] Optimizer should have a super class as an interface.
Some developers want to replace `Optimizer` to fit their projects but can't do so because currently `Optimizer` is an `object`.

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

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

abbc53c [Takuya UESHIN] Re-rename Optimizer object.
4d2e1bc [Takuya UESHIN] Rename Optimizer object.
9547a23 [Takuya UESHIN] Extract abstract class from Optimizer for developers to be able to replace Optimizer.
2014-10-20 17:09:12 -07:00
luogankun fce1d41611 [SPARK-3945]Properties of hive-site.xml is invalid in running the Thrift JDBC server
Write properties of hive-site.xml to HiveContext when initilize session state in SparkSQLEnv.scala.

The method of SparkSQLEnv.init() in HiveThriftServer2.scala can not write the properties of hive-site.xml to HiveContext. Such as: add configuration property spark.sql.shuffle.partititions in the hive-site.xml.

Author: luogankun <luogankun@gmail.com>

Closes #2800 from luogankun/SPARK-3945 and squashes the following commits:

3679efc [luogankun] [SPARK-3945]Write properties of hive-site.xml to HiveContext when initilize session state In SparkSQLEnv.scala
2014-10-20 16:50:51 -07:00
Takuya UESHIN 364d52b707 [SPARK-3966][SQL] Fix nullabilities of Cast related to DateType.
Author: Takuya UESHIN <ueshin@happy-camper.st>

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

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

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

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

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

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

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

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

fa2cc05 [Takuya UESHIN] Fix package names to fit their directory names.
2014-10-20 11:31:51 -07:00
Sean Owen f406a83918 SPARK-3926 [CORE] Result of JavaRDD.collectAsMap() is not Serializable
Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are

Author: Sean Owen <sowen@cloudera.com>

Closes #2805 from srowen/SPARK-3926 and squashes the following commits:

ecb78ee [Sean Owen] Fix conflict between java.io.Serializable and use of Scala's Serializable
f4717f9 [Sean Owen] Oops, fix compile problem
ae1b36f [Sean Owen] Expand to cover Maps returned from other Java API methods as well
51c26c2 [Sean Owen] Make JavaPairRDD.collectAsMap result Serializable since Java Maps generally are
2014-10-18 12:38:18 -07:00
Michael Armbrust adcb7d3350 [SPARK-3855][SQL] Preserve the result attribute of python UDFs though transformations
In the current implementation it was possible for the reference to change after analysis.

Author: Michael Armbrust <michael@databricks.com>

Closes #2717 from marmbrus/pythonUdfResults and squashes the following commits:

da14879 [Michael Armbrust] Fix test
6343bcb [Michael Armbrust] add test
9533286 [Michael Armbrust] Correctly preserve the result attribute of python UDFs though transformations
2014-10-17 14:12:07 -07:00
Prashant Sharma 2fe0ba9561 SPARK-3874: Provide stable TaskContext API
This is a small number of clean-up changes on top of #2782. Closes #2782.

Author: Prashant Sharma <prashant.s@imaginea.com>
Author: Patrick Wendell <pwendell@gmail.com>

Closes #2803 from pwendell/pr-2782 and squashes the following commits:

56d5b7a [Patrick Wendell] Minor clean-up
44089ec [Patrick Wendell] Clean-up the TaskContext API.
ed551ce [Prashant Sharma] Fixed a typo
df261d0 [Prashant Sharma] Josh's suggestion
facf3b1 [Prashant Sharma] Fixed the mima issue.
7ecc2fe [Prashant Sharma] CR, Moved implementations to TaskContextImpl
bbd9e05 [Prashant Sharma] adding missed out files to git.
ef633f5 [Prashant Sharma] SPARK-3874, Provide stable TaskContext API
2014-10-16 21:38:45 -04:00
Cheng Lian 99e416b6d6 [SQL] Fixes the race condition that may cause test failure
The removed `Future` was used to end the test case as soon as the Spark SQL CLI process exits. When the process exits prematurely, this mechanism prevents the test case to wait until timeout. But it also creates a race condition: when `foundAllExpectedAnswers.tryFailure` is called, there are chances that the last expected output line of the CLI process hasn't been caught by the main logics of the test code, thus fails the test case.

Removing this `Future` doesn't affect correctness.

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

Closes #2823 from liancheng/clean-clisuite and squashes the following commits:

489a97c [Cheng Lian] Fixes the race condition that may cause test failure
2014-10-16 16:15:55 -07:00
Cheng Lian 9eb49d4134 [SPARK-3809][SQL] Fixes test suites in hive-thriftserver
As scwf pointed out, `HiveThriftServer2Suite` isn't effective anymore after the Thrift server was made a daemon. On the other hand, these test suites were known flaky, PR #2214 tried to fix them but failed because of unknown Jenkins build error. This PR fixes both sets of issues.

In this PR, instead of watching `start-thriftserver.sh` output, the test code start a `tail` process to watch the log file. A `Thread.sleep` has to be introduced because the `kill` command used in `stop-thriftserver.sh` is not synchronous.

As for the root cause of the mysterious Jenkins build failure. Please refer to [this comment](https://github.com/apache/spark/pull/2675#issuecomment-58464189) below for details.

----

(Copied from PR description of #2214)

This PR fixes two issues of `HiveThriftServer2Suite` and brings 1 enhancement:

1. Although metastore, warehouse directories and listening port are randomly chosen, all test cases share the same configuration. Due to parallel test execution, one of the two test case is doomed to fail
2. We caught any exceptions thrown from a test case and print diagnosis information, but forgot to re-throw the exception...
3. When the forked server process ends prematurely (e.g., fails to start), the `serverRunning` promise is completed with a failure, preventing the test code to keep waiting until timeout.

So, embarrassingly, this test suite was failing continuously for several days but no one had ever noticed it... Fortunately no bugs in the production code were covered under the hood.

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

Closes #2675 from liancheng/fix-thriftserver-tests and squashes the following commits:

1c384b7 [Cheng Lian] Minor code cleanup, restore the logging level hack in TestHive.scala
7805c33 [wangfei]  reset SPARK_TESTING to avoid loading Log4J configurations in testing class paths
af2b5a9 [Cheng Lian] Removes log level hacks from TestHiveContext
d116405 [wangfei] make sure that log4j level is INFO
ee92a82 [Cheng Lian] Relaxes timeout
7fd6757 [Cheng Lian] Fixes test suites in hive-thriftserver
2014-10-13 13:50:27 -07:00
Liquan Pei 9d9ca91fef [SQL]Small bug in unresolved.scala
name should throw exception with name instead of exprId.

Author: Liquan Pei <liquanpei@gmail.com>

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

aa36a3b [Liquan Pei] small bug
2014-10-13 13:49:11 -07:00
chirag e6e37701f1 SPARK-3807: SparkSql does not work for tables created using custom serde
SparkSql crashes on selecting tables using custom serde.

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

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

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

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

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

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

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

(cherry picked from commit 925e22d313)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2014-10-13 13:47:51 -07:00
Michael Armbrust 371321cade [SQL] Add type checking debugging functions
Adds some functions that were very useful when trying to track down the bug from #2656.  This change also changes the tree output for query plans to include the `'` prefix to unresolved nodes and `!` prefix to nodes that refer to non-existent attributes.

Author: Michael Armbrust <michael@databricks.com>

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

654b926 [Michael Armbrust] Clean-up, add tests
763af15 [Michael Armbrust] Add typeChecking debugging functions
8c69303 [Michael Armbrust] Add inputSet, references to QueryPlan. Improve tree string with a prefix to denote invalid or unresolved nodes.
fbeab54 [Michael Armbrust] Better toString, factories for AttributeSet.
2014-10-13 13:46:34 -07:00
Venkata Ramana Gollamudi e10d71e7e5 [SPARK-3559][SQL] Remove unnecessary columns from List of needed Column Ids in Hive Conf
Author: Venkata Ramana G <ramana.gollamudihuawei.com>

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

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

b7ba768 [Venkata Ramana Gollamudi] Added comment and checkstyle fix
6a93459 [Venkata Ramana Gollamudi] cloned hiveconf for each TableScanOperators so that only required columns are added
2014-10-13 13:45:34 -07:00
Takuya UESHIN 73da9c26b0 [SPARK-3771][SQL] AppendingParquetOutputFormat should use reflection to prevent from breaking binary-compatibility.
Original problem is [SPARK-3764](https://issues.apache.org/jira/browse/SPARK-3764).

`AppendingParquetOutputFormat` uses a binary-incompatible method `context.getTaskAttemptID`.
This causes binary-incompatible of Spark itself, i.e. if Spark itself is built against hadoop-1, the artifact is for only hadoop-1, and vice versa.

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

Closes #2638 from ueshin/issues/SPARK-3771 and squashes the following commits:

efd3784 [Takuya UESHIN] Add a comment to explain the reason to use reflection.
ec213c1 [Takuya UESHIN] Use reflection to prevent breaking binary-compatibility.
2014-10-13 13:43:41 -07:00
Cheng Hao d3cdf9128a [SPARK-3529] [SQL] Delete the temp files after test exit
There are lots of temporal files created by TestHive under the /tmp by default, which may cause potential performance issue for testing. This PR will automatically delete them after test exit.

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

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

3a6511f [Cheng Hao] Remove the temp dir after text exit
2014-10-13 13:40:20 -07:00
Cheng Lian 56102dc2d8 [SPARK-2066][SQL] Adds checks for non-aggregate attributes with aggregation
This PR adds a new rule `CheckAggregation` to the analyzer to provide better error message for non-aggregate attributes with aggregation.

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

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

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

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

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

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

2824216 [Daoyuan Wang] remove redundant typeName
fbaf340 [Daoyuan Wang] typename
2014-10-13 13:31:14 -07:00
Reynold Xin 39ccabacf1 [SPARK-3861][SQL] Avoid rebuilding hash tables for broadcast joins on each partition
Author: Reynold Xin <rxin@apache.org>

Closes #2727 from rxin/SPARK-3861-broadcast-hash-2 and squashes the following commits:

9c7b1a2 [Reynold Xin] Revert "Reuse CompactBuffer in UniqueKeyHashedRelation."
97626a1 [Reynold Xin] Reuse CompactBuffer in UniqueKeyHashedRelation.
7fcffb5 [Reynold Xin] Make UniqueKeyHashedRelation private[joins].
18eb214 [Reynold Xin] Merge branch 'SPARK-3861-broadcast-hash' into SPARK-3861-broadcast-hash-1
4b9d0c9 [Reynold Xin] UniqueKeyHashedRelation.get should return null if the value is null.
e0ebdd1 [Reynold Xin] Added a test case.
90b58c0 [Reynold Xin] [SPARK-3861] Avoid rebuilding hash tables on each partition
0c0082b [Reynold Xin] Fix line length.
cbc664c [Reynold Xin] Rename join -> joins package.
a070d44 [Reynold Xin] Fix line length in HashJoin
a39be8c [Reynold Xin] [SPARK-3857] Create a join package for various join operators.
2014-10-13 11:50:42 -07:00
ravipesala 6f98902a3d [SPARK-3834][SQL] Backticks not correctly handled in subquery aliases
The queries like SELECT a.key FROM (SELECT key FROM src) \`a\` does not work as backticks in subquery aliases are not handled properly. This PR fixes that.

Author : ravipesala ravindra.pesalahuawei.com

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

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

0e0ab98 [ravipesala] Fixing issue in backtick handling for subquery aliases
2014-10-09 18:41:36 -07:00
Cheng Lian 421382d0e7 [SPARK-3824][SQL] Sets in-memory table default storage level to MEMORY_AND_DISK
Using `MEMORY_AND_DISK` as default storage level for in-memory table caching. Due to the in-memory columnar representation, recomputing an in-memory cached table partitions can be very expensive.

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

Closes #2686 from liancheng/spark-3824 and squashes the following commits:

35d2ed0 [Cheng Lian] Removes extra space
1ab7967 [Cheng Lian] Reduces test data size to fit DiskStore.getBytes()
ba565f0 [Cheng Lian] Maks CachedBatch serializable
07f0204 [Cheng Lian] Sets in-memory table default storage level to MEMORY_AND_DISK
2014-10-09 18:26:43 -07:00
Cheng Lian edf02da389 [SPARK-3654][SQL] Unifies SQL and HiveQL parsers
This PR is a follow up of #2590, and tries to introduce a top level SQL parser entry point for all SQL dialects supported by Spark SQL.

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

The `ExtendedHiveQlParser` now only handle Hive specific extensions.

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

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

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

ceada76 [Cheng Lian] Minor styling fixes
9738934 [Cheng Lian] Minor refactoring, removes optional trailing ";" in the parser
bb2ab12 [Cheng Lian] SET property value can be empty string
ce8860b [Cheng Lian] Passes test suites
e86968e [Cheng Lian] Removes debugging code
8bcace5 [Cheng Lian] Replaces digit.+ to rep1(digit) (Scala style checking doesn't like it)
d15d54f [Cheng Lian] Unifies SQL and HiveQL parsers
2014-10-09 18:25:06 -07:00
Michael Armbrust 2837bf8548 [SPARK-3798][SQL] Store the output of a generator in a val
This prevents it from changing during serialization, leading to corrupted results.

Author: Michael Armbrust <michael@databricks.com>

Closes #2656 from marmbrus/generateBug and squashes the following commits:

efa32eb [Michael Armbrust] Store the output of a generator in a val. This prevents it from changing during serialization.
2014-10-09 17:54:02 -07:00
ravipesala ac30205287 [SPARK-3813][SQL] Support "case when" conditional functions in Spark SQL.
"case when" conditional function is already supported in Spark SQL but there is no support in SqlParser. So added parser support to it.

Author : ravipesala ravindra.pesalahuawei.com

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

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

70c75a7 [ravipesala] Fixed styles
713ea84 [ravipesala] Updated as per admin comments
709684f [ravipesala] Changed parser to support case when function.
2014-10-09 15:14:58 -07:00
Nathan Howell bc3b6cb061 [SPARK-3858][SQL] Pass the generator alias into logical plan node
The alias parameter is being ignored, which makes it more difficult to specify a qualifier for Generator expressions.

Author: Nathan Howell <nhowell@godaddy.com>

Closes #2721 from NathanHowell/SPARK-3858 and squashes the following commits:

8aa0f43 [Nathan Howell] [SPARK-3858][SQL] Pass the generator alias into logical plan node
2014-10-09 15:03:01 -07:00
Daoyuan Wang 0c0e09f567 [SPARK-3412][SQL]add missing row api
chenghao-intel assigned this to me, check PR #2284 for previous discussion

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

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

c6594b2 [Daoyuan Wang] using boxed
7b7e6e3 [Daoyuan Wang] update pattern match
7a39456 [Daoyuan Wang] rename file and refresh getAs[T]
4c18c29 [Daoyuan Wang] remove setAs[T] and null judge
1614493 [Daoyuan Wang] add missing row api
2014-10-09 14:59:03 -07:00
Yin Huai 1c7f0ab302 [SPARK-3339][SQL] Support for skipping json lines that fail to parse
This PR aims to provide a way to skip/query corrupt JSON records. To do so, we introduce an internal column to hold corrupt records (the default name is `_corrupt_record`. This name can be changed by setting the value of `spark.sql.columnNameOfCorruptRecord`). When there is a parsing error, we will put the corrupt record in its unparsed format to the internal column. Users can skip/query this column through SQL.

* To query those corrupt records
```
-- For Hive parser
SELECT `_corrupt_record`
FROM jsonTable
WHERE `_corrupt_record` IS NOT NULL
-- For our SQL parser
SELECT _corrupt_record
FROM jsonTable
WHERE _corrupt_record IS NOT NULL
```
* To skip corrupt records and query regular records
```
-- For Hive parser
SELECT field1, field2
FROM jsonTable
WHERE `_corrupt_record` IS NULL
-- For our SQL parser
SELECT field1, field2
FROM jsonTable
WHERE _corrupt_record IS NULL
```

Generally, it is not recommended to change the name of the internal column. If the name has to be changed to avoid possible name conflicts, you can use `sqlContext.setConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD, <new column name>)` or `sqlContext.sql(SET spark.sql.columnNameOfCorruptRecord=<new column name>)`.

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

Closes #2680 from yhuai/corruptJsonRecord and squashes the following commits:

4c9828e [Yin Huai] Merge remote-tracking branch 'upstream/master' into corruptJsonRecord
309616a [Yin Huai] Change the default name of corrupt record to "_corrupt_record".
b4a3632 [Yin Huai] Merge remote-tracking branch 'upstream/master' into corruptJsonRecord
9375ae9 [Yin Huai] Set the column name of corrupt json record back to the default one after the unit test.
ee584c0 [Yin Huai] Provide a way to query corrupt json records as unparsed strings.
2014-10-09 14:57:27 -07:00
Mike Timper ec4d40e481 [SPARK-3853][SQL] JSON Schema support for Timestamp fields
In JSONRDD.scala, add 'case TimestampType' in the enforceCorrectType function and a toTimestamp function.

Author: Mike Timper <mike@aurorafeint.com>

Closes #2720 from mtimper/master and squashes the following commits:

9386ab8 [Mike Timper] Fix and tests for SPARK-3853
2014-10-09 14:02:27 -07:00
scwf 2c8851343a [SPARK-3806][SQL] Minor fix for CliSuite
To fix two issues in CliSuite
1 CliSuite throw IndexOutOfBoundsException:
Exception in thread "Thread-6" java.lang.IndexOutOfBoundsException: 6
	at scala.collection.mutable.ResizableArray$class.apply(ResizableArray.scala:43)
	at scala.collection.mutable.ArrayBuffer.apply(ArrayBuffer.scala:47)
	at org.apache.spark.sql.hive.thriftserver.CliSuite.org$apache$spark$sql$hive$thriftserver$CliSuite$$captureOutput$1(CliSuite.scala:67)
	at org.apache.spark.sql.hive.thriftserver.CliSuite$$anonfun$4.apply(CliSuite.scala:78)
	at org.apache.spark.sql.hive.thriftserver.CliSuite$$anonfun$4.apply(CliSuite.scala:78)
	at scala.sys.process.ProcessLogger$$anon$1.out(ProcessLogger.scala:96)
	at scala.sys.process.BasicIO$$anonfun$processOutFully$1.apply(BasicIO.scala:135)
	at scala.sys.process.BasicIO$$anonfun$processOutFully$1.apply(BasicIO.scala:135)
	at scala.sys.process.BasicIO$.readFully$1(BasicIO.scala:175)
	at scala.sys.process.BasicIO$.processLinesFully(BasicIO.scala:179)
	at scala.sys.process.BasicIO$$anonfun$processFully$1.apply(BasicIO.scala:164)
	at scala.sys.process.BasicIO$$anonfun$processFully$1.apply(BasicIO.scala:162)
	at scala.sys.process.ProcessBuilderImpl$Simple$$anonfun$3.apply$mcV$sp(ProcessBuilderImpl.scala:73)
	at scala.sys.process.ProcessImpl$Spawn$$anon$1.run(ProcessImpl.scala:22)

Actually, it is the Mutil-Threads lead to this problem.

2 Using ```line.startsWith``` instead ```line.contains``` to assert expected answer. This is a tiny bug in CliSuite, for test case "Simple commands", there is a expected answers "5", if we use ```contains``` that means output like "14/10/06 11:```5```4:36 INFO CliDriver: Time taken: 1.078 seconds" or "14/10/06 11:54:36 INFO StatsReportListener: 	0%	```5```%	10%	25%	50%	75%	90%	95%	100%" will make the assert true.

Author: scwf <wangfei1@huawei.com>

Closes #2666 from scwf/clisuite and squashes the following commits:

11430db [scwf] fix-clisuite
2014-10-09 13:22:41 -07:00
Yash Datta 752e90f15e [SPARK-3711][SQL] Optimize where in clause filter queries
The In case class is replaced by a InSet class in case all the filters are literals, which uses a hashset instead of Sequence, thereby giving significant performance improvement (earlier the seq was using a worst case linear match (exists method) since expressions were assumed in the filter list) . Maximum improvement should be visible in case small percentage of large data matches the filter list.

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

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

4bf2d19 [Yash Datta] SPARK-3711: 1. Fix code style and import order             2. Fix optimization condition             3. Add tests for null in filter list             4. Add test case that optimization is not triggered in case of attributes in filter list
afedbcd [Yash Datta] SPARK-3711: 1. Add test cases for InSet class in ExpressionEvaluationSuite             2. Add class OptimizedInSuite on the lines of ConstantFoldingSuite, for the optimized In clause
0fc902f [Yash Datta] SPARK-3711: UnaryMinus will be handled by constantFolding
bd84c67 [Yash Datta] SPARK-3711: Incorporate review comments. Move optimization of In clause to Optimizer.scala by adding a rule. Add appropriate comments
430f5d1 [Yash Datta] SPARK-3711: Optimize the filter list in case of negative values as well
bee98aa [Yash Datta] SPARK-3711: Optimize where in clause filter queries
2014-10-09 13:17:13 -07:00
Vida Ha b77a02f41c [SPARK-3752][SQL]: Add tests for different UDF's
Author: Vida Ha <vida@databricks.com>

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

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

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

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

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

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

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

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

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

def1a1a [Cheng Lian] Makes PreInsertionCasts handle partitions properly
2014-10-08 18:11:18 -07:00
Cheng Hao 4ec931951f [SPARK-3707] [SQL] Fix bug of type coercion in DIV
Calling `BinaryArithmetic.dataType` will throws exception until it's resolved, but in type coercion rule `Division`, seems doesn't follow this.

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

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

199a85d [Cheng Hao] Simplify the divide rule
dc55218 [Cheng Hao] fix bug of type coercion in div
2014-10-08 17:52:27 -07:00
Liquan Pei 00b7791720 [SQL][Doc] Keep Spark SQL README.md up to date
marmbrus
Update README.md to be consistent with Spark 1.1

Author: Liquan Pei <liquanpei@gmail.com>

Closes #2706 from Ishiihara/SparkSQL-readme and squashes the following commits:

33b9d4b [Liquan Pei] keep README.md up to date
2014-10-08 17:16:54 -07:00
Cheng Lian a42cc08d21 [SPARK-3713][SQL] Uses JSON to serialize DataType objects
This PR uses JSON instead of `toString` to serialize `DataType`s. The latter is not only hard to parse but also flaky in many cases.

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

JoshRosen davies Please help review PySpark related changes, thanks!

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

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

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

    SELECT * from person WHERE FALSE;

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

And current optimizer cannot optimize the case NOT is duplicated like

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

The filter rule above should be simplified

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

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

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

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

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

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

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

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

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

fe92287 [Cheng Lian] Makes table caching eager by default and adds syntax for lazy caching
2014-10-05 17:51:59 -07:00
scwf 58f5361caa [SPARK-3792][SQL] Enable JavaHiveQLSuite
Do not use TestSQLContext in JavaHiveQLSuite, that may lead to two SparkContexts in one jvm and enable JavaHiveQLSuite

Author: scwf <wangfei1@huawei.com>

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

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

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

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

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

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

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

Target directory structure:

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

Hadoop 2.4.1 result:

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

Hadoop 1.0.4 resuet:

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

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

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

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

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

0177dae [Cheng Lian] Fixes dynamic partitioning support for lower Hadoop versions
2014-10-05 11:19:17 -07:00
Michael Armbrust 6a1d48f4f0 [SPARK-3212][SQL] Use logical plan matching instead of temporary tables for table caching
_Also addresses: SPARK-1671, SPARK-1379 and SPARK-3641_

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

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

Author: Michael Armbrust <michael@databricks.com>

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

63fbc2c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into caching.
0ea889e [Michael Armbrust] Address comments.
1e23287 [Michael Armbrust] Add support for cache invalidation for hive inserts.
65ed04a [Michael Armbrust] fix tests.
bdf9a3f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into caching
b4b77f2 [Michael Armbrust] Address comments
6923c9d [Michael Armbrust] More comments / tests
80f26ac [Michael Armbrust] First draft of improved semantics for Spark SQL caching.
2014-10-03 12:34:27 -07:00
Cheng Lian bec0d0eaa3 [SPARK-3007][SQL] Adds dynamic partitioning support
PR #2226 was reverted because it broke Jenkins builds for unknown reason. This debugging PR aims to fix the Jenkins build.

This PR also fixes two bugs:

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

9749850 [Michael Armbrust] Initilize session state before creating CommandProcessor
2014-10-02 19:32:21 -07:00
ravipesala bbdf1de84f [SPARK-3371][SQL] Renaming a function expression with group by gives error
The following code gives error.
```
sqlContext.registerFunction("len", (s: String) => s.length)
sqlContext.sql("select len(foo) as a, count(1) from t1 group by len(foo)").collect()
```
Because SQl parser creates the aliases to the functions in grouping expressions with generated alias names. So if user gives the alias names to the functions inside projection then it does not match the generated alias name of grouping expression.
This kind of queries are working in Hive.
So the fix I have given that if user provides alias to the function in projection then don't generate alias in grouping expression,use the same alias.

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

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

9fb973f [ravipesala] Removed aliases to grouping expressions.
f8ace79 [ravipesala] Fixed the testcase issue
bad2fd0 [ravipesala] SPARK-3371 : Fixed Renaming a function expression with group by gives error
2014-10-01 23:53:21 -07:00
scwf 1b9f0d67f2 [SPARK-3704][SQL] Fix ColumnValue type for Short values in thrift server
case ```ShortType```, we should add short value to hive row. Int value may lead to some problems.

Author: scwf <wangfei1@huawei.com>

Closes #2551 from scwf/fix-addColumnValue and squashes the following commits:

08bcc59 [scwf] ColumnValue.shortValue for short type
2014-10-01 16:38:15 -07:00
Michael Armbrust 45e058ca4b [SPARK-3729][SQL] Do all hive session state initialization in lazy val
This change avoids a NPE during context initialization when settings are present.

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

1bfc16a [Cheng Lian] Made Command.sideEffectResult protected
2014-10-01 16:00:29 -07:00
Venkata Ramana Gollamudi f84b228c40 [SPARK-3593][SQL] Add support for sorting BinaryType
BinaryType is derived from NativeType and added Ordering support.

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

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

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

1cf26f3 [Venkata Ramana Gollamudi] Supported Sorting of BinaryType
2014-10-01 15:57:09 -07:00
scwf f315fb7efc [SPARK-3705][SQL] Add case for VoidObjectInspector to cover NullType
add case for VoidObjectInspector in ```inspectorToDataType```

Author: scwf <wangfei1@huawei.com>

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

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

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

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

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

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

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

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

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

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

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

Author: Reynold Xin <rxin@apache.org>

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

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

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

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

e69ce88 [Cheng Lian] Adds tests to verify dynamic partitioning folder layout
b20a3dc [Cheng Lian] Addresses @yhuai's comments
096bbbc [baishuo(白硕)] Merge pull request #1 from liancheng/refactor-dp
1093c20 [Cheng Lian] Adds more tests
5004542 [Cheng Lian] Minor refactoring
fae9eff [Cheng Lian] Refactors InsertIntoHiveTable to a Command
528e84c [Cheng Lian] Fixes typo in test name, regenerated golden answer files
c464b26 [Cheng Lian] Refactors dynamic partitioning support
5033928 [baishuo] pass check style
2201c75 [baishuo] use HiveConf.DEFAULTPARTITIONNAME to replace hive.exec.default.partition.name
b47c9bf [baishuo] modify according micheal's advice
c3ab36d [baishuo] modify for some bad indentation
7ce2d9f [baishuo] modify code to pass scala style checks
37c1c43 [baishuo] delete a empty else branch
66e33fc [baishuo] do a little modify
88d0110 [baishuo] update file after test
a3961d9 [baishuo(白硕)] Update Cast.scala
f7467d0 [baishuo(白硕)] Update InsertIntoHiveTable.scala
c1a59dd [baishuo(白硕)] Update Cast.scala
0e18496 [baishuo(白硕)] Update HiveQuerySuite.scala
60f70aa [baishuo(白硕)] Update InsertIntoHiveTable.scala
0a50db9 [baishuo(白硕)] Update HiveCompatibilitySuite.scala
491c7d0 [baishuo(白硕)] Update InsertIntoHiveTable.scala
a2374a8 [baishuo(白硕)] Update InsertIntoHiveTable.scala
701a814 [baishuo(白硕)] Update SparkHadoopWriter.scala
dc24c41 [baishuo(白硕)] Update HiveQl.scala
2014-09-29 15:51:55 -07:00
Reynold Xin f350cd3070 [SPARK-3543] TaskContext remaining cleanup work.
Author: Reynold Xin <rxin@apache.org>

Closes #2560 from rxin/TaskContext and squashes the following commits:

9eff95a [Reynold Xin] [SPARK-3543] remaining cleanup work.
2014-09-28 20:32:54 -07:00
Michael Armbrust f0c7e19550 [SPARK-3680][SQL] Fix bug caused by eager typing of HiveGenericUDFs
Typing of UDFs should be lazy as it is often not valid to call `dataType` on an expression until after all of its children are `resolved`.

Author: Michael Armbrust <michael@databricks.com>

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

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

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

Author: w00228970 <wangfei1@huawei.com>

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

0cb5e8d [w00228970] delete golden answer of division-0 and timestamp cast #1
1df3964 [w00228970] Jdk version leads to different query output for Double, this make HiveQuerySuite failed
2014-09-27 12:06:16 -07:00
Michael Armbrust d8a9d1d442 [SPARK-3675][SQL] Allow starting a JDBC server on an existing context
Author: Michael Armbrust <michael@databricks.com>

Closes #2515 from marmbrus/jdbcExistingContext and squashes the following commits:

7866fad [Michael Armbrust] Allows starting a JDBC server on an existing context.
2014-09-26 22:30:12 -07:00
Cheng Hao 7364fa5a17 [SPARK-3393] [SQL] Align the log4j configuration for Spark & SparkSQLCLI
User may be confused for the HQL logging & configurations, we'd better provide a default templates.

Both files are copied from Hive.

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

Closes #2263 from chenghao-intel/hive_template and squashes the following commits:

53bffa9 [Cheng Hao] Remove the hive-log4j.properties initialization
2014-09-26 12:06:01 -07:00
Daoyuan Wang 0ec2d2e8f0 [SPARK-3531][SQL]select null from table would throw a MatchError
Author: Daoyuan Wang <daoyuan.wang@intel.com>

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

2458229 [Daoyuan Wang] rebase solution
2014-09-26 12:04:37 -07:00
Michael Armbrust a08153f8a3 [SPARK-3646][SQL] Copy SQL configuration from SparkConf when a SQLContext is created.
This will allow us to take advantage of things like the spark.defaults file.

Author: Michael Armbrust <michael@databricks.com>

Closes #2493 from marmbrus/copySparkConf and squashes the following commits:

0bd1377 [Michael Armbrust] Copy SQL configuration from SparkConf when a SQLContext is created.
2014-09-23 12:27:12 -07:00
Venkata Ramana Gollamudi 1c62f97e94 [SPARK-3268][SQL] DoubleType, FloatType and DecimalType modulus support
Supported modulus operation using % operator on fractional datatypes FloatType, DoubleType and DecimalType
Example:
SELECT 1388632775.0 % 60 from tablename LIMIT 1

Author : Venkata Ramana Gollamudi ramana.gollamudihuawei.com

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

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

79172a8 [Venkata Ramana Gollamudi] Add hive cache to testcase
c09bd5b [Venkata Ramana Gollamudi] Added a HiveQuerySuite testcase
193fa81 [Venkata Ramana Gollamudi] corrected testcase
3624471 [Venkata Ramana Gollamudi] modified testcase
e112c09 [Venkata Ramana Gollamudi] corrected the testcase
513d0e0 [Venkata Ramana Gollamudi] modified to add modulus support to fractional types float,double,decimal
296d253 [Venkata Ramana Gollamudi] modified to add modulus support to fractional types float,double,decimal
2014-09-23 12:17:47 -07:00
wangfei ae60f8fb2d [SPARK-3481][SQL] removes the evil MINOR HACK
a follow up of https://github.com/apache/spark/pull/2377 and https://github.com/apache/spark/pull/2352, see detail there.

Author: wangfei <wangfei1@huawei.com>

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

4874ec8 [wangfei] removes the evil MINOR HACK
2014-09-23 11:59:44 -07:00
ravipesala 3b8eefa9b8 [SPARK-3536][SQL] SELECT on empty parquet table throws exception
It returns null metadata from parquet if querying on empty parquet file while calculating splits.So added null check and returns the empty splits.

Author : ravipesala ravindra.pesalahuawei.com

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

Closes #2456 from ravipesala/SPARK-3536 and squashes the following commits:

1e81a50 [ravipesala] Fixed the issue when querying on empty parquet file.
2014-09-23 11:52:13 -07:00
Daoyuan Wang 116016b481 [SPARK-3582][SQL] not limit argument type for hive simple udf
Since we have moved to `ConventionHelper`, it is quite easy to avoid call `javaClassToDataType` in hive simple udf. This will solve SPARK-3582.

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

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

450c28e [Daoyuan Wang] not limit argument type for hive simple udf
2014-09-23 11:47:53 -07:00
Daoyuan Wang 66bc0f2d67 [SPARK-3598][SQL]cast to timestamp should be the same as hive
this patch fixes timestamp smaller than 0 and cast int as timestamp

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

32687dc [Cheng Lian] Moved the test case to PlannerSuite
5595a91 [Cheng Lian] Removes debugging code
73faf69 [Cheng Lian] Test case for auto choosing broadcast hash join
f30fe1d [Cheng Lian] Adds sizeInBytes estimation for Limit when all output types are native types
2014-09-20 16:30:49 -07:00
Daoyuan Wang ba68a51c40 [SPARK-3485][SQL] Use GenericUDFUtils.ConversionHelper for Simple UDF type conversions
This is just another solution to SPARK-3485, in addition to PR #2355
In this patch, we will use ConventionHelper and FunctionRegistry to invoke a simple udf evaluation, which rely more on hive, but much cleaner and safer.
We can discuss which one is better.

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

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

15762d2 [Daoyuan Wang] add posmod test which would fail the test but now ok
0d69eb4 [Daoyuan Wang] another way to pass to hive simple udf
2014-09-19 15:39:31 -07:00
Sandy Ryza 3b9cd13ebc SPARK-3605. Fix typo in SchemaRDD.
Author: Sandy Ryza <sandy@cloudera.com>

Closes #2460 from sryza/sandy-spark-3605 and squashes the following commits:

09d940b [Sandy Ryza] SPARK-3605. Fix typo in SchemaRDD.
2014-09-19 15:34:48 -07:00
ravipesala 5522151eb1 [SPARK-2594][SQL] Support CACHE TABLE <name> AS SELECT ...
This feature allows user to add cache table from the select query.
Example : ```CACHE TABLE testCacheTable AS SELECT * FROM TEST_TABLE```
Spark takes this type of SQL as command and it does lazy caching just like ```SQLContext.cacheTable```, ```CACHE TABLE <name>``` does.
It can be executed from both SQLContext and HiveContext.

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

Author : ravipesala ravindra.pesalahuawei.com

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

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

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

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

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

5c9c3a5 [Cheng Hao] make more clear code
49dfc50 [Cheng Hao] Add no-op for Cast and revert the position of SimplifyCasts
b804abd [Cheng Hao] Add unit test to show the failure in identical data type casting
330a5c8 [Cheng Hao] Update Code based on comments
b834ed4 [Cheng Hao] Fix bug of HiveSimpleUDF with unnecessary type cast which cause exception in constant folding
2014-09-19 15:29:22 -07:00
Aaron Staple 8e7ae477ba [SPARK-2314][SQL] Override collect and take in python library, and count in java library, with optimized versions.
SchemaRDD overrides RDD functions, including collect, count, and take, with optimized versions making use of the query optimizer.  The java and python interface classes wrapping SchemaRDD need to ensure the optimized versions are called as well.  This patch overrides relevant calls in the python and java interfaces with optimized versions.

Adds a new Row serialization pathway between python and java, based on JList[Array[Byte]] versus the existing RDD[Array[Byte]]. I wasn’t overjoyed about doing this, but I noticed that some QueryPlans implement optimizations in executeCollect(), which outputs an Array[Row] rather than the typical RDD[Row] that can be shipped to python using the existing serialization code. To me it made sense to ship the Array[Row] over to python directly instead of converting it back to an RDD[Row] just for the purpose of sending the Rows to python using the existing serialization code.

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

Closes #1592 from staple/SPARK-2314 and squashes the following commits:

89ff550 [Aaron Staple] Merge with master.
6bb7b6c [Aaron Staple] Fix typo.
b56d0ac [Aaron Staple] [SPARK-2314][SQL] Override count in JavaSchemaRDD, forwarding to SchemaRDD's count.
0fc9d40 [Aaron Staple] Fix comment typos.
f03cdfa [Aaron Staple] [SPARK-2314][SQL] Override collect and take in sql.py, forwarding to SchemaRDD's collect.
2014-09-16 11:45:35 -07:00
Michael Armbrust 30f288ae34 [SPARK-2890][SQL] Allow reading of data when case insensitive resolution could cause possible ambiguity.
Throwing an error in the constructor makes it possible to run queries, even when there is no actual ambiguity.  Remove this check in favor of throwing an error in analysis when they query is actually is ambiguous.

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

Author: Michael Armbrust <michael@databricks.com>

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

729cca4 [Michael Armbrust] Better tests.
a003aeb [Michael Armbrust] Remove error (it'll be caught in analysis).
2014-09-16 11:42:26 -07:00
Yin Huai 7583699873 [SPARK-3308][SQL] Ability to read JSON Arrays as tables
This PR aims to support reading top level JSON arrays and take every element in such an array as a row (an empty array will not generate a row).

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

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

Closes #2400 from yhuai/SPARK-3308 and squashes the following commits:

990077a [Yin Huai] Handle top level JSON arrays.
2014-09-16 11:40:28 -07:00
Cheng Hao 86d253ec4e [SPARK-3527] [SQL] Strip the string message
Author: Cheng Hao <hao.cheng@intel.com>

Closes #2392 from chenghao-intel/trim and squashes the following commits:

e52024f [Cheng Hao] trim the string message
2014-09-16 11:21:30 -07:00
Bertrand Bossy c243b21a8b SPARK-3039: Allow spark to be built using avro-mapred for hadoop2
SPARK-3039: Adds the maven property "avro.mapred.classifier" to build spark-assembly with avro-mapred with support for the new Hadoop API. Sets this property to hadoop2 for Hadoop 2 profiles.

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

Author: Bertrand Bossy <bertrandbossy@gmail.com>

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

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

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

0da1e8c [Michael Armbrust] test hax
ef2d985 [Michael Armbrust] more test hacks.
2dabae3 [Michael Armbrust] more test fixes
0bdbf21 [Michael Armbrust] Make parquet tests less order dependent
b42eeab [Michael Armbrust] increase test parallelism
80453d5 [Michael Armbrust] Decrease partitions when testing
2014-09-13 16:08:04 -07:00
Cheng Lian 74049249ab [SPARK-3294][SQL] Eliminates boxing costs from in-memory columnar storage
This is a major refactoring of the in-memory columnar storage implementation, aims to eliminate boxing costs from critical paths (building/accessing column buffers) as much as possible. The basic idea is to refactor all major interfaces into a row-based form and use them together with `SpecificMutableRow`. The difficult part is how to adapt all compression schemes, esp. `RunLengthEncoding` and `DictionaryEncoding`, to this design. Since in-memory compression is disabled by default for now, and this PR should be strictly better than before no matter in-memory compression is enabled or not, maybe I'll finish that part in another PR.

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

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

TODO

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

## Micro benchmark

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

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

Speedup:

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

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

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

Before:

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

After:

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

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

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

4419fe4 [Cheng Lian] Addressing comments
e5d2cf2 [Cheng Lian] Bug fix: should call setNullAt when field value is null to avoid NPE
8b8552b [Cheng Lian] Only checks for partition batch pruning flag once
489f97b [Cheng Lian] Bug fix: TableReader.fillObject uses wrong ordinals
97bbc4e [Cheng Lian] Optimizes hive.TableReader by by providing specific Writable unwrappers a head of time
3dc1f94 [Cheng Lian] Minor changes to eliminate row object creation
5b39cb9 [Cheng Lian] Lowers log level of compression scheme details
f2a7890 [Cheng Lian] Use SpecificMutableRow in InMemoryColumnarTableScan to avoid boxing
9cf30b0 [Cheng Lian] Added row based ColumnType.append/extract
456c366 [Cheng Lian] Made compression decoder row based
edac3cd [Cheng Lian] Makes ColumnAccessor.extractSingle row based
8216936 [Cheng Lian] Removes boxing cost in IntDelta and LongDelta by providing specialized implementations
b70d519 [Cheng Lian] Made some in-memory columnar storage interfaces row-based
2014-09-13 15:08:30 -07:00
Cheng Lian 184cd51c42 [SPARK-3481][SQL] Removes the evil MINOR HACK
This is a follow up of #2352. Now we can finally remove the evil "MINOR HACK", which covered up the eldest bug in the history of Spark SQL (see details [here](https://github.com/apache/spark/pull/2352#issuecomment-55440621)).

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

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

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

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

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

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

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

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

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

74fd76b [Cheng Hao] eliminate the error log
2014-09-12 11:29:30 -07:00
Cody Koeninger f858f46686 SPARK-3462 push down filters and projections into Unions
Author: Cody Koeninger <cody.koeninger@mediacrossing.com>

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

5c8d24d [Cody Koeninger] SPARK-3462 remove now-unused parameter
0788691 [Cody Koeninger] SPARK-3462 add tests, handle compatible schema with different aliases, per marmbrus feedback
ef47b3b [Cody Koeninger] SPARK-3462 push down filters and projections into Unions
2014-09-11 17:49:36 -07:00
Yin Huai 4bc9e046cb [SPARK-3390][SQL] sqlContext.jsonRDD fails on a complex structure of JSON array and JSON object nesting
This PR aims to correctly handle JSON arrays in the type of `ArrayType(...(ArrayType(StructType)))`.

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

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

Closes #2364 from yhuai/SPARK-3390 and squashes the following commits:

46db418 [Yin Huai] Handle JSON arrays in the type of ArrayType(...(ArrayType(StructType))).
2014-09-11 15:23:33 -07:00
Cheng Hao ca83f1e2c4 [SPARK-2917] [SQL] Avoid table creation in logical plan analyzing for CTAS
Author: Cheng Hao <hao.cheng@intel.com>

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

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

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

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

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

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

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

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

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

32683c4 [Aaron Staple] Fix compilation failure due to merge.
7c77fda [Aaron Staple] Move ExtractPythonUdfs to Analyzer's extendedRules in HiveContext.
d49bfb3 [Aaron Staple] Address review comments.
915b690 [Aaron Staple] Fix merge issue causing compilation failure.
701dcd2 [Aaron Staple] [SPARK-2781][SQL] Check resolution of LogicalPlans in Analyzer.
2014-09-10 21:01:53 -07:00
Michael Armbrust f92cde24e8 [SPARK-3447][SQL] Remove explicit conversion with JListWrapper to avoid NPE
Author: Michael Armbrust <michael@databricks.com>

Closes #2323 from marmbrus/kryoJListNPE and squashes the following commits:

9634f11 [Michael Armbrust] Rollback JSON RDD changes
4d4d93c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into kryoJListNPE
646976b [Michael Armbrust] Fix JSON RDD Conversion too
59065bc [Michael Armbrust] Remove explicit conversion to avoid NPE
2014-09-10 20:59:40 -07:00
Michael Armbrust 84e2c8bfe4 [SQL] Add test case with workaround for reading partitioned Avro files
In order to read from partitioned Avro files we need to also set the `SERDEPROPERTIES` since `TBLPROPERTIES` are not passed to the initialization.  This PR simply adds a test to make sure we don't break this workaround.

Author: Michael Armbrust <michael@databricks.com>

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

6b969d6 [Michael Armbrust] fix style
fea2124 [Michael Armbrust] Add test case with workaround for reading partitioned avro files.
2014-09-10 20:57:38 -07:00
Wenchen Fan e4f4886d71 [SPARK-2096][SQL] Correctly parse dot notations
First let me write down the current `projections` grammar of spark sql:

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

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

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

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

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

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

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

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

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

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

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

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

3332f6e [Daoyuan Wang] remove wrong comment
83b536c [Daoyuan Wang] a comment to trigger retest
d7315b3 [Daoyuan Wang] code improve
eed35fc [Daoyuan Wang] bug in casewhen resolve
2014-09-10 10:45:24 -07:00
Eric Liang b734ed0c22 [SPARK-3395] [SQL] DSL sometimes incorrectly reuses attribute ids, breaking queries
This resolves https://issues.apache.org/jira/browse/SPARK-3395

Author: Eric Liang <ekl@google.com>

Closes #2266 from ericl/spark-3395 and squashes the following commits:

7f2b6f0 [Eric Liang] add regression test
05bd1e4 [Eric Liang] in the dsl, create a new schema instance in each applySchema
2014-09-09 23:47:12 -07:00
Cheng Lian c110614b33 [SPARK-3448][SQL] Check for null in SpecificMutableRow.update
`SpecificMutableRow.update` doesn't check for null, and breaks existing `MutableRow` contract.

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

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

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

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

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

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

71d15e7 [xinyunh] remove POWER part
8843643 [xinyunh] fix the code style issue
39f0309 [bomeng] Modify the code of POWER and ABS. Move them to the file arithmetic
ff8e51e [bomeng] add abs() function support
7f6980a [xinyunh] fix the bug in 'Last' component
b3df91b [xinyunh] add 'Last' component
2014-09-09 16:55:39 -07:00
Cheng Hao 1e03cf79f8 [SPARK-3455] [SQL] **HOT FIX** Fix the unit test failure
Unit test failed due to can not resolve the attribute references. Temporally disable this test case for a quick fixing, otherwise it will block the others.

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

Closes #2334 from chenghao-intel/unit_test_failure and squashes the following commits:

661f784 [Cheng Hao] temporally disable the failed test case
2014-09-09 10:18:25 -07:00
William Benton 2b7ab814f9 [SPARK-3329][SQL] Don't depend on Hive SET pair ordering in tests.
This fixes some possible spurious test failures in `HiveQuerySuite` by comparing sets of key-value pairs as sets, rather than as lists.

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

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

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

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

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

d9fa1d6 [Cheng Lian] Stores analyzed logical plan when registering a temp table
2014-09-08 19:08:05 -07:00
William Benton ca0348e682 SPARK-3423: [SQL] Implement BETWEEN for SQLParser
This patch improves the SQLParser by adding support for BETWEEN conditions

Author: William Benton <willb@redhat.com>

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

0016d30 [William Benton] Implement BETWEEN for SQLParser
2014-09-08 19:05:02 -07:00
Eric Liang 7db53391f1 [SPARK-3349][SQL] Output partitioning of limit should not be inherited from child
This resolves https://issues.apache.org/jira/browse/SPARK-3349

Author: Eric Liang <ekl@google.com>

Closes #2262 from ericl/spark-3349 and squashes the following commits:

3e1b05c [Eric Liang] add regression test
ac32723 [Eric Liang] make limit/takeOrdered output SinglePartition
2014-09-08 16:14:36 -07:00
Reynold Xin e2614038e7 [SPARK-3408] Fixed Limit operator so it works with sort-based shuffle.
Author: Reynold Xin <rxin@apache.org>

Closes #2281 from rxin/sql-limit-sort and squashes the following commits:

1ef7780 [Reynold Xin] [SPARK-3408] Fixed Limit operator so it works with sort-based shuffle.
2014-09-07 18:42:24 -07:00
Michael Armbrust 39db1bfdab [SQL] Update SQL Programming Guide
Author: Michael Armbrust <michael@databricks.com>
Author: Yin Huai <huai@cse.ohio-state.edu>

Closes #2258 from marmbrus/sqlDocUpdate and squashes the following commits:

f3d450b [Michael Armbrust] fix brackets
bea3bfa [Michael Armbrust] Davies suggestions
3a29fe2 [Michael Armbrust] tighten visibility
a71aa36 [Michael Armbrust] Draft of doc updates
52932c0 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into sqlDocUpdate
1e8c849 [Yin Huai] Update the example used for applySchema.
9457c39 [Yin Huai] Update doc.
31ba240 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeDoc
29bc668 [Yin Huai] Draft doc for data type and schema APIs.
2014-09-07 21:34:46 -04:00
GuoQiang Li 607ae39c22 [SPARK-3397] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT
Author: GuoQiang Li <witgo@qq.com>

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

eaf913f [GuoQiang Li] Bump pom.xml version number of master branch to 1.2.0-SNAPSHOT
2014-09-06 15:04:50 -07:00
Reynold Xin 1b9001f78d [SPARK-3409][SQL] Avoid pulling in Exchange operator itself in Exchange's closures.
This is a tiny teeny optimization to move the if check of sortBasedShuffledOn to outside the closures so the closures don't need to pull in the entire Exchange operator object.

Author: Reynold Xin <rxin@apache.org>

Closes #2282 from rxin/SPARK-3409 and squashes the following commits:

1de3f88 [Reynold Xin] [SPARK-3409][SQL] Avoid pulling in Exchange operator itself in Exchange's closures.
2014-09-06 00:33:00 -07:00
Cheng Hao 1904bac38d [SPARK-3392] [SQL] Show value spark.sql.shuffle.partitions for mapred.reduce.tasks
This is a tiny fix for getting the value of "mapred.reduce.tasks", which make more sense for the hive user.
As well as the command "set -v", which should output verbose information for all of the key/values.

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

Closes #2261 from chenghao-intel/set_mapreduce_tasks and squashes the following commits:

653858a [Cheng Hao] show value spark.sql.shuffle.partitions for mapred.reduce.tasks
2014-09-04 19:16:12 -07:00
Cheng Lian ee575f12f2 [SPARK-2219][SQL] Added support for the "add jar" command
Adds logical and physical command classes for the "add jar" command.

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

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

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

e43a2f1 [Cheng Lian] Updates AddJar according to conventions introduced in #2215
b99107f [Cheng Lian] Added test case for ADD JAR command
095b2c7 [Cheng Lian] Also forward ADD JAR command to Hive
9be031b [Cheng Lian] Trims Jar path string
8195056 [Cheng Lian] Added support for the "add jar" command
2014-09-04 18:47:45 -07:00
Liang-Chi Hsieh 3eb6ef316c [SPARK-3310][SQL] Directly use currentTable without unnecessary implicit conversion
We can directly use currentTable there without unnecessary implicit conversion.

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

Closes #2203 from viirya/direct_use_inmemoryrelation and squashes the following commits:

4741d02 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into direct_use_inmemoryrelation
b671f67 [Liang-Chi Hsieh] Can directly use currentTable there without unnecessary implicit conversion.
2014-09-04 18:46:09 -07:00
Kousuke Saruta dc1ba9e9fc [SPARK-3378] [DOCS] Replace the word "SparkSQL" with right word "Spark SQL"
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

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

0bfe234 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-3378
bb5938f [Kousuke Saruta] Replaced rest of "SparkSQL" with "Spark SQL"
6df66de [Kousuke Saruta] Replaced "SparkSQL" with "Spark SQL"
2014-09-04 15:06:08 -07:00
Davies Liu c5cbc49233 [SPARK-3335] [SQL] [PySpark] support broadcast in Python UDF
After this patch, broadcast can be used in Python UDF.

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

Closes #2243 from davies/udf_broadcast and squashes the following commits:

7b88861 [Davies Liu] support broadcast in UDF
2014-09-03 19:08:39 -07:00
Cheng Lian 248067adbe [SPARK-2961][SQL] Use statistics to prune batches within cached partitions
This PR is based on #1883 authored by marmbrus. Key differences:

1. Batch pruning instead of partition pruning

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

1. More filters are supported

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

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

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

68cf019 [Cheng Lian] Marked sqlContext as @transient
4254f6c [Cheng Lian] Enables in-memory partition pruning in PartitionBatchPruningSuite
3784105 [Cheng Lian] Overrides InMemoryColumnarTableScan.sqlContext
d2a1d66 [Cheng Lian] Disables in-memory partition pruning by default
062c315 [Cheng Lian] HiveCompatibilitySuite code cleanup
16b77bf [Cheng Lian] Fixed pruning predication conjunctions and disjunctions
16195c5 [Cheng Lian] Enabled both disjunction and conjunction
89950d0 [Cheng Lian] Worked around Scala style check
9c167f6 [Cheng Lian] Minor code cleanup
3c4d5c7 [Cheng Lian] Minor code cleanup
ea59ee5 [Cheng Lian] Renamed PartitionSkippingSuite to PartitionBatchPruningSuite
fc517d0 [Cheng Lian] More test cases
1868c18 [Cheng Lian] Code cleanup, bugfix, and adding tests
cb76da4 [Cheng Lian] Added more predicate filters, fixed table scan stats for testing purposes
385474a [Cheng Lian] Merge branch 'inMemStats' into in-mem-batch-pruning
2014-09-03 18:59:26 -07:00
Cheng Lian f48420fde5 [SPARK-2973][SQL] Lightweight SQL commands without distributed jobs when calling .collect()
By overriding `executeCollect()` in physical plan classes of all commands, we can avoid to kick off a distributed job when collecting result of a SQL command, e.g. `sql("SET").collect()`.

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

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

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

3fbef60 [Cheng Lian] Factored execute() method of physical commands to parent class Command
5a0e16c [Cheng Lian] Passes test suites
e0e12e9 [Cheng Lian] Refactored Command.sideEffectResult and Command.executeCollect
995bdd8 [Cheng Lian] Cleaned up DescribeHiveTableCommand
542977c [Cheng Lian] Avoids confusion between logical and physical plan by adding package prefixes
55b2aa5 [Cheng Lian] Avoids distributed jobs when execution SQL commands
2014-09-03 18:57:20 -07:00
Liang-Chi Hsieh 24ab384018 [SPARK-3300][SQL] No need to call clear() and shorten build()
The function `ensureFreeSpace` in object `ColumnBuilder` clears old buffer before copying its content to new buffer. This PR fixes it.

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

Closes #2195 from viirya/fix_buffer_clear and squashes the following commits:

792f009 [Liang-Chi Hsieh] no need to call clear(). use flip() instead of calling limit(), position() and rewind().
df2169f [Liang-Chi Hsieh] should clean old buffer after copying its content.
2014-09-02 20:51:25 -07:00
Cheng Lian 19d3e1e8e9 [SQL] Renamed ColumnStat to ColumnMetrics to avoid confusion between ColumnStats
Class names of these two are just too similar.

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

Closes #2189 from liancheng/column-metrics and squashes the following commits:

8bb3b21 [Cheng Lian] Renamed ColumnStat to ColumnMetrics to avoid confusion between ColumnStats
2014-09-02 20:49:36 -07:00
Takuya UESHIN 0cd91f666d [SPARK-3341][SQL] The dataType of Sqrt expression should be DoubleType.
Author: Takuya UESHIN <ueshin@happy-camper.st>

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

e497320 [Takuya UESHIN] Fix data type of Sqrt expression.
2014-09-02 20:31:15 -07:00
scwf 725715cbf3 [SPARK-3010] fix redundant conditional
https://issues.apache.org/jira/browse/SPARK-3010

this pr is to fix redundant conditional in spark, such as
1.
private[spark] def codegenEnabled: Boolean =
if (getConf(CODEGEN_ENABLED, "false") == "true") true else false
2.
x => if (x == 2) true else false
...

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

Closes #1992 from scwf/condition and squashes the following commits:

b2a044a [scwf] merge SecurityManager
e16239c [scwf] fix confilct
6811401 [scwf] fix merge confilct
0824df4 [scwf] Merge branch 'master' of https://github.com/apache/spark into patch-4
e274515 [scwf] fix redundant conditions
d032bf9 [wangfei] [SQL]Excess judgment
2014-08-31 14:02:11 -07:00
Cheng Lian 32b18dd52c [SPARK-3320][SQL] Made batched in-memory column buffer building work for SchemaRDDs with empty partitions
Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2213 from liancheng/spark-3320 and squashes the following commits:

45a0139 [Cheng Lian] Fixed typo in InMemoryColumnarQuerySuite
f67067d [Cheng Lian] Fixed SPARK-3320
2014-08-29 18:16:47 -07:00
Zdenek Farana 98ddbe6cdb [SPARK-3173][SQL] Timestamp support in the parser
If you have a table with TIMESTAMP column, that column can't be used in WHERE clause properly - it is not evaluated properly. [More](https://issues.apache.org/jira/browse/SPARK-3173)

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

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

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

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

442b59d [Zdenek Farana] Fixed test merge conflict
2dbf4f6 [Zdenek Farana] Merge remote-tracking branch 'origin/SPARK-3173' into SPARK-3173
65b6215 [Zdenek Farana] Fixed timezone sensitivity in the test
47b27b4 [Zdenek Farana] Now works in the case of "StringLiteral=TimestampColumn"
96a661b [Zdenek Farana] Code style change
491dfcf [Zdenek Farana] Added test cases for SPARK-3173
4446b1e [Zdenek Farana] A string literal is casted into Timestamp when the column is Timestamp.
59af397 [Zdenek Farana] Added a new TIMESTAMP keyword; CAST to TIMESTAMP now can be used in SQL expression.
2014-08-29 15:39:15 -07:00
qiping.lqp 634d04b87c [SPARK-3291][SQL]TestcaseName in createQueryTest should not contain ":"
":" is not allowed to appear in a file name of Windows system. If file name contains ":", this file can't be checked out in a Windows system and developers using Windows must be careful to not commit the deletion of such files, Which is very inconvenient.

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

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

0e943a1 [qiping.lqp] rename golden file
60a863f [qiping.lqp] TestcaseName in createQueryTest should not contain ":"
2014-08-29 15:37:43 -07:00
Cheng Lian d94a44d7ca [SPARK-3269][SQL] Decreases initial buffer size for row set to prevent OOM
When a large batch size is specified, `SparkSQLOperationManager` OOMs even if the whole result set is much smaller than the batch size.

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

Closes #2171 from liancheng/jdbc-fetch-size and squashes the following commits:

5e1623b [Cheng Lian] Decreases initial buffer size for row set to prevent OOM
2014-08-29 15:36:04 -07:00
Cheng Lian b1eccfc88a [SQL] Turns on in-memory columnar compression in HiveCompatibilitySuite
`HiveCompatibilitySuite` already turns on in-memory columnar caching, it would be good to also enable compression to improve test coverage.

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

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

88b536c [Cheng Lian] Code cleanup, narrowed field visibility
d13efd2 [Cheng Lian] Turns on in-memory columnar compression in HiveCompatibilitySuite
2014-08-29 15:34:59 -07:00
Cheng Hao dc4d577c65 [SPARK-3198] [SQL] Remove the TreeNode.id
Thus id property of the TreeNode API does save time in a faster way to compare 2 TreeNodes, it is kind of performance bottleneck during the expression object creation in a multi-threading env (because of the memory barrier).
Fortunately, the tree node comparison only happen once in master, so even we remove it, the entire performance will not be affected.

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

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

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

Author: William Benton <willb@redhat.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

189722f [Michael Armbrust] Merge remote-tracking branch 'origin/master' into structUdfs
8e29b1c [Michael Armbrust] Use existing function
d8d0b76 [Michael Armbrust] Fix udfs that return structs
2014-08-28 00:15:23 -07:00
Cheng Lian 68f75dcdfe [SQL] Fixed 2 comment typos in SQLConf
Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #2172 from liancheng/sqlconf-typo and squashes the following commits:

115cc71 [Cheng Lian] Fixed 2 comment typos in SQLConf
2014-08-28 00:08:09 -07:00
Michael Armbrust 024178c574 [HOTFIX][SQL] Remove cleaning of UDFs
It is not safe to run the closure cleaner on slaves.  #2153 introduced this which broke all UDF execution on slaves.  Will re-add cleaning of UDF closures in a follow-up PR.

Author: Michael Armbrust <michael@databricks.com>

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

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

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

5390360 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into inMemDefaultSize
14204d3 [Michael Armbrust] Set the context before creating SparkLogicalPlans.
8da4414 [Michael Armbrust] Make sure we throw errors when leaf nodes fail to provide statistcs
18ce029 [Michael Armbrust] Ensure in-memory tables don't always broadcast.
2014-08-27 15:14:08 -07:00
luogankun 65253502b9 [SPARK-3065][SQL] Add locale setting to fix results do not match for udf_unix_timestamp format "yyyy MMM dd h:mm:ss a" run with not "America/Los_Angeles" TimeZone in HiveCompatibilitySuite
When run the udf_unix_timestamp of org.apache.spark.sql.hive.execution.HiveCompatibilitySuite testcase
with not "America/Los_Angeles" TimeZone throws error. [https://issues.apache.org/jira/browse/SPARK-3065]
add locale setting on beforeAll and afterAll method to fix the bug of HiveCompatibilitySuite testcase

Author: luogankun <luogankun@gmail.com>

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

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

Author: Aaron Davidson <aaron@databricks.com>

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

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

Author: viirya <viirya@gmail.com>

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

b1c0f52 [viirya] add missing condition.
2014-08-27 14:55:05 -07:00
chutium 48f42781de [SPARK-3138][SQL] sqlContext.parquetFile should be able to take a single file as parameter
```if (!fs.getFileStatus(path).isDir) throw Exception``` make no sense after this commit #1370

be careful if someone is working on SPARK-2551, make sure the new change passes test case ```test("Read a parquet file instead of a directory")```

Author: chutium <teng.qiu@gmail.com>

Closes #2044 from chutium/parquet-singlefile and squashes the following commits:

4ae477f [chutium] [SPARK-3138][SQL] sqlContext.parquetFile should be able to take a single file as parameter
2014-08-27 13:13:04 -07:00
Cheng Hao 4238c17dc9 [SPARK-3197] [SQL] Reduce the Expression tree object creations for aggregation function (min/max)
Aggregation function min/max in catalyst will create expression tree for each single row, however, the expression tree creation is quite expensive in a multithreading env currently. Hence we got a very bad performance for the min/max.
Here is the benchmark that I've done in my local.

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

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

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

object DataPrepare extends App {
  import TestHive2._

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

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

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

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

object PerformanceTest extends App {
  import TestHive2._

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

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

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

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

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

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

db40395 [Cheng Hao] remove the transient and add val for the expression property
d56167d [Cheng Hao] Reduce the Expressions creation
2014-08-27 12:50:47 -07:00
u0jing 3b5eb7083d [SPARK-3118][SQL]add "SHOW TBLPROPERTIES tblname;" and "SHOW COLUMNS (FROM|IN) table_name [(FROM|IN) db_name]" support
JIRA issue: [SPARK-3118] https://issues.apache.org/jira/browse/SPARK-3118

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

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

Author: u0jing <u9jing@gmail.com>

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

b231d87 [u0jing] add golden answer files
35f4885 [u0jing] add 'show columns' and 'show tblproperties' support
2014-08-27 12:47:14 -07:00
Michael Armbrust e1139dd60e [SPARK-3237][SQL] Fix parquet filters with UDFs
Author: Michael Armbrust <michael@databricks.com>

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

712731a [Michael Armbrust] Use closure serializer for sending filters.
1e83f80 [Michael Armbrust] Clean udf functions.
2014-08-27 00:59:23 -07:00
Takuya UESHIN 727cb25bcc [SPARK-3036][SPARK-3037][SQL] Add MapType/ArrayType containing null value support to Parquet.
JIRA:
- https://issues.apache.org/jira/browse/SPARK-3036
- https://issues.apache.org/jira/browse/SPARK-3037

Currently this uses the following Parquet schema for `MapType` when `valueContainsNull` is `true`:

```
message root {
  optional group a (MAP) {
    repeated group map (MAP_KEY_VALUE) {
      required int32 key;
      optional int32 value;
    }
  }
}
```

for `ArrayType` when `containsNull` is `true`:

```
message root {
  optional group a (LIST) {
    repeated group bag {
      optional int32 array;
    }
  }
}
```

We have to think about compatibilities with older version of Spark or Hive or others I mentioned in the JIRA issues.

Notice:
This PR is based on #1963 and #1889.
Please check them first.

/cc marmbrus, yhuai

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

Closes #2032 from ueshin/issues/SPARK-3036_3037 and squashes the following commits:

4e8e9e7 [Takuya UESHIN] Add ArrayType containing null value support to Parquet.
013c2ca [Takuya UESHIN] Add MapType containing null value support to Parquet.
62989de [Takuya UESHIN] Merge branch 'issues/SPARK-2969' into issues/SPARK-3036_3037
8e38b53 [Takuya UESHIN] Merge branch 'issues/SPARK-3063' into issues/SPARK-3036_3037
2014-08-26 18:28:41 -07:00
Michael Armbrust c4787a3690 [SPARK-3194][SQL] Add AttributeSet to fix bugs with invalid comparisons of AttributeReferences
It is common to want to describe sets of attributes that are in various parts of a query plan.  However, the semantics of putting `AttributeReference` objects into a standard Scala `Set` result in subtle bugs when references differ cosmetically.  For example, with case insensitive resolution it is possible to have two references to the same attribute whose names are not equal.

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

1c0dae5 [Michael Armbrust] work on serialization bug.
9ba868d [Michael Armbrust] Merge remote-tracking branch 'origin/master' into attributeSets
3ae5288 [Michael Armbrust] review comments
40ce7f6 [Michael Armbrust] style
d577cc7 [Michael Armbrust] Scaladoc
cae5d22 [Michael Armbrust] remove more references implementations
d6e16be [Michael Armbrust] Remove more instances of "def references" and normal sets of attributes.
fc26b49 [Michael Armbrust] Add AttributeSet class, remove references from Expression.
2014-08-26 16:29:14 -07:00
Takuya UESHIN 6b5584ef1c [SPARK-3063][SQL] ExistingRdd should convert Map to catalyst Map.
Currently `ExistingRdd.convertToCatalyst` doesn't convert `Map` value.

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

Closes #1963 from ueshin/issues/SPARK-3063 and squashes the following commits:

3ba41f2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063
4d7bae2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063
9321379 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063
d8a900a [Takuya UESHIN] Make ExistingRdd.convertToCatalyst be able to convert Map value.
2014-08-26 15:04:08 -07:00
Takuya UESHIN 98c2bb0bbd [SPARK-2969][SQL] Make ScalaReflection be able to handle ArrayType.containsNull and MapType.valueContainsNull.
Make `ScalaReflection` be able to handle like:

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

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

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

24f1c5c [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Python API.
79f5b65 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Java API.
7cd1a7a [Takuya UESHIN] Fix json test failures.
2cfb862 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true.
2f38e61 [Takuya UESHIN] Revert the default value of MapTypes.valueContainsNull.
9fa02f5 [Takuya UESHIN] Fix a test failure.
1a9a96b [Takuya UESHIN] Modify ScalaReflection to handle ArrayType.containsNull and MapType.valueContainsNull.
2014-08-26 13:22:55 -07:00
chutium 8856c3d860 [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext
There are 4 different compression codec available for ```ParquetOutputFormat```

in Spark SQL, it was set as a hard-coded value in ```ParquetRelation.defaultCompression```

original discuss:
https://github.com/apache/spark/pull/195#discussion-diff-11002083

i added a new config property in SQLConf to allow user to change this compression codec, and i used similar short names syntax as described in SPARK-2953 #1873 (https://github.com/apache/spark/pull/1873/files#diff-0)

btw, which codec should we use as default? it was set to GZIP (https://github.com/apache/spark/pull/195/files#diff-4), but i think maybe we should change this to SNAPPY, since SNAPPY is already the default codec for shuffling in spark-core (SPARK-2469, #1415), and parquet-mr supports Snappy codec natively (e440108de5).

Author: chutium <teng.qiu@gmail.com>

Closes #2039 from chutium/parquet-compression and squashes the following commits:

2f44964 [chutium] [SPARK-3131][SQL] parquet compression default codec set to snappy, also in test suite
e578e21 [chutium] [SPARK-3131][SQL] compression codec config property name and default codec set to snappy
21235dc [chutium] [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext
2014-08-26 11:51:26 -07:00
Daoyuan Wang 52fbdc2ded [Spark-3222] [SQL] Cross join support in HiveQL
We can simple treat cross join as inner join without join conditions.

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

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

8c9b7c5 [Daoyuan Wang] add a test
7d47bbb [adrian-wang] add cross join support for hql
2014-08-25 22:56:35 -07:00
Chia-Yung Su 4243bb6634 [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile
fix compile error on hadoop 0.23 for the pull request #1924.

Author: Chia-Yung Su <chiayung@appier.com>

Closes #1959 from joesu/bugfix-spark3011 and squashes the following commits:

be30793 [Chia-Yung Su] remove .* and _* except _metadata
8fe2398 [Chia-Yung Su] add note to explain
40ea9bd [Chia-Yung Su] fix hadoop-0.23 compile error
c7e44f2 [Chia-Yung Su] match syntax
f8fc32a [Chia-Yung Su] filter out tmp dir
2014-08-25 18:20:19 -07:00
wangfei 507a1b5200 [SQL] logWarning should be logInfo in getResultSetSchema
Author: wangfei <wangfei_hello@126.com>

Closes #1939 from scwf/patch-5 and squashes the following commits:

f952d10 [wangfei] [SQL] logWarning should be logInfo in getResultSetSchema
2014-08-25 17:46:43 -07:00
Cheng Hao 156eb39661 [SPARK-3058] [SQL] Support EXTENDED for EXPLAIN
Provide `extended` keyword support for `explain` command in SQL. e.g.
```
explain extended select key as a1, value as a2 from src where key=1;
== Parsed Logical Plan ==
Project ['key AS a1#3,'value AS a2#4]
 Filter ('key = 1)
  UnresolvedRelation None, src, None

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

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

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

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

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

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

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

295db74 [Cheng Hao] Fix bug in printing the simple execution plan
48bc989 [Cheng Hao] Support EXTENDED for EXPLAIN
2014-08-25 17:43:56 -07:00
Cheng Lian cae9414d38 [SPARK-2929][SQL] Refactored Thrift server and CLI suites
Removed most hard coded timeout, timing assumptions and all `Thread.sleep`. Simplified IPC and synchronization with `scala.sys.process` and future/promise so that the test suites can run more robustly and faster.

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

Closes #1856 from liancheng/thriftserver-tests and squashes the following commits:

2d914ca [Cheng Lian] Minor refactoring
0e12e71 [Cheng Lian] Cleaned up test output
0ee921d [Cheng Lian] Refactored Thrift server and CLI suites
2014-08-25 16:29:59 -07:00
Takuya UESHIN d299e2bf2f [SPARK-3204][SQL] MaxOf would be foldable if both left and right are foldable.
Author: Takuya UESHIN <ueshin@happy-camper.st>

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

7d9b107 [Takuya UESHIN] Make MaxOf foldable if both left and right are foldable.
2014-08-25 16:27:00 -07:00
Michael Armbrust 3519b5e8e5 [SPARK-2967][SQL] Follow-up: Also copy hash expressions in sort based shuffle fix.
Follow-up to #2066

Author: Michael Armbrust <michael@databricks.com>

Closes #2072 from marmbrus/sortShuffle and squashes the following commits:

2ff8114 [Michael Armbrust] Fix bug
2014-08-23 16:21:08 -07:00
Michael Armbrust 7e191fe29b [SPARK-2554][SQL] CountDistinct partial aggregation and object allocation improvements
Author: Michael Armbrust <michael@databricks.com>
Author: Gregory Owen <greowen@gmail.com>

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

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

cc: marmbrus

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

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

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

Author: Alex Liu <alex_liu68@yahoo.com>

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

e4bdc4c [Alex Liu] SPARK-SQL-2846 add configureInputJobPropertiesForStorageHandler to initial job conf
2014-08-20 16:14:06 -07:00
Michael Armbrust a2e658dcda [SPARK-2967][SQL] Fix sort based shuffle for spark sql.
Add explicit row copies when sort based shuffle is on.

Author: Michael Armbrust <michael@databricks.com>

Closes #2066 from marmbrus/sortShuffle and squashes the following commits:

fcd7bb2 [Michael Armbrust] Fix sort based shuffle for spark sql.
2014-08-20 15:51:14 -07:00
Cheng Lian cf46e72581 [SPARK-3126][SPARK-3127][SQL] Fixed HiveThriftServer2Suite
This PR fixes two issues:

1. Fixes wrongly quoted command line option in `HiveThriftServer2Suite` that makes test cases hang until timeout.
1. Asks `dev/run-test` to run Spark SQL tests when `bin/spark-sql` and/or `sbin/start-thriftserver.sh` are modified.

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

Closes #2036 from liancheng/fix-thriftserver-test and squashes the following commits:

f38c4eb [Cheng Lian] Fixed the same quotation issue in CliSuite
26b82a0 [Cheng Lian] Run SQL tests when dff contains bin/spark-sql and/or sbin/start-thriftserver.sh
a87f83d [Cheng Lian] Extended timeout
e5aa31a [Cheng Lian] Fixed metastore JDBC URI quotation
2014-08-20 12:57:39 -07:00
wangfei 0e3ab94d41 [SQL] add note of use synchronizedMap in SQLConf
Refer to:
http://stackoverflow.com/questions/510632/whats-the-difference-between-concurrenthashmap-and-collections-synchronizedmap
Collections.synchronizedMap(map) creates a blocking Map which will degrade performance, albeit ensure consistency. So use ConcurrentHashMap(a more effective thread-safe hashmap) instead.

also update HiveQuerySuite to fix test error when changed to ConcurrentHashMap.

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

Closes #1996 from scwf/sqlconf and squashes the following commits:

93bc0c5 [wangfei] revert change of HiveQuerySuite
0cc05dd [wangfei] add note for use synchronizedMap
3c224d31 [scwf] fix formate
a7bcb98 [scwf] use ConcurrentHashMap in sql conf, intead synchronizedMap
2014-08-19 19:37:02 -07:00
Michael Armbrust 3abd0c1cda [SPARK-2406][SQL] Initial support for using ParquetTableScan to read HiveMetaStore tables.
This PR adds an experimental flag `spark.sql.hive.convertMetastoreParquet` that when true causes the planner to detects tables that use Hive's Parquet SerDe and instead plans them using Spark SQL's native `ParquetTableScan`.

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

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

1620079 [Michael Armbrust] Revert "remove hive parquet bundle"
cc30430 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore
4f3d54f [Michael Armbrust] fix style
41ebc5f [Michael Armbrust] remove hive parquet bundle
a43e0da [Michael Armbrust] Merge remote-tracking branch 'origin/master' into parquetMetastore
4c4dc19 [Michael Armbrust] Fix bug with tree splicing.
ebb267e [Michael Armbrust] include parquet hive to tests pass (Remove this later).
c0d9b72 [Michael Armbrust] Avoid creating a HadoopRDD per partition.  Add dirty hacks to retrieve partition values from the InputSplit.
8cdc93c [Michael Armbrust] Merge pull request #8 from yhuai/parquetMetastore
a0baec7 [Yin Huai] Partitioning columns can be resolved.
1161338 [Michael Armbrust] Add a test to make sure conversion is actually happening
212d5cd [Michael Armbrust] Initial support for using ParquetTableScan to read HiveMetaStore tables.
2014-08-18 13:17:10 -07:00
Matei Zaharia 9eb74c7d2c [SPARK-3091] [SQL] Add support for caching metadata on Parquet files
For larger Parquet files, reading the file footers (which is done in parallel on up to 5 threads) and HDFS block locations (which is serial) can take multiple seconds. We can add an option to cache this data within FilteringParquetInputFormat. Unfortunately ParquetInputFormat only caches footers within each instance of ParquetInputFormat, not across them.

Note: this PR leaves this turned off by default for 1.1, but I believe it's safe to turn it on after. The keys in the hash maps are FileStatus objects that include a modification time, so this will work fine if files are modified. The location cache could become invalid if files have moved within HDFS, but that's rare so I just made it invalidate entries every 15 minutes.

Author: Matei Zaharia <matei@databricks.com>

Closes #2005 from mateiz/parquet-cache and squashes the following commits:

dae8efe [Matei Zaharia] Bug fix
c71e9ed [Matei Zaharia] Handle empty statuses directly
22072b0 [Matei Zaharia] Use Guava caches and add a config option for caching metadata
8fb56ce [Matei Zaharia] Cache file block locations too
453bd21 [Matei Zaharia] Bug fix
4094df6 [Matei Zaharia] First attempt at caching Parquet footers
2014-08-18 11:00:10 -07:00
Patrick Wendell 6bca8898a1 SPARK-3025 [SQL]: Allow JDBC clients to set a fair scheduler pool
This definitely needs review as I am not familiar with this part of Spark.
I tested this locally and it did seem to work.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #1937 from pwendell/scheduler and squashes the following commits:

b858e33 [Patrick Wendell] SPARK-3025: Allow JDBC clients to set a fair scheduler pool
2014-08-18 10:52:20 -07:00
Matei Zaharia 4bf3de7107 [SPARK-3085] [SQL] Use compact data structures in SQL joins
This reuses the CompactBuffer from Spark Core to save memory and pointer
dereferences. I also tried AppendOnlyMap instead of java.util.HashMap
but unfortunately that slows things down because it seems to do more
equals() calls and the equals on GenericRow, and especially JoinedRow,
is pretty expensive.

Author: Matei Zaharia <matei@databricks.com>

Closes #1993 from mateiz/spark-3085 and squashes the following commits:

188221e [Matei Zaharia] Remove unneeded import
5f903ee [Matei Zaharia] [SPARK-3085] [SQL] Use compact data structures in SQL joins
2014-08-18 10:45:24 -07:00
Matei Zaharia 6a13dca12f [SPARK-3084] [SQL] Collect broadcasted tables in parallel in joins
BroadcastHashJoin has a broadcastFuture variable that tries to collect
the broadcasted table in a separate thread, but this doesn't help
because it's a lazy val that only gets initialized when you attempt to
build the RDD. Thus queries that broadcast multiple tables would collect
and broadcast them sequentially. I changed this to a val to let it start
collecting right when the operator is created.

Author: Matei Zaharia <matei@databricks.com>

Closes #1990 from mateiz/spark-3084 and squashes the following commits:

f468766 [Matei Zaharia] [SPARK-3084] Collect broadcasted tables in parallel in joins
2014-08-18 10:05:52 -07:00
Patrick Wendell 7ae28d1247 SPARK-3096: Include parquet hive serde by default in build
A small change - we should just add this dependency. It doesn't have any recursive deps and it's needed for reading have parquet tables.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #2009 from pwendell/parquet and squashes the following commits:

e411f9f [Patrick Wendell] SPARk-309: Include parquet hive serde by default in build
2014-08-18 10:00:46 -07:00
Michael Armbrust bfa09b01d7 [SQL] Improve debug logging and toStrings.
Author: Michael Armbrust <michael@databricks.com>

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

b7a7e41 [Michael Armbrust] Improve debug logging and toStrings.
2014-08-17 19:00:38 -07:00
Michael Armbrust 5ecb08ea06 Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled"
Revert #1891 due to issues with hadoop 1 compatibility.

Author: Michael Armbrust <michael@databricks.com>

Closes #2007 from marmbrus/revert1891 and squashes the following commits:

68706c0 [Michael Armbrust] Revert "[SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled"
2014-08-17 18:10:45 -07:00
Sean Owen e1b85f3102 SPARK-2955 [BUILD] Test code fails to compile with "mvn compile" without "install"
(This is the corrected follow-up to https://issues.apache.org/jira/browse/SPARK-2903)

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

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

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

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

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

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

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

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

Author: Sean Owen <srowen@gmail.com>

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

ad8242f [Sean Owen] Generate test-jar on test-compile for modules whose tests are needed by others' tests
2014-08-14 22:08:44 -07:00
Michael Armbrust a7f8a4f5ee Revert [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile
Reverts #1924 due to build failures with hadoop 0.23.

Author: Michael Armbrust <michael@databricks.com>

Closes #1949 from marmbrus/revert1924 and squashes the following commits:

6bff940 [Michael Armbrust] Revert "[SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile"
2014-08-14 13:00:21 -07:00
Yin Huai add75d4831 [SPARK-2927][SQL] Add a conf to configure if we always read Binary columns stored in Parquet as String columns
This PR adds a new conf flag `spark.sql.parquet.binaryAsString`. When it is `true`, if there is no parquet metadata file available to provide the schema of the data, we will always treat binary fields stored in parquet as string fields. This conf is used to provide a way to read string fields generated without UTF8 decoration.

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

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

Closes #1855 from yhuai/parquetBinaryAsString and squashes the following commits:

689ffa9 [Yin Huai] Add missing "=".
80827de [Yin Huai] Unit test.
1765ca4 [Yin Huai] Use .toBoolean.
9d3f199 [Yin Huai] Merge remote-tracking branch 'upstream/master' into parquetBinaryAsString
5d436a1 [Yin Huai] The initial support of adding a conf to treat binary columns stored in Parquet as string columns.
2014-08-14 10:46:33 -07:00
Chia-Yung Su 078f3fbda8 [SPARK-3011][SQL] _temporary directory should be filtered out by sqlContext.parquetFile
Author: Chia-Yung Su <chiayung@appier.com>

Closes #1924 from joesu/bugfix-spark3011 and squashes the following commits:

c7e44f2 [Chia-Yung Su] match syntax
f8fc32a [Chia-Yung Su] filter out tmp dir
2014-08-14 10:43:08 -07:00
guowei 63d6777737 [SPARK-2986] [SQL] fixed: setting properties does not effect
it seems that set command does not run by SparkSQLDriver. it runs on hive api.
user can not change reduce number by setting spark.sql.shuffle.partitions

but i think setting hive properties seems just a role to spark sql.

Author: guowei <guowei@upyoo.com>

Closes #1904 from guowei2/temp-branch and squashes the following commits:

7d47dde [guowei] fixed: setting properties like spark.sql.shuffle.partitions does not effective
2014-08-13 17:45:24 -07:00
Kousuke Saruta 905dc4b405 [SPARK-2970] [SQL] spark-sql script ends with IOException when EventLogging is enabled
Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #1891 from sarutak/SPARK-2970 and squashes the following commits:

4a2d2fe [Kousuke Saruta] Modified comment style
8bd833c [Kousuke Saruta] Modified style
6c0997c [Kousuke Saruta] Modified the timing of shutdown hook execution. It should be executed before shutdown hook of o.a.h.f.FileSystem
2014-08-13 17:42:38 -07:00
Michael Armbrust 9fde1ff5fc [SPARK-2935][SQL]Fix parquet predicate push down bug
Author: Michael Armbrust <michael@databricks.com>

Closes #1863 from marmbrus/parquetPredicates and squashes the following commits:

10ad202 [Michael Armbrust] left <=> right
f249158 [Michael Armbrust] quiet parquet tests.
802da5b [Michael Armbrust] Add test case.
eab2eda [Michael Armbrust] Fix parquet predicate push down bug
2014-08-13 17:40:59 -07:00
Cheng Lian 376a82e196 [SPARK-2650][SQL] More precise initial buffer size estimation for in-memory column buffer
This is a follow up of #1880.

Since the row number within a single batch is known, we can estimate a much more precise initial buffer size when building an in-memory column buffer.

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

Closes #1901 from liancheng/precise-init-buffer-size and squashes the following commits:

d5501fa [Cheng Lian] More precise initial buffer size estimation for in-memory column buffer
2014-08-13 17:37:55 -07:00
Michael Armbrust 9256d4a9c8 [SPARK-2994][SQL] Support for udfs that take complex types
Author: Michael Armbrust <michael@databricks.com>

Closes #1915 from marmbrus/arrayUDF and squashes the following commits:

a1c503d [Michael Armbrust] Support for udfs that take complex types
2014-08-13 17:35:38 -07:00
tianyi 13f54e2b97 [SPARK-2817] [SQL] add "show create table" support
In spark sql component, the "show create table" syntax had been disabled.
We thought it is a useful funciton to describe a hive table.

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

Closes #1760 from tianyi/spark-2817 and squashes the following commits:

7d28b15 [tianyi] [SPARK-2817] fix too short prefix problem
cbffe8b [tianyi] [SPARK-2817] fix the case problem
565ec14 [tianyi] [SPARK-2817] fix the case problem
60d48a9 [tianyi] [SPARK-2817] use system temporary folder instead of temporary files in the source tree, and also clean some empty line
dbe1031 [tianyi] [SPARK-2817] move some code out of function rewritePaths, as it may be called multiple times
9b2ba11 [tianyi] [SPARK-2817] fix the line length problem
9f97586 [tianyi] [SPARK-2817] remove test.tmp.dir from pom.xml
bfc2999 [tianyi] [SPARK-2817] add "File.separator" support, create a "testTmpDir" outside the rewritePaths
bde800a [tianyi] [SPARK-2817] add "${system:test.tmp.dir}" support add "last_modified_by" to nonDeterministicLineIndicators in HiveComparisonTest
bb82726 [tianyi] [SPARK-2817] remove test which requires a system from the whitelist.
bbf6b42 [tianyi] [SPARK-2817] add a systemProperties named "test.tmp.dir" to pass the test which contains "${system:test.tmp.dir}"
a337bd6 [tianyi] [SPARK-2817] add "show create table" support
a03db77 [tianyi] [SPARK-2817] add "show create table" support
2014-08-13 16:50:02 -07:00
Cheng Lian bdc7a1a474 [SPARK-3004][SQL] Added null checking when retrieving row set
JIRA issue: [SPARK-3004](https://issues.apache.org/jira/browse/SPARK-3004)

HiveThriftServer2 throws exception when the result set contains `NULL`. Should check `isNullAt` in `SparkSQLOperationManager.getNextRowSet`.

Note that simply using `row.addColumnValue(null)` doesn't work, since Hive set the column type of a null `ColumnValue` to String by default.

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

Closes #1920 from liancheng/spark-3004 and squashes the following commits:

1b1db1c [Cheng Lian] Adding NULL column values in the Hive way
2217722 [Cheng Lian] Fixed SPARK-3004: added null checking when retrieving row set
2014-08-13 16:27:50 -07:00
Cheng Hao 5d54d71ddb [SQL] [SPARK-2826] Reduce the memory copy while building the hashmap for HashOuterJoin
This is a follow up for #1147 , this PR will improve the performance about 10% - 15% in my local tests.
```
Before:
LeftOuterJoin: took 16750 ms ([3000000] records)
LeftOuterJoin: took 15179 ms ([3000000] records)
RightOuterJoin: took 15515 ms ([3000000] records)
RightOuterJoin: took 15276 ms ([3000000] records)
FullOuterJoin: took 19150 ms ([6000000] records)
FullOuterJoin: took 18935 ms ([6000000] records)

After:
LeftOuterJoin: took 15218 ms ([3000000] records)
LeftOuterJoin: took 13503 ms ([3000000] records)
RightOuterJoin: took 13663 ms ([3000000] records)
RightOuterJoin: took 14025 ms ([3000000] records)
FullOuterJoin: took 16624 ms ([6000000] records)
FullOuterJoin: took 16578 ms ([6000000] records)
```

Besides the performance improvement, I also do some clean up as suggested in #1147

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

Closes #1765 from chenghao-intel/hash_outer_join_fixing and squashes the following commits:

ab1f9e0 [Cheng Hao] Reduce the memory copy while building the hashmap
2014-08-11 20:45:14 -07:00
Michael Armbrust bad21ed085 [SPARK-2650][SQL] Build column buffers in smaller batches
Author: Michael Armbrust <michael@databricks.com>

Closes #1880 from marmbrus/columnBatches and squashes the following commits:

0649987 [Michael Armbrust] add test
4756fad [Michael Armbrust] fix compilation
2314532 [Michael Armbrust] Build column buffers in smaller batches
2014-08-11 20:21:56 -07:00
Takuya UESHIN c686b7dd46 [SPARK-2968][SQL] Fix nullabilities of Explode.
Output nullabilities of `Explode` could be detemined by `ArrayType.containsNull` or `MapType.valueContainsNull`.

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

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

d128c95 [Takuya UESHIN] Fix nullability of Explode.
2014-08-11 20:18:03 -07:00
Takuya UESHIN c9c89c31b6 [SPARK-2965][SQL] Fix HashOuterJoin output nullabilities.
Output attributes of opposite side of `OuterJoin` should be nullable.

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

Closes #1887 from ueshin/issues/SPARK-2965 and squashes the following commits:

bcb2d37 [Takuya UESHIN] Fix HashOuterJoin output nullabilities.
2014-08-11 20:15:01 -07:00
Yin Huai 647aeba3a9 [SQL] A tiny refactoring in HiveContext#analyze
I should use `EliminateAnalysisOperators` in  `analyze` instead of manually pattern matching.

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

Closes #1881 from yhuai/useEliminateAnalysisOperators and squashes the following commits:

f3e1e7f [Yin Huai] Use EliminateAnalysisOperators.
2014-08-11 20:11:29 -07:00
wangfei e83fdcd421 [sql]use SparkSQLEnv.stop() in ShutdownHook
Author: wangfei <wangfei1@huawei.com>

Closes #1852 from scwf/patch-3 and squashes the following commits:

ae28c29 [wangfei] use SparkSQLEnv.stop() in ShutdownHook
2014-08-11 20:10:13 -07:00
Cheng Lian 21a95ef051 [SPARK-2590][SQL] Added option to handle incremental collection, disabled by default
JIRA issue: [SPARK-2590](https://issues.apache.org/jira/browse/SPARK-2590)

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

Closes #1853 from liancheng/inc-collect-option and squashes the following commits:

cb3ea45 [Cheng Lian] Moved incremental collection option to Thrift server
43ce3aa [Cheng Lian] Changed incremental collect option name
623abde [Cheng Lian] Added option to handle incremental collection, disabled by default
2014-08-11 20:08:06 -07:00
Reynold Xin 5b6585de6b Updated Spark SQL README to include the hive-thriftserver module
Author: Reynold Xin <rxin@apache.org>

Closes #1867 from rxin/sql-readme and squashes the following commits:

42a5307 [Reynold Xin] Updated Spark SQL README to include the hive-thriftserver module
2014-08-09 22:05:36 -07:00
chutium b7c89a7f0c [SPARK-2700] [SQL] Hidden files (such as .impala_insert_staging) should be filtered out by sqlContext.parquetFile
Author: chutium <teng.qiu@gmail.com>

Closes #1691 from chutium/SPARK-2700 and squashes the following commits:

b76ae8c [chutium] [SPARK-2700] [SQL] fixed styling issue
d75a8bd [chutium] [SPARK-2700] [SQL] Hidden files (such as .impala_insert_staging) should be filtered out by sqlContext.parquetFile
2014-08-08 13:31:08 -07:00
Yin Huai 45d8f4deab [SPARK-2919] [SQL] Basic support for analyze command in HiveQl
The command we will support is
```
ANALYZE TABLE tablename COMPUTE STATISTICS noscan
```
Other cases shown in https://cwiki.apache.org/confluence/display/Hive/StatsDev#StatsDev-ExistingTables will still be treated as Hive native commands.

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

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

Closes #1848 from yhuai/sqlAnalyze and squashes the following commits:

0b79d36 [Yin Huai] Typo and format.
c59d94b [Yin Huai] Support "ANALYZE TABLE tableName COMPUTE STATISTICS noscan".
2014-08-08 11:23:58 -07:00
Yin Huai c874723fa8 [SPARK-2877] [SQL] MetastoreRelation should use SparkClassLoader when creating the tableDesc
JIRA: https://issues.apache.org/jira/browse/SPARK-2877

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

Closes #1806 from yhuai/SPARK-2877 and squashes the following commits:

4142bcb [Yin Huai] Use Spark's classloader.
2014-08-08 11:15:16 -07:00
Yin Huai 0489cee6b2 [SPARK-2908] [SQL] JsonRDD.nullTypeToStringType does not convert all NullType to StringType
JIRA: https://issues.apache.org/jira/browse/SPARK-2908

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

Closes #1840 from yhuai/SPARK-2908 and squashes the following commits:

86e833e [Yin Huai] Update test.
cb11759 [Yin Huai] nullTypeToStringType should check columns with the type of array of structs.
2014-08-08 11:10:11 -07:00
Yin Huai 9016af3f27 [SPARK-2888] [SQL] Fix addColumnMetadataToConf in HiveTableScan
JIRA: https://issues.apache.org/jira/browse/SPARK-2888

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

Closes #1817 from yhuai/fixAddColumnMetadataToConf and squashes the following commits:

fba728c [Yin Huai] Fix addColumnMetadataToConf.
2014-08-08 11:01:51 -07:00
Cheng Lian a6cd31108f [SPARK-2678][Core][SQL] A workaround for SPARK-2678
JIRA issues:

- Main: [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678)
- Related: [SPARK-2874](https://issues.apache.org/jira/browse/SPARK-2874)

Related PR:

- #1715

This PR is both a fix for SPARK-2874 and a workaround for SPARK-2678. Fixing SPARK-2678 completely requires some API level changes that need further discussion, and we decided not to include it in Spark 1.1 release. As currently SPARK-2678 only affects Spark SQL scripts, this workaround is enough for Spark 1.1. Command line option handling logic in bash scripts looks somewhat dirty and duplicated, but it helps to provide a cleaner user interface as well as retain full downward compatibility for now.

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

Closes #1801 from liancheng/spark-2874 and squashes the following commits:

8045d7a [Cheng Lian] Make sure test suites pass
8493a9e [Cheng Lian] Using eval to retain quoted arguments
aed523f [Cheng Lian] Fixed typo in bin/spark-sql
f12a0b1 [Cheng Lian] Worked arount SPARK-2678
daee105 [Cheng Lian] Fixed usage messages of all Spark SQL related scripts
2014-08-06 12:28:35 -07:00
Davies Liu 48789117c2 [SPARK-2875] [PySpark] [SQL] handle null in schemaRDD()
Handle null in schemaRDD during converting them into Python.

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

Closes #1802 from davies/json and squashes the following commits:

88e6b1f [Davies Liu] handle null in schemaRDD()
2014-08-06 11:08:12 -07:00
Michael Armbrust 5a826c00c3 [SQL] Fix logging warn -> debug
Author: Michael Armbrust <michael@databricks.com>

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

8ea9cf1 [Michael Armbrust] [SQL] Fix logging warn -> debug.
2014-08-05 22:30:32 -07:00
Reynold Xin b70bae40eb [SQL] Tighten the visibility of various SQLConf methods and renamed setter/getters
Author: Reynold Xin <rxin@apache.org>

Closes #1794 from rxin/sql-conf and squashes the following commits:

3ac11ef [Reynold Xin] getAllConfs return an immutable Map instead of an Array.
4b19d6c [Reynold Xin] Tighten the visibility of various SQLConf methods and renamed setter/getters.
2014-08-05 22:29:19 -07:00
Michael Armbrust 1d70c4f66d [SPARK-2866][SQL] Support attributes in ORDER BY that aren't in SELECT
Minor refactoring to allow resolution either using a nodes input or output.

Author: Michael Armbrust <michael@databricks.com>

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

237f580 [Michael Armbrust] style
74d833b [Michael Armbrust] newline
705d963 [Michael Armbrust] Add a rule for resolving ORDER BY expressions that reference attributes not present in the SELECT clause.
82cabda [Michael Armbrust] Generalize attribute resolution.
2014-08-05 20:55:02 -07:00
Yin Huai 69ec678d3a [SPARK-2854][SQL] Finalize _acceptable_types in pyspark.sql
This PR aims to finalize accepted data value types in Python RDDs provided to Python `applySchema`.

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

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

Closes #1793 from yhuai/SPARK-2854 and squashes the following commits:

32f0708 [Yin Huai] LongType only accepts long values.
c2b23dd [Yin Huai] Do data type conversions based on the specified Spark SQL data type.
2014-08-05 18:56:10 -07:00
Cheng Lian d0ae3f3912 [SPARK-2650][SQL] Try to partially fix SPARK-2650 by adjusting initial buffer size and reducing memory allocation
JIRA issue: [SPARK-2650](https://issues.apache.org/jira/browse/SPARK-2650)

Please refer to [comments](https://issues.apache.org/jira/browse/SPARK-2650?focusedCommentId=14084397&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-14084397) of SPARK-2650 for some other details.

This PR adjusts the initial in-memory columnar buffer size to 1MB, same as the default value of Shark's `shark.column.partitionSize.mb` property when running in local mode. Will add Shark style partition size estimation in another PR.

Also, before this PR, `NullableColumnBuilder` copies the whole buffer to add the null positions section, and then `CompressibleColumnBuilder` copies and compresses the buffer again, even if compression is disabled (`PassThrough` compression scheme is used to disable compression). In this PR the first buffer copy is eliminated to reduce memory consumption.

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

Closes #1769 from liancheng/spark-2650 and squashes the following commits:

88a042e [Cheng Lian] Fixed method visibility and removed dead code
001f2e5 [Cheng Lian] Try fixing SPARK-2650 by adjusting initial buffer size and reducing memory allocation
2014-08-05 18:50:37 -07:00
wangfei d94f5990e5 [sql] rename project name in pom.xml of hive-thriftserver module
module spark-hive-thriftserver_2.10 and spark-hive_2.10 both named "Spark Project Hive" in pom.xml, so rename spark-hive-thriftserver_2.10 project name to "Spark Project Hive Thrift Server"

Author: wangfei <wangfei1@huawei.com>

Closes #1789 from scwf/patch-1 and squashes the following commits:

ca1f5e9 [wangfei] [sql] rename module name of hive-thriftserver
2014-08-05 18:30:02 -07:00
Michael Armbrust 6e821e3d1a [SPARK-2860][SQL] Fix coercion of CASE WHEN.
Author: Michael Armbrust <michael@databricks.com>

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

126006d [Michael Armbrust] better error message
2fe357f [Michael Armbrust] Fix coercion of CASE WHEN.
2014-08-05 11:17:50 -07:00
Yin Huai e139e2be60 [SPARK-2783][SQL] Basic support for analyze in HiveContext
JIRA: https://issues.apache.org/jira/browse/SPARK-2783

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

Closes #1741 from yhuai/analyzeTable and squashes the following commits:

7bb5f02 [Yin Huai] Use sql instead of hql.
4d09325 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable
e3ebcd4 [Yin Huai] Renaming.
c170f4e [Yin Huai] Do not use getContentSummary.
62393b6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable
db233a6 [Yin Huai] Trying to debug jenkins...
fee84f0 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable
f0501f3 [Yin Huai] Fix compilation error.
24ad391 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable
8918140 [Yin Huai] Wording.
23df227 [Yin Huai] Add a simple analyze method to get the size of a table and update the "totalSize" property of this table in the Hive metastore.
2014-08-03 14:54:41 -07:00
Cheng Lian ac33cbbf33 [SPARK-2814][SQL] HiveThriftServer2 throws NPE when executing native commands
JIRA issue: [SPARK-2814](https://issues.apache.org/jira/browse/SPARK-2814)

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

Closes #1753 from liancheng/spark-2814 and squashes the following commits:

c74a3b2 [Cheng Lian] Fixed SPARK-2814
2014-08-03 12:34:46 -07:00
Michael Armbrust 236dfac676 [SPARK-2784][SQL] Deprecate hql() method in favor of a config option, 'spark.sql.dialect'
Many users have reported being confused by the distinction between the `sql` and `hql` methods.  Specifically, many users think that `sql(...)` cannot be used to read hive tables.  In this PR I introduce a new configuration option `spark.sql.dialect` that picks which dialect with be used for parsing.  For SQLContext this must be set to `sql`.  In `HiveContext` it defaults to `hiveql` but can also be set to `sql`.

The `hql` and `hiveql` methods continue to act the same but are now marked as deprecated.

**This is a possibly breaking change for some users unless they set the dialect manually, though this is unlikely.**

For example: `hiveContex.sql("SELECT 1")` will now throw a parsing exception by default.

Author: Michael Armbrust <michael@databricks.com>

Closes #1746 from marmbrus/sqlLanguageConf and squashes the following commits:

ad375cc [Michael Armbrust] Merge remote-tracking branch 'apache/master' into sqlLanguageConf
20c43f8 [Michael Armbrust] override function instead of just setting the value
7e4ae93 [Michael Armbrust] Deprecate hql() method in favor of a config option, 'spark.sql.dialect'
2014-08-03 12:28:29 -07:00
Michael Armbrust 1a8043739d [SPARK-2739][SQL] Rename registerAsTable to registerTempTable
There have been user complaints that the difference between `registerAsTable` and `saveAsTable` is too subtle.  This PR addresses this by renaming `registerAsTable` to `registerTempTable`, which more clearly reflects what is happening.  `registerAsTable` remains, but will cause a deprecation warning.

Author: Michael Armbrust <michael@databricks.com>

Closes #1743 from marmbrus/registerTempTable and squashes the following commits:

d031348 [Michael Armbrust] Merge remote-tracking branch 'apache/master' into registerTempTable
4dff086 [Michael Armbrust] Fix .java files too
89a2f12 [Michael Armbrust] Merge remote-tracking branch 'apache/master' into registerTempTable
0b7b71e [Michael Armbrust] Rename registerAsTable to registerTempTable
2014-08-02 18:27:04 -07:00
Cheng Lian 866cf1f822 [SPARK-2729][SQL] Added test case for SPARK-2729
This is a follow up of #1636.

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

Closes #1738 from liancheng/test-for-spark-2729 and squashes the following commits:

b13692a [Cheng Lian] Added test case for SPARK-2729
2014-08-02 17:12:49 -07:00
Michael Armbrust 198df11f1a [SPARK-2785][SQL] Remove assertions that throw when users try unsupported Hive commands.
Author: Michael Armbrust <michael@databricks.com>

Closes #1742 from marmbrus/asserts and squashes the following commits:

5182d54 [Michael Armbrust] Remove assertions that throw when users try unsupported Hive commands.
2014-08-02 16:48:07 -07:00
Michael Armbrust 158ad0bba9 [SPARK-2097][SQL] UDF Support
This patch adds the ability to register lambda functions written in Python, Java or Scala as UDFs for use in SQL or HiveQL.

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: GuoQiang Li <witgo@qq.com>

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

422646b [GuoQiang Li] Remove scalalogging-slf4j dependency
2014-08-02 13:59:58 -07:00
Yin Huai 67bd8e3c21 [SQL] Set outputPartitioning of BroadcastHashJoin correctly.
I think we will not generate the plan triggering this bug at this moment. But, let me explain it...

Right now, we are using `left.outputPartitioning` as the `outputPartitioning` of a `BroadcastHashJoin`. We may have a wrong physical plan for cases like...
```sql
SELECT l.key, count(*)
FROM (SELECT key, count(*) as cnt
      FROM src
      GROUP BY key) l // This is buildPlan
JOIN r // This is the streamedPlan
ON (l.cnt = r.value)
GROUP BY l.key
```
Let's say we have a `BroadcastHashJoin` on `l` and `r`. For this case, we will pick `l`'s `outputPartitioning` for the `outputPartitioning`of the `BroadcastHashJoin` on `l` and `r`. Also, because the last `GROUP BY` is using `l.key` as the key, we will not introduce an `Exchange` for this aggregation. However, `r`'s outputPartitioning may not match the required distribution of the last `GROUP BY` and we fail to group data correctly.

JIRA is being reindexed. I will create a JIRA ticket once it is back online.

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

Closes #1735 from yhuai/BroadcastHashJoin and squashes the following commits:

96d9cb3 [Yin Huai] Set outputPartitioning correctly.
2014-08-02 13:16:41 -07:00
Anand Avati 08c095b664 [SPARK-1812] sql/catalyst - Provide explicit type information
For Scala 2.11 compatibility.

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

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

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

Author: Anand Avati <avati@redhat.com>

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

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

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

66a1641 [GuoQiang Li] IncompatibleResultTypeProblem
73a89ba [GuoQiang Li] Use the scala-logging wrapper instead of the directly sfl4j api.
2014-08-01 23:55:11 -07:00
Yin Huai 3822f33f3c [SPARK-2212][SQL] Hash Outer Join (follow-up bug fix).
We need to carefully set the ouputPartitioning of the HashOuterJoin Operator. Otherwise, we may not correctly handle nulls.

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

Closes #1721 from yhuai/SPARK-2212-BugFix and squashes the following commits:

ed5eef7 [Yin Huai] Correctly choosing outputPartitioning for the HashOuterJoin operator.
2014-08-01 18:52:01 -07:00
Davies Liu 880eabec37 [SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDD
Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes.

This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance.

root
 |-- field1: integer (nullable = true)
 |-- field2: string (nullable = true)
 |-- field3: struct (nullable = true)
 |    |-- field4: integer (nullable = true)
 |    |-- field5: array (nullable = true)
 |    |    |-- element: integer (containsNull = false)
 |-- field6: array (nullable = true)
 |    |-- element: struct (containsNull = false)
 |    |    |-- field7: string (nullable = true)

Then we can access them by row.field3.field5[0]  or row.field6[5].field7

It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType.

You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as:

ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1]))

Or you could use Row to create a class just like namedtuple, for example:

Person = Row("name", "age")
ctx.inferSchema(rdd.map(lambda x: Person(*x)))

Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details.

schema = StructType([StructField("name, StringType, True),
                                    StructType("age", IntegerType, True)])
ctx.applySchema(rdd, schema)

PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable.

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

Closes #1598 from davies/nested and squashes the following commits:

f1d15b6 [Davies Liu] verify schema with the first few rows
8852aaf [Davies Liu] check type of schema
abe9e6e [Davies Liu] address comments
61b2292 [Davies Liu] add @deprecated to pythonToJavaMap
1e5b801 [Davies Liu] improve cache of classes
51aa135 [Davies Liu] use Row to infer schema
e9c0d5c [Davies Liu] remove string typed schema
353a3f2 [Davies Liu] fix code style
63de8f8 [Davies Liu] fix typo
c79ca67 [Davies Liu] fix serialization of nested data
6b258b5 [Davies Liu] fix pep8
9d8447c [Davies Liu] apply schema provided by string of names
f5df97f [Davies Liu] refactor, address comments
9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python
84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested
0eaaf56 [Davies Liu] fix doc tests
b3559b4 [Davies Liu] use generated Row instead of namedtuple
c4ddc30 [Davies Liu] fix conflict between name of fields and variables
7f6f251 [Davies Liu] address all comments
d69d397 [Davies Liu] refactor
2cc2d45 [Davies Liu] refactor
182fb46 [Davies Liu] refactor
bc6e9e1 [Davies Liu] switch to new Schema API
547bf3e [Davies Liu] Merge branch 'master' into nested
a435b5a [Davies Liu] add docs and code refactor
2c8debc [Davies Liu] Merge branch 'master' into nested
644665a [Davies Liu] use tuple and namedtuple for schemardd
2014-08-01 18:47:41 -07:00
Cheng Hao c0b47bada3 [SPARK-2767] [SQL] SparkSQL CLI doens't output error message if query failed.
Author: Cheng Hao <hao.cheng@intel.com>

Closes #1686 from chenghao-intel/spark_sql_cli and squashes the following commits:

eb664cc [Cheng Hao] Output detailed failure message in console
93b0382 [Cheng Hao] Fix Bug of no output in cli if exception thrown internally
2014-08-01 11:42:05 -07:00
chutium 580c7011ca [SPARK-2729] [SQL] Forgot to match Timestamp type in ColumnBuilder
just a match forgot, found after SPARK-2710 , TimestampType can be used by a SchemaRDD generated from JDBC ResultSet

Author: chutium <teng.qiu@gmail.com>

Closes #1636 from chutium/SPARK-2729 and squashes the following commits:

71af77a [chutium] [SPARK-2729] [SQL] added Timestamp in NullableColumnAccessorSuite
39cf9f8 [chutium] [SPARK-2729] add Timestamp Type into ColumnBuilder TestSuite, ref. #1636
ab6ff97 [chutium] [SPARK-2729] Forgot to match Timestamp type in ColumnBuilder
2014-08-01 11:31:44 -07:00
Cheng Hao 4415722e91 [SQL][SPARK-2212]Hash Outer Join
This patch is to support the hash based outer join. Currently, outer join for big relations are resort to `BoradcastNestedLoopJoin`, which is super slow. This PR will create 2 hash tables for both relations in the same partition, which greatly reduce the table scans.

Here is the testing code that I used:
```
package org.apache.spark.sql.hive

import org.apache.spark.SparkContext
import org.apache.spark.SparkConf
import org.apache.spark.sql._

case class Record(key: String, value: String)

object JoinTablePrepare extends App {
  import TestHive2._

  val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"${i % 828193}", s"val_$i")))

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

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

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

object JoinTablePerformanceTest extends App {
  import TestHive2._

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

  val leftOuterJoin = "insert overwrite table result select a.key, b.value from a left outer join b on a.key=b.key"
  val rightOuterJoin = "insert overwrite table result select a.key, b.value from a right outer join b on a.key=b.key"
  val fullOuterJoin = "insert overwrite table result select a.key, b.value from a full outer join b on a.key=b.key"

  val results = ("LeftOuterJoin", benchmark(leftOuterJoin)) :: ("LeftOuterJoin", benchmark(leftOuterJoin)) ::
                ("RightOuterJoin", benchmark(rightOuterJoin)) :: ("RightOuterJoin", benchmark(rightOuterJoin)) ::
                ("FullOuterJoin", benchmark(fullOuterJoin)) :: ("FullOuterJoin", benchmark(fullOuterJoin)) :: Nil
  val explains = hql(s"explain $leftOuterJoin").collect ++ hql(s"explain $rightOuterJoin").collect ++ hql(s"explain $fullOuterJoin").collect
  println(explains.mkString(",\n"))
  results.foreach { case (prompt, result) => {
      println(s"$prompt: took ${result._1} ms (${result._2} records)")
    }
  }

  def benchmark(cmd: String) = {
    val begin = System.currentTimeMillis()
    val result = hql(cmd)
    val end = System.currentTimeMillis()
    val count = hql("select count(1) from result").collect.mkString("")
    ((end - begin), count)
  }
}
```
And the result as shown below:
```
[Physical execution plan:],
[InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
[ Project [key#95,value#98]],
[  HashOuterJoin [key#95], [key#97], LeftOuter, None],
[   Exchange (HashPartitioning [key#95], 20)],
[    HiveTableScan [key#95], (MetastoreRelation default, a, None), None],
[   Exchange (HashPartitioning [key#97], 20)],
[    HiveTableScan [key#97,value#98], (MetastoreRelation default, b, None), None],
[Physical execution plan:],
[InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
[ Project [key#102,value#105]],
[  HashOuterJoin [key#102], [key#104], RightOuter, None],
[   Exchange (HashPartitioning [key#102], 20)],
[    HiveTableScan [key#102], (MetastoreRelation default, a, None), None],
[   Exchange (HashPartitioning [key#104], 20)],
[    HiveTableScan [key#104,value#105], (MetastoreRelation default, b, None), None],
[Physical execution plan:],
[InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
[ Project [key#109,value#112]],
[  HashOuterJoin [key#109], [key#111], FullOuter, None],
[   Exchange (HashPartitioning [key#109], 20)],
[    HiveTableScan [key#109], (MetastoreRelation default, a, None), None],
[   Exchange (HashPartitioning [key#111], 20)],
[    HiveTableScan [key#111,value#112], (MetastoreRelation default, b, None), None]
LeftOuterJoin: took 16072 ms ([3000000] records)
LeftOuterJoin: took 14394 ms ([3000000] records)
RightOuterJoin: took 14802 ms ([3000000] records)
RightOuterJoin: took 14747 ms ([3000000] records)
FullOuterJoin: took 17715 ms ([6000000] records)
FullOuterJoin: took 17629 ms ([6000000] records)
```

Without this PR, the benchmark will run seems never end.

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

Closes #1147 from chenghao-intel/hash_based_outer_join and squashes the following commits:

65c599e [Cheng Hao] Fix issues with the community comments
72b1394 [Cheng Hao] Fix bug of stale value in joinedRow
55baef7 [Cheng Hao] Add HashOuterJoin
2014-08-01 11:27:12 -07:00
Yin Huai c41fdf04f4 [SPARK-2179][SQL] A minor refactoring Java data type APIs (2179 follow-up).
It is a follow-up PR of SPARK-2179 (https://issues.apache.org/jira/browse/SPARK-2179). It makes package names of data type APIs more consistent across languages (Scala: `org.apache.spark.sql`, Java: `org.apache.spark.sql.api.java`, Python: `pyspark.sql`).

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

Closes #1712 from yhuai/javaDataType and squashes the following commits:

62eb705 [Yin Huai] Move package-info.
add4bcb [Yin Huai] Make the package names of data type classes consistent across languages by moving all Java data type classes to package sql.api.java.
2014-08-01 11:14:53 -07:00
GuoQiang Li 9998efab96 SPARK-2766: ScalaReflectionSuite throw an llegalArgumentException in JDK 6
Author: GuoQiang Li <witgo@qq.com>

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

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

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

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

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

00d72fd [Yin Huai] Use scala.collection.Map.
2014-07-31 21:02:11 -07:00