Commit graph

296 commits

Author SHA1 Message Date
Michael Armbrust 3a845d3c04 [SQL] Update Hive test harness for Hive 12 and 13
As part of the upgrade I also copy the newest version of the query tests, and whitelist a bunch of new ones that are now passing.

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: Liquan Pei <liquanpei@gmail.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

The `ExtendedHiveQlParser` now only handle Hive specific extensions.

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

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

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

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

Author : ravipesala ravindra.pesalahuawei.com

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

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

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

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

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

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

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

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

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

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

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

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

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

JoshRosen davies Please help review PySpark related changes, thanks!

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

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

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

    SELECT * from person WHERE FALSE;

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

And current optimizer cannot optimize the case NOT is duplicated like

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

The filter rule above should be simplified

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

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

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

Author : Venkata Ramana Gollamudi ramana.gollamudihuawei.com

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

Author : ravipesala ravindra.pesalahuawei.com

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

TODO

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

## Micro benchmark

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

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

Speedup:

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

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

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

Before:

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

After:

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: William Benton <willb@redhat.com>

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

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

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

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

1. Batch pruning instead of partition pruning

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

1. More filters are supported

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: William Benton <willb@redhat.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

object DataPrepare extends App {
  import TestHive2._

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

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

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

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

object PerformanceTest extends App {
  import TestHive2._

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: Sean Owen <srowen@gmail.com>

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: GuoQiang Li <witgo@qq.com>

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

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

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

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

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

Author: Anand Avati <avati@redhat.com>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

6f295a2 [Aaron Staple] Fix typos in comment about ExprId.
8566467 [Aaron Staple] Fix off by one column indentation in SqlParser.
2014-07-29 01:35:26 -07:00
Cheng Lian a7a9d14479 [SPARK-2410][SQL] Merging Hive Thrift/JDBC server (with Maven profile fix)
JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410)

Another try for #1399 & #1600. Those two PR breaks Jenkins builds because we made a separate profile `hive-thriftserver` in sub-project `assembly`, but the `hive-thriftserver` module is defined outside the `hive-thriftserver` profile. Thus every time a pull request that doesn't touch SQL code will also execute test suites defined in `hive-thriftserver`, but tests fail because related .class files are not included in the assembly jar.

In the most recent commit, module `hive-thriftserver` is moved into its own profile to fix this problem. All previous commits are squashed for clarity.

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

Closes #1620 from liancheng/jdbc-with-maven-fix and squashes the following commits:

629988e [Cheng Lian] Moved hive-thriftserver module definition into its own profile
ec3c7a7 [Cheng Lian] Cherry picked the Hive Thrift server
2014-07-28 12:07:30 -07:00
Patrick Wendell e5bbce9a60 Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"
This reverts commit f6ff2a61d0.
2014-07-27 18:46:58 -07:00
Cheng Lian f6ff2a61d0 [SPARK-2410][SQL] Merging Hive Thrift/JDBC server
(This is a replacement of #1399, trying to fix potential `HiveThriftServer2` port collision between parallel builds. Please refer to [these comments](https://github.com/apache/spark/pull/1399#issuecomment-50212572) for details.)

JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410)

Merging the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc).

Thanks chenghao-intel for his initial contribution of the Spark SQL CLI.

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

Closes #1600 from liancheng/jdbc and squashes the following commits:

ac4618b [Cheng Lian] Uses random port for HiveThriftServer2 to avoid collision with parallel builds
090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR
21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs
fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd]
199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver
1083e9d [Cheng Lian] Fixed failed test suites
7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic
9cc0f06 [Cheng Lian] Starts beeline with spark-submit
cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile
061880f [Cheng Lian] Addressed all comments by @pwendell
7755062 [Cheng Lian] Adapts test suites to spark-submit settings
40bafef [Cheng Lian] Fixed more license header issues
e214aab [Cheng Lian] Added missing license headers
b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh
f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft
3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit
a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit
61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit
2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server
2014-07-27 13:03:38 -07:00
Michael Armbrust 8904791230 [SPARK-2659][SQL] Fix division semantics for hive
Author: Michael Armbrust <michael@databricks.com>

Closes #1557 from marmbrus/fixDivision and squashes the following commits:

b85077f [Michael Armbrust] Fix unit tests.
af98f29 [Michael Armbrust] Change DIV to long type
0c29ae8 [Michael Armbrust] Fix division semantics for hive
2014-07-25 19:17:49 -07:00
Michael Armbrust afd757a241 Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"
This reverts commit 06dc0d2c6b.

#1399 is making Jenkins fail.  We should investigate and put this back after its passing tests.

Author: Michael Armbrust <michael@databricks.com>

Closes #1594 from marmbrus/revertJDBC and squashes the following commits:

59748da [Michael Armbrust] Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"
2014-07-25 15:36:57 -07:00
Cheng Lian 06dc0d2c6b [SPARK-2410][SQL] Merging Hive Thrift/JDBC server
JIRA issue:

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

Cherry picked the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc).

(Thanks chenghao-intel for his initial contribution of the Spark SQL CLI.)

TODO

- [x] Use `spark-submit` to launch the server, the CLI and beeline
- [x] Migration guideline draft for Shark users

----

Hit by a bug in `SparkSubmitArguments` while working on this PR: all application options that are recognized by `SparkSubmitArguments` are stolen as `SparkSubmit` options. For example:

```bash
$ spark-submit --class org.apache.hive.beeline.BeeLine spark-internal --help
```

This actually shows usage information of `SparkSubmit` rather than `BeeLine`.

~~Fixed this bug here since the `spark-internal` related stuff also touches `SparkSubmitArguments` and I'd like to avoid conflict.~~

**UPDATE** The bug mentioned above is now tracked by [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678). Decided to revert changes to this bug since it involves more subtle considerations and worth a separate PR.

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

Closes #1399 from liancheng/thriftserver and squashes the following commits:

090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR
21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs
fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd]
199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver
1083e9d [Cheng Lian] Fixed failed test suites
7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic
9cc0f06 [Cheng Lian] Starts beeline with spark-submit
cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile
061880f [Cheng Lian] Addressed all comments by @pwendell
7755062 [Cheng Lian] Adapts test suites to spark-submit settings
40bafef [Cheng Lian] Fixed more license header issues
e214aab [Cheng Lian] Added missing license headers
b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh
f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft
3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit
a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit
61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit
2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server
2014-07-25 12:20:49 -07:00
Cheng Hao 184aa1c6c0 [SPARK-2665] [SQL] Add EqualNS & Unit Tests
Hive Supports the operator "<=>", which returns same result with EQUAL(=) operator for non-null operands, but returns TRUE if both are NULL, FALSE if one of the them is NULL.

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

Closes #1570 from chenghao-intel/equalns and squashes the following commits:

8d6c789 [Cheng Hao] Remove the test case orc_predicate_pushdown
5b2ca88 [Cheng Hao] Add cases into whitelist
8e66cdd [Cheng Hao] Rename the EqualNSTo ==> EqualNullSafe
7af4b0b [Cheng Hao] Add EqualNS & Unit Tests
2014-07-25 01:30:22 -07:00
Michael Armbrust 78d18fdbaa [SPARK-2658][SQL] Add rule for true = 1.
Author: Michael Armbrust <michael@databricks.com>

Closes #1556 from marmbrus/fixBooleanEqualsOne and squashes the following commits:

ad8edd4 [Michael Armbrust] Add rule for true = 1 and false = 0.
2014-07-23 22:52:49 -07:00
William Benton e060d3ee2d SPARK-2226: [SQL] transform HAVING clauses with aggregate expressions that aren't in the aggregation list
This change adds an analyzer rule to
  1. find expressions in `HAVING` clause filters that depend on unresolved attributes,
  2. push these expressions down to the underlying aggregates, and then
  3. project them away above the filter.

It also enables the `HAVING` queries in the Hive compatibility suite.

Author: William Benton <willb@redhat.com>

Closes #1497 from willb/spark-2226 and squashes the following commits:

92c9a93 [William Benton] Removed unnecessary import
f1d4f34 [William Benton] Cleanups missed in prior commit
0e1624f [William Benton] Incorporated suggestions from @marmbrus; thanks!
541d4ee [William Benton] Cleanups from review
5a12647 [William Benton] Explanatory comments and stylistic cleanups.
c7f2b2c [William Benton] Whitelist HAVING queries.
29a26e3 [William Benton] Added rule to handle unresolved attributes in HAVING clauses (SPARK-2226)
2014-07-23 16:25:32 -07:00
Takuya UESHIN 1b790cf775 [SPARK-2588][SQL] Add some more DSLs.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #1491 from ueshin/issues/SPARK-2588 and squashes the following commits:

43d0a46 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-2588
1023ea0 [Takuya UESHIN] Modify tests to use DSLs.
2310bf1 [Takuya UESHIN] Add some more DSLs.
2014-07-23 14:47:23 -07:00
Cheng Lian cd273a2381 [SPARK-2190][SQL] Specialized ColumnType for Timestamp
JIRA issue: [SPARK-2190](https://issues.apache.org/jira/browse/SPARK-2190)

Added specialized in-memory column type for `Timestamp`. Whitelisted all timestamp related Hive tests except `timestamp_udf`, which is timezone sensitive.

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

Closes #1440 from liancheng/timestamp-column-type and squashes the following commits:

e682175 [Cheng Lian] Enabled more timezone sensitive Hive tests.
53a358f [Cheng Lian] Fixed failed test suites
01b592d [Cheng Lian] Fixed SimpleDateFormat thread safety issue
2a59343 [Cheng Lian] Removed timezone sensitive Hive timestamp tests
45dd05d [Cheng Lian] Added Timestamp specific in-memory columnar representation
2014-07-21 00:46:28 -07:00
chutium 2a732110d4 SPARK-2407: Added Parser of SQL SUBSTR()
follow-up of #1359

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

Closes #1442 from chutium/master and squashes the following commits:

b49cc8a [chutium] SPARK-2407: Added Parser of SQL SUBSTRING() #1442
9a60ccf [chutium] SPARK-2407: Added Parser of SQL SUBSTR() #1442
06e933b [chutium] Merge https://github.com/apache/spark
c870172 [chutium] Merge https://github.com/apache/spark
094f773 [chutium] Merge https://github.com/apache/spark
88cb37d [chutium] Merge https://github.com/apache/spark
1de83a7 [chutium] SPARK-2407: Added Parse of SQL SUBSTR()
2014-07-19 11:04:41 -05:00
Takuya UESHIN 3a1709fa55 [SPARK-2535][SQL] Add StringComparison case to NullPropagation.
`StringComparison` expressions including `null` literal cases could be added to `NullPropagation`.

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

Closes #1451 from ueshin/issues/SPARK-2535 and squashes the following commits:

e99c237 [Takuya UESHIN] Add some tests.
8f9b984 [Takuya UESHIN] Add StringComparison case to NullPropagation.
2014-07-18 16:24:00 -05:00
Takuya UESHIN cc965eea51 [SPARK-2518][SQL] Fix foldability of Substring expression.
This is a follow-up of #1428.

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

Closes #1432 from ueshin/issues/SPARK-2518 and squashes the following commits:

37d1ace [Takuya UESHIN] Fix foldability of Substring expression.
2014-07-16 11:13:38 -07:00
Reynold Xin 1c5739f685 [SQL] Cleaned up ConstantFolding slightly.
Moved couple rules out of NullPropagation and added more comments.

Author: Reynold Xin <rxin@apache.org>

Closes #1430 from rxin/sql-folding-rule and squashes the following commits:

7f9a197 [Reynold Xin] Updated documentation for ConstantFolding.
7f8cf61 [Reynold Xin] [SQL] Cleaned up ConstantFolding slightly.
2014-07-16 10:55:47 -07:00
Takuya UESHIN 632fb3d9a9 [SPARK-2504][SQL] Fix nullability of Substring expression.
This is a follow-up of #1359 with nullability narrowing.

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

Closes #1426 from ueshin/issues/SPARK-2504 and squashes the following commits:

5157832 [Takuya UESHIN] Remove unnecessary white spaces.
80958ac [Takuya UESHIN] Fix nullability of Substring expression.
2014-07-15 22:43:48 -07:00
Takuya UESHIN 9b38b7c713 [SPARK-2509][SQL] Add optimization for Substring.
`Substring` including `null` literal cases could be added to `NullPropagation`.

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

Closes #1428 from ueshin/issues/SPARK-2509 and squashes the following commits:

d9eb85f [Takuya UESHIN] Add Substring cases to NullPropagation.
2014-07-15 22:35:34 -07:00
Zongheng Yang c2048a5165 [SPARK-2498] [SQL] Synchronize on a lock when using scala reflection inside data type objects.
JIRA ticket: https://issues.apache.org/jira/browse/SPARK-2498

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

Closes #1423 from concretevitamin/scala-ref-catalyst and squashes the following commits:

325a149 [Zongheng Yang] Synchronize on a lock when initializing data type objects in Catalyst.
2014-07-15 17:58:28 -07:00
William Benton 61de65bc69 SPARK-2407: Added internal implementation of SQL SUBSTR()
This replaces the Hive UDF for SUBSTR(ING) with an implementation in Catalyst
and adds tests to verify correct operation.

Author: William Benton <willb@redhat.com>

Closes #1359 from willb/internalSqlSubstring and squashes the following commits:

ccedc47 [William Benton] Fixed too-long line.
a30a037 [William Benton] replace view bounds with implicit parameters
ec35c80 [William Benton] Adds fixes from review:
4f3bfdb [William Benton] Added internal implementation of SQL SUBSTR()
2014-07-15 14:11:57 -07:00
Yin Huai 8af46d5846 [SPARK-2474][SQL] For a registered table in OverrideCatalog, the Analyzer failed to resolve references in the format of "tableName.fieldName"
Please refer to JIRA (https://issues.apache.org/jira/browse/SPARK-2474) for how to reproduce the problem and my understanding of the root cause.

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

Closes #1406 from yhuai/SPARK-2474 and squashes the following commits:

96b1627 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2474
af36d65 [Yin Huai] Fix comment.
be86ba9 [Yin Huai] Correct SQL console settings.
c43ad00 [Yin Huai] Wrap the relation in a Subquery named by the table name in OverrideCatalog.lookupRelation.
a5c2145 [Yin Huai] Support sql/console.
2014-07-15 14:06:45 -07:00
Michael Armbrust 1a7d7cc85f [SPARK-2405][SQL] Reusue same byte buffers when creating new instance of InMemoryRelation
Reuse byte buffers when creating unique attributes for multiple instances of an InMemoryRelation in a single query plan.

Author: Michael Armbrust <michael@databricks.com>

Closes #1332 from marmbrus/doubleCache and squashes the following commits:

4a19609 [Michael Armbrust] Clean up concurrency story by calculating buffersn the constructor.
b39c931 [Michael Armbrust] Allocations are kind of a side effect.
f67eff7 [Michael Armbrust] Reusue same byte buffers when creating new instance of InMemoryRelation
2014-07-12 12:13:32 -07:00
Takuya UESHIN f62c427289 [SPARK-2431][SQL] Refine StringComparison and related codes.
Refine `StringComparison` and related codes as follows:
- `StringComparison` could be similar to `StringRegexExpression` or `CaseConversionExpression`.
- Nullability of `StringRegexExpression` could depend on children's nullabilities.
- Add a case that the like condition includes no wildcard to `LikeSimplification`.

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

Closes #1357 from ueshin/issues/SPARK-2431 and squashes the following commits:

77766f5 [Takuya UESHIN] Add a case that the like condition includes no wildcard to LikeSimplification.
b9da9d2 [Takuya UESHIN] Fix nullability of StringRegexExpression.
680bb72 [Takuya UESHIN] Refine StringComparison.
2014-07-10 19:20:00 -07:00
Prashant Sharma 628932b8d0 [SPARK-1776] Have Spark's SBT build read dependencies from Maven.
Patch introduces the new way of working also retaining the existing ways of doing things.

For example build instruction for yarn in maven is
`mvn -Pyarn -PHadoop2.2 clean package -DskipTests`
in sbt it can become
`MAVEN_PROFILES="yarn, hadoop-2.2" sbt/sbt clean assembly`
Also supports
`sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 clean assembly`

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

Closes #772 from ScrapCodes/sbt-maven and squashes the following commits:

a8ac951 [Prashant Sharma] Updated sbt version.
62b09bb [Prashant Sharma] Improvements.
fa6221d [Prashant Sharma] Excluding sql from mima
4b8875e [Prashant Sharma] Sbt assembly no longer builds tools by default.
72651ca [Prashant Sharma] Addresses code reivew comments.
acab73d [Prashant Sharma] Revert "Small fix to run-examples script."
ac4312c [Prashant Sharma] Revert "minor fix"
6af91ac [Prashant Sharma] Ported oldDeps back. + fixes issues with prev commit.
65cf06c [Prashant Sharma] Servelet API jars mess up with the other servlet jars on the class path.
446768e [Prashant Sharma] minor fix
89b9777 [Prashant Sharma] Merge conflicts
d0a02f2 [Prashant Sharma] Bumped up pom versions, Since the build now depends on pom it is better updated there. + general cleanups.
dccc8ac [Prashant Sharma] updated mima to check against 1.0
a49c61b [Prashant Sharma] Fix for tools jar
a2f5ae1 [Prashant Sharma] Fixes a bug in dependencies.
cf88758 [Prashant Sharma] cleanup
9439ea3 [Prashant Sharma] Small fix to run-examples script.
96cea1f [Prashant Sharma] SPARK-1776 Have Spark's SBT build read dependencies from Maven.
36efa62 [Patrick Wendell] Set project name in pom files and added eclipse/intellij plugins.
4973dbd [Patrick Wendell] Example build using pom reader.
2014-07-10 11:03:37 -07:00
Michael Armbrust cc3e0a14da [SPARK-2395][SQL] Optimize common LIKE patterns.
Author: Michael Armbrust <michael@databricks.com>

Closes #1325 from marmbrus/slowLike and squashes the following commits:

023c3eb [Michael Armbrust] add comment.
8b421c2 [Michael Armbrust] Handle the case where the final % is actually escaped.
d34d37e [Michael Armbrust] add periods.
3bbf35f [Michael Armbrust] Roll back changes to SparkBuild
53894b1 [Michael Armbrust] Fix grammar.
4094462 [Michael Armbrust] Fix grammar.
6d3d0a0 [Michael Armbrust] Optimize common LIKE patterns.
2014-07-08 10:36:18 -07:00
Yanjie Gao 50561f4396 [SPARK-2235][SQL]Spark SQL basicOperator add Intersect operator
Hi all,
I want to submit a basic operator Intersect
For example , in sql case
select * from table1
intersect
select * from table2
So ,i want use this operator support this function in Spark SQL
This operator will return the  the intersection of SparkPlan child table RDD .
JIRA:https://issues.apache.org/jira/browse/SPARK-2235

Author: Yanjie Gao <gaoyanjie55@163.com>
Author: YanjieGao <396154235@qq.com>

Closes #1150 from YanjieGao/patch-5 and squashes the following commits:

4629afe [YanjieGao] reformat the code
bdc2ac0 [YanjieGao] reformat the code as Michael's suggestion
3b29ad6 [YanjieGao] Merge remote branch 'upstream/master' into patch-5
1cfbfe6 [YanjieGao] refomat some files
ea78f33 [YanjieGao] resolve conflict and add annotation on basicOperator and remove HiveQl
0c7cca5 [YanjieGao] modify format problem
a802ca8 [YanjieGao] Merge remote branch 'upstream/master' into patch-5
5e374c7 [YanjieGao] resolve conflict in SparkStrategies and basicOperator
f7961f6 [Yanjie Gao] update the line less than
bdc4a05 [Yanjie Gao] Update basicOperators.scala
0b49837 [Yanjie Gao] delete the annotation
f1288b4 [Yanjie Gao] delete annotation
e2b64be [Yanjie Gao] Update basicOperators.scala
4dd453e [Yanjie Gao] Update SQLQuerySuite.scala
790765d [Yanjie Gao] Update SparkStrategies.scala
ac73e60 [Yanjie Gao] Update basicOperators.scala
d4ac5e5 [Yanjie Gao] Update HiveQl.scala
61e88e7 [Yanjie Gao] Update SqlParser.scala
469f099 [Yanjie Gao] Update basicOperators.scala
e5bff61 [Yanjie Gao] Spark SQL basicOperator add Intersect operator
2014-07-07 19:40:04 -07:00
Yin Huai c0b4cf097d [SPARK-2339][SQL] SQL parser in sql-core is case sensitive, but a table alias is converted to lower case when we create Subquery
Reported by http://apache-spark-user-list.1001560.n3.nabble.com/Spark-SQL-Join-throws-exception-td8599.html
After we get the table from the catalog, because the table has an alias, we will temporarily insert a Subquery. Then, we convert the table alias to lower case no matter if the parser is case sensitive or not.
To see the issue ...
```
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext.createSchemaRDD

case class Person(name: String, age: Int)

val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt))
people.registerAsTable("people")

sqlContext.sql("select PEOPLE.name from people PEOPLE")
```
The plan is ...
```
== Query Plan ==
Project ['PEOPLE.name]
 ExistingRdd [name#0,age#1], MapPartitionsRDD[4] at mapPartitions at basicOperators.scala:176
```
You can find that `PEOPLE.name` is not resolved.

This PR introduces three changes.
1.  If a table has an alias, the catalog will not lowercase the alias. If a lowercase alias is needed, the analyzer will do the work.
2.  A catalog has a new val caseSensitive that indicates if this catalog is case sensitive or not. For example, a SimpleCatalog is case sensitive, but
3.  Corresponding unit tests.
With this PR, case sensitivity of database names and table names is handled by the catalog. Case sensitivity of other identifiers are handled by the analyzer.

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

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

Closes #1317 from yhuai/SPARK-2339 and squashes the following commits:

12d8006 [Yin Huai] Handling case sensitivity correctly. This patch introduces three changes. 1. If a table has an alias, the catalog will not lowercase the alias. If a lowercase alias is needed, the analyzer will do the work. 2. A catalog has a new val caseSensitive that indicates if this catalog is case sensitive or not. For example, a SimpleCatalog is case sensitive, but 3. Corresponding unit tests. With this patch, case sensitivity of database names and table names is handled by the catalog. Case sensitivity of other identifiers is handled by the analyzer.
2014-07-07 17:01:44 -07:00
Takuya UESHIN 9d5ecf8205 [SPARK-2327] [SQL] Fix nullabilities of Join/Generate/Aggregate.
Fix nullabilities of `Join`/`Generate`/`Aggregate` because:
- Output attributes of opposite side of `OuterJoin` should be nullable.
- Output attributes of generater side of `Generate` should be nullable if `join` is `true` and `outer` is `true`.
- `AttributeReference` of `computedAggregates` of `Aggregate` should be the same as `aggregateExpression`'s.

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

Closes #1266 from ueshin/issues/SPARK-2327 and squashes the following commits:

3ace83a [Takuya UESHIN] Add withNullability to Attribute and use it to change nullabilities.
df1ae53 [Takuya UESHIN] Modify nullabilize to leave attribute if not resolved.
799ce56 [Takuya UESHIN] Add nullabilization to Generate of SparkPlan.
a0fc9bc [Takuya UESHIN] Fix scalastyle errors.
0e31e37 [Takuya UESHIN] Fix Aggregate resultAttribute nullabilities.
09532ec [Takuya UESHIN] Fix Generate output nullabilities.
f20f196 [Takuya UESHIN] Fix Join output nullabilities.
2014-07-05 11:51:48 -07:00
Takuya UESHIN 3da8df939e [SPARK-2366] [SQL] Add column pruning for the right side of LeftSemi join.
The right side of `LeftSemi` join needs columns only used in join condition.

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

Closes #1301 from ueshin/issues/SPARK-2366 and squashes the following commits:

7677a39 [Takuya UESHIN] Update comments.
786d3a0 [Takuya UESHIN] Rename method name.
e0957b1 [Takuya UESHIN] Add column pruning for the right side of LeftSemi join.
2014-07-05 11:48:08 -07:00
Yanjie Gao 5dadda8645 [SPARK-2234][SQL]Spark SQL basicOperators add Except operator
Hi all,
I want to submit a Except operator in basicOperators.scala
In SQL case.SQL support two table do except operator.
select * from table1
except
select * from table2
This operator support the substract function .Return an table with the elements from `this` that are not in `other`.This operator should limit the input SparkPlan Seq only has two member.The check will later support
JIRA:https://issues.apache.org/jira/browse/SPARK-2234

Author: Yanjie Gao <gaoyanjie55@163.com>
Author: YanjieGao <396154235@qq.com>
Author: root <root@node4.(none)>
Author: gaoyanjie <gaoyanjie55@163.com>

Closes #1151 from YanjieGao/patch-6 and squashes the following commits:

f19f899 [YanjieGao] add a new blank line in basicoperators.scala
2ff7d73 [YanjieGao] resolve the identation in SqlParser and SparkStrategies
fdb5227 [YanjieGao] Merge remote branch 'upstream/master' into patch-6
9940d19 [YanjieGao] make comment less than 100c
09c7413 [YanjieGao] pr 1151 SqlParser add cache ,basic Operator rename Except and modify comment
b4b5867 [root] Merge remote branch 'upstream/master' into patch-6
b4c3869 [Yanjie Gao] change SparkStrategies Sparkcontext to SqlContext
7e0ec29 [Yanjie Gao] delete multi test
7e7c83f [Yanjie Gao] delete conflict except
b01beb8 [YanjieGao] resolve conflict sparkstrategies and basicOperators
4dc8166 [YanjieGao] resolve conflict
fa68a98 [Yanjie Gao] Update joins.scala
8e6bb00 [Yanjie Gao] delete conflict except
dd9ba5e [Yanjie Gao] Update joins.scala
a0d4e73 [Yanjie Gao] delete skew join
60f5ddd [Yanjie Gao] update less than 100c
0e72233 [Yanjie Gao] update SQLQuerySuite on master branch
7f916b5 [Yanjie Gao] update execution/basicOperators on master branch
a28dece [Yanjie Gao] Update logical/basicOperators on master branch
a639935 [Yanjie Gao] Update SparkStrategies.scala
3bf7def [Yanjie Gao] update SqlParser on master branch
26f833f [Yanjie Gao] update SparkStrategies.scala on master branch
8dd063f [Yanjie Gao] Update logical/basicOperators on master branch
9847dcf [Yanjie Gao] update SqlParser on masterbranch
d6a4604 [Yanjie Gao] Update joins.scala
424c507 [Yanjie Gao] Update joins.scala
7680742 [Yanjie Gao] Update SqlParser.scala
a7193d8 [gaoyanjie] [SPARK-2234][SQL]Spark SQL basicOperators add Except operator #1151
5c8a224 [Yanjie Gao] update the line less than 100c
ee066b3 [Yanjie Gao] Update basicOperators.scala
32a80ab [Yanjie Gao] remove except in HiveQl
cf232eb [Yanjie Gao] update 1comment 2space3 left.out
f1ea3f3 [Yanjie Gao] remove comment
7ea9b91 [Yanjie Gao] remove annotation
7f3d613 [Yanjie Gao] update .map(_.copy())
670a1bb [Yanjie Gao] Update HiveQl.scala
3fe7746 [Yanjie Gao] Update SQLQuerySuite.scala
a36eb0a [Yanjie Gao] Update basicOperators.scala
7859e56 [Yanjie Gao] Update SparkStrategies.scala
052346d [Yanjie Gao] Subtract is conflict with Subtract(e1,e2)
aab3785 [Yanjie Gao] Update SQLQuerySuite.scala
4bf80b1 [Yanjie Gao] update subtract to except
4bdd520 [Yanjie Gao] Update SqlParser.scala
2d4bfbd [Yanjie Gao] Update SQLQuerySuite.scala
0808921 [Yanjie Gao] SQLQuerySuite
a8a1948 [Yanjie Gao] SparkStrategies
1fe96c0 [Yanjie Gao] HiveQl.scala update
3305e40 [Yanjie Gao] SqlParser
7a98c37 [Yanjie Gao] Update basicOperators.scala
cf5b9d0 [Yanjie Gao] Update basicOperators.scala
8945835 [Yanjie Gao] object SkewJoin extends Strategy
2b98962 [Yanjie Gao] Update SqlParser.scala
dd32980 [Yanjie Gao] update1
68815b2 [Yanjie Gao] Reformat the code style
4eb43ec [Yanjie Gao] Update basicOperators.scala
aa06072 [Yanjie Gao] Reformat the code sytle
2014-07-04 02:43:57 -07:00
Reynold Xin b3e768e154 [SPARK-2059][SQL] Add analysis checks
This replaces #1263 with a test case.

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

Closes #1265 from rxin/sql-analysis-error and squashes the following commits:

a639e01 [Reynold Xin] Added a test case for unresolved attribute analysis.
7371e1b [Reynold Xin] Merge pull request #1263 from marmbrus/analysisChecks
448c088 [Michael Armbrust] Add analysis checks
2014-07-04 00:53:41 -07:00
Yijie Shen a9b52e5623 [SPARK-2342] Evaluation helper's output type doesn't conform to input ty...
The function cast doesn't conform to the intention of "Those expressions are supposed to be in the same data type, and also the return type." comment

Author: Yijie Shen <henry.yijieshen@gmail.com>

Closes #1283 from yijieshen/master and squashes the following commits:

c7aaa4b [Yijie Shen] [SPARK-2342] Evaluation helper's output type doesn't conform to input type
2014-07-03 13:22:13 -07:00
Takuya UESHIN bc7041a42d [SPARK-2287] [SQL] Make ScalaReflection be able to handle Generic case classes.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #1226 from ueshin/issues/SPARK-2287 and squashes the following commits:

32ef7c3 [Takuya UESHIN] Add execution of `SHOW TABLES` before `TestHive.reset()`.
541dc8d [Takuya UESHIN] Merge branch 'master' into issues/SPARK-2287
fac5fae [Takuya UESHIN] Remove unnecessary method receiver.
d306e60 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-2287
7de5706 [Takuya UESHIN] Make ScalaReflection be able to handle Generic case classes.
2014-07-02 10:10:36 -07:00
Ximo Guanter Gonzalbez 5c6ec94da1 SPARK-2186: Spark SQL DSL support for simple aggregations such as SUM and AVG
**Description** This patch enables using the `.select()` function in SchemaRDD with functions such as `Sum`, `Count` and other.
**Testing** Unit tests added.

Author: Ximo Guanter Gonzalbez <ximo@tid.es>

Closes #1211 from edrevo/add-expression-support-in-select and squashes the following commits:

fe4a1e1 [Ximo Guanter Gonzalbez] Extend SQL DSL to functions
e1d344a [Ximo Guanter Gonzalbez] SPARK-2186: Spark SQL DSL support for simple aggregations such as SUM and AVG
2014-07-02 10:03:44 -07:00
CodingCat 6596392da0 update the comments in SqlParser
SqlParser has been case-insensitive after dab5439a08 was merged

Author: CodingCat <zhunansjtu@gmail.com>

Closes #1275 from CodingCat/master and squashes the following commits:

17931cd [CodingCat] update the comments in SqlParser
2014-07-01 20:37:10 -07:00
Cheng Hao 981bde9b05 [SQL]Extract the joinkeys from join condition
Extract the join keys from equality conditions, that can be evaluated using equi-join.

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

Closes #1190 from chenghao-intel/extract_join_keys and squashes the following commits:

4a1060a [Cheng Hao] Fix some of the small issues
ceb4924 [Cheng Hao] Remove the redundant pattern of join keys extraction
cec34e8 [Cheng Hao] Update the code style issues
dcc4584 [Cheng Hao] Extract the joinkeys from join condition
2014-06-26 19:18:11 -07:00
Takuya UESHIN e4899a2537 [SPARK-2254] [SQL] ScalaRefection should mark primitive types as non-nullable.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #1193 from ueshin/issues/SPARK-2254 and squashes the following commits:

cfd6088 [Takuya UESHIN] Modify ScalaRefection.schemaFor method to return nullability of Scala Type.
2014-06-25 23:55:31 -07:00
Zongheng Yang 9d824fed8c [SQL] SPARK-1800 Add broadcast hash join operator & associated hints.
This PR is based off Michael's [PR 734](https://github.com/apache/spark/pull/734) and includes a bunch of cleanups.

Moreover, this PR also
- makes `SparkLogicalPlan` take a `tableName: String`, which facilitates testing.
- moves join-related tests to a single file.

Author: Zongheng Yang <zongheng.y@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #1163 from concretevitamin/auto-broadcast-hash-join and squashes the following commits:

d0f4991 [Zongheng Yang] Fix bug in broadcast hash join & add test to cover it.
af080d7 [Zongheng Yang] Fix in joinIterators()'s next().
440d277 [Zongheng Yang] Fixes to imports; add back requiredChildDistribution (lost when merging)
208d5f6 [Zongheng Yang] Make LeftSemiJoinHash mix in HashJoin.
ad6c7cc [Zongheng Yang] Minor cleanups.
814b3bf [Zongheng Yang] Merge branch 'master' into auto-broadcast-hash-join
a8a093e [Zongheng Yang] Minor cleanups.
6fd8443 [Zongheng Yang] Cut down size estimation related stuff.
a4267be [Zongheng Yang] Add test for broadcast hash join and related necessary refactorings:
0e64b08 [Zongheng Yang] Scalastyle fix.
91461c2 [Zongheng Yang] Merge branch 'master' into auto-broadcast-hash-join
7c7158b [Zongheng Yang] Prototype of auto conversion to broadcast hash join.
0ad122f [Zongheng Yang] Merge branch 'master' into auto-broadcast-hash-join
3e5d77c [Zongheng Yang] WIP: giant and messy WIP.
a92ed0c [Michael Armbrust] Formatting.
76ca434 [Michael Armbrust] A simple strategy that broadcasts tables only when they are found in a configuration hint.
cf6b381 [Michael Armbrust] Split out generic logic for hash joins and create two concrete physical operators: BroadcastHashJoin and ShuffledHashJoin.
a8420ca [Michael Armbrust] Copy records in executeCollect to avoid issues with mutable rows.
2014-06-25 18:06:33 -07:00
Cheng Hao 133495d826 [SQL]Add base row updating methods for JoinedRow
This will be helpful in join operators.

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

Closes #1187 from chenghao-intel/joinedRow and squashes the following commits:

87c19e3 [Cheng Hao] Add base row set methods for JoinedRow
2014-06-24 19:07:02 -07:00
Reynold Xin 2f6a835e1a [SPARK-2218] rename Equals to EqualTo in Spark SQL expressions.
Due to the existence of scala.Equals, it is very error prone to name the expression Equals, especially because we use a lot of partial functions and pattern matching in the optimizer.

Note that this sits on top of #1144.

Author: Reynold Xin <rxin@apache.org>

Closes #1146 from rxin/equals and squashes the following commits:

f8583fd [Reynold Xin] Merge branch 'master' of github.com:apache/spark into equals
326b388 [Reynold Xin] Merge branch 'master' of github.com:apache/spark into equals
bd19807 [Reynold Xin] Rename EqualsTo to EqualTo.
81148d1 [Reynold Xin] [SPARK-2218] rename Equals to EqualsTo in Spark SQL expressions.
c4e543d [Reynold Xin] [SPARK-2210] boolean cast on boolean value should be removed.
2014-06-20 00:34:59 -07:00
Takuya UESHIN 3249528920 [SPARK-2196] [SQL] Fix nullability of CaseWhen.
`CaseWhen` should use `branches.length` to check if `elseValue` is provided or not.

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

Closes #1133 from ueshin/issues/SPARK-2196 and squashes the following commits:

510f12d [Takuya UESHIN] Add some tests.
dc25e8d [Takuya UESHIN] Fix nullable of CaseWhen to be nullable if the elseValue is nullable.
4f049cc [Takuya UESHIN] Fix nullability of CaseWhen.
2014-06-20 00:12:52 -07:00
Reynold Xin c55bbb49f7 [SPARK-2209][SQL] Cast shouldn't do null check twice.
Also took the chance to clean up cast a little bit. Too many arrows on each line before!

Author: Reynold Xin <rxin@apache.org>

Closes #1143 from rxin/cast and squashes the following commits:

dd006cb [Reynold Xin] Code review feedback.
c2b88ae [Reynold Xin] [SPARK-2209][SQL] Cast shouldn't do null check twice.
2014-06-20 00:01:19 -07:00
Reynold Xin 6175640973 [SPARK-2210] cast to boolean on boolean value gets turned into NOT((boolean_condition) = 0)
```
explain select cast(cast(key=0 as boolean) as boolean) aaa from src
```
should be
```
[Physical execution plan:]
[Project [(key#10:0 = 0) AS aaa#7]]
[ HiveTableScan [key#10], (MetastoreRelation default, src, None), None]
```

However, it is currently
```
[Physical execution plan:]
[Project [NOT((key#10=0) = 0) AS aaa#7]]
[ HiveTableScan [key#10], (MetastoreRelation default, src, None), None]
```

Author: Reynold Xin <rxin@apache.org>

Closes #1144 from rxin/booleancast and squashes the following commits:

c4e543d [Reynold Xin] [SPARK-2210] boolean cast on boolean value should be removed.
2014-06-19 23:58:23 -07:00
Andre Schumacher f479cf3743 SPARK-1293 [SQL] Parquet support for nested types
It should be possible to import and export data stored in Parquet's columnar format that contains nested types. For example:
```java
message AddressBook {
   required binary owner;
   optional group ownerPhoneNumbers {
      repeated binary array;
   }
   optional group contacts {
      repeated group array {
         required binary name;
         optional binary phoneNumber;
      }
   }
   optional group nameToApartmentNumber {
      repeated group map {
         required binary key;
         required int32 value;
      }
   }
}
```
The example could model a type (AddressBook) that contains records made of strings (owner), lists (ownerPhoneNumbers) and a table of contacts (e.g., a list of pairs or a map that can contain null values but keys must not be null). The list of tasks are as follows:

<h6>Implement support for converting nested Parquet types to Spark/Catalyst types:</h6>
- [x] Structs
- [x] Lists
- [x] Maps (note: currently keys need to be Strings)

<h6>Implement import (via ``parquetFile``) of nested Parquet types (first version in this PR)</h6>
- [x] Initial version

<h6>Implement export (via ``saveAsParquetFile``)</h6>
- [x] Initial version

<h6>Test support for AvroParquet, etc.</h6>
- [x] Initial testing of import of avro-generated Parquet data (simple + nested)

Example:
```scala
val data = TestSQLContext
  .parquetFile("input.dir")
  .toSchemaRDD
data.registerAsTable("data")
sql("SELECT owner, contacts[1].name, nameToApartmentNumber['John'] FROM data").collect()
```

Author: Andre Schumacher <andre.schumacher@iki.fi>
Author: Michael Armbrust <michael@databricks.com>

Closes #360 from AndreSchumacher/nested_parquet and squashes the following commits:

30708c8 [Andre Schumacher] Taking out AvroParquet test for now to remove Avro dependency
95c1367 [Andre Schumacher] Changes to ParquetRelation and its metadata
7eceb67 [Andre Schumacher] Review feedback
94eea3a [Andre Schumacher] Scalastyle
403061f [Andre Schumacher] Fixing some issues with tests and schema metadata
b8a8b9a [Andre Schumacher] More fixes to short and byte conversion
63d1b57 [Andre Schumacher] Cleaning up and Scalastyle
88e6bdb [Andre Schumacher] Attempting to fix loss of schema
37e0a0a [Andre Schumacher] Cleaning up
14c3fd8 [Andre Schumacher] Attempting to fix Spark-Parquet schema conversion
3e1456c [Michael Armbrust] WIP: Directly serialize catalyst attributes.
f7aeba3 [Michael Armbrust] [SPARK-1982] Support for ByteType and ShortType.
3104886 [Michael Armbrust] Nested Rows should be Rows, not Seqs.
3c6b25f [Andre Schumacher] Trying to reduce no-op changes wrt master
31465d6 [Andre Schumacher] Scalastyle: fixing commented out bottom
de02538 [Andre Schumacher] Cleaning up ParquetTestData
2f5a805 [Andre Schumacher] Removing stripMargin from test schemas
191bc0d [Andre Schumacher] Changing to Seq for ArrayType, refactoring SQLParser for nested field extension
cbb5793 [Andre Schumacher] Code review feedback
32229c7 [Andre Schumacher] Removing Row nested values and placing by generic types
0ae9376 [Andre Schumacher] Doc strings and simplifying ParquetConverter.scala
a6b4f05 [Andre Schumacher] Cleaning up ArrayConverter, moving classTag to NativeType, adding NativeRow
431f00f [Andre Schumacher] Fixing problems introduced during rebase
c52ff2c [Andre Schumacher] Adding native-array converter
619c397 [Andre Schumacher] Completing Map testcase
79d81d5 [Andre Schumacher] Replacing field names for array and map in WriteSupport
f466ff0 [Andre Schumacher] Added ParquetAvro tests and revised Array conversion
adc1258 [Andre Schumacher] Optimizing imports
e99cc51 [Andre Schumacher] Fixing nested WriteSupport and adding tests
1dc5ac9 [Andre Schumacher] First version of WriteSupport for nested types
d1911dc [Andre Schumacher] Simplifying ArrayType conversion
f777b4b [Andre Schumacher] Scalastyle
824500c [Andre Schumacher] Adding attribute resolution for MapType
b539fde [Andre Schumacher] First commit for MapType
a594aed [Andre Schumacher] Scalastyle
4e25fcb [Andre Schumacher] Adding resolution of complex ArrayTypes
f8f8911 [Andre Schumacher] For primitive rows fall back to more efficient converter, code reorg
6dbc9b7 [Andre Schumacher] Fixing some problems intruduced during rebase
b7fcc35 [Andre Schumacher] Documenting conversions, bugfix, wrappers of Rows
ee70125 [Andre Schumacher] fixing one problem with arrayconverter
98219cf [Andre Schumacher] added struct converter
5d80461 [Andre Schumacher] fixing one problem with nested structs and breaking up files
1b1b3d6 [Andre Schumacher] Fixing one problem with nested arrays
ddb40d2 [Andre Schumacher] Extending tests for nested Parquet data
745a42b [Andre Schumacher] Completing testcase for nested data (Addressbook(
6125c75 [Andre Schumacher] First working nested Parquet record input
4d4892a [Andre Schumacher] First commit nested Parquet read converters
aa688fe [Andre Schumacher] Adding conversion of nested Parquet schemas
2014-06-19 23:47:45 -07:00
Yin Huai f397e92eb2 [SPARK-2177][SQL] describe table result contains only one column
```
scala> hql("describe src").collect().foreach(println)

[key                 	string              	None                ]
[value               	string              	None                ]
```

The result should contain 3 columns instead of one. This screws up JDBC or even the downstream consumer of the Scala/Java/Python APIs.

I am providing a workaround. We handle a subset of describe commands in Spark SQL, which are defined by ...
```
DESCRIBE [EXTENDED] [db_name.]table_name
```
All other cases are treated as Hive native commands.

Also, if we upgrade Hive to 0.13, we need to check the results of context.sessionState.isHiveServerQuery() to determine how to split the result. This method is introduced by https://issues.apache.org/jira/browse/HIVE-4545. We may want to set Hive to use JsonMetaDataFormatter for the output of a DDL statement (`set hive.ddl.output.format=json` introduced by https://issues.apache.org/jira/browse/HIVE-2822).

The link to JIRA: https://issues.apache.org/jira/browse/SPARK-2177

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

Closes #1118 from yhuai/SPARK-2177 and squashes the following commits:

fd2534c [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177
b9b9aa5 [Yin Huai] rxin's comments.
e7c4e72 [Yin Huai] Fix unit test.
656b068 [Yin Huai] 100 characters.
6387217 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177
8003cf3 [Yin Huai] Generate strings with the format like Hive for unit tests.
9787fff [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177
440c5af [Yin Huai] rxin's comments.
f1a417e [Yin Huai] Update doc.
83adb2f [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177
366f891 [Yin Huai] Add describe command.
74bd1d4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177
342fdf7 [Yin Huai] Split to up to 3 parts.
725e88c [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2177
bb8bbef [Yin Huai] Split every string in the result of a describe command.
2014-06-19 23:41:38 -07:00
Reynold Xin 278ec8a203 More minor scaladoc cleanup for Spark SQL.
Author: Reynold Xin <rxin@apache.org>

Closes #1142 from rxin/sqlclean and squashes the following commits:

67a789e [Reynold Xin] More minor scaladoc cleanup for Spark SQL.
2014-06-19 22:34:21 -07:00
Reynold Xin 5464e79175 A few minor Spark SQL Scaladoc fixes.
Author: Reynold Xin <rxin@apache.org>

Closes #1139 from rxin/sparksqldoc and squashes the following commits:

c3049d8 [Reynold Xin] Fixed line length.
66dc72c [Reynold Xin] A few minor Spark SQL Scaladoc fixes.
2014-06-19 18:24:05 -07:00
Michael Armbrust 5ff75c748a [SPARK-2184][SQL] AddExchange isn't idempotent
...redPartitioning.

Author: Michael Armbrust <michael@databricks.com>

Closes #1122 from marmbrus/fixAddExchange and squashes the following commits:

3417537 [Michael Armbrust] Don't bind partitioning expressions as that breaks comparison with requiredPartitioning.
2014-06-18 17:52:42 -07:00
Yin Huai d2f4f30b12 [SPARK-2060][SQL] Querying JSON Datasets with SQL and DSL in Spark SQL
JIRA: https://issues.apache.org/jira/browse/SPARK-2060

Programming guide: http://yhuai.github.io/site/sql-programming-guide.html

Scala doc of SQLContext: http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.SQLContext

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

Closes #999 from yhuai/newJson and squashes the following commits:

227e89e [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
ce8eedd [Yin Huai] rxin's comments.
bc9ac51 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
94ffdaa [Yin Huai] Remove "get" from method names.
ce31c81 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
e2773a6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
79ea9ba [Yin Huai] Fix typos.
5428451 [Yin Huai] Newline
1f908ce [Yin Huai] Remove extra line.
d7a005c [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
7ea750e [Yin Huai] marmbrus's comments.
6a5f5ef [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
83013fb [Yin Huai] Update Java Example.
e7a6c19 [Yin Huai] SchemaRDD.javaToPython should convert a field with the StructType to a Map.
6d20b85 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
4fbddf0 [Yin Huai] Programming guide.
9df8c5a [Yin Huai] Python API.
7027634 [Yin Huai] Java API.
cff84cc [Yin Huai] Use a SchemaRDD for a JSON dataset.
d0bd412 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
ab810b0 [Yin Huai] Make JsonRDD private.
6df0891 [Yin Huai] Apache header.
8347f2e [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
66f9e76 [Yin Huai] Update docs and use the entire dataset to infer the schema.
8ffed79 [Yin Huai] Update the example.
a5a4b52 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
4325475 [Yin Huai] If a sampled dataset is used for schema inferring, update the schema of the JsonTable after first execution.
65b87f0 [Yin Huai] Fix sampling...
8846af5 [Yin Huai] API doc.
52a2275 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
0387523 [Yin Huai] Address PR comments.
666b957 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
a2313a6 [Yin Huai] Address PR comments.
f3ce176 [Yin Huai] After type conflict resolution, if a NullType is found, StringType is used.
0576406 [Yin Huai] Add Apache license header.
af91b23 [Yin Huai] Merge remote-tracking branch 'upstream/master' into newJson
f45583b [Yin Huai] Infer the schema of a JSON dataset (a text file with one JSON object per line or a RDD[String] with one JSON object per string) and returns a SchemaRDD.
f31065f [Yin Huai] A query plan or a SchemaRDD can print out its schema.
2014-06-17 19:14:59 -07:00
Zongheng Yang e243c5ffac [SPARK-2053][SQL] Add Catalyst expressions for CASE WHEN.
JIRA ticket: https://issues.apache.org/jira/browse/SPARK-2053

This PR adds support for two types of CASE statements present in Hive. The first type is of the form `CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END`, with the semantics like a chain of if statements. The second type is of the form `CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END`, with the semantics like a switch statement on key `a`. Both forms are implemented in `CaseWhen`.

[This link](https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions) contains more detailed descriptions on their semantics.

Notes / Open issues:

* Please check if any implicit contracts / invariants are broken in the implementations (especially for the operators). I am not very familiar with them and I currently find them tricky to spot.
* We should decide whether or not a non-boolean condition is allowed in a branch of `CaseWhen`. Hive throws a `SemanticException` for this situation and I think it'd be good to mimic it -- the question is where in the whole Spark SQL pipeline should we signal an exception for such a query.

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

Closes #1055 from concretevitamin/caseWhen and squashes the following commits:

4226eb9 [Zongheng Yang] Comment.
79d26fc [Zongheng Yang] Merge branch 'master' into caseWhen
caf9383 [Zongheng Yang] Update a FIXME.
9d26ab8 [Zongheng Yang] Add @transient marker.
788a0d9 [Zongheng Yang] Implement CastNulls, which fixes udf_case and udf_when.
7ef284f [Zongheng Yang] Refactors: remove redundant passes, improve toString, mark transient.
f47ae7b [Zongheng Yang] Modify queries in tests to have shorter golden files.
1c1fbfc [Zongheng Yang] Cleanups per review comments.
7d2b7e2 [Zongheng Yang] Translate CaseKeyWhen to CaseWhen at parsing time.
47d406a [Zongheng Yang] Do toArray once and lazily outside of eval().
bb3d109 [Zongheng Yang] Update scaladoc of a method.
aea3195 [Zongheng Yang] Fix bug that branchesArr is not used; remove unused import.
96870a8 [Zongheng Yang] Turn off scalastyle for some comments.
7392f3a [Zongheng Yang] Minor cleanup.
2cf08bb [Zongheng Yang] Merge branch 'master' into caseWhen
9f84b40 [Zongheng Yang] Add golden outputs from Hive.
db51a85 [Zongheng Yang] Add allCondBooleans check; uncomment tests.
3f9ef0a [Zongheng Yang] Cleanups and bug fixes (mainly in eval() and resolved).
be54bc8 [Zongheng Yang] Rewrite eval() to a low-level implementation. Separate two CASE stmts.
f2bcb9d [Zongheng Yang] WIP
5906f75 [Zongheng Yang] WIP
efd019b [Zongheng Yang] eval() and toString() bug fixes.
7d81e95 [Zongheng Yang] Clean up resolved.
a31d782 [Zongheng Yang] Finish up Case.
2014-06-17 13:30:17 +02:00
Michael Armbrust 269fc62b20 [SQL] Support transforming TreeNodes with Option children.
Thanks goes to @marmbrus for his implementation.

Author: Michael Armbrust <michael@databricks.com>
Author: Zongheng Yang <zongheng.y@gmail.com>

Closes #1074 from concretevitamin/option-treenode and squashes the following commits:

ef27b85 [Zongheng Yang] Merge pull request #1 from marmbrus/pr/1074
73133c2 [Michael Armbrust] TreeNodes can't be inner classes.
ab78420 [Zongheng Yang] Add a test.
2ccb721 [Michael Armbrust] Add support for transformation of optional children.
2014-06-15 11:28:34 +02:00
Cheng Lian ac96d9657c [SPARK-2094][SQL] "Exactly once" semantics for DDL and command statements
## Related JIRA issues

- Main issue:

  - [SPARK-2094](https://issues.apache.org/jira/browse/SPARK-2094): Ensure exactly once semantics for DDL/Commands

- Issues resolved as dependencies:

  - [SPARK-2081](https://issues.apache.org/jira/browse/SPARK-2081): Undefine output() from the abstract class Command and implement it in concrete subclasses
  - [SPARK-2128](https://issues.apache.org/jira/browse/SPARK-2128): No plan for DESCRIBE
  - [SPARK-1852](https://issues.apache.org/jira/browse/SPARK-1852): SparkSQL Queries with Sorts run before the user asks them to

- Other related issue:

  - [SPARK-2129](https://issues.apache.org/jira/browse/SPARK-2129): NPE thrown while lookup a view

    Two test cases, `join_view` and `mergejoin_mixed`, within the `HiveCompatibilitySuite` are removed from the whitelist to workaround this issue.

## PR Overview

This PR defines physical plans for DDL statements and commands and wraps their side effects in a lazy field `PhysicalCommand.sideEffectResult`, so that they are executed eagerly and exactly once.  Also, as a positive side effect, now DDL statements and commands can be turned into proper `SchemaRDD`s and let user query the execution results.

This PR defines schemas for the following DDL/commands:

- EXPLAIN command

  - `plan`: String, the plan explanation

- SET command

  - `key`: String, the key(s) of the propert(y/ies) being set or queried
  - `value`: String, the value(s) of the propert(y/ies) being queried

- Other Hive native command

  - `result`: String, execution result returned by Hive

  **NOTE**: We should refine schemas for different native commands by defining physical plans for them in the future.

## Examples

### EXPLAIN command

Take the "EXPLAIN" command as an example, we first execute the command and obtain a `SchemaRDD` at the same time, then query the `plan` field with the schema DSL:

```
scala> loadTestTable("src")
...

scala> val q0 = hql("EXPLAIN SELECT key, COUNT(*) FROM src GROUP BY key")
...
q0: org.apache.spark.sql.SchemaRDD =
SchemaRDD[0] at RDD at SchemaRDD.scala:98
== Query Plan ==
ExplainCommandPhysical [plan#11:0]
 Aggregate false, [key#4], [key#4,SUM(PartialCount#6L) AS c_1#2L]
  Exchange (HashPartitioning [key#4:0], 200)
   Exchange (HashPartitioning [key#4:0], 200)
    Aggregate true, [key#4], [key#4,COUNT(1) AS PartialCount#6L]
     HiveTableScan [key#4], (MetastoreRelation default, src, None), None

scala> q0.select('plan).collect()
...
[ExplainCommandPhysical [plan#24:0]
 Aggregate false, [key#17], [key#17,SUM(PartialCount#19L) AS c_1#2L]
  Exchange (HashPartitioning [key#17:0], 200)
   Exchange (HashPartitioning [key#17:0], 200)
    Aggregate true, [key#17], [key#17,COUNT(1) AS PartialCount#19L]
     HiveTableScan [key#17], (MetastoreRelation default, src, None), None]

scala>
```

### SET command

In this example we query all the properties set in `SQLConf`, register the result as a table, and then query the table with HiveQL:

```
scala> val q1 = hql("SET")
...
q1: org.apache.spark.sql.SchemaRDD =
SchemaRDD[7] at RDD at SchemaRDD.scala:98
== Query Plan ==
<SET command: executed by Hive, and noted by SQLContext>

scala> q1.registerAsTable("properties")

scala> hql("SELECT key, value FROM properties ORDER BY key LIMIT 10").foreach(println)
...
== Query Plan ==
TakeOrdered 10, [key#51:0 ASC]
 Project [key#51:0,value#52:1]
  SetCommandPhysical None, None, [key#55:0,value#56:1]), which has no missing parents
14/06/12 12:19:27 INFO scheduler.DAGScheduler: Submitting 1 missing tasks from Stage 5 (SchemaRDD[21] at RDD at SchemaRDD.scala:98
== Query Plan ==
TakeOrdered 10, [key#51:0 ASC]
 Project [key#51:0,value#52:1]
  SetCommandPhysical None, None, [key#55:0,value#56:1])
...
[datanucleus.autoCreateSchema,true]
[datanucleus.autoStartMechanismMode,checked]
[datanucleus.cache.level2,false]
[datanucleus.cache.level2.type,none]
[datanucleus.connectionPoolingType,BONECP]
[datanucleus.fixedDatastore,false]
[datanucleus.identifierFactory,datanucleus1]
[datanucleus.plugin.pluginRegistryBundleCheck,LOG]
[datanucleus.rdbms.useLegacyNativeValueStrategy,true]
[datanucleus.storeManagerType,rdbms]

scala>
```

### "Exactly once" semantics

At last, an example of the "exactly once" semantics:

```
scala> val q2 = hql("CREATE TABLE t1(key INT, value STRING)")
...
q2: org.apache.spark.sql.SchemaRDD =
SchemaRDD[28] at RDD at SchemaRDD.scala:98
== Query Plan ==
<Native command: executed by Hive>

scala> table("t1")
...
res9: org.apache.spark.sql.SchemaRDD =
SchemaRDD[32] at RDD at SchemaRDD.scala:98
== Query Plan ==
HiveTableScan [key#58,value#59], (MetastoreRelation default, t1, None), None

scala> q2.collect()
...
res10: Array[org.apache.spark.sql.Row] = Array([])

scala>
```

As we can see, the "CREATE TABLE" command is executed eagerly right after the `SchemaRDD` is created, and referencing the `SchemaRDD` again won't trigger a duplicated execution.

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

Closes #1071 from liancheng/exactlyOnceCommand and squashes the following commits:

d005b03 [Cheng Lian] Made "SET key=value" returns the newly set key value pair
f6c7715 [Cheng Lian] Added test cases for DDL/command statement RDDs
1d00937 [Cheng Lian] Makes SchemaRDD DSLs work for DDL/command statement RDDs
5c7e680 [Cheng Lian] Bug fix: wrong type used in pattern matching
48aa2e5 [Cheng Lian] Refined SQLContext.emptyResult as an empty RDD[Row]
cc64f32 [Cheng Lian] Renamed physical plan classes for DDL/commands
74789c1 [Cheng Lian] Fixed failing test cases
0ad343a [Cheng Lian] Added physical plan for DDL and commands to ensure the "exactly once" semantics
2014-06-13 12:59:48 -07:00
Takuya UESHIN 9a2448daf9 [SPARK-2052] [SQL] Add optimization for CaseConversionExpression's.
Add optimization for `CaseConversionExpression`'s.

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

Closes #990 from ueshin/issues/SPARK-2052 and squashes the following commits:

2568666 [Takuya UESHIN] Move some rules back.
dde7ede [Takuya UESHIN] Add tests to check if ConstantFolding can handle null literals and remove the unneeded rules from NullPropagation.
c4eea67 [Takuya UESHIN] Fix toString methods.
23e2363 [Takuya UESHIN] Make CaseConversionExpressions foldable if the child is foldable.
0ff7568 [Takuya UESHIN] Add tests for collapsing case statements.
3977d80 [Takuya UESHIN] Add optimization for CaseConversionExpression's.
2014-06-11 17:58:35 -07:00
Sameer Agarwal 4107cce58c [SPARK-2042] Prevent unnecessary shuffle triggered by take()
This PR implements `take()` on a `SchemaRDD` by inserting a logical limit that is followed by a `collect()`. This is also accompanied by adding a catalyst optimizer rule for collapsing adjacent limits. Doing so prevents an unnecessary shuffle that is sometimes triggered by `take()`.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #1048 from sameeragarwal/master and squashes the following commits:

3eeb848 [Sameer Agarwal] Fixing Tests
1b76ff1 [Sameer Agarwal] Deprecating limit(limitExpr: Expression) in v1.1.0
b723ac4 [Sameer Agarwal] Added limit folding tests
a0ff7c4 [Sameer Agarwal] Adding catalyst rule to fold two consecutive limits
8d42d03 [Sameer Agarwal] Implement trigger() as limit() followed by collect()
2014-06-11 12:01:04 -07:00
Qiuzhuang.Lian 6e11930310 SPARK-2107: FilterPushdownSuite doesn't need Junit jar.
Author: Qiuzhuang.Lian <Qiuzhuang.Lian@gmail.com>

Closes #1046 from Qiuzhuang/master and squashes the following commits:

0a9921a [Qiuzhuang.Lian] SPARK-2107: FilterPushdownSuite doesn't need Junit jar.
2014-06-11 00:36:06 -07:00
Cheng Lian 0266a0c8a7 [SPARK-1968][SQL] SQL/HiveQL command for caching/uncaching tables
JIRA issue: [SPARK-1968](https://issues.apache.org/jira/browse/SPARK-1968)

This PR added support for SQL/HiveQL command for caching/uncaching tables:

```
scala> sql("CACHE TABLE src")
...
res0: org.apache.spark.sql.SchemaRDD =
SchemaRDD[0] at RDD at SchemaRDD.scala:98
== Query Plan ==
CacheCommandPhysical src, true

scala> table("src")
...
res1: org.apache.spark.sql.SchemaRDD =
SchemaRDD[3] at RDD at SchemaRDD.scala:98
== Query Plan ==
InMemoryColumnarTableScan [key#0,value#1], (HiveTableScan [key#0,value#1], (MetastoreRelation default, src, None), None), false

scala> isCached("src")
res2: Boolean = true

scala> sql("CACHE TABLE src")
...
res3: org.apache.spark.sql.SchemaRDD =
SchemaRDD[4] at RDD at SchemaRDD.scala:98
== Query Plan ==
CacheCommandPhysical src, false

scala> table("src")
...
res4: org.apache.spark.sql.SchemaRDD =
SchemaRDD[11] at RDD at SchemaRDD.scala:98
== Query Plan ==
HiveTableScan [key#2,value#3], (MetastoreRelation default, src, None), None

scala> isCached("src")
res5: Boolean = false
```

Things also work for `hql`.

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

Closes #1038 from liancheng/sqlCacheTable and squashes the following commits:

ecb7194 [Cheng Lian] Trimmed the SQL string before parsing special commands
6f4ce42 [Cheng Lian] Moved logical command classes to a separate file
3458a24 [Cheng Lian] Added comment for public API
f0ffacc [Cheng Lian] Added isCached() predicate
15ec6d2 [Cheng Lian] Added "(UN)CACHE TABLE" SQL/HiveQL statements
2014-06-11 00:06:50 -07:00
Takuya UESHIN 0402bd77ec [SPARK-2093] [SQL] NullPropagation should use exact type value.
`NullPropagation` should use exact type value when transform `Count` or `Sum`.

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

Closes #1034 from ueshin/issues/SPARK-2093 and squashes the following commits:

65b6ff1 [Takuya UESHIN] Modify the literal value of the result of transformation from Sum to long value.
830c20b [Takuya UESHIN] Add Cast to the result of transformation from Count.
9314806 [Takuya UESHIN] Fix NullPropagation to use exact type value.
2014-06-10 23:13:48 -07:00
Cheng Hao db0c038a66 [SPARK-2076][SQL] Pushdown the join filter & predication for outer join
As the rule described in https://cwiki.apache.org/confluence/display/Hive/OuterJoinBehavior, we can optimize the SQL Join by pushing down the Join predicate and Where predicate.

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

Closes #1015 from chenghao-intel/join_predicate_push_down and squashes the following commits:

10feff9 [Cheng Hao] fix bug of changing the join type in PredicatePushDownThroughJoin
44c6700 [Cheng Hao] Add logical to support pushdown the join filter
0bce426 [Cheng Hao] Pushdown the join filter & predicate for outer join
2014-06-10 12:59:52 -07:00
Zongheng Yang 08ed9ad813 [SPARK-1508][SQL] Add SQLConf to SQLContext.
This PR (1) introduces a new class SQLConf that stores key-value properties for a SQLContext (2) clean up the semantics of various forms of SET commands.

The SQLConf class unlocks user-controllable optimization opportunities; for example, user can now override the number of partitions used during an Exchange. A SQLConf can be accessed and modified programmatically through its getters and setters. It can also be modified through SET commands executed by `sql()` or `hql()`. Note that users now have the ability to change a particular property for different queries inside the same Spark job, unlike settings configured in SparkConf.

For SET commands: "SET" will return all properties currently set in a SQLConf, "SET key" will return the key-value pair (if set) or an undefined message, and "SET key=value" will call the setter on SQLConf, and if a HiveContext is used, it will be executed in Hive as well.

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

Closes #956 from concretevitamin/sqlconf and squashes the following commits:

4968c11 [Zongheng Yang] Very minor cleanup.
d74dde5 [Zongheng Yang] Remove the redundant mkQueryExecution() method.
c129b86 [Zongheng Yang] Merge remote-tracking branch 'upstream/master' into sqlconf
26c40eb [Zongheng Yang] Make SQLConf a trait and have SQLContext mix it in.
dd19666 [Zongheng Yang] Update a comment.
baa5d29 [Zongheng Yang] Remove default param for shuffle partitions accessor.
5f7e6d8 [Zongheng Yang] Add default num partitions.
22d9ed7 [Zongheng Yang] Fix output() of Set physical. Add SQLConf param accessor method.
e9856c4 [Zongheng Yang] Use java.util.Collections.synchronizedMap on a Java HashMap.
88dd0c8 [Zongheng Yang] Remove redundant SET Keyword.
271f0b1 [Zongheng Yang] Minor change.
f8983d1 [Zongheng Yang] Minor changes per review comments.
1ce8a5e [Zongheng Yang] Invoke runSqlHive() in SQLConf#get for the HiveContext case.
b766af9 [Zongheng Yang] Remove a test.
d52e1bd [Zongheng Yang] De-hardcode number of shuffle partitions for BasicOperators (read from SQLConf).
555599c [Zongheng Yang] Bullet-proof (relatively) parsing SET per review comment.
c2067e8 [Zongheng Yang] Mark SQLContext transient and put it in a second param list.
2ea8cdc [Zongheng Yang] Wrap long line.
41d7f09 [Zongheng Yang] Fix imports.
13279e6 [Zongheng Yang] Refactor the logic of eagerly processing SET commands.
b14b83e [Zongheng Yang] In a HiveContext, make SQLConf a subset of HiveConf.
6983180 [Zongheng Yang] Move a SET test to SQLQuerySuite and make it complete.
5b67985 [Zongheng Yang] New line at EOF.
c651797 [Zongheng Yang] Add commands.scala.
efd82db [Zongheng Yang] Clean up semantics of several cases of SET.
c1017c2 [Zongheng Yang] WIP in changing SetCommand to take two Options (for different semantics of SETs).
0f00d86 [Zongheng Yang] Add a test for singleton set command in SQL.
41acd75 [Zongheng Yang] Add a test for hql() in HiveQuerySuite.
2276929 [Zongheng Yang] Fix default hive result for set commands in HiveComparisonTest.
3b0c71b [Zongheng Yang] Remove Parser for set commands. A few other fixes.
d0c4578 [Zongheng Yang] Tmux typo.
0ecea46 [Zongheng Yang] Changes for HiveQl and HiveContext.
ce22d80 [Zongheng Yang] Fix parsing issues.
cb722c1 [Zongheng Yang] Finish up SQLConf patch.
4ebf362 [Zongheng Yang] First cut at SQLConf inside SQLContext.
2014-06-10 00:49:09 -07:00
Zongheng Yang a9ec033c8c [SPARK-1704][SQL] Fully support EXPLAIN commands as SchemaRDD.
This PR attempts to resolve [SPARK-1704](https://issues.apache.org/jira/browse/SPARK-1704) by introducing a physical plan for EXPLAIN commands, which just prints out the debug string (containing various SparkSQL's plans) of the corresponding QueryExecution for the actual query.

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

Closes #1003 from concretevitamin/explain-cmd and squashes the following commits:

5b7911f [Zongheng Yang] Add a regression test.
1bfa379 [Zongheng Yang] Modify output().
719ada9 [Zongheng Yang] Override otherCopyArgs for ExplainCommandPhysical.
4318fd7 [Zongheng Yang] Make all output one Row.
439c6ab [Zongheng Yang] Minor cleanups.
408f574 [Zongheng Yang] SPARK-1704: Add CommandStrategy and ExplainCommandPhysical.
2014-06-09 16:47:44 -07:00
Daoyuan 0cf6002801 [SPARK-1495][SQL]add support for left semi join
Just submit another solution for #395

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

Closes #837 from adrian-wang/left-semi-join-support and squashes the following commits:

d39cd12 [Daoyuan Wang] Merge pull request #1 from marmbrus/pr/837
6713c09 [Michael Armbrust] Better debugging for failed query tests.
035b73e [Michael Armbrust] Add test for left semi that can't be done with a hash join.
5ec6fa4 [Michael Armbrust] Add left semi to SQL Parser.
4c726e5 [Daoyuan] improvement according to Michael
8d4a121 [Daoyuan] add golden files for leftsemijoin
83a3c8a [Daoyuan] scala style fix
14cff80 [Daoyuan] add support for left semi join
2014-06-09 11:31:36 -07:00
Michael Armbrust c7a183b2c2 [SPARK-2041][SQL] Correctly analyze queries where columnName == tableName.
Author: Michael Armbrust <michael@databricks.com>

Closes #985 from marmbrus/tableName and squashes the following commits:

3caaa27 [Michael Armbrust] Correctly analyze queries where columnName == tableName.
2014-06-05 17:42:08 -07:00
Takuya UESHIN e4c11eef2f [SPARK-2036] [SQL] CaseConversionExpression should check if the evaluated value is null.
`CaseConversionExpression` should check if the evaluated value is `null`.

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

Closes #982 from ueshin/issues/SPARK-2036 and squashes the following commits:

61e1c54 [Takuya UESHIN] Add check if the evaluated value is null.
2014-06-05 12:00:31 -07:00
Takuya UESHIN 7c160293d6 [SPARK-2029] Bump pom.xml version number of master branch to 1.1.0-SNAPSHOT.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #974 from ueshin/issues/SPARK-2029 and squashes the following commits:

e19e8f4 [Takuya UESHIN] Bump version number to 1.1.0-SNAPSHOT.
2014-06-05 11:27:33 -07:00
egraldlo ec8be274a7 [SPARK-1995][SQL] system function upper and lower can be supported
I don't know whether it's time to implement system function about string operation in spark sql now.

Author: egraldlo <egraldlo@gmail.com>

Closes #936 from egraldlo/stringoperator and squashes the following commits:

3c6c60a [egraldlo] Add UPPER, LOWER, MAX and MIN into hive parser
ea76d0a [egraldlo] modify the formatting issues
b49f25e [egraldlo] modify the formatting issues
1f0bbb5 [egraldlo] system function upper and lower supported
13d3267 [egraldlo] system function upper and lower supported
2014-06-02 18:02:57 -07:00
Takuya UESHIN 3ce81494c5 [SPARK-1947] [SQL] Child of SumDistinct or Average should be widened to prevent overflows the same as Sum.
Child of `SumDistinct` or `Average` should be widened to prevent overflows the same as `Sum`.

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

Closes #902 from ueshin/issues/SPARK-1947 and squashes the following commits:

99c3dcb [Takuya UESHIN] Insert Cast for SumDistinct and Average.
2014-05-31 11:30:03 -07:00
Takuya UESHIN 9df86835b6 [SPARK-1938] [SQL] ApproxCountDistinctMergeFunction should return Int value.
`ApproxCountDistinctMergeFunction` should return `Int` value because the `dataType` of `ApproxCountDistinct` is `IntegerType`.

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

Closes #893 from ueshin/issues/SPARK-1938 and squashes the following commits:

3970e88 [Takuya UESHIN] Remove a superfluous line.
5ad7ec1 [Takuya UESHIN] Make dataType for each of CountDistinct, ApproxCountDistinctMerge and ApproxCountDistinct LongType.
cbe7c71 [Takuya UESHIN] Revert a change.
fc3ac0f [Takuya UESHIN] Fix evaluated value type of ApproxCountDistinctMergeFunction to Int.
2014-05-27 22:17:50 -07:00
Takuya UESHIN 3b0babad1f [SPARK-1915] [SQL] AverageFunction should not count if the evaluated value is null.
Average values are difference between the calculation is done partially or not partially.
Because `AverageFunction` (in not-partially calculation) counts even if the evaluated value is null.

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

Closes #862 from ueshin/issues/SPARK-1915 and squashes the following commits:

b1ff3c0 [Takuya UESHIN] Modify AverageFunction not to count if the evaluated value is null.
2014-05-27 14:55:23 -07:00
Takuya UESHIN d1375a2bff [SPARK-1926] [SQL] Nullability of Max/Min/First should be true.
Nullability of `Max`/`Min`/`First` should be `true` because they return `null` if there are no rows.

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

Closes #881 from ueshin/issues/SPARK-1926 and squashes the following commits:

322610f [Takuya UESHIN] Fix nullability of Min/Max/First.
2014-05-27 14:53:57 -07:00
Takuya UESHIN d6395d86f9 [SPARK-1914] [SQL] Simplify CountFunction not to traverse to evaluate all child expressions.
`CountFunction` should count up only if the child's evaluated value is not null.

Because it traverses to evaluate all child expressions, even if the child is null, it counts up if one of the all children is not null.

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

Closes #861 from ueshin/issues/SPARK-1914 and squashes the following commits:

3b37315 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-1914
2afa238 [Takuya UESHIN] Simplify CountFunction not to traverse to evaluate all child expressions.
2014-05-26 00:17:20 -07:00
Kan Zhang 6052db9dc1 [SPARK-1822] SchemaRDD.count() should use query optimizer
Author: Kan Zhang <kzhang@apache.org>

Closes #841 from kanzhang/SPARK-1822 and squashes the following commits:

2f8072a [Kan Zhang] [SPARK-1822] Minor style update
cf4baa4 [Kan Zhang] [SPARK-1822] Adding Scaladoc
e67c910 [Kan Zhang] [SPARK-1822] SchemaRDD.count() should use optimizer
2014-05-25 00:06:42 -07:00
Takuya UESHIN bb88875ad5 [SPARK-1889] [SQL] Apply splitConjunctivePredicates to join condition while finding join ke...
...ys.

When tables are equi-joined by multiple-keys `HashJoin` should be used, but `CartesianProduct` and then `Filter` are used.
The join keys are paired by `And` expression so we need to apply `splitConjunctivePredicates` to join condition while finding join keys.

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

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

fe1c387 [Takuya UESHIN] Apply splitConjunctivePredicates to join condition while finding join keys.
2014-05-21 15:37:47 -07:00
Cheng Hao a20fea9881 [Spark-1461] Deferred Expression Evaluation (short-circuit evaluation)
This patch unify the foldable & nullable interface for Expression.
1) Deterministic-less UDF (like Rand()) can not be folded.
2) Short-circut will significantly improves the performance in Expression Evaluation, however, the stateful UDF should not be ignored in a short-circuit evaluation(e.g. in expression: col1 > 0 and row_sequence() < 1000, row_sequence() can not be ignored even if col1 > 0 is false)

I brought an concept of DeferredObject from Hive, which has 2 kinds of children classes (EagerResult / DeferredResult), the former requires triggering the evaluation before it's created, while the later trigger the evaluation when first called its get() method.

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

Closes #446 from chenghao-intel/expression_deferred_evaluation and squashes the following commits:

d2729de [Cheng Hao] Fix the codestyle issues
a08f09c [Cheng Hao] fix bug in or/and short-circuit evaluation
af2236b [Cheng Hao] revert the short-circuit expression evaluation for IF
b7861d2 [Cheng Hao] Add Support for Deferred Expression Evaluation
2014-05-15 22:12:34 -07:00
Takuya UESHIN 94c9d6f598 [SPARK-1819] [SQL] Fix GetField.nullable.
`GetField.nullable` should be `true` not only when `field.nullable` is `true` but also when `child.nullable` is `true`.

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

Closes #757 from ueshin/issues/SPARK-1819 and squashes the following commits:

8781a11 [Takuya UESHIN] Modify a test to use named parameters.
5bfc77d [Takuya UESHIN] Fix GetField.nullable.
2014-05-15 11:21:33 -07:00
Prashant Sharma 46324279da Package docs
This is a few changes based on the original patch by @scrapcodes.

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

Closes #785 from pwendell/package-docs and squashes the following commits:

c32b731 [Patrick Wendell] Changes based on Prashant's patch
c0463d3 [Prashant Sharma] added eof new line
ce8bf73 [Prashant Sharma] Added eof new line to all files.
4c35f2e [Prashant Sharma] SPARK-1563 Add package-info.java and package.scala files for all packages that appear in docs
2014-05-14 22:24:41 -07:00