Commit graph

585 commits

Author SHA1 Message Date
Michael Armbrust 3abd0c1cda [SPARK-2406][SQL] Initial support for using ParquetTableScan to read HiveMetaStore tables.
This PR adds an experimental flag `spark.sql.hive.convertMetastoreParquet` that when true causes the planner to detects tables that use Hive's Parquet SerDe and instead plans them using Spark SQL's native `ParquetTableScan`.

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

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

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

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

Author: Matei Zaharia <matei@databricks.com>

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

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

Author: Patrick Wendell <pwendell@gmail.com>

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

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

Author: Matei Zaharia <matei@databricks.com>

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

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

Author: Matei Zaharia <matei@databricks.com>

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

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

Author: Patrick Wendell <pwendell@gmail.com>

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

Author: Sean Owen <srowen@gmail.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

Author: guowei <guowei@upyoo.com>

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

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

Related PR:

- #1715

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

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

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

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

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

Author: wangfei <wangfei1@huawei.com>

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

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

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

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

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

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

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

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

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

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

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

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

Author: Michael Armbrust <michael@databricks.com>

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

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

Author: GuoQiang Li <witgo@qq.com>

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

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

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

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

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

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

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

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

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

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

Author: Anand Avati <avati@redhat.com>

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

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

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

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

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

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

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

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

root
 |-- field1: integer (nullable = true)
 |-- field2: string (nullable = true)
 |-- field3: struct (nullable = true)
 |    |-- field4: integer (nullable = true)
 |    |-- field5: array (nullable = true)
 |    |    |-- element: integer (containsNull = false)
 |-- field6: array (nullable = true)
 |    |-- element: struct (containsNull = false)
 |    |    |-- field7: string (nullable = true)

Then we can access them by row.field3.field5[0]  or row.field6[5].field7

It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType.

You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as:

ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1]))

Or you could use Row to create a class just like namedtuple, for example:

Person = Row("name", "age")
ctx.inferSchema(rdd.map(lambda x: Person(*x)))

Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details.

schema = StructType([StructField("name, StringType, True),
                                    StructType("age", IntegerType, True)])
ctx.applySchema(rdd, schema)

PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable.

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

Closes #1598 from davies/nested and squashes the following commits:

f1d15b6 [Davies Liu] verify schema with the first few rows
8852aaf [Davies Liu] check type of schema
abe9e6e [Davies Liu] address comments
61b2292 [Davies Liu] add @deprecated to pythonToJavaMap
1e5b801 [Davies Liu] improve cache of classes
51aa135 [Davies Liu] use Row to infer schema
e9c0d5c [Davies Liu] remove string typed schema
353a3f2 [Davies Liu] fix code style
63de8f8 [Davies Liu] fix typo
c79ca67 [Davies Liu] fix serialization of nested data
6b258b5 [Davies Liu] fix pep8
9d8447c [Davies Liu] apply schema provided by string of names
f5df97f [Davies Liu] refactor, address comments
9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python
84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested
0eaaf56 [Davies Liu] fix doc tests
b3559b4 [Davies Liu] use generated Row instead of namedtuple
c4ddc30 [Davies Liu] fix conflict between name of fields and variables
7f6f251 [Davies Liu] address all comments
d69d397 [Davies Liu] refactor
2cc2d45 [Davies Liu] refactor
182fb46 [Davies Liu] refactor
bc6e9e1 [Davies Liu] switch to new Schema API
547bf3e [Davies Liu] Merge branch 'master' into nested
a435b5a [Davies Liu] add docs and code refactor
2c8debc [Davies Liu] Merge branch 'master' into nested
644665a [Davies Liu] use tuple and namedtuple for schemardd
2014-08-01 18:47:41 -07:00
Cheng Hao c0b47bada3 [SPARK-2767] [SQL] SparkSQL CLI doens't output error message if query failed.
Author: Cheng Hao <hao.cheng@intel.com>

Closes #1686 from chenghao-intel/spark_sql_cli and squashes the following commits:

eb664cc [Cheng Hao] Output detailed failure message in console
93b0382 [Cheng Hao] Fix Bug of no output in cli if exception thrown internally
2014-08-01 11:42:05 -07:00
chutium 580c7011ca [SPARK-2729] [SQL] Forgot to match Timestamp type in ColumnBuilder
just a match forgot, found after SPARK-2710 , TimestampType can be used by a SchemaRDD generated from JDBC ResultSet

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

Closes #1636 from chutium/SPARK-2729 and squashes the following commits:

71af77a [chutium] [SPARK-2729] [SQL] added Timestamp in NullableColumnAccessorSuite
39cf9f8 [chutium] [SPARK-2729] add Timestamp Type into ColumnBuilder TestSuite, ref. #1636
ab6ff97 [chutium] [SPARK-2729] Forgot to match Timestamp type in ColumnBuilder
2014-08-01 11:31:44 -07:00
Cheng Hao 4415722e91 [SQL][SPARK-2212]Hash Outer Join
This patch is to support the hash based outer join. Currently, outer join for big relations are resort to `BoradcastNestedLoopJoin`, which is super slow. This PR will create 2 hash tables for both relations in the same partition, which greatly reduce the table scans.

Here is the testing code that I used:
```
package org.apache.spark.sql.hive

import org.apache.spark.SparkContext
import org.apache.spark.SparkConf
import org.apache.spark.sql._

case class Record(key: String, value: String)

object JoinTablePrepare extends App {
  import TestHive2._

  val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"${i % 828193}", s"val_$i")))

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

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

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

object JoinTablePerformanceTest extends App {
  import TestHive2._

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

  val leftOuterJoin = "insert overwrite table result select a.key, b.value from a left outer join b on a.key=b.key"
  val rightOuterJoin = "insert overwrite table result select a.key, b.value from a right outer join b on a.key=b.key"
  val fullOuterJoin = "insert overwrite table result select a.key, b.value from a full outer join b on a.key=b.key"

  val results = ("LeftOuterJoin", benchmark(leftOuterJoin)) :: ("LeftOuterJoin", benchmark(leftOuterJoin)) ::
                ("RightOuterJoin", benchmark(rightOuterJoin)) :: ("RightOuterJoin", benchmark(rightOuterJoin)) ::
                ("FullOuterJoin", benchmark(fullOuterJoin)) :: ("FullOuterJoin", benchmark(fullOuterJoin)) :: Nil
  val explains = hql(s"explain $leftOuterJoin").collect ++ hql(s"explain $rightOuterJoin").collect ++ hql(s"explain $fullOuterJoin").collect
  println(explains.mkString(",\n"))
  results.foreach { case (prompt, result) => {
      println(s"$prompt: took ${result._1} ms (${result._2} records)")
    }
  }

  def benchmark(cmd: String) = {
    val begin = System.currentTimeMillis()
    val result = hql(cmd)
    val end = System.currentTimeMillis()
    val count = hql("select count(1) from result").collect.mkString("")
    ((end - begin), count)
  }
}
```
And the result as shown below:
```
[Physical execution plan:],
[InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
[ Project [key#95,value#98]],
[  HashOuterJoin [key#95], [key#97], LeftOuter, None],
[   Exchange (HashPartitioning [key#95], 20)],
[    HiveTableScan [key#95], (MetastoreRelation default, a, None), None],
[   Exchange (HashPartitioning [key#97], 20)],
[    HiveTableScan [key#97,value#98], (MetastoreRelation default, b, None), None],
[Physical execution plan:],
[InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
[ Project [key#102,value#105]],
[  HashOuterJoin [key#102], [key#104], RightOuter, None],
[   Exchange (HashPartitioning [key#102], 20)],
[    HiveTableScan [key#102], (MetastoreRelation default, a, None), None],
[   Exchange (HashPartitioning [key#104], 20)],
[    HiveTableScan [key#104,value#105], (MetastoreRelation default, b, None), None],
[Physical execution plan:],
[InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true],
[ Project [key#109,value#112]],
[  HashOuterJoin [key#109], [key#111], FullOuter, None],
[   Exchange (HashPartitioning [key#109], 20)],
[    HiveTableScan [key#109], (MetastoreRelation default, a, None), None],
[   Exchange (HashPartitioning [key#111], 20)],
[    HiveTableScan [key#111,value#112], (MetastoreRelation default, b, None), None]
LeftOuterJoin: took 16072 ms ([3000000] records)
LeftOuterJoin: took 14394 ms ([3000000] records)
RightOuterJoin: took 14802 ms ([3000000] records)
RightOuterJoin: took 14747 ms ([3000000] records)
FullOuterJoin: took 17715 ms ([6000000] records)
FullOuterJoin: took 17629 ms ([6000000] records)
```

Without this PR, the benchmark will run seems never end.

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

Closes #1147 from chenghao-intel/hash_based_outer_join and squashes the following commits:

65c599e [Cheng Hao] Fix issues with the community comments
72b1394 [Cheng Hao] Fix bug of stale value in joinedRow
55baef7 [Cheng Hao] Add HashOuterJoin
2014-08-01 11:27:12 -07:00
Yin Huai c41fdf04f4 [SPARK-2179][SQL] A minor refactoring Java data type APIs (2179 follow-up).
It is a follow-up PR of SPARK-2179 (https://issues.apache.org/jira/browse/SPARK-2179). It makes package names of data type APIs more consistent across languages (Scala: `org.apache.spark.sql`, Java: `org.apache.spark.sql.api.java`, Python: `pyspark.sql`).

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

Closes #1712 from yhuai/javaDataType and squashes the following commits:

62eb705 [Yin Huai] Move package-info.
add4bcb [Yin Huai] Make the package names of data type classes consistent across languages by moving all Java data type classes to package sql.api.java.
2014-08-01 11:14:53 -07:00
GuoQiang Li 9998efab96 SPARK-2766: ScalaReflectionSuite throw an llegalArgumentException in JDK 6
Author: GuoQiang Li <witgo@qq.com>

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

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

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

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

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

00d72fd [Yin Huai] Use scala.collection.Map.
2014-07-31 21:02:11 -07:00
Zongheng Yang 8f51491ea7 [SPARK-2531 & SPARK-2436] [SQL] Optimize the BuildSide when planning BroadcastNestedLoopJoin.
This PR resolves the following two tickets:

- [SPARK-2531](https://issues.apache.org/jira/browse/SPARK-2531): BNLJ currently assumes the build side is the right relation. This patch refactors some of its logic to take into account a BuildSide properly.
- [SPARK-2436](https://issues.apache.org/jira/browse/SPARK-2436): building on top of the above, we simply use the physical size statistics (if available) of both relations, and make the smaller relation the build side in the planner.

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

Closes #1448 from concretevitamin/bnlj-buildSide and squashes the following commits:

1780351 [Zongheng Yang] Use size estimation to decide optimal build side of BNLJ.
68e6c5b [Zongheng Yang] Consolidate two adjacent pattern matchings.
96d312a [Zongheng Yang] Use a while loop instead of collection methods chaining.
4bc525e [Zongheng Yang] Make BroadcastNestedLoopJoin take a BuildSide.
2014-07-31 19:32:16 -07:00
Yin Huai 49b361298b [SPARK-2523] [SQL] Hadoop table scan bug fixing (fix failing Jenkins maven test)
This PR tries to resolve the broken Jenkins maven test issue introduced by #1439. Now, we create a single query test to run both the setup work and the test query.

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

Closes #1669 from yhuai/SPARK-2523-fixTest and squashes the following commits:

358af1a [Yin Huai] Make partition_based_table_scan_with_different_serde run atomically.
2014-07-31 13:05:24 -07:00
Michael Armbrust 72cfb13987 [SPARK-2397][SQL] Deprecate LocalHiveContext
LocalHiveContext is redundant with HiveContext.  The only difference is it creates `./metastore` instead of `./metastore_db`.

Author: Michael Armbrust <michael@databricks.com>

Closes #1641 from marmbrus/localHiveContext and squashes the following commits:

e5ec497 [Michael Armbrust] Add deprecation version
626e056 [Michael Armbrust] Don't remove from imports yet
905cc5f [Michael Armbrust] Merge remote-tracking branch 'apache/master' into localHiveContext
1c2727e [Michael Armbrust] Deprecate LocalHiveContext
2014-07-31 11:26:43 -07:00
Michael Armbrust 3072b96026 [SPARK-2743][SQL] Resolve original attributes in ParquetTableScan
Author: Michael Armbrust <michael@databricks.com>

Closes #1647 from marmbrus/parquetCase and squashes the following commits:

a1799b7 [Michael Armbrust] move comment
2a2a68b [Michael Armbrust] Merge remote-tracking branch 'apache/master' into parquetCase
bb35d5b [Michael Armbrust] Fix test case that produced an invalid plan.
e6870bf [Michael Armbrust] Better error message.
539a2e1 [Michael Armbrust] Resolve original attributes in ParquetTableScan
2014-07-31 11:15:25 -07:00
Matei Zaharia e966284409 SPARK-2045 Sort-based shuffle
This adds a new ShuffleManager based on sorting, as described in https://issues.apache.org/jira/browse/SPARK-2045. The bulk of the code is in an ExternalSorter class that is similar to ExternalAppendOnlyMap, but sorts key-value pairs by partition ID and can be used to create a single sorted file with a map task's output. (Longer-term I think this can take on the remaining functionality in ExternalAppendOnlyMap and replace it so we don't have code duplication.)

The main TODOs still left are:
- [x] enabling ExternalSorter to merge across spilled files
  - [x] with an Ordering
  - [x] without an Ordering, using the keys' hash codes
- [x] adding more tests (e.g. a version of our shuffle suite that runs on this)
- [x] rebasing on top of the size-tracking refactoring in #1165 when that is merged
- [x] disabling spilling if spark.shuffle.spill is set to false

Despite this though, this seems to work pretty well (running successfully in cases where the hash shuffle would OOM, such as 1000 reduce tasks on executors with only 1G memory), and it seems to be comparable in speed or faster than hash-based shuffle (it will create much fewer files for the OS to keep track of). So I'm posting it to get some early feedback.

After these TODOs are done, I'd also like to enable ExternalSorter to sort data within each partition by a key as well, which will allow us to use it to implement external spilling in reduce tasks in `sortByKey`.

Author: Matei Zaharia <matei@databricks.com>

Closes #1499 from mateiz/sort-based-shuffle and squashes the following commits:

bd841f9 [Matei Zaharia] Various review comments
d1c137fd [Matei Zaharia] Various review comments
a611159 [Matei Zaharia] Compile fixes due to rebase
62c56c8 [Matei Zaharia] Fix ShuffledRDD sometimes not returning Tuple2s.
f617432 [Matei Zaharia] Fix a failing test (seems to be due to change in SizeTracker logic)
9464d5f [Matei Zaharia] Simplify code and fix conflicts after latest rebase
0174149 [Matei Zaharia] Add cleanup behavior and cleanup tests for sort-based shuffle
eb4ee0d [Matei Zaharia] Remove customizable element type in ShuffledRDD
fa2e8db [Matei Zaharia] Allow nextBatchStream to be called after we're done looking at all streams
a34b352 [Matei Zaharia] Fix tracking of indices within a partition in SpillReader, and add test
03e1006 [Matei Zaharia] Add a SortShuffleSuite that runs ShuffleSuite with sort-based shuffle
3c7ff1f [Matei Zaharia] Obey the spark.shuffle.spill setting in ExternalSorter
ad65fbd [Matei Zaharia] Rebase on top of Aaron's Sorter change, and use Sorter in our buffer
44d2a93 [Matei Zaharia] Use estimateSize instead of atGrowThreshold to test collection sizes
5686f71 [Matei Zaharia] Optimize merging phase for in-memory only data:
5461cbb [Matei Zaharia] Review comments and more tests (e.g. tests with 1 element per partition)
e9ad356 [Matei Zaharia] Update ContextCleanerSuite to make sure shuffle cleanup tests use hash shuffle (since they were written for it)
c72362a [Matei Zaharia] Added bug fix and test for when iterators are empty
de1fb40 [Matei Zaharia] Make trait SizeTrackingCollection private[spark]
4988d16 [Matei Zaharia] tweak
c1b7572 [Matei Zaharia] Small optimization
ba7db7f [Matei Zaharia] Handle null keys in hash-based comparator, and add tests for collisions
ef4e397 [Matei Zaharia] Support for partial aggregation even without an Ordering
4b7a5ce [Matei Zaharia] More tests, and ability to sort data if a total ordering is given
e1f84be [Matei Zaharia] Fix disk block manager test
5a40a1c [Matei Zaharia] More tests
614f1b4 [Matei Zaharia] Add spill metrics to map tasks
cc52caf [Matei Zaharia] Add more error handling and tests for error cases
bbf359d [Matei Zaharia] More work
3a56341 [Matei Zaharia] More partial work towards sort-based shuffle
7a0895d [Matei Zaharia] Some more partial work towards sort-based shuffle
b615476 [Matei Zaharia] Scaffolding for sort-based shuffle
2014-07-30 18:07:59 -07:00
Michael Armbrust 88a519db90 [SPARK-2734][SQL] Remove tables from cache when DROP TABLE is run.
Author: Michael Armbrust <michael@databricks.com>

Closes #1650 from marmbrus/dropCached and squashes the following commits:

e6ab80b [Michael Armbrust] Support if exists.
83426c6 [Michael Armbrust] Remove tables from cache when DROP TABLE is run.
2014-07-30 17:30:51 -07:00
Sean Owen 6ab96a6fd0 SPARK-2749 [BUILD]. Spark SQL Java tests aren't compiling in Jenkins' Maven builds; missing junit:junit dep
The Maven-based builds in the build matrix have been failing for a few days:

https://amplab.cs.berkeley.edu/jenkins/view/Spark/

On inspection, it looks like the Spark SQL Java tests don't compile:

https://amplab.cs.berkeley.edu/jenkins/view/Spark/job/Spark-Master-Maven-pre-YARN/hadoop.version=1.0.4,label=centos/244/consoleFull

I confirmed it by repeating the command vs master:

`mvn -Dhadoop.version=1.0.4 -Dlabel=centos -DskipTests clean package`

The problem is that this module doesn't depend on JUnit. In fact, none of the modules do, but `com.novocode:junit-interface` (the SBT-JUnit bridge) pulls it in, in most places. However this module doesn't depend on `com.novocode:junit-interface`

Adding the `junit:junit` dependency fixes the compile problem. In fact, the other modules with Java tests should probably depend on it explicitly instead of happening to get it via `com.novocode:junit-interface`, since that is a bit SBT/Scala-specific (and I am not even sure it's needed).

Author: Sean Owen <srowen@gmail.com>

Closes #1660 from srowen/SPARK-2749 and squashes the following commits:

858ff7c [Sean Owen] Add explicit junit dep to other modules with Java tests for robustness
9636794 [Sean Owen] Add junit dep so that Spark SQL Java tests compile
2014-07-30 15:04:33 -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 077f633b47 [SQL] Handle null values in debug()
Author: Michael Armbrust <michael@databricks.com>

Closes #1646 from marmbrus/nullDebug and squashes the following commits:

49050a8 [Michael Armbrust] Handle null values in debug()
2014-07-29 22:42:54 -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 86534d0f52 [SPARK-2631][SQL] Use SQLConf to configure in-memory columnar caching
Author: Michael Armbrust <michael@databricks.com>

Closes #1638 from marmbrus/cachedConfig and squashes the following commits:

2362082 [Michael Armbrust] Use SQLConf to configure in-memory columnar caching
2014-07-29 18:20:51 -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
Davies Liu f0d880e288 [SPARK-2674] [SQL] [PySpark] support datetime type for SchemaRDD
Datetime and time in Python will be converted into java.util.Calendar after serialization, it will be converted into java.sql.Timestamp during inferSchema().

In javaToPython(), Timestamp will be converted into Calendar, then be converted into datetime in Python after pickling.

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

Closes #1601 from davies/date and squashes the following commits:

f0599b0 [Davies Liu] remove tests for sets and tuple in sql, fix list of list
c9d607a [Davies Liu] convert datetype for runtime
709d40d [Davies Liu] remove brackets
96db384 [Davies Liu] support datetime type for SchemaRDD
2014-07-29 12:31:39 -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
Daoyuan 0c5c6a63d1 [SQL]change some test lists
1. there's no `hook_context.q` but a `hook_context_cs.q` in query folder
2. there's no `compute_stats_table.q` in query folder
3. there's no `having1.q` in query folder
4. `udf_E` and `udf_PI` appear twice in white list

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

Closes #1634 from adrian-wang/testcases and squashes the following commits:

d7482ce [Daoyuan] change some test lists
2014-07-29 12:22:48 -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
Yadong Qi 16ef4d110f Excess judgment
Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #1629 from watermen/bug-fix2 and squashes the following commits:

59b7237 [Yadong Qi] Update HiveQl.scala
2014-07-28 21:39:02 -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
Cheng Hao 2b8d89e30e [SPARK-2523] [SQL] Hadoop table scan bug fixing
In HiveTableScan.scala, ObjectInspector was created for all of the partition based records, which probably causes ClassCastException if the object inspector is not identical among table & partitions.

This is the follow up with:
https://github.com/apache/spark/pull/1408
https://github.com/apache/spark/pull/1390

I've run a micro benchmark in my local with 15000000 records totally, and got the result as below:

With This Patch  |  Partition-Based Table  |  Non-Partition-Based Table
------------ | ------------- | -------------
No  |  1927 ms  |  1885 ms
Yes  | 1541 ms  |  1524 ms

It showed this patch will also improve the performance.

PS:  the benchmark code is also attached. (thanks liancheng )
```
package org.apache.spark.sql.hive

import org.apache.spark.SparkContext
import org.apache.spark.SparkConf
import org.apache.spark.sql._

object HiveTableScanPrepare extends App {
  case class Record(key: String, value: String)

  val sparkContext = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))

  val hiveContext = new LocalHiveContext(sparkContext)

  val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"$i", s"val_$i")))

  import hiveContext._

  hql("SHOW TABLES")
  hql("DROP TABLE if exists part_scan_test")
  hql("DROP TABLE if exists scan_test")
  hql("DROP TABLE if exists records")
  rdd.registerAsTable("records")

  hql("""CREATE TABLE part_scan_test (key STRING, value STRING) PARTITIONED BY (part1 string, part2 STRING)
                 | ROW FORMAT SERDE
                 | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'
                 | STORED AS RCFILE
               """.stripMargin)
  hql("""CREATE TABLE scan_test (key STRING, value STRING)
                 | ROW FORMAT SERDE
                 | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe'
                 | STORED AS RCFILE
               """.stripMargin)

  for (part1 <- 2000 until 2001) {
    for (part2 <- 1 to 5) {
      hql(s"""from records
                 | insert into table part_scan_test PARTITION (part1='$part1', part2='2010-01-$part2')
                 | select key, value
               """.stripMargin)
      hql(s"""from records
                 | insert into table scan_test select key, value
               """.stripMargin)
    }
  }
}

object HiveTableScanTest extends App {
  val sparkContext = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))

  val hiveContext = new LocalHiveContext(sparkContext)

  import hiveContext._

  hql("SHOW TABLES")
  val part_scan_test = hql("select key, value from part_scan_test")
  val scan_test = hql("select key, value from scan_test")

  val r_part_scan_test = (0 to 5).map(i => benchmark(part_scan_test))
  val r_scan_test = (0 to 5).map(i => benchmark(scan_test))
  println("Scanning Partition-Based Table")
  r_part_scan_test.foreach(printResult)
  println("Scanning Non-Partition-Based Table")
  r_scan_test.foreach(printResult)

  def printResult(result: (Long, Long)) {
    println(s"Duration: ${result._1} ms Result: ${result._2}")
  }

  def benchmark(srdd: SchemaRDD) = {
    val begin = System.currentTimeMillis()
    val result = srdd.count()
    val end = System.currentTimeMillis()
    ((end - begin), result)
  }
}
```

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

Closes #1439 from chenghao-intel/hadoop_table_scan and squashes the following commits:

888968f [Cheng Hao] Fix issues in code style
27540ba [Cheng Hao] Fix the TableScan Bug while partition serde differs
40a24a7 [Cheng Hao] Add Unit Test
2014-07-28 10:59:53 -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
baishuo(白硕) ab3c6a455c [SQL]Update HiveMetastoreCatalog.scala
I think it's better to defined hiveQlTable as a val

Author: baishuo(白硕) <vc_java@hotmail.com>

Closes #1569 from baishuo/patch-1 and squashes the following commits:

dc2f895 [baishuo(白硕)] Update HiveMetastoreCatalog.scala
a7b32a2 [baishuo(白硕)] Update HiveMetastoreCatalog.scala
2014-07-25 13:59:45 -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
Yin Huai b352ef175c [SPARK-2603][SQL] Remove unnecessary toMap and toList in converting Java collections to Scala collections JsonRDD.scala
In JsonRDD.scalafy, we are using toMap/toList to convert a Java Map/List to a Scala one. These two operations are pretty expensive because they read elements from a Java Map/List and then load to a Scala Map/List. We can use Scala wrappers to wrap those Java collections instead of using toMap/toList.

I did a quick test to see the performance. I had a 2.9GB cached RDD[String] storing one JSON object per record (twitter dataset). My simple test program is attached below.
```scala
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext._

val jsonData = sc.textFile("...")
jsonData.cache.count

val jsonSchemaRDD = sqlContext.jsonRDD(jsonData)
jsonSchemaRDD.registerAsTable("jt")

sqlContext.sql("select count(*) from jt").collect
```
Stages for the schema inference and the table scan both had 48 tasks. These tasks were executed sequentially. For the current implementation, scanning the JSON dataset will materialize values of all fields of a record. The inferred schema of the dataset can be accessed at https://gist.github.com/yhuai/05fe8a57c638c6666f8d.

From the result, there was no significant difference on running `jsonRDD`. For the simple aggregation query, results are attached below.
```
Original:
Run 1: 26.1s
Run 2: 27.03s
Run 3: 27.035s

With this change:
Run 1: 21.086s
Run 2: 21.035s
Run 3: 21.029s
```

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

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

Closes #1504 from yhuai/removeToMapToList and squashes the following commits:

6831b77 [Yin Huai] Fix failed tests.
09b9bca [Yin Huai] Merge remote-tracking branch 'upstream/master' into removeToMapToList
d1abdb8 [Yin Huai] Remove unnecessary toMap and toList.
2014-07-24 11:19:19 -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
witgo 60f0ae3d87 [SPARK-2484][SQL] Build should not run hivecompatibility tests by default.
Author: witgo <witgo@qq.com>

Closes #1403 from witgo/hive_compatibility and squashes the following commits:

4e5ecdb [witgo] The default does not run hive compatibility tests
2014-07-23 18:17:05 -07:00
Ian O Connell efdaeb1119 [SPARK-2102][SQL][CORE] Add option for kryo registration required and use a resource pool in Spark SQL for Kryo instances.
Author: Ian O Connell <ioconnell@twitter.com>

Closes #1377 from ianoc/feature/SPARK-2102 and squashes the following commits:

5498566 [Ian O Connell] Docs update suggested by Patrick
20e8555 [Ian O Connell] Slight style change
f92c294 [Ian O Connell] Add docs for new KryoSerializer option
f3735c8 [Ian O Connell] Add using a kryo resource pool for the SqlSerializer
4e5c342 [Ian O Connell] Register the SparkConf for kryo, it gets swept into serialization
665805a [Ian O Connell] Add a spark.kryo.registrationRequired option for configuring the Kryo Serializer
2014-07-23 16:30:11 -07:00
Michael Armbrust 1871574a24 [SPARK-2569][SQL] Fix shipping of TEMPORARY hive UDFs.
Instead of shipping just the name and then looking up the info on the workers, we now ship the whole classname.  Also, I refactored the file as it was getting pretty large to move out the type conversion code to its own file.

Author: Michael Armbrust <michael@databricks.com>

Closes #1552 from marmbrus/fixTempUdfs and squashes the following commits:

b695904 [Michael Armbrust] Make add jar execute with Hive.  Ship the whole function class name since sometimes we cannot lookup temporary functions on the workers.
2014-07-23 16:26:55 -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 Hao 79fe7634f6 [SPARK-2615] [SQL] Add Equal Sign "==" Support for HiveQl
Currently, the "==" in HiveQL expression will cause exception thrown, this patch will fix it.

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

Closes #1522 from chenghao-intel/equal and squashes the following commits:

f62a0ff [Cheng Hao] Add == Support for HiveQl
2014-07-22 18:13:28 -07:00
Michael Armbrust 511a731403 [SPARK-2561][SQL] Fix apply schema
We need to use the analyzed attributes otherwise we end up with a tree that will never resolve.

Author: Michael Armbrust <michael@databricks.com>

Closes #1470 from marmbrus/fixApplySchema and squashes the following commits:

f968195 [Michael Armbrust] Use analyzed attributes when applying the schema.
4969015 [Michael Armbrust] Add test case.
2014-07-21 18:18:17 -07:00
Aaron Davidson abeacffb7b Fix flakey HiveQuerySuite test
Result may not be returned in the expected order, so relax that constraint.

Author: Aaron Davidson <aaron@databricks.com>

Closes #1514 from aarondav/flakey and squashes the following commits:

e5af823 [Aaron Davidson] Fix flakey HiveQuerySuite test
2014-07-21 14:35:15 -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
Cheng Hao 7f17208137 [SPARK-2540] [SQL] Add HiveDecimal & HiveVarchar support in unwrapping data
Author: Cheng Hao <hao.cheng@intel.com>

Closes #1436 from chenghao-intel/unwrapdata and squashes the following commits:

34cc21a [Cheng Hao] update the table scan accodringly since the unwrapData function changed
afc39da [Cheng Hao] Polish the code
39d6475 [Cheng Hao] Add HiveDecimal & HiveVarchar support in unwrap data
2014-07-18 16:38:11 -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
Yin Huai df95d82da7 [SPARK-2525][SQL] Remove as many compilation warning messages as possible in Spark SQL
JIRA: https://issues.apache.org/jira/browse/SPARK-2525.

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

Closes #1444 from yhuai/SPARK-2517 and squashes the following commits:

edbac3f [Yin Huai] Removed some compiler type erasure warnings.
2014-07-16 10:53:59 -07:00
Cheng Lian efc452a163 [SPARK-2119][SQL] Improved Parquet performance when reading off S3
JIRA issue: [SPARK-2119](https://issues.apache.org/jira/browse/SPARK-2119)

Essentially this PR fixed three issues to gain much better performance when reading large Parquet file off S3.

1. When reading the schema, fetching Parquet metadata from a part-file rather than the `_metadata` file

   The `_metadata` file contains metadata of all row groups, and can be very large if there are many row groups. Since schema information and row group metadata are coupled within a single Thrift object, we have to read the whole `_metadata` to fetch the schema. On the other hand, schema is replicated among footers of all part-files, which are fairly small.

1. Only add the root directory of the Parquet file rather than all the part-files to input paths

   HDFS API can automatically filter out all hidden files and underscore files (`_SUCCESS` & `_metadata`), there's no need to filter out all part-files and add them individually to input paths. What make it much worse is that, `FileInputFormat.listStatus()` calls `FileSystem.globStatus()` on each individual input path sequentially, each results a blocking remote S3 HTTP request.

1. Worked around [PARQUET-16](https://issues.apache.org/jira/browse/PARQUET-16)

   Essentially PARQUET-16 is similar to the above issue, and results lots of sequential `FileSystem.getFileStatus()` calls, which are further translated into a bunch of remote S3 HTTP requests.

   `FilteringParquetRowInputFormat` should be cleaned up once PARQUET-16 is fixed.

Below is the micro benchmark result. The dataset used is a S3 Parquet file consists of 3,793 partitions, about 110MB per partition in average. The benchmark is done with a 9-node AWS cluster.

- Creating a Parquet `SchemaRDD` (Parquet schema is fetched)

  ```scala
  val tweets = parquetFile(uri)
  ```

  - Before: 17.80s
  - After: 8.61s

- Fetching partition information

  ```scala
  tweets.getPartitions
  ```

  - Before: 700.87s
  - After: 21.47s

- Counting the whole file (both steps above are executed altogether)

  ```scala
  parquetFile(uri).count()
  ```

  - Before: ??? (haven't test yet)
  - After: 53.26s

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

Closes #1370 from liancheng/faster-parquet and squashes the following commits:

94a2821 [Cheng Lian] Added comments about schema consistency
d2c4417 [Cheng Lian] Worked around PARQUET-16 to improve Parquet performance
1c0d1b9 [Cheng Lian] Accelerated Parquet schema retrieving
5bd3d29 [Cheng Lian] Fixed Parquet log level
2014-07-16 12:44:51 -04: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
Aaron Staple 90ca532a0f [SPARK-2314][SQL] Override collect and take in JavaSchemaRDD, forwarding to SchemaRDD implementations.
Author: Aaron Staple <aaron.staple@gmail.com>

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

73e04dc [Aaron Staple] [SPARK-2314] Override collect and take in JavaSchemaRDD, forwarding to SchemaRDD implementations.
2014-07-15 21:35:36 -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
Michael Armbrust 502f90782a [SQL] Attribute equality comparisons should be done by exprId.
Author: Michael Armbrust <michael@databricks.com>

Closes #1414 from marmbrus/exprIdResolution and squashes the following commits:

97b47bc [Michael Armbrust] Attribute equality comparisons should be done by exprId.
2014-07-15 17:56:17 -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 bcd0c30c7e [SQL] Whitelist more Hive tests.
Author: Michael Armbrust <michael@databricks.com>

Closes #1396 from marmbrus/moreTests and squashes the following commits:

6660b60 [Michael Armbrust] Blacklist a test that requires DFS command.
8b6001c [Michael Armbrust] Add golden files.
ccd8f97 [Michael Armbrust] Whitelist more tests.
2014-07-15 14:04:01 -07:00
Michael Armbrust 0f98ef1a2c [SPARK-2483][SQL] Fix parsing of repeated, nested data access.
Author: Michael Armbrust <michael@databricks.com>

Closes #1411 from marmbrus/nestedRepeated and squashes the following commits:

044fa09 [Michael Armbrust] Fix parsing of repeated, nested data access.
2014-07-15 14:01:48 -07:00
Michael Armbrust c7c7ac8339 [SPARK-2485][SQL] Lock usage of hive client.
Author: Michael Armbrust <michael@databricks.com>

Closes #1412 from marmbrus/lockHiveClient and squashes the following commits:

4bc9d5a [Michael Armbrust] protected[hive]
22e9177 [Michael Armbrust] Add comments.
7aa8554 [Michael Armbrust] Don't lock on hive's object.
a6edc5f [Michael Armbrust] Lock usage of hive client.
2014-07-15 00:13:51 -07:00
Takuya UESHIN 9fe693b5b6 [SPARK-2446][SQL] Add BinaryType support to Parquet I/O.
Note that this commit changes the semantics when loading in data that was created with prior versions of Spark SQL.  Before, we were writing out strings as Binary data without adding any other annotations. Thus, when data is read in from prior versions, data that was StringType will now become BinaryType.  Users that need strings can CAST that column to a String.  It was decided that while this breaks compatibility, it does make us compatible with other systems (Hive, Thrift, etc) and adds support for Binary data, so this is the right decision long term.

To support `BinaryType`, the following changes are needed:
- Make `StringType` use `OriginalType.UTF8`
- Add `BinaryType` using `PrimitiveTypeName.BINARY` without `OriginalType`

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

Closes #1373 from ueshin/issues/SPARK-2446 and squashes the following commits:

ecacb92 [Takuya UESHIN] Add BinaryType support to Parquet I/O.
616e04a [Takuya UESHIN] Make StringType use OriginalType.UTF8.
2014-07-14 15:42:35 -07:00
Zongheng Yang d60b09bb60 [SPARK-2443][SQL] Fix slow read from partitioned tables
This fix obtains a comparable performance boost as [PR #1390](https://github.com/apache/spark/pull/1390) by moving an array update and deserializer initialization out of a potentially very long loop. Suggested by yhuai. The below results are updated for this fix.

## Benchmarks
Generated a local text file with 10M rows of simple key-value pairs. The data is loaded as a table through Hive. Results are obtained on my local machine using hive/console.

Without the fix:

Type | Non-partitioned | Partitioned (1 part)
------------ | ------------ | -------------
First run | 9.52s end-to-end (1.64s Spark job) | 36.6s (28.3s)
Stablized runs | 1.21s (1.18s) | 27.6s (27.5s)

With this fix:

Type | Non-partitioned | Partitioned (1 part)
------------ | ------------ | -------------
First run | 9.57s (1.46s) | 11.0s (1.69s)
Stablized runs | 1.13s (1.10s) | 1.23s (1.19s)

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

Closes #1408 from concretevitamin/slow-read-2 and squashes the following commits:

d86e437 [Zongheng Yang] Move update & initialization out of potentially long loop.
2014-07-14 13:22:24 -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
Michael Armbrust 7e26b57615 [SPARK-2441][SQL] Add more efficient distinct operator.
Author: Michael Armbrust <michael@databricks.com>

Closes #1366 from marmbrus/partialDistinct and squashes the following commits:

12a31ab [Michael Armbrust] Add more efficient distinct operator.
2014-07-12 12:07:27 -07:00
Takuya UESHIN 10b59ba230 [SPARK-2428][SQL] Add except and intersect methods to SchemaRDD.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #1355 from ueshin/issues/SPARK-2428 and squashes the following commits:

b6fa264 [Takuya UESHIN] Add except and intersect methods to SchemaRDD.
2014-07-10 19:27:24 -07:00
Takuya UESHIN f5abd27129 [SPARK-2415] [SQL] RowWriteSupport should handle empty ArrayType correctly.
`RowWriteSupport` doesn't write empty `ArrayType` value, so the read value becomes `null`.
It should write empty `ArrayType` value as it is.

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

Closes #1339 from ueshin/issues/SPARK-2415 and squashes the following commits:

32afc87 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-2415
2f05196 [Takuya UESHIN] Fix RowWriteSupport to handle empty ArrayType correctly.
2014-07-10 19:23:44 -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
Patrick Wendell 553c578de1 HOTFIX: Remove persistently failing test in master.
Apparently this functionality is going to be removed soon anywyas.
2014-07-09 19:44:24 -07:00
Patrick Wendell dd22bc2d57 Revert "[HOTFIX] Synchronize on SQLContext.settings in tests."
This reverts commit d4c30cd991.
2014-07-09 19:36:38 -07:00
Reynold Xin 32516f866a [SPARK-2409] Make SQLConf thread safe.
Author: Reynold Xin <rxin@apache.org>

Closes #1334 from rxin/sqlConfThreadSafetuy and squashes the following commits:

c1e0a5a [Reynold Xin] Fixed the duplicate comment.
7614372 [Reynold Xin] [SPARK-2409] Make SQLConf thread safe.
2014-07-08 14:00:47 -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
Michael Armbrust 5a4063645d [SPARK-2391][SQL] Custom take() for LIMIT queries.
Using Spark's take can result in an entire in-memory partition to be shipped in order to retrieve a single row.

Author: Michael Armbrust <michael@databricks.com>

Closes #1318 from marmbrus/takeLimit and squashes the following commits:

77289a5 [Michael Armbrust] Update scala doc
32f0674 [Michael Armbrust] Custom take implementation for LIMIT queries.
2014-07-08 00:41:46 -07:00
witgo 3cd5029be7 Resolve sbt warnings during build Ⅱ
Author: witgo <witgo@qq.com>

Closes #1153 from witgo/expectResult and squashes the following commits:

97541d8 [witgo] merge master
ead26e7 [witgo] Resolve sbt warnings during build
2014-07-08 00:31:42 -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 4352a2fdaa [SPARK-2376][SQL] Selecting list values inside nested JSON objects raises java.lang.IllegalArgumentException
JIRA: https://issues.apache.org/jira/browse/SPARK-2376

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

Closes #1320 from yhuai/SPARK-2376 and squashes the following commits:

0107417 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2376
480803d [Yin Huai] Correctly handling JSON arrays in PySpark.
2014-07-07 18:37:38 -07:00
Yin Huai f0496ee108 [SPARK-2375][SQL] JSON schema inference may not resolve type conflicts correctly for a field inside an array of structs
For example, for
```
{"array": [{"field":214748364700}, {"field":1}]}
```
the type of field is resolved as IntType. While, for
```
{"array": [{"field":1}, {"field":214748364700}]}
```
the type of field is resolved as LongType.

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

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

Closes #1308 from yhuai/SPARK-2375 and squashes the following commits:

3e2e312 [Yin Huai] Update unit test.
1b2ff9f [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2375
10794eb [Yin Huai] Correctly resolve the type of a field inside an array of structs.
2014-07-07 17:05:59 -07:00
Takuya UESHIN 4deeed17c4 [SPARK-2386] [SQL] RowWriteSupport should use the exact types to cast.
When execute `saveAsParquetFile` with non-primitive type, `RowWriteSupport` uses wrong type `Int` for `ByteType` and `ShortType`.

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

Closes #1315 from ueshin/issues/SPARK-2386 and squashes the following commits:

20d89ec [Takuya UESHIN] Use None instead of null.
bd88741 [Takuya UESHIN] Add a test.
323d1d2 [Takuya UESHIN] Modify RowWriteSupport to use the exact types to cast.
2014-07-07 17:04:02 -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
Michael Armbrust 9d006c9737 [SPARK-2370][SQL] Decrease metadata retrieved for partitioned hive queries.
Author: Michael Armbrust <michael@databricks.com>

Closes #1305 from marmbrus/usePrunerPartitions and squashes the following commits:

744aa20 [Michael Armbrust] Use getAllPartitionsForPruner instead of getPartitions, which avoids retrieving auth data
2014-07-04 19:15:48 -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
baishuo(白硕) 0bbe61223e Update SQLConf.scala
use concurrent.ConcurrentHashMap instead of util.Collections.synchronizedMap

Author: baishuo(白硕) <vc_java@hotmail.com>

Closes #1272 from baishuo/master and squashes the following commits:

51ec55d [baishuo(白硕)] Update SQLConf.scala
63da043 [baishuo(白硕)] Update SQLConf.scala
36b6dbd [baishuo(白硕)] Update SQLConf.scala
864faa0 [baishuo(白硕)] Update SQLConf.scala
593096b [baishuo(白硕)] Update SQLConf.scala
7304d9b [baishuo(白硕)] Update SQLConf.scala
843581c [baishuo(白硕)] Update SQLConf.scala
1d3e4a2 [baishuo(白硕)] Update SQLConf.scala
0740f28 [baishuo(白硕)] Update SQLConf.scala
2014-07-04 00:25:31 -07:00
Cheng Lian 544880457d [SPARK-2059][SQL] Don't throw TreeNodeException in execution.ExplainCommand
This is a fix for the problem revealed by PR #1265.

Currently `HiveComparisonSuite` ignores output of `ExplainCommand` since Catalyst query plan is quite different from Hive query plan. But exceptions throw from `CheckResolution` still breaks test cases. This PR catches any `TreeNodeException` and reports it as part of the query explanation.

After merging this PR, PR #1265 can also be merged safely.

For a normal query:

```
scala> hql("explain select key from src").foreach(println)
...
[Physical execution plan:]
[HiveTableScan [key#9], (MetastoreRelation default, src, None), None]
```

For a wrong query with unresolved attribute(s):

```
scala> hql("explain select kay from src").foreach(println)
...
[Error occurred during query planning: ]
[Unresolved attributes: 'kay, tree:]
[Project ['kay]]
[ LowerCaseSchema ]
[  MetastoreRelation default, src, None]
```

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

Closes #1294 from liancheng/safe-explain and squashes the following commits:

4318911 [Cheng Lian] Don't throw TreeNodeException in `execution.ExplainCommand`
2014-07-03 23:41:54 -07:00
Zongheng Yang d4c30cd991 [HOTFIX] Synchronize on SQLContext.settings in tests.
Let's see if this fixes the ongoing series of test failures in a master build machine (https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT-pre-YARN/SPARK_HADOOP_VERSION=1.0.4,label=centos/81/).

pwendell marmbrus

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

Closes #1277 from concretevitamin/test-fix and squashes the following commits:

28c88bd [Zongheng Yang] Synchronize on SQLContext.settings in tests.
2014-07-03 17:37:53 -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
Takuya UESHIN 1e2c26c83d [SPARK-2328] [SQL] Add execution of SHOW TABLES before TestHive.reset().
`PruningSuite` is executed first of Hive tests unfortunately, `TestHive.reset()` breaks the test environment.
To prevent this, we must run a query before calling reset the first time.

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

Closes #1268 from ueshin/issues/SPARK-2328 and squashes the following commits:

043ceac [Takuya UESHIN] Add execution of `SHOW TABLES` before `TestHive.reset()`.
2014-07-02 10:07:01 -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 32a1ad7531 [SPARK-2295] [SQL] Make JavaBeans nullability stricter.
Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #1235 from ueshin/issues/SPARK-2295 and squashes the following commits:

201c508 [Takuya UESHIN] Make JavaBeans nullability stricter.
2014-06-26 13:37:19 -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
Cheng Lian 7f196b009d [SPARK-2283][SQL] Reset test environment before running PruningSuite
JIRA issue: [SPARK-2283](https://issues.apache.org/jira/browse/SPARK-2283)

If `PruningSuite` is run right after `HiveCompatibilitySuite`, the first test case fails because `srcpart` table is cached in-memory by `HiveCompatibilitySuite`, but column pruning is not implemented for `InMemoryColumnarTableScan` operator yet.

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

Closes #1221 from liancheng/spark-2283 and squashes the following commits:

dc0b663 [Cheng Lian] SPARK-2283: reset test environment before running PruningSuite
2014-06-25 18:41:47 -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 Lian 22036aeb1b [BUGFIX][SQL] Should match java.math.BigDecimal when wnrapping Hive output
The `BigDecimal` branch in `unwrap` matches to `scala.math.BigDecimal` rather than `java.math.BigDecimal`.

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

Closes #1199 from liancheng/javaBigDecimal and squashes the following commits:

e9bb481 [Cheng Lian] Should match java.math.BigDecimal when wnrapping Hive output
2014-06-25 00:17:28 -07:00
Cheng Lian 8fade8973e [SPARK-2263][SQL] Support inserting MAP<K, V> to Hive tables
JIRA issue: [SPARK-2263](https://issues.apache.org/jira/browse/SPARK-2263)

Map objects were not converted to Hive types before inserting into Hive tables.

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

Closes #1205 from liancheng/spark-2263 and squashes the following commits:

c7a4373 [Cheng Lian] Addressed @concretevitamin's comment
784940b [Cheng Lian] SARPK-2263: support inserting MAP<K, V> to Hive tables
2014-06-25 00:14:34 -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
Michael Armbrust a162c9b337 [SPARK-2264][SQL] Fix failing CachedTableSuite
Author: Michael Armbrust <michael@databricks.com>

Closes #1201 from marmbrus/fixCacheTests and squashes the following commits:

9d87ed1 [Michael Armbrust] Use analyzer (which runs to fixed point) instead of manually removing analysis operators.
2014-06-24 19:04:29 -07:00
Patrick Wendell 221909e678 HOTFIX: Disabling tests per SPARK-2264 2014-06-24 15:09:38 -07:00
jerryshao 56eb8af187 [SPARK-2124] Move aggregation into shuffle implementations
This PR is a sub-task of SPARK-2044 to move the execution of aggregation into shuffle implementations.

I leave `CoGoupedRDD` and `SubtractedRDD` unchanged because they have their implementations of aggregation. I'm not sure is it suitable to change these two RDDs.

Also I do not move sort related code of `OrderedRDDFunctions` into shuffle, this will be solved in another sub-task.

Author: jerryshao <saisai.shao@intel.com>

Closes #1064 from jerryshao/SPARK-2124 and squashes the following commits:

4a05a40 [jerryshao] Modify according to comments
1f7dcc8 [jerryshao] Style changes
50a2fd6 [jerryshao] Fix test suite issue after moving aggregator to Shuffle reader and writer
1a96190 [jerryshao] Code modification related to the ShuffledRDD
308f635 [jerryshao] initial works of move combiner to ShuffleManager's reader and writer
2014-06-23 20:25:46 -07:00
Reynold Xin 51c8168377 [SPARK-2227] Support dfs command in SQL.
Note that nothing gets printed to the console because we don't properly maintain session state right now.

I will have a followup PR that fixes it.

Author: Reynold Xin <rxin@apache.org>

Closes #1167 from rxin/commands and squashes the following commits:

56f04f8 [Reynold Xin] [SPARK-2227] Support dfs command in SQL.
2014-06-23 18:34:54 -07:00
Cheng Lian a4bc442ca2 [SPARK-1669][SQL] Made cacheTable idempotent
JIRA issue: [SPARK-1669](https://issues.apache.org/jira/browse/SPARK-1669)

Caching the same table multiple times should end up with only 1 in-memory columnar representation of this table.

Before:

```
scala> loadTestTable("src")
...
scala> cacheTable("src")
...
scala> cacheTable("src")
...
scala> table("src")
...
== Query Plan ==
InMemoryColumnarTableScan [key#2,value#3], (InMemoryRelation [key#2,value#3], false, (InMemoryColumnarTableScan [key#2,value#3], (InMemoryRelation [key#2,value#3], false, (HiveTableScan [key#2,value#3], (MetastoreRelation default, src, None), None))))
```

After:

```
scala> loadTestTable("src")
...
scala> cacheTable("src")
...
scala> cacheTable("src")
...
scala> table("src")
...
== Query Plan ==
InMemoryColumnarTableScan [key#2,value#3], (InMemoryRelation [key#2,value#3], false, (HiveTableScan [key#2,value#3], (MetastoreRelation default, src, None), None))
```

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

Closes #1183 from liancheng/spark-1669 and squashes the following commits:

68f8a20 [Cheng Lian] Removed an unused import
51bae90 [Cheng Lian] Made cacheTable idempotent
2014-06-23 13:24:33 -07:00
Reynold Xin ec935abce1 [SQL] Break hiveOperators.scala into multiple files.
The single file was getting very long (500+ loc).

Author: Reynold Xin <rxin@apache.org>

Closes #1166 from rxin/hiveOperators and squashes the following commits:

5b43068 [Reynold Xin] [SQL] Break hiveOperators.scala into multiple files.
2014-06-21 12:04:18 -07:00
Reynold Xin ca5d8b5904 [SQL] Pass SQLContext instead of SparkContext into physical operators.
This makes it easier to use config options in operators.

Author: Reynold Xin <rxin@apache.org>

Closes #1164 from rxin/sqlcontext and squashes the following commits:

797b2fd [Reynold Xin] Pass SQLContext instead of SparkContext into physical operators.
2014-06-20 22:49:48 -07:00
Aaron Davidson 2044784915 [SQL] Use hive.SessionState, not the thread local SessionState
Note that this is simply mimicing lookupRelation(). I do not have a concrete notion of why this solution is necessarily right-er than SessionState.get, but SessionState.get is returning null, which is bad.

Author: Aaron Davidson <aaron@databricks.com>

Closes #1148 from aarondav/createtable and squashes the following commits:

37c3e7c [Aaron Davidson] [SQL] Use hive.SessionState, not the thread local SessionState
2014-06-20 17:55:54 -07:00
Reynold Xin d4c7572dba Move ScriptTransformation into the appropriate place.
Author: Reynold Xin <rxin@apache.org>

Closes #1162 from rxin/script and squashes the following commits:

2c836b9 [Reynold Xin] Move ScriptTransformation into the appropriate place.
2014-06-20 17:16:56 -07:00
Reynold Xin 0ac71d1284 [SPARK-2225] Turn HAVING without GROUP BY into WHERE.
@willb

Author: Reynold Xin <rxin@apache.org>

Closes #1161 from rxin/having-filter and squashes the following commits:

fa8359a [Reynold Xin] [SPARK-2225] Turn HAVING without GROUP BY into WHERE.
2014-06-20 15:38:02 -07:00
William Benton 171ebb3a82 SPARK-2180: support HAVING clauses in Hive queries
This PR extends Spark's HiveQL support to handle HAVING clauses in aggregations.  The HAVING test from the Hive compatibility suite doesn't appear to be runnable from within Spark, so I added a simple comparable test to `HiveQuerySuite`.

Author: William Benton <willb@redhat.com>

Closes #1136 from willb/SPARK-2180 and squashes the following commits:

3bbaf26 [William Benton] Added casts to HAVING expressions
83f1340 [William Benton] scalastyle fixes
18387f1 [William Benton] Add test for HAVING without GROUP BY
b880bef [William Benton] Added semantic error for HAVING without GROUP BY
942428e [William Benton] Added test coverage for SPARK-2180.
56084cc [William Benton] Add support for HAVING clauses in Hive queries.
2014-06-20 13:41:38 -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
Michael Armbrust d3b7671c1f [SQL] Improve Speed of InsertIntoHiveTable
Author: Michael Armbrust <michael@databricks.com>

Closes #1130 from marmbrus/noFunctional and squashes the following commits:

ccdb68c [Michael Armbrust] Remove functional programming and Array allocations from fast path in InsertIntoHiveTable.
2014-06-19 23:39:03 -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 777c5958c4 [SPARK-2191][SQL] Make sure InsertIntoHiveTable doesn't execute more than once.
Author: Michael Armbrust <michael@databricks.com>

Closes #1129 from marmbrus/doubleCreateAs and squashes the following commits:

9c6d9e4 [Michael Armbrust] Fix typo.
5128fe2 [Michael Armbrust] Make sure InsertIntoHiveTable doesn't execute each time you ask for its result.
2014-06-19 14:14:03 -07:00
Reynold Xin 640c294369 [SPARK-2187] Explain should not run the optimizer twice.
@yhuai @marmbrus @concretevitamin

Author: Reynold Xin <rxin@apache.org>

Closes #1123 from rxin/explain and squashes the following commits:

def83b0 [Reynold Xin] Update unit tests for explain.
a9d3ba8 [Reynold Xin] [SPARK-2187] Explain should not run the optimizer twice.
2014-06-18 22:44:12 -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 587d32012c [SPARK-2176][SQL] Extra unnecessary exchange operator in the result of an explain command
```
hql("explain select * from src group by key").collect().foreach(println)

[ExplainCommand [plan#27:0]]
[ Aggregate false, [key#25], [key#25,value#26]]
[  Exchange (HashPartitioning [key#25:0], 200)]
[   Exchange (HashPartitioning [key#25:0], 200)]
[    Aggregate true, [key#25], [key#25]]
[     HiveTableScan [key#25,value#26], (MetastoreRelation default, src, None), None]
```

There are two exchange operators.

However, if we do not use explain...
```
hql("select * from src group by key")

res4: org.apache.spark.sql.SchemaRDD =
SchemaRDD[8] at RDD at SchemaRDD.scala:100
== Query Plan ==
Aggregate false, [key#8], [key#8,value#9]
 Exchange (HashPartitioning [key#8:0], 200)
  Aggregate true, [key#8], [key#8]
   HiveTableScan [key#8,value#9], (MetastoreRelation default, src, None), None
```
The plan is fine.

The cause of this bug is explained below.

When we create an `execution.ExplainCommand`, we use the `executedPlan` as the child of this `ExplainCommand`. But, this `executedPlan` is prepared for execution again when we generate the `executedPlan` for the `ExplainCommand`. Basically, `prepareForExecution` is called twice on a physical plan. Because after `prepareForExecution` we have already bounded those references (in `BoundReference`s), `AddExchange` cannot figure out we are using the same partitioning (we use `AttributeReference`s to create an `ExchangeOperator` and then those references will be changed to `BoundReference`s after `prepareForExecution` is called). So, an extra `ExchangeOperator` is inserted.

I think in `CommandStrategy`, we should just use the `sparkPlan` (`sparkPlan` is the input of `prepareForExecution`) to initialize the `ExplainCommand` instead of using `executedPlan`.

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

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

Closes #1116 from yhuai/SPARK-2176 and squashes the following commits:

197c19c [Yin Huai] Use sparkPlan to initialize a Physical Explain Command instead of using executedPlan.
2014-06-18 10:51:32 -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
Xi Liu f5a4049e53 [SPARK-2164][SQL] Allow Hive UDF on columns of type struct
Author: Xi Liu <xil@conviva.com>

Closes #796 from xiliu82/sqlbug and squashes the following commits:

328dfc4 [Xi Liu] [Spark SQL] remove a temporary function after test
354386a [Xi Liu] [Spark SQL] add test suite for UDF on struct
8fc6f51 [Xi Liu] [SparkSQL] allow UDF on struct
2014-06-17 13:16:02 +02:00
Cheng Lian 237b96bc59 Minor fix: made "EXPLAIN" output to play well with JDBC output format
Fixed the broken JDBC output. Test from Shark `beeline`:

```
beeline> !connect jdbc:hive2://localhost:10000/
scan complete in 2ms
Connecting to jdbc:hive2://localhost:10000/
Enter username for jdbc:hive2://localhost:10000/: lian
Enter password for jdbc:hive2://localhost:10000/:
Connected to: Hive (version 0.12.0)
Driver: Hive (version 0.12.0)
Transaction isolation: TRANSACTION_REPEATABLE_READ
0: jdbc:hive2://localhost:10000/>
0: jdbc:hive2://localhost:10000/> explain select * from src;
+-------------------------------------------------------------------------------+
|                                     plan                                      |
+-------------------------------------------------------------------------------+
| ExplainCommand [plan#2:0]                                                     |
|  HiveTableScan [key#0,value#1], (MetastoreRelation default, src, None), None  |
+-------------------------------------------------------------------------------+
2 rows selected (1.386 seconds)
```

Before this change, the output looked something like this:

```
+-------------------------------------------------------------------------------+
|                                     plan                                      |
+-------------------------------------------------------------------------------+
| ExplainCommand [plan#2:0]
 HiveTableScan [key#0,value#1], (MetastoreRelation default, src, None), None  |
+-------------------------------------------------------------------------------+
```

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

Closes #1097 from liancheng/multiLineExplain and squashes the following commits:

eb37967 [Cheng Lian] Made output of "EXPLAIN" play well with JDBC output format
2014-06-16 16:42:17 -07:00
Cheng Lian 273afcb254 [SQL][SPARK-2094] Follow up of PR #1071 for Java API
Updated `JavaSQLContext` and `JavaHiveContext` similar to what we've done to `SQLContext` and `HiveContext` in PR #1071. Added corresponding test case for Spark SQL Java API.

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

Closes #1085 from liancheng/spark-2094-java and squashes the following commits:

29b8a51 [Cheng Lian] Avoided instantiating JavaSparkContext & JavaHiveContext to workaround test failure
92bb4fb [Cheng Lian] Marked test cases in JavaHiveQLSuite with "ignore"
22aec97 [Cheng Lian] Follow up of PR #1071 for Java API
2014-06-16 21:32:51 +02:00
Kan Zhang 4fdb491775 [SPARK-2010] Support for nested data in PySpark SQL
JIRA issue https://issues.apache.org/jira/browse/SPARK-2010

This PR adds support for nested collection types in PySpark SQL, including
array, dict, list, set, and tuple. Example,

```
>>> from array import array
>>> from pyspark.sql import SQLContext
>>> sqlCtx = SQLContext(sc)
>>> rdd = sc.parallelize([
...         {"f1" : array('i', [1, 2]), "f2" : {"row1" : 1.0}},
...         {"f1" : array('i', [2, 3]), "f2" : {"row2" : 2.0}}])
>>> srdd = sqlCtx.inferSchema(rdd)
>>> srdd.collect() == [{"f1" : array('i', [1, 2]), "f2" : {"row1" : 1.0}},
...                    {"f1" : array('i', [2, 3]), "f2" : {"row2" : 2.0}}]
True
>>> rdd = sc.parallelize([
...         {"f1" : [[1, 2], [2, 3]], "f2" : set([1, 2]), "f3" : (1, 2)},
...         {"f1" : [[2, 3], [3, 4]], "f2" : set([2, 3]), "f3" : (2, 3)}])
>>> srdd = sqlCtx.inferSchema(rdd)
>>> srdd.collect() == \
... [{"f1" : [[1, 2], [2, 3]], "f2" : set([1, 2]), "f3" : (1, 2)},
...  {"f1" : [[2, 3], [3, 4]], "f2" : set([2, 3]), "f3" : (2, 3)}]
True
```

Author: Kan Zhang <kzhang@apache.org>

Closes #1041 from kanzhang/SPARK-2010 and squashes the following commits:

1b2891d [Kan Zhang] [SPARK-2010] minor doc change and adding a TODO
504f27e [Kan Zhang] [SPARK-2010] Support for nested data in PySpark SQL
2014-06-16 11:11:29 -07: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
Kan Zhang 2550533a28 [SPARK-2079] Support batching when serializing SchemaRDD to Python
Added batching with default batch size 10 in SchemaRDD.javaToPython

Author: Kan Zhang <kzhang@apache.org>

Closes #1023 from kanzhang/SPARK-2079 and squashes the following commits:

2d1915e [Kan Zhang] [SPARK-2079] Add batching in SchemaRDD.javaToPython
19b0c09 [Kan Zhang] [SPARK-2079] Removing unnecessary wrapping in SchemaRDD.javaToPython
2014-06-14 13:17:22 -07:00
Yin Huai 8919685091 [Spark-2137][SQL] Timestamp UDFs broken
https://issues.apache.org/jira/browse/SPARK-2137

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

Closes #1081 from yhuai/SPARK-2137 and squashes the following commits:

c04f910 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SPARK-2137
205f17b [Yin Huai] Make Hive UDF wrapper support Timestamp.
2014-06-13 23:28:57 -07: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
Michael Armbrust 1c2fd015b0 [SPARK-1964][SQL] Add timestamp to HiveMetastoreTypes.toMetastoreType
Author: Michael Armbrust <michael@databricks.com>

Closes #1061 from marmbrus/timestamp and squashes the following commits:

79c3903 [Michael Armbrust] Add timestamp to HiveMetastoreTypes.toMetastoreType()
2014-06-13 12:55:15 -07:00
Michael Armbrust 13f8cfdc04 [SPARK-2135][SQL] Use planner for in-memory scans
Author: Michael Armbrust <michael@databricks.com>

Closes #1072 from marmbrus/cachedStars and squashes the following commits:

8757c8e [Michael Armbrust] Use planner for in-memory scans.
2014-06-12 23:09:41 -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
Patrick Wendell d45e0c6b98 HOTFIX: Forgot to remove false change in previous commit 2014-06-11 15:55:41 -07:00
Patrick Wendell 14e6dc94f6 HOTFIX: PySpark tests should be order insensitive.
This has been messing up the SQL PySpark tests on Jenkins.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #1054 from pwendell/pyspark and squashes the following commits:

1eb5487 [Patrick Wendell] False change
06f062d [Patrick Wendell] HOTFIX: PySpark tests should be order insensitive
2014-06-11 15:54:41 -07:00
Daoyuan ce6deb1e5b [SQL] Code Cleanup: Left Semi Hash Join
Some improvement for PR #837, add another case to white list and use `filter` to build result iterator.

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

Closes #1049 from adrian-wang/clean-LeftSemiJoinHash and squashes the following commits:

b314d5a [Daoyuan] change hashSet name
27579a9 [Daoyuan] add semijoin to white list and use filter to create new iterator in LeftSemiJoinBNL

Signed-off-by: Michael Armbrust <michael@databricks.com>
2014-06-11 12:09:42 -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
Zongheng Yang 601032f5bf HOTFIX: clear() configs in SQLConf-related unit tests.
Thanks goes to @liancheng, who pointed out that `sql/test-only *.SQLConfSuite *.SQLQuerySuite` passed but `sql/test-only *.SQLQuerySuite *.SQLConfSuite` failed. The reason is that some tests use the same test keys and without clear()'ing, they get carried over to other tests. This hotfix simply adds some `clear()` calls.

This problem was not evident on Jenkins before probably because `parallelExecution` is not set to `false` for `sqlCoreSettings`.

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

Closes #1040 from concretevitamin/sqlconf-tests and squashes the following commits:

6d14ceb [Zongheng Yang] HOTFIX: clear() confs in SQLConf related unit tests.
2014-06-10 21:59:01 -07:00
egraldlo 1abbde0e89 [SQL] Add average overflow test case from #978
By @egraldlo.

Author: egraldlo <egraldlo@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #1033 from marmbrus/pr/978 and squashes the following commits:

e228c5e [Michael Armbrust] Remove "test".
762aeaf [Michael Armbrust] Remove unneeded rule. More descriptive name for test table.
d414cd7 [egraldlo] fommatting issues
1153f75 [egraldlo] do best to avoid overflowing in function avg().
2014-06-10 14:07:55 -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
Cheng Lian a9a461c594 Moved hiveOperators.scala to the right package folder
The package is `org.apache.spark.sql.hive.execution`, while the file was placed under `sql/hive/src/main/scala/org/apache/spark/sql/hive/`.

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

Closes #1029 from liancheng/moveHiveOperators and squashes the following commits:

d632eb8 [Cheng Lian] Moved hiveOperators.scala to the right package folder
2014-06-10 01:14:44 -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
Michael Armbrust c6e041d171 [SQL] Simple framework for debugging query execution
Only records number of tuples and unique dataTypes output right now...

Example:
```scala
scala> import org.apache.spark.sql.execution.debug._
scala> hql("SELECT value FROM src WHERE key > 10").debug(sparkContext)

Results returned: 489
== Project [value#1:0] ==
Tuples output: 489
 value StringType: {java.lang.String}
== Filter (key#0:1 > 10) ==
Tuples output: 489
 value StringType: {java.lang.String}
 key IntegerType: {java.lang.Integer}
== HiveTableScan [value#1,key#0], (MetastoreRelation default, src, None), None ==
Tuples output: 500
 value StringType: {java.lang.String}
 key IntegerType: {java.lang.Integer}
```

Author: Michael Armbrust <michael@databricks.com>

Closes #1005 from marmbrus/debug and squashes the following commits:

dcc3ca6 [Michael Armbrust] Add comments.
c9dded2 [Michael Armbrust] Simple framework for debugging query execution
2014-06-09 14:24:19 -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 a6c72ab16e [SPARK-1994][SQL] Weird data corruption bug when running Spark SQL on data in HDFS
Basically there is a race condition (possibly a scala bug?) when these values are recomputed on all of the slaves that results in an incorrect projection being generated (possibly because the GUID uniqueness contract is broken?).

In general we should probably enforce that all expression planing occurs on the driver, as is now occurring here.

Author: Michael Armbrust <michael@databricks.com>

Closes #1004 from marmbrus/fixAggBug and squashes the following commits:

e0c116c [Michael Armbrust] Compute aggregate expression during planning instead of lazily on workers.
2014-06-07 14:20:33 -07:00
witgo 41c4a33105 [SPARK-1841]: update scalatest to version 2.1.5
Author: witgo <witgo@qq.com>

Closes #713 from witgo/scalatest and squashes the following commits:

b627a6a [witgo] merge master
51fb3d6 [witgo] merge master
3771474 [witgo] fix RDDSuite
996d6f9 [witgo] fix TimeStampedWeakValueHashMap test
9dfa4e7 [witgo] merge bug
1479b22 [witgo] merge master
29b9194 [witgo] fix code style
022a7a2 [witgo] fix test dependency
a52c0fa [witgo] fix test dependency
cd8f59d [witgo] Merge branch 'master' of https://github.com/apache/spark into scalatest
046540d [witgo] fix RDDSuite.scala
2c543b9 [witgo] fix ReplSuite.scala
c458928 [witgo] update scalatest to version 2.1.5
2014-06-06 11:45:21 -07:00
Michael Armbrust 8d210560be [SPARK-2050 - 2][SQL] DIV and BETWEEN should not be case sensitive.
Followup: #989

Author: Michael Armbrust <michael@databricks.com>

Closes #994 from marmbrus/caseSensitiveFunctions2 and squashes the following commits:

9d9c8ed [Michael Armbrust] Fix DIV and BETWEEN.
2014-06-06 11:31:37 -07:00
Michael Armbrust 41db44c428 [SPARK-2050][SQL] LIKE, RLIKE and IN in HQL should not be case sensitive.
Author: Michael Armbrust <michael@databricks.com>

Closes #989 from marmbrus/caseSensitiveFuncitons and squashes the following commits:

681de54 [Michael Armbrust] LIKE, RLIKE and IN in HQL should not be case sensitive.
2014-06-05 23:20:59 -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
Syed Hashmi 7782a304ad [SPARK-1942] Stop clearing spark.driver.port in unit tests
stop resetting spark.driver.port in unit tests (scala, java and python).

Author: Syed Hashmi <shashmi@cloudera.com>
Author: CodingCat <zhunansjtu@gmail.com>

Closes #943 from syedhashmi/master and squashes the following commits:

885f210 [Syed Hashmi] Removing unnecessary file (created by mergetool)
b8bd4b5 [Syed Hashmi] Merge remote-tracking branch 'upstream/master'
b895e59 [Syed Hashmi] Revert "[SPARK-1784] Add a new partitioner"
57b6587 [Syed Hashmi] Revert "[SPARK-1784] Add a balanced partitioner"
1574769 [Syed Hashmi] [SPARK-1942] Stop clearing spark.driver.port in unit tests
4354836 [Syed Hashmi] Revert "SPARK-1686: keep schedule() calling in the main thread"
fd36542 [Syed Hashmi] [SPARK-1784] Add a balanced partitioner
6668015 [CodingCat] SPARK-1686: keep schedule() calling in the main thread
4ca94cc [Syed Hashmi] [SPARK-1784] Add a new partitioner
2014-06-03 12:04:47 -07:00
Cheng Lian 862283e9cc Avoid dynamic dispatching when unwrapping Hive data.
This is a follow up of PR #758.

The `unwrapHiveData` function is now composed statically before actual rows are scanned according to the field object inspector to avoid dynamic dispatching cost.

According to the same micro benchmark used in PR #758, this simple change brings slight performance boost: 2.5% for CSV table and 1% for RCFile table.

```
Optimized version:

CSV: 6870 ms, RCFile: 5687 ms
CSV: 6832 ms, RCFile: 5800 ms
CSV: 6822 ms, RCFile: 5679 ms
CSV: 6704 ms, RCFile: 5758 ms
CSV: 6819 ms, RCFile: 5725 ms

Original version:

CSV: 7042 ms, RCFile: 5667 ms
CSV: 6883 ms, RCFile: 5703 ms
CSV: 7115 ms, RCFile: 5665 ms
CSV: 7020 ms, RCFile: 5981 ms
CSV: 6871 ms, RCFile: 5906 ms
```

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

Closes #935 from liancheng/staticUnwrapping and squashes the following commits:

c49c70c [Cheng Lian] Avoid dynamic dispatching when unwrapping Hive data.
2014-06-02 19:20:23 -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
Cheng Lian d000ca98a8 [SPARK-1958] Calling .collect() on a SchemaRDD should call executeCollect() on the underlying query plan.
In cases like `Limit` and `TakeOrdered`, `executeCollect()` makes optimizations that `execute().collect()` will not.

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

Closes #939 from liancheng/spark-1958 and squashes the following commits:

bdc4a14 [Cheng Lian] Copy rows to present immutable data to users
8250976 [Cheng Lian] Added return type explicitly for public API
192a25c [Cheng Lian] [SPARK-1958] Calling .collect() on a SchemaRDD should call executeCollect() on the underlying query plan.
2014-06-02 12:09:43 -07:00
Michael Armbrust 1a0da0ec57 [SQL] SPARK-1964 Add timestamp to hive metastore type parser.
Author: Michael Armbrust <michael@databricks.com>

Closes #913 from marmbrus/timestampMetastore and squashes the following commits:

8e0154f [Michael Armbrust] Add timestamp to hive metastore type parser.
2014-05-31 12:34:22 -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
Cheng Lian cf989601d0 [SPARK-1959] String "NULL" shouldn't be interpreted as null value
JIRA issue: [SPARK-1959](https://issues.apache.org/jira/browse/SPARK-1959)

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

Closes #909 from liancheng/spark-1959 and squashes the following commits:

306659c [Cheng Lian] [SPARK-1959] String "NULL" shouldn't be interpreted as null value
2014-05-30 22:13:11 -07:00
Cheng Lian 8f7141fbc0 [SPARK-1368][SQL] Optimized HiveTableScan
JIRA issue: [SPARK-1368](https://issues.apache.org/jira/browse/SPARK-1368)

This PR introduces two major updates:

- Replaced FP style code with `while` loop and reusable `GenericMutableRow` object in critical path of `HiveTableScan`.
- Using `ColumnProjectionUtils` to help optimizing RCFile and ORC column pruning.

My quick micro benchmark suggests these two optimizations made the optimized version 2x and 2.5x faster when scanning CSV table and RCFile table respectively:

```
Original:

[info] CSV: 27676 ms, RCFile: 26415 ms
[info] CSV: 27703 ms, RCFile: 26029 ms
[info] CSV: 27511 ms, RCFile: 25962 ms

Optimized:

[info] CSV: 13820 ms, RCFile: 10402 ms
[info] CSV: 14158 ms, RCFile: 10691 ms
[info] CSV: 13606 ms, RCFile: 10346 ms
```

The micro benchmark loads a 609MB CVS file (structurally similar to the `src` test table) into a normal Hive table with `LazySimpleSerDe` and a RCFile table, then scans these tables respectively.

Preparation code:

```scala
package org.apache.spark.examples.sql.hive

import org.apache.spark.sql.hive.LocalHiveContext
import org.apache.spark.{SparkConf, SparkContext}

object HiveTableScanPrepare extends App {
  val sparkContext = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))

  val hiveContext = new LocalHiveContext(sparkContext)

  import hiveContext._

  hql("drop table scan_csv")
  hql("drop table scan_rcfile")

  hql("""create table scan_csv (key int, value string)
        |  row format serde 'org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe'
        |  with serdeproperties ('field.delim'=',')
      """.stripMargin)

  hql(s"""load data local inpath "${args(0)}" into table scan_csv""")

  hql("""create table scan_rcfile (key int, value string)
        |  row format serde 'org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe'
        |stored as
        |  inputformat 'org.apache.hadoop.hive.ql.io.RCFileInputFormat'
        |  outputformat 'org.apache.hadoop.hive.ql.io.RCFileOutputFormat'
      """.stripMargin)

  hql(
    """
      |from scan_csv
      |insert overwrite table scan_rcfile
      |select scan_csv.key, scan_csv.value
    """.stripMargin)
}
```

Benchmark code:

```scala
package org.apache.spark.examples.sql.hive

import org.apache.spark.sql.hive.LocalHiveContext
import org.apache.spark.{SparkConf, SparkContext}

object HiveTableScanBenchmark extends App {
  val sparkContext = new SparkContext(
    new SparkConf()
      .setMaster("local")
      .setAppName(getClass.getSimpleName.stripSuffix("$")))

  val hiveContext = new LocalHiveContext(sparkContext)

  import hiveContext._

  val scanCsv = hql("select key from scan_csv")
  val scanRcfile = hql("select key from scan_rcfile")

  val csvDuration = benchmark(scanCsv.count())
  val rcfileDuration = benchmark(scanRcfile.count())

  println(s"CSV: $csvDuration ms, RCFile: $rcfileDuration ms")

  def benchmark(f: => Unit) = {
    val begin = System.currentTimeMillis()
    f
    val end = System.currentTimeMillis()
    end - begin
  }
}
```

@marmbrus Please help review, thanks!

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

Closes #758 from liancheng/fastHiveTableScan and squashes the following commits:

4241a19 [Cheng Lian] Distinguishes sorted and possibly not sorted operations more accurately in HiveComparisonTest
cf640d8 [Cheng Lian] More HiveTableScan optimisations:
bf0e7dc [Cheng Lian] Added SortedOperation pattern to match *some* definitely sorted operations and avoid some sorting cost in HiveComparisonTest.
6d1c642 [Cheng Lian] Using ColumnProjectionUtils to optimise RCFile and ORC column pruning
eb62fd3 [Cheng Lian] [SPARK-1368] Optimized HiveTableScan
2014-05-29 15:24: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
LY Lai 0682567450 [SQL] SPARK-1922
Allow underscore in column name of a struct field https://issues.apache.org/jira/browse/SPARK-1922 .

Author: LY Lai <ly.lai@vpon.com>

Closes #873 from lyuanlai/master and squashes the following commits:

2253263 [LY Lai] Allow underscore in struct field column name
2014-05-27 16:08:38 -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
Aaron Davidson c3576ffcd7 [SQL] Minor: Introduce SchemaRDD#aggregate() for simple aggregations
```scala
rdd.aggregate(Sum('val))
```
is just shorthand for

```scala
rdd.groupBy()(Sum('val))
```

but seems be more natural than doing a groupBy with no grouping expressions when you really just want an aggregation over all rows.

Did not add a JavaSchemaRDD or Python API, as these seem to be lacking several other methods like groupBy() already -- leaving that cleanup for future patches.

Author: Aaron Davidson <aaron@databricks.com>

Closes #874 from aarondav/schemardd and squashes the following commits:

e9e68ee [Aaron Davidson] Add comment
db6afe2 [Aaron Davidson] Introduce SchemaRDD#aggregate() for simple aggregations
2014-05-25 18:37:44 -07:00
Reynold Xin d66642e397 SPARK-1822: Some minor cleanup work on SchemaRDD.count()
Minor cleanup following #841.

Author: Reynold Xin <rxin@apache.org>

Closes #868 from rxin/schema-count and squashes the following commits:

5442651 [Reynold Xin] SPARK-1822: Some minor cleanup work on SchemaRDD.count()
2014-05-25 01:44:49 -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
Cheng Lian 5afe6af0b1 [SPARK-1913][SQL] Bug fix: column pruning error in Parquet support
JIRA issue: [SPARK-1913](https://issues.apache.org/jira/browse/SPARK-1913)

When scanning Parquet tables, attributes referenced only in predicates that are pushed down are not passed to the `ParquetTableScan` operator and causes exception.

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

Closes #863 from liancheng/spark-1913 and squashes the following commits:

f976b73 [Cheng Lian] Addessed the readability issue commented by @rxin
f5b257d [Cheng Lian] Added back comments deleted by mistake
ae60ab3 [Cheng Lian] [SPARK-1913] Attributes referenced only in predicates pushed down should remain in ParquetTableScan operator
2014-05-24 20:42:01 -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
Tathagata Das 7f0cfe47f4 [Hotfix] Blacklisted flaky HiveCompatibility test
`lateral_view_outer` query sometimes returns a different set of 10 rows.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #838 from tdas/hive-test-fix2 and squashes the following commits:

9128a0d [Tathagata Das] Blacklisted flaky HiveCompatibility test.
2014-05-20 10:27:12 -07:00
witgo 6a2c5c610c [SPARK-1875]NoClassDefFoundError: StringUtils when building with hadoop 1.x and hive
Author: witgo <witgo@qq.com>

Closes #824 from witgo/SPARK-1875_commons-lang-2.6 and squashes the following commits:

ef7231d [witgo] review commit
ead3c3b [witgo] SPARK-1875:NoClassDefFoundError: StringUtils when building against Hadoop 1
2014-05-19 19:40:29 -07:00
Andre Schumacher 40d6acd6ba SPARK-1487 [SQL] Support record filtering via predicate pushdown in Parquet
Simple filter predicates such as LessThan, GreaterThan, etc., where one side is a literal and the other one a NamedExpression are now pushed down to the underlying ParquetTableScan. Here are some results for a microbenchmark with a simple schema of six fields of different types where most records failed the test:

             | Uncompressed  | Compressed
-------------| ------------- | -------------
File size  |     10 GB  | 2 GB
Speedup |      2         | 1.8

Since mileage may vary I added a new option to SparkConf:

`org.apache.spark.sql.parquet.filter.pushdown`

Default value would be `true` and setting it to `false` disables the pushdown. When most rows are expected to pass the filter or when there are few fields performance can be better when pushdown is disabled. The default should fit situations with a reasonable number of (possibly nested) fields where not too many records on average pass the filter.

Because of an issue with Parquet ([see here](https://github.com/Parquet/parquet-mr/issues/371])) currently only predicates on non-nullable attributes are pushed down. If one would know that for a given table no optional fields have missing values one could also allow overriding this.

Author: Andre Schumacher <andre.schumacher@iki.fi>

Closes #511 from AndreSchumacher/parquet_filter and squashes the following commits:

16bfe83 [Andre Schumacher] Removing leftovers from merge during rebase
7b304ca [Andre Schumacher] Fixing formatting
c36d5cb [Andre Schumacher] Scalastyle
3da98db [Andre Schumacher] Second round of review feedback
7a78265 [Andre Schumacher] Fixing broken formatting in ParquetFilter
a86553b [Andre Schumacher] First round of code review feedback
b0f7806 [Andre Schumacher] Optimizing imports in ParquetTestData
85fea2d [Andre Schumacher] Adding SparkConf setting to disable filter predicate pushdown
f0ad3cf [Andre Schumacher] Undoing changes not needed for this PR
210e9cb [Andre Schumacher] Adding disjunctive filter predicates
a93a588 [Andre Schumacher] Adding unit test for filtering
6d22666 [Andre Schumacher] Extending ParquetFilters
93e8192 [Andre Schumacher] First commit Parquet record filtering
2014-05-16 13:41:41 -07:00
Michael Armbrust 032d6632ad [SQL] Implement between in hql
Author: Michael Armbrust <michael@databricks.com>

Closes #804 from marmbrus/between and squashes the following commits:

ae24672 [Michael Armbrust] add golden answer.
d9997ef [Michael Armbrust] Implement between in hql.
9bd4433 [Michael Armbrust] Better error on parse failures.
2014-05-16 11:47:00 -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
Michael Armbrust a4aafe5f9f [SQL] Fix tiny/small ints from HiveMetastore.
Author: Michael Armbrust <michael@databricks.com>

Closes #797 from marmbrus/smallInt and squashes the following commits:

2db9dae [Michael Armbrust] Fix tiny/small ints from HiveMetastore.
2014-05-15 16:50:42 -07:00
Stevo Slavić e66e31be51 SPARK-1803 Replaced colon in filenames with a dash
This patch replaces colon in several filenames with dash to make these filenames Windows compatible.

Author: Stevo Slavić <sslavic@gmail.com>
Author: Stevo Slavic <sslavic@gmail.com>

Closes #739 from sslavic/SPARK-1803 and squashes the following commits:

3ec66eb [Stevo Slavic] Removed extra empty line which was causing test to fail
b967cc3 [Stevo Slavić] Aligned tests and names of test resources
2b12776 [Stevo Slavić] Fixed a typo in file name
1c5dfff [Stevo Slavić] Replaced colon in file name with dash
8f5bf7f [Stevo Slavić] Replaced colon in file name with dash
c5b5083 [Stevo Slavić] Replaced colon in file name with dash
a49801f [Stevo Slavić] Replaced colon in file name with dash
401d99e [Stevo Slavić] Replaced colon in file name with dash
40a9621 [Stevo Slavić] Replaced colon in file name with dash
4774580 [Stevo Slavić] Replaced colon in file name with dash
004f8bb [Stevo Slavić] Replaced colon in file name with dash
d6a3e2c [Stevo Slavić] Replaced colon in file name with dash
b585126 [Stevo Slavić] Replaced colon in file name with dash
028e48a [Stevo Slavić] Replaced colon in file name with dash
ece0507 [Stevo Slavić] Replaced colon in file name with dash
84f5d2f [Stevo Slavić] Replaced colon in file name with dash
2fc7854 [Stevo Slavić] Replaced colon in file name with dash
9e1467d [Stevo Slavić] Replaced colon in file name with dash
2014-05-15 16:44:14 -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
Takuya UESHIN db8cc6f28a [SPARK-1845] [SQL] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of ...
...Scala collections.

When I execute `orderBy` or `limit` for `SchemaRDD` including `ArrayType` or `MapType`, `SparkSqlSerializer` throws the following exception:

```
com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.$colon$colon
```

or

```
com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.Vector
```

or

```
com.esotericsoftware.kryo.KryoException: Class cannot be created (missing no-arg constructor): scala.collection.immutable.HashMap$HashTrieMap
```

and so on.

This is because registrations of serializers for each concrete collections are missing in `SparkSqlSerializer`.
I believe it should use `AllScalaRegistrar`.
`AllScalaRegistrar` covers a lot of serializers for concrete classes of `Seq`, `Map` for `ArrayType`, `MapType`.

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

Closes #790 from ueshin/issues/SPARK-1845 and squashes the following commits:

d1ed992 [Takuya UESHIN] Use AllScalaRegistrar for SparkSqlSerializer to register serializers of Scala collections.
2014-05-15 11:20:21 -07:00
witgo bae07e36a6 fix different versions of commons-lang dependency and apache/spark#746 addendum
Author: witgo <witgo@qq.com>

Closes #754 from witgo/commons-lang and squashes the following commits:

3ebab31 [witgo] merge master
f3b8fa2 [witgo] merge master
2083fae [witgo] repeat definition
5599cdb [witgo] multiple version of sbt  dependency
c1b66a1 [witgo] fix different versions of commons-lang dependency
2014-05-14 22:26:26 -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
wangfei 44165fc91a [SPARK-1826] fix the head notation of package object dsl
Author: wangfei <scnbwf@yeah.net>

Closes #765 from scwf/dslfix and squashes the following commits:

d2d1a9d [wangfei] Update package.scala
66ff53b [wangfei] fix the head notation of package object dsl
2014-05-14 17:59:11 -07:00
Patrick Wendell d58cb33ffa SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies
See https://issues.apache.org/jira/browse/SPARK-1828 for more information.

This is being submitted to Jenkin's for testing. The dependency won't fully
propagate in Maven central for a few more hours.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #767 from pwendell/hive-shaded and squashes the following commits:

ea10ac5 [Patrick Wendell] SPARK-1828: Created forked version of hive-exec that doesn't bundle other dependencies
2014-05-14 09:51:01 -07:00
Michael Armbrust 6ce0884446 [SQL] Improve column pruning.
Fixed a bug that was preventing us from ever pruning beneath Joins.

## TPC-DS Q3
### Before:
```
Aggregate false, [d_year#12,i_brand#65,i_brand_id#64], [d_year#12,i_brand_id#64 AS brand_id#0,i_brand#65 AS brand#1,SUM(PartialSum#79) AS sum_agg#2]
 Exchange (HashPartitioning [d_year#12:0,i_brand#65:1,i_brand_id#64:2], 150)
  Aggregate true, [d_year#12,i_brand#65,i_brand_id#64], [d_year#12,i_brand#65,i_brand_id#64,SUM(CAST(ss_ext_sales_price#49, DoubleType)) AS PartialSum#79]
   Project [d_year#12:6,i_brand#65:59,i_brand_id#64:58,ss_ext_sales_price#49:43]
    HashJoin [ss_item_sk#36], [i_item_sk#57], BuildRight
     Exchange (HashPartitioning [ss_item_sk#36:30], 150)
      HashJoin [d_date_sk#6], [ss_sold_date_sk#34], BuildRight
       Exchange (HashPartitioning [d_date_sk#6:0], 150)
        Filter (d_moy#14:8 = 12)
         HiveTableScan [d_date_sk#6,d_date_id#7,d_date#8,d_month_seq#9,d_week_seq#10,d_quarter_seq#11,d_year#12,d_dow#13,d_moy#14,d_dom#15,d_qoy#16,d_fy_year#17,d_fy_quarter_seq#18,d_fy_week_seq#19,d_day_name#20,d_quarter_name#21,d_holiday#22,d_weekend#23,d_following_holiday#24,d_first_dom#25,d_last_dom#26,d_same_day_ly#27,d_same_day_lq#28,d_current_day#29,d_current_week#30,d_current_month#31,d_current_quarter#32,d_current_year#33], (MetastoreRelation default, date_dim, Some(dt)), None
       Exchange (HashPartitioning [ss_sold_date_sk#34:0], 150)
        HiveTableScan [ss_sold_date_sk#34,ss_sold_time_sk#35,ss_item_sk#36,ss_customer_sk#37,ss_cdemo_sk#38,ss_hdemo_sk#39,ss_addr_sk#40,ss_store_sk#41,ss_promo_sk#42,ss_ticket_number#43,ss_quantity#44,ss_wholesale_cost#45,ss_list_price#46,ss_sales_price#47,ss_ext_discount_amt#48,ss_ext_sales_price#49,ss_ext_wholesale_cost#50,ss_ext_list_price#51,ss_ext_tax#52,ss_coupon_amt#53,ss_net_paid#54,ss_net_paid_inc_tax#55,ss_net_profit#56], (MetastoreRelation default, store_sales, None), None
     Exchange (HashPartitioning [i_item_sk#57:0], 150)
      Filter (i_manufact_id#70:13 = 436)
       HiveTableScan [i_item_sk#57,i_item_id#58,i_rec_start_date#59,i_rec_end_date#60,i_item_desc#61,i_current_price#62,i_wholesale_cost#63,i_brand_id#64,i_brand#65,i_class_id#66,i_class#67,i_category_id#68,i_category#69,i_manufact_id#70,i_manufact#71,i_size#72,i_formulation#73,i_color#74,i_units#75,i_container#76,i_manager_id#77,i_product_name#78], (MetastoreRelation default, item, None), None
```
### After
```
Aggregate false, [d_year#172,i_brand#225,i_brand_id#224], [d_year#172,i_brand_id#224 AS brand_id#160,i_brand#225 AS brand#161,SUM(PartialSum#239) AS sum_agg#162]
 Exchange (HashPartitioning [d_year#172:0,i_brand#225:1,i_brand_id#224:2], 150)
  Aggregate true, [d_year#172,i_brand#225,i_brand_id#224], [d_year#172,i_brand#225,i_brand_id#224,SUM(CAST(ss_ext_sales_price#209, DoubleType)) AS PartialSum#239]
   Project [d_year#172:1,i_brand#225:5,i_brand_id#224:3,ss_ext_sales_price#209:0]
    HashJoin [ss_item_sk#196], [i_item_sk#217], BuildRight
     Exchange (HashPartitioning [ss_item_sk#196:2], 150)
      Project [ss_ext_sales_price#209:2,d_year#172:1,ss_item_sk#196:3]
       HashJoin [d_date_sk#166], [ss_sold_date_sk#194], BuildRight
        Exchange (HashPartitioning [d_date_sk#166:0], 150)
         Project [d_date_sk#166:0,d_year#172:1]
          Filter (d_moy#174:2 = 12)
           HiveTableScan [d_date_sk#166,d_year#172,d_moy#174], (MetastoreRelation default, date_dim, Some(dt)), None
        Exchange (HashPartitioning [ss_sold_date_sk#194:2], 150)
         HiveTableScan [ss_ext_sales_price#209,ss_item_sk#196,ss_sold_date_sk#194], (MetastoreRelation default, store_sales, None), None
     Exchange (HashPartitioning [i_item_sk#217:1], 150)
      Project [i_brand_id#224:0,i_item_sk#217:1,i_brand#225:2]
       Filter (i_manufact_id#230:3 = 436)
        HiveTableScan [i_brand_id#224,i_item_sk#217,i_brand#225,i_manufact_id#230], (MetastoreRelation default, item, None), None
```

Author: Michael Armbrust <michael@databricks.com>

Closes #729 from marmbrus/fixPruning and squashes the following commits:

5feeff0 [Michael Armbrust] Improve column pruning.
2014-05-13 23:27:22 -07:00
larvaboy c33b8dcbf6 Implement ApproximateCountDistinct for SparkSql
Add the implementation for ApproximateCountDistinct to SparkSql. We use the HyperLogLog algorithm implemented in stream-lib, and do the count in two phases: 1) counting the number of distinct elements in each partitions, and 2) merge the HyperLogLog results from different partitions.

A simple serializer and test cases are added as well.

Author: larvaboy <larvaboy@gmail.com>

Closes #737 from larvaboy/master and squashes the following commits:

bd8ef3f [larvaboy] Add support of user-provided standard deviation to ApproxCountDistinct.
9ba8360 [larvaboy] Fix alignment and null handling issues.
95b4067 [larvaboy] Add a test case for count distinct and approximate count distinct.
f57917d [larvaboy] Add the parser for the approximate count.
a2d5d10 [larvaboy] Add ApproximateCountDistinct aggregates and functions.
7ad273a [larvaboy] Add SparkSql serializer for HyperLogLog.
1d9aacf [larvaboy] Fix a minor typo in the toString method of the Count case class.
653542b [larvaboy] Fix a couple of minor typos.
2014-05-13 21:26:08 -07:00
Michael Armbrust 44233865cf [SQL] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext.
Author: Michael Armbrust <michael@databricks.com>

Closes #761 from marmbrus/existingContext and squashes the following commits:

4651051 [Michael Armbrust] Make it possible to create Java/Python SQLContexts from an existing Scala SQLContext.
2014-05-13 21:23:51 -07:00
Andrew Ash 156df87e7c SPARK-1757 Failing test for saving null primitives with .saveAsParquetFile()
https://issues.apache.org/jira/browse/SPARK-1757

The first test succeeds, but the second test fails with exception:

```
[info] - save and load case class RDD with Nones as parquet *** FAILED *** (14 milliseconds)
[info]   java.lang.RuntimeException: Unsupported datatype StructType(List())
[info]   at scala.sys.package$.error(package.scala:27)
[info]   at org.apache.spark.sql.parquet.ParquetTypesConverter$.fromDataType(ParquetRelation.scala:201)
[info]   at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235)
[info]   at org.apache.spark.sql.parquet.ParquetTypesConverter$$anonfun$1.apply(ParquetRelation.scala:235)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
[info]   at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
[info]   at scala.collection.immutable.List.foreach(List.scala:318)
[info]   at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
[info]   at scala.collection.AbstractTraversable.map(Traversable.scala:105)
[info]   at org.apache.spark.sql.parquet.ParquetTypesConverter$.convertFromAttributes(ParquetRelation.scala:234)
[info]   at org.apache.spark.sql.parquet.ParquetTypesConverter$.writeMetaData(ParquetRelation.scala:267)
[info]   at org.apache.spark.sql.parquet.ParquetRelation$.createEmpty(ParquetRelation.scala:143)
[info]   at org.apache.spark.sql.parquet.ParquetRelation$.create(ParquetRelation.scala:122)
[info]   at org.apache.spark.sql.execution.SparkStrategies$ParquetOperations$.apply(SparkStrategies.scala:139)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$1.apply(QueryPlanner.scala:58)
[info]   at scala.collection.Iterator$$anon$13.hasNext(Iterator.scala:371)
[info]   at org.apache.spark.sql.catalyst.planning.QueryPlanner.apply(QueryPlanner.scala:59)
[info]   at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan$lzycompute(SQLContext.scala:264)
[info]   at org.apache.spark.sql.SQLContext$QueryExecution.sparkPlan(SQLContext.scala:264)
[info]   at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan$lzycompute(SQLContext.scala:265)
[info]   at org.apache.spark.sql.SQLContext$QueryExecution.executedPlan(SQLContext.scala:265)
[info]   at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:268)
[info]   at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:268)
[info]   at org.apache.spark.sql.SchemaRDDLike$class.saveAsParquetFile(SchemaRDDLike.scala:66)
[info]   at org.apache.spark.sql.SchemaRDD.saveAsParquetFile(SchemaRDD.scala:98)
```

Author: Andrew Ash <andrew@andrewash.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #690 from ash211/rdd-parquet-save and squashes the following commits:

747a0b9 [Andrew Ash] Merge pull request #1 from marmbrus/pr/690
54bd00e [Michael Armbrust] Need to put Option first since Option <: Seq.
8f3f281 [Andrew Ash] SPARK-1757 Add failing test for saving SparkSQL Schemas with Option[?] fields as parquet
2014-05-12 19:23:39 -07:00
Michael Armbrust 2f1a337358 [SQL] Make Hive Metastore conversion functions publicly visible.
I need this to be public for the implementation of SharkServer2.  However, I think this functionality is generally useful and should be pretty stable.

Author: Michael Armbrust <michael@databricks.com>

Closes #750 from marmbrus/metastoreTypes and squashes the following commits:

f51b62e [Michael Armbrust] Make Hive Metastore conversion functions publicly visible.
2014-05-12 18:40:30 -07:00
Sean Owen 8586bf564f SPARK-1802. Audit dependency graph when Spark is built with -Phive
This initial commit resolves the conflicts in the Hive profiles as noted in https://issues.apache.org/jira/browse/SPARK-1802 .

Most of the fix was to note that Hive drags in Avro, and so if the hive module depends on Spark's version of the `avro-*` dependencies, it will pull in our exclusions as needed too. But I found we need to copy some exclusions between the two Avro dependencies to get this right. And then had to squash some commons-logging intrusions.

This turned up another annoying find, that `hive-exec` is basically an "assembly" artifact that _also_ packages all of its transitive dependencies. This means the final assembly shows lots of collisions between itself and its dependencies, and even other project dependencies. I have a TODO to examine whether that is going to be a deal-breaker or not.

In the meantime I'm going to tack on a second commit to this PR that will also fix some similar, last collisions in the YARN profile.

Author: Sean Owen <sowen@cloudera.com>

Closes #744 from srowen/SPARK-1802 and squashes the following commits:

a856604 [Sean Owen] Resolve JAR version conflicts specific to Hive profile
2014-05-12 14:17:25 -07:00
Sean Owen 7120a2979d SPARK-1798. Tests should clean up temp files
Three issues related to temp files that tests generate – these should be touched up for hygiene but are not urgent.

Modules have a log4j.properties which directs the unit-test.log output file to a directory like `[module]/target/unit-test.log`. But this ends up creating `[module]/[module]/target/unit-test.log` instead of former.

The `work/` directory is not deleted by "mvn clean", in the parent and in modules. Neither is the `checkpoint/` directory created under the various external modules.

Many tests create a temp directory, which is not usually deleted. This can be largely resolved by calling `deleteOnExit()` at creation and trying to call `Utils.deleteRecursively` consistently to clean up, sometimes in an `@After` method.

_If anyone seconds the motion, I can create a more significant change that introduces a new test trait along the lines of `LocalSparkContext`, which provides management of temp directories for subclasses to take advantage of._

Author: Sean Owen <sowen@cloudera.com>

Closes #732 from srowen/SPARK-1798 and squashes the following commits:

5af578e [Sean Owen] Try to consistently delete test temp dirs and files, and set deleteOnExit() for each
b21b356 [Sean Owen] Remove work/ and checkpoint/ dirs with mvn clean
bdd0f41 [Sean Owen] Remove duplicate module dir in log4j.properties output path for tests
2014-05-12 14:16:19 -07:00
Matei Zaharia 7eefc9d2b3 SPARK-1708. Add a ClassTag on Serializer and things that depend on it
This pull request contains a rebased patch from @heathermiller (https://github.com/heathermiller/spark/pull/1) to add ClassTags on Serializer and types that depend on it (Broadcast and AccumulableCollection). Putting these in the public API signatures now will allow us to use Scala Pickling for serialization down the line without breaking binary compatibility.

One question remaining is whether we also want them on Accumulator -- Accumulator is passed as part of a bigger Task or TaskResult object via the closure serializer so it doesn't seem super useful to add the ClassTag there. Broadcast and AccumulableCollection in contrast were being serialized directly.

CC @rxin, @pwendell, @heathermiller

Author: Matei Zaharia <matei@databricks.com>

Closes #700 from mateiz/spark-1708 and squashes the following commits:

1a3d8b0 [Matei Zaharia] Use fake ClassTag in Java
3b449ed [Matei Zaharia] test fix
2209a27 [Matei Zaharia] Code style fixes
9d48830 [Matei Zaharia] Add a ClassTag on Serializer and things that depend on it
2014-05-10 12:10:24 -07:00
Takuya UESHIN 8e94d2721a [SPARK-1778] [SQL] Add 'limit' transformation to SchemaRDD.
Add `limit` transformation to `SchemaRDD`.

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

Closes #711 from ueshin/issues/SPARK-1778 and squashes the following commits:

33169df [Takuya UESHIN] Add 'limit' transformation to SchemaRDD.
2014-05-10 12:03:27 -07:00
Takuya UESHIN 322b1808d2 [SPARK-1754] [SQL] Add missing arithmetic DSL operations.
Add missing arithmetic DSL operations: `unary_-`, `%`.

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

Closes #689 from ueshin/issues/SPARK-1754 and squashes the following commits:

a09ef69 [Takuya UESHIN] Add also missing ! (not) operation.
f73ae2c [Takuya UESHIN] Remove redundant tests.
5b3f087 [Takuya UESHIN] Add tests relating DSL operations.
e09c5b8 [Takuya UESHIN] Add missing arithmetic DSL operations.
2014-05-08 15:31:47 -07:00
Michael Armbrust 19c8fb02bc [SQL] Improve SparkSQL Aggregates
* Add native min/max (was using hive before).
* Handle nulls correctly in Avg and Sum.

Author: Michael Armbrust <michael@databricks.com>

Closes #683 from marmbrus/aggFixes and squashes the following commits:

64fe30b [Michael Armbrust] Improve SparkSQL Aggregates * Add native min/max (was using hive before). * Handle nulls correctly in Avg and Sum.
2014-05-08 01:08:43 -04:00
Cheng Hao ca43186867 [SQL] Fix Performance Issue in data type casting
Using lazy val object instead of function in the class Cast, which improved the performance nearly by 2X in my local micro-benchmark.

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

Closes #679 from chenghao-intel/fix_type_casting and squashes the following commits:

71b0902 [Cheng Hao] using lazy val object instead of function for data type casting
2014-05-07 16:54:58 -04:00
Kan Zhang 967635a242 [SPARK-1460] Returning SchemaRDD instead of normal RDD on Set operations...
... that do not change schema

Author: Kan Zhang <kzhang@apache.org>

Closes #448 from kanzhang/SPARK-1460 and squashes the following commits:

111e388 [Kan Zhang] silence MiMa errors in EdgeRDD and VertexRDD
91dc787 [Kan Zhang] Taking into account newly added Ordering param
79ed52a [Kan Zhang] [SPARK-1460] Returning SchemaRDD on Set operations that do not change schema
2014-05-07 09:41:31 -07:00
Cheng Hao 3eb53bd59e [WIP][Spark-SQL] Optimize the Constant Folding for Expression
Currently, expression does not support the "constant null" well in constant folding.
e.g. Sum(a, 0) actually always produces Literal(0, NumericType) in runtime.

For example:
```
explain select isnull(key+null)  from src;
== Logical Plan ==
Project [HiveGenericUdf#isnull((key#30 + CAST(null, IntegerType))) AS c_0#28]
 MetastoreRelation default, src, None

== Optimized Logical Plan ==
Project [true AS c_0#28]
 MetastoreRelation default, src, None

== Physical Plan ==
Project [true AS c_0#28]
 HiveTableScan [], (MetastoreRelation default, src, None), None
```

I've create a new Optimization rule called NullPropagation for such kind of constant folding.

Author: Cheng Hao <hao.cheng@intel.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #482 from chenghao-intel/optimize_constant_folding and squashes the following commits:

2f14b50 [Cheng Hao] Fix code style issues
68b9fad [Cheng Hao] Remove the Literal pattern matching for NullPropagation
29c8166 [Cheng Hao] Update the code for feedback of code review
50444cc [Cheng Hao] Remove the unnecessary null checking
80f9f18 [Cheng Hao] Update the UnitTest for aggregation constant folding
27ea3d7 [Cheng Hao] Fix Constant Folding Bugs & Add More Unittests
b28e03a [Cheng Hao] Merge pull request #1 from marmbrus/pr/482
9ccefdb [Michael Armbrust] Add tests for optimized expression evaluation.
543ef9d [Cheng Hao] fix code style issues
9cf0396 [Cheng Hao] update code according to the code review comment
536c005 [Cheng Hao] Add Exceptional case for constant folding
3c045c7 [Cheng Hao] Optimize the Constant Folding by adding more rules
2645d4f [Cheng Hao] Constant Folding(null propagation)
2014-05-07 03:37:12 -04:00
Michael Armbrust 3c64750bdd [SQL] SPARK-1732 - Support for null primitive values.
I also removed a println that I bumped into.

Author: Michael Armbrust <michael@databricks.com>

Closes #658 from marmbrus/nullPrimitives and squashes the following commits:

a3ec4f3 [Michael Armbrust] Remove println.
695606b [Michael Armbrust] Support for null primatives from using scala and java reflection.
2014-05-05 22:59:42 -07:00
Cheng Lian 6d721c5f71 [SPARK-1678][SPARK-1679] In-memory compression bug fix and made compression configurable, disabled by default
In-memory compression is now configurable in `SparkConf` by the `spark.sql.inMemoryCompression.enabled` property, and is disabled by default.

To help code review, the bug fix is in [the first commit](d537a367ed), compression configuration is in [the second one](4ce09aa8aa).

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

Closes #608 from liancheng/spark-1678 and squashes the following commits:

66c3a8d [Cheng Lian] Renamed in-memory compression configuration key
f8fb3a0 [Cheng Lian] Added assertion for testing .hasNext of various decoder
4ce09aa [Cheng Lian] Made in-memory compression configurable via SparkConf
d537a36 [Cheng Lian] Fixed SPARK-1678
2014-05-05 19:38:59 -07:00
Michael Armbrust 92b2902ca0 Whitelist Hive Tests
This is ready when Jenkins is.

Author: Michael Armbrust <michael@databricks.com>

Closes #596 from marmbrus/moreTests and squashes the following commits:

85be703 [Michael Armbrust] Blacklist MR required tests.
35bc311 [Michael Armbrust] Add hive golden answers.
ede98fd [Michael Armbrust] More hive gitignore
da096ea [Michael Armbrust] update whitelist
2014-05-03 23:13:53 -07:00
Michael Armbrust b295714708 [SQL] Better logging when applying rules.
Author: Michael Armbrust <michael@databricks.com>

Closes #616 from marmbrus/ruleLogging and squashes the following commits:

39c09fe [Michael Armbrust] Fix off by one error.
5af3537 [Michael Armbrust] Better logging when applying rules.
2014-05-03 18:38:44 -07:00
ArcherShao 9347565f41 Update SchemaRDD.scala
Modify spelling errors

Author: ArcherShao <ArcherShao@users.noreply.github.com>

Closes #619 from ArcherShao/patch-1 and squashes the following commits:

2957195 [ArcherShao] Update SchemaRDD.scala
2014-05-03 00:17:36 -07:00
Michael Armbrust a43d9c14f2 [SQL] SPARK-1661 - Fix regex_serde test
The JIRA in question is actually reporting a bug with Shark, but I wanted to make sure Spark SQL did not have similar problems.  This fixes a bug in our parsing code that was preventing the test from executing, but it looks like the RegexSerDe is working in Spark SQL.

Author: Michael Armbrust <michael@databricks.com>

Closes #595 from marmbrus/fixRegexSerdeTest and squashes the following commits:

a4dc612 [Michael Armbrust] Add files created by hive to gitignore.
efa6402 [Michael Armbrust] Fix Hive serde_regex test.
2014-05-01 21:32:43 -07:00
witgo 030f2c2126 Improved build configuration
1, Fix SPARK-1441: compile spark core error with hadoop 0.23.x
2, Fix SPARK-1491: maven hadoop-provided profile fails to build
3, Fix org.scala-lang: * ,org.apache.avro:* inconsistent versions dependency
4, A modified on the sql/catalyst/pom.xml,sql/hive/pom.xml,sql/core/pom.xml (Four spaces formatted into two spaces)

Author: witgo <witgo@qq.com>

Closes #480 from witgo/format_pom and squashes the following commits:

03f652f [witgo] review commit
b452680 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom
bee920d [witgo] revert fix SPARK-1629: Spark Core missing commons-lang dependence
7382a07 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom
6902c91 [witgo] fix SPARK-1629: Spark Core missing commons-lang dependence
0da4bc3 [witgo] merge master
d1718ed [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom
e345919 [witgo] add avro dependency to yarn-alpha
77fad08 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom
62d0862 [witgo] Fix org.scala-lang: * inconsistent versions dependency
1a162d7 [witgo] Merge branch 'master' of https://github.com/apache/spark into format_pom
934f24d [witgo] review commit
cf46edc [witgo] exclude jruby
06e7328 [witgo] Merge branch 'SparkBuild' into format_pom
99464d2 [witgo] fix maven hadoop-provided profile fails to build
0c6c1fc [witgo] Fix compile spark core error with hadoop 0.23.x
6851bec [witgo] Maintain consistent SparkBuild.scala, pom.xml
2014-04-28 22:51:46 -07:00
Cheng Hao f735884414 [SQL]Append some missing types for HiveUDF
Add the missing types

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

Closes #459 from chenghao-intel/missing_types and squashes the following commits:

21cba2e [Cheng Hao] Append some missing types for HiveUDF
2014-04-27 23:59:42 -07:00
Takuya UESHIN 8e37ed6eb8 [SPARK-1608] [SQL] Fix Cast.nullable when cast from StringType to NumericType/TimestampType.
`Cast.nullable` should be `true` when cast from `StringType` to `NumericType` or `TimestampType`.
Because if `StringType` expression has an illegal number string or illegal timestamp string, the casted value becomes `null`.

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

Closes #532 from ueshin/issues/SPARK-1608 and squashes the following commits:

065d37c [Takuya UESHIN] Add tests to check nullabilities of cast expressions.
f278ed7 [Takuya UESHIN] Revert test to keep it readable and concise.
9fc9380 [Takuya UESHIN] Fix Cast.nullable when cast from StringType to NumericType/TimestampType.
2014-04-26 14:39:54 -07:00
Michael Armbrust 86ff8b1027 Generalize pattern for planning hash joins.
This will be helpful for [SPARK-1495](https://issues.apache.org/jira/browse/SPARK-1495) and other cases where we want to have custom hash join implementations but don't want to repeat the logic for finding the join keys.

Author: Michael Armbrust <michael@databricks.com>

Closes #418 from marmbrus/hashFilter and squashes the following commits:

d5cc79b [Michael Armbrust] Address @rxin 's comments.
366b6d9 [Michael Armbrust] style fixes
14560eb [Michael Armbrust] Generalize pattern for planning hash joins.
f4809c1 [Michael Armbrust] Move common functions to PredicateHelper.
2014-04-24 21:42:33 -07:00
Mridul Muralidharan 968c0187a1 SPARK-1586 Windows build fixes
Unfortunately, this is not exhaustive - particularly hive tests still fail due to path issues.

Author: Mridul Muralidharan <mridulm80@apache.org>

This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>

Closes #505 from mridulm/windows_fixes and squashes the following commits:

ef12283 [Mridul Muralidharan] Move to org.apache.commons.lang3 for StringEscapeUtils. Earlier version was buggy appparently
cdae406 [Mridul Muralidharan] Remove leaked changes from > 2G fix branch
3267f4b [Mridul Muralidharan] Fix build failures
35b277a [Mridul Muralidharan] Fix Scalastyle failures
bc69d14 [Mridul Muralidharan] Change from hardcoded path separator
10c4d78 [Mridul Muralidharan] Use explicit encoding while using getBytes
1337abd [Mridul Muralidharan] fix classpath while running in windows
2014-04-24 20:48:33 -07:00
Michael Armbrust 4660991e67 [SQL] Add support for parsing indexing into arrays in SQL.
Author: Michael Armbrust <michael@databricks.com>

Closes #518 from marmbrus/parseArrayIndex and squashes the following commits:

afd2d6b [Michael Armbrust] 100 chars
c3d6026 [Michael Armbrust] Add support for parsing indexing into arrays in SQL.
2014-04-24 18:21:00 -07:00
Arun Ramakrishnan 35e3d199f0 SPARK-1438 RDD.sample() make seed param optional
copying form previous pull request https://github.com/apache/spark/pull/462

Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None.

In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention.

Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params.
sample(fraction, withReplacement=false, seed=math.random)
Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it.

If backward compatible is important, 3 new method can be introduced (without default params) like this
sample(fraction)
sample(fraction, withReplacement)
sample(fraction, withReplacement, seed)

Added some tests for the scala RDD takeSample method.

Author: Arun Ramakrishnan <smartnut007@gmail.com>

This patch had conflicts when merged, resolved by
Committer: Matei Zaharia <matei@databricks.com>

Closes #477 from smartnut007/master and squashes the following commits:

07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues
b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler
8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance.
69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue
0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample
2014-04-24 17:27:16 -07:00
Sandeep a03ac222d8 Fix Scala Style
Any comments are welcome

Author: Sandeep <sandeep@techaddict.me>

Closes #531 from techaddict/stylefix-1 and squashes the following commits:

7492730 [Sandeep] Pass 4
98b2428 [Sandeep] fix rxin suggestions
b5e2e6f [Sandeep] Pass 3
05932d7 [Sandeep] fix if else styling 2
08690e5 [Sandeep] fix if else styling
2014-04-24 15:07:23 -07:00
Takuya UESHIN 27b2821cf1 [SPARK-1610] [SQL] Fix Cast to use exact type value when cast from BooleanType to NumericTy...
...pe.

`Cast` from `BooleanType` to `NumericType` are all using `Int` value.
But it causes `ClassCastException` when the casted value is used by the following evaluation like the code below:

``` scala
scala> import org.apache.spark.sql.catalyst._
import org.apache.spark.sql.catalyst._

scala> import types._
import types._

scala> import expressions._
import expressions._

scala> Add(Cast(Literal(true), ShortType), Literal(1.toShort)).eval()
java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Short
	at scala.runtime.BoxesRunTime.unboxToShort(BoxesRunTime.java:102)
	at scala.math.Numeric$ShortIsIntegral$.plus(Numeric.scala:72)
	at org.apache.spark.sql.catalyst.expressions.Add$$anonfun$eval$2.apply(arithmetic.scala:58)
	at org.apache.spark.sql.catalyst.expressions.Add$$anonfun$eval$2.apply(arithmetic.scala:58)
	at org.apache.spark.sql.catalyst.expressions.Expression.n2(Expression.scala:114)
	at org.apache.spark.sql.catalyst.expressions.Add.eval(arithmetic.scala:58)
	at .<init>(<console>:17)
	at .<clinit>(<console>)
	at .<init>(<console>:7)
	at .<clinit>(<console>)
	at $print(<console>)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:483)
	at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:734)
	at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:983)
	at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:573)
	at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:604)
	at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:568)
	at scala.tools.nsc.interpreter.ILoop.reallyInterpret$1(ILoop.scala:760)
	at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:805)
	at scala.tools.nsc.interpreter.ILoop.command(ILoop.scala:717)
	at scala.tools.nsc.interpreter.ILoop.processLine$1(ILoop.scala:581)
	at scala.tools.nsc.interpreter.ILoop.innerLoop$1(ILoop.scala:588)
	at scala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:591)
	at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply$mcZ$sp(ILoop.scala:882)
	at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837)
	at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837)
	at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135)
	at scala.tools.nsc.interpreter.ILoop.process(ILoop.scala:837)
	at scala.tools.nsc.MainGenericRunner.runTarget$1(MainGenericRunner.scala:83)
	at scala.tools.nsc.MainGenericRunner.process(MainGenericRunner.scala:96)
	at scala.tools.nsc.MainGenericRunner$.main(MainGenericRunner.scala:105)
	at scala.tools.nsc.MainGenericRunner.main(MainGenericRunner.scala)
```

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

Closes #533 from ueshin/issues/SPARK-1610 and squashes the following commits:

70f36e8 [Takuya UESHIN] Fix Cast to use exact type value when cast from BooleanType to NumericType.
2014-04-24 09:57:28 -07:00
Michael Armbrust aa77f8a6a6 SPARK-1562 Fix visibility / annotation of Spark SQL APIs
Author: Michael Armbrust <michael@databricks.com>

Closes #489 from marmbrus/sqlDocFixes and squashes the following commits:

acee4f3 [Michael Armbrust] Fix visibility / annotation of Spark SQL APIs
2014-04-22 20:02:33 -07:00
Kan Zhang ea8cea82a0 [SPARK-1570] Fix classloading in JavaSQLContext.applySchema
I think I hit a class loading issue when running JavaSparkSQL example using spark-submit in local mode.

Author: Kan Zhang <kzhang@apache.org>

Closes #484 from kanzhang/SPARK-1570 and squashes the following commits:

feaaeba [Kan Zhang] [SPARK-1570] Fix classloading in JavaSQLContext.applySchema
2014-04-22 15:05:12 -07:00
Andrew Or b3e5366f69 [Fix #274] Document + fix annotation usages
... so that we don't follow an unspoken set of forbidden rules for adding **@AlphaComponent**, **@DeveloperApi**, and **@Experimental** annotations in the code.

In addition, this PR
(1) removes unnecessary `:: * ::` tags,
(2) adds missing `:: * ::` tags, and
(3) removes annotations for internal APIs.

Author: Andrew Or <andrewor14@gmail.com>

Closes #470 from andrewor14/annotations-fix and squashes the following commits:

92a7f42 [Andrew Or] Document + fix annotation usages
2014-04-21 22:24:44 -07:00
Cheng Lian 89f47434e2 Reuses Row object in ExistingRdd.productToRowRdd()
Author: Cheng Lian <lian.cs.zju@gmail.com>

Closes #432 from liancheng/reuseRow and squashes the following commits:

9e6d083 [Cheng Lian] Simplified code with BufferedIterator
52acec9 [Cheng Lian] Reuses Row object in ExistingRdd.productToRowRdd()
2014-04-18 10:02:27 -07:00
CodingCat e31c8ffca6 SPARK-1483: Rename minSplits to minPartitions in public APIs
https://issues.apache.org/jira/browse/SPARK-1483

From the original JIRA: " The parameter name is part of the public API in Scala and Python, since you can pass named parameters to a method, so we should name it to this more descriptive term. Everywhere else we refer to "splits" as partitions." - @mateiz

Author: CodingCat <zhunansjtu@gmail.com>

Closes #430 from CodingCat/SPARK-1483 and squashes the following commits:

4b60541 [CodingCat] deprecate defaultMinSplits
ba2c663 [CodingCat] Rename minSplits to minPartitions in public APIs
2014-04-18 10:01:16 -07:00
Cheng Lian fec462c153 Loads test tables when running "sbt hive/console" without HIVE_DEV_HOME
When running Hive tests, the working directory is `$SPARK_HOME/sql/hive`, while when running `sbt hive/console`, it becomes `$SPARK_HOME`, and test tables are not loaded if `HIVE_DEV_HOME` is not defined.

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

Closes #417 from liancheng/loadTestTables and squashes the following commits:

7cea8d6 [Cheng Lian] Loads test tables when running "sbt hive/console" without HIVE_DEV_HOME
2014-04-16 08:54:34 -07:00
Michael Armbrust 273c2fd08d [SQL] SPARK-1424 Generalize insertIntoTable functions on SchemaRDDs
This makes it possible to create tables and insert into them using the DSL and SQL for the scala and java apis.

Author: Michael Armbrust <michael@databricks.com>

Closes #354 from marmbrus/insertIntoTable and squashes the following commits:

6c6f227 [Michael Armbrust] Create random temporary files in python parquet unit tests.
f5e6d5c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into insertIntoTable
765c506 [Michael Armbrust] Add to JavaAPI.
77b512c [Michael Armbrust] typos.
5c3ef95 [Michael Armbrust] use names for boolean args.
882afdf [Michael Armbrust] Change createTableAs to saveAsTable.  Clean up api annotations.
d07d94b [Michael Armbrust] Add tests, support for creating parquet files and hive tables.
fa3fe81 [Michael Armbrust] Make insertInto available on JavaSchemaRDD as well.  Add createTableAs function.
2014-04-15 20:40:40 -07:00
Ahir Reddy c99bcb7fea SPARK-1374: PySpark API for SparkSQL
An initial API that exposes SparkSQL functionality in PySpark. A PythonRDD composed of dictionaries, with string keys and primitive values (boolean, float, int, long, string) can be converted into a SchemaRDD that supports sql queries.

```
from pyspark.context import SQLContext
sqlCtx = SQLContext(sc)
rdd = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}])
srdd = sqlCtx.applySchema(rdd)
sqlCtx.registerRDDAsTable(srdd, "table1")
srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1")
srdd2.collect()
```
The last line yields ```[{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, {"f1" : 3, "f2": "row3"}]```

Author: Ahir Reddy <ahirreddy@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #363 from ahirreddy/pysql and squashes the following commits:

0294497 [Ahir Reddy] Updated log4j properties to supress Hive Warns
307d6e0 [Ahir Reddy] Style fix
6f7b8f6 [Ahir Reddy] Temporary fix MIMA checker. Since we now assemble Spark jar with Hive, we don't want to check the interfaces of all of our hive dependencies
3ef074a [Ahir Reddy] Updated documentation because classes moved to sql.py
29245bf [Ahir Reddy] Cache underlying SchemaRDD instead of generating and caching PythonRDD
f2312c7 [Ahir Reddy] Moved everything into sql.py
a19afe4 [Ahir Reddy] Doc fixes
6d658ba [Ahir Reddy] Remove the metastore directory created by the HiveContext tests in SparkSQL
521ff6d [Ahir Reddy] Trying to get spark to build with hive
ab95eba [Ahir Reddy] Set SPARK_HIVE=true on jenkins
ded03e7 [Ahir Reddy] Added doc test for HiveContext
22de1d4 [Ahir Reddy] Fixed maven pyrolite dependency
e4da06c [Ahir Reddy] Display message if hive is not built into spark
227a0be [Michael Armbrust] Update API links. Fix Hive example.
58e2aa9 [Michael Armbrust] Build Docs for pyspark SQL Api.  Minor fixes.
4285340 [Michael Armbrust] Fix building of Hive API Docs.
38a92b0 [Michael Armbrust] Add note to future non-python developers about python docs.
337b201 [Ahir Reddy] Changed com.clearspring.analytics stream version from 2.4.0 to 2.5.1 to match SBT build, and added pyrolite to maven build
40491c9 [Ahir Reddy] PR Changes + Method Visibility
1836944 [Michael Armbrust] Fix comments.
e00980f [Michael Armbrust] First draft of python sql programming guide.
b0192d3 [Ahir Reddy] Added Long, Double and Boolean as usable types + unit test
f98a422 [Ahir Reddy] HiveContexts
79621cf [Ahir Reddy] cleaning up cruft
b406ba0 [Ahir Reddy] doctest formatting
20936a5 [Ahir Reddy] Added tests and documentation
e4d21b4 [Ahir Reddy] Added pyrolite dependency
79f739d [Ahir Reddy] added more tests
7515ba0 [Ahir Reddy] added more tests :)
d26ec5e [Ahir Reddy] added test
e9f5b8d [Ahir Reddy] adding tests
906d180 [Ahir Reddy] added todo explaining cost of creating Row object in python
251f99d [Ahir Reddy] for now only allow dictionaries as input
09b9980 [Ahir Reddy] made jrdd explicitly lazy
c608947 [Ahir Reddy] SchemaRDD now has all RDD operations
725c91e [Ahir Reddy] awesome row objects
55d1c76 [Ahir Reddy] return row objects
4fe1319 [Ahir Reddy] output dictionaries correctly
be079de [Ahir Reddy] returning dictionaries works
cd5f79f [Ahir Reddy] Switched to using Scala SQLContext
e948bd9 [Ahir Reddy] yippie
4886052 [Ahir Reddy] even better
c0fb1c6 [Ahir Reddy] more working
043ca85 [Ahir Reddy] working
5496f9f [Ahir Reddy] doesn't crash
b8b904b [Ahir Reddy] Added schema rdd class
67ba875 [Ahir Reddy] java to python, and python to java
bcc0f23 [Ahir Reddy] Java to python
ab6025d [Ahir Reddy] compiling
2014-04-15 00:07:55 -07:00
Sean Owen 0247b5c546 SPARK-1488. Resolve scalac feature warnings during build
For your consideration: scalac currently notes a number of feature warnings during compilation:

```
[warn] there were 65 feature warning(s); re-run with -feature for details
```

Warnings are like:

```
[warn] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SparkContext.scala:1261: implicit conversion method rddToPairRDDFunctions should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scala docs for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn]   implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) =
[warn]                ^
```

scalac is suggesting that it's just best practice to explicitly enable certain language features by importing them where used.

This PR simply adds the imports it suggests (and squashes one other Java warning along the way). This leaves just deprecation warnings in the build.

Author: Sean Owen <sowen@cloudera.com>

Closes #404 from srowen/SPARK-1488 and squashes the following commits:

8598980 [Sean Owen] Quiet scalac warnings about language features by explicitly importing language features.
39bc831 [Sean Owen] Enable -feature in scalac to emit language feature warnings
2014-04-14 19:50:00 -07:00
Cheng Lian 7dbca68e92 [BUGFIX] In-memory columnar storage bug fixes
Fixed several bugs of in-memory columnar storage to make `HiveInMemoryCompatibilitySuite` pass.

@rxin @marmbrus It is reasonable to include `HiveInMemoryCompatibilitySuite` in this PR, but I didn't, since it significantly increases test execution time. What do you think?

**UPDATE** `HiveCompatibilitySuite` has been made to cache tables in memory. `HiveInMemoryCompatibilitySuite` was removed.

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

Closes #374 from liancheng/inMemBugFix and squashes the following commits:

6ad6d9b [Cheng Lian] Merged HiveCompatibilitySuite and HiveInMemoryCompatibilitySuite
5bdbfe7 [Cheng Lian] Revert 882c538 & 8426ddc, which introduced regression
882c538 [Cheng Lian] Remove attributes field from InMemoryColumnarTableScan
32cc9ce [Cheng Lian] Code style cleanup
99382bf [Cheng Lian] Enable compression by default
4390bcc [Cheng Lian] Report error for any Throwable in HiveComparisonTest
d1df4fd [Michael Armbrust] Remove test tables that might always get created anyway?
ab9e807 [Michael Armbrust] Fix the logged console version of failed test cases to use the new syntax.
1965123 [Michael Armbrust] Don't use coalesce for gathering all data to a single partition, as it does not work correctly with mutable rows.
e36cdd0 [Michael Armbrust] Spelling.
2d0e168 [Michael Armbrust] Run Hive tests in-memory too.
6360723 [Cheng Lian] Made PreInsertionCasts support SparkLogicalPlan and InMemoryColumnarTableScan
c9b0f6f [Cheng Lian] Let InsertIntoTable support InMemoryColumnarTableScan
9c8fc40 [Cheng Lian] Disable compression by default
e619995 [Cheng Lian] Bug fix: incorrect byte order in CompressionScheme.columnHeaderSize
8426ddc [Cheng Lian] Bug fix: InMemoryColumnarTableScan should cache columns specified by the attributes argument
036cd09 [Cheng Lian] Clean up unused imports
44591a5 [Cheng Lian] Bug fix: NullableColumnAccessor.hasNext must take nulls into account
052bf41 [Cheng Lian] Bug fix: should only gather compressibility info for non-null values
95b3301 [Cheng Lian] Fixed bugs in IntegralDelta
2014-04-14 15:22:43 -07:00
Patrick Wendell 4bc07eebbf SPARK-1480: Clean up use of classloaders
The Spark codebase is a bit fast-and-loose when accessing classloaders and this has caused a few bugs to surface in master.

This patch defines some utility methods for accessing classloaders. This makes the intention when accessing a classloader much more explicit in the code and fixes a few cases where the wrong one was chosen.

case (a) -> We want the classloader that loaded Spark
case (b) -> We want the context class loader, or if not present, we want (a)

This patch provides a better fix for SPARK-1403 (https://issues.apache.org/jira/browse/SPARK-1403) than the current work around, which it reverts. It also fixes a previously unreported bug that the `./spark-submit` script did not work for running with `local` master. It didn't work because the executor classloader did not properly delegate to the context class loader (if it is defined) and in local mode the context class loader is set by the `./spark-submit` script. A unit test is added for that case.

Author: Patrick Wendell <pwendell@gmail.com>

Closes #398 from pwendell/class-loaders and squashes the following commits:

b4a1a58 [Patrick Wendell] Minor clean up
14f1272 [Patrick Wendell] SPARK-1480: Clean up use of classloaders
2014-04-13 08:58:37 -07:00
Michael Armbrust f99401a630 [SQL] Improve column pruning in the optimizer.
Author: Michael Armbrust <michael@databricks.com>

Closes #378 from marmbrus/columnPruning and squashes the following commits:

779da56 [Michael Armbrust] More consistent naming.
1a4e9ea [Michael Armbrust] More comments.
2f4e7b9 [Michael Armbrust] Improve column pruning in the optimizer.
2014-04-10 16:20:33 -07:00
Sandeep 930b70f052 Remove Unnecessary Whitespace's
stack these together in a commit else they show up chunk by chunk in different commits.

Author: Sandeep <sandeep@techaddict.me>

Closes #380 from techaddict/white_space and squashes the following commits:

b58f294 [Sandeep] Remove Unnecessary Whitespace's
2014-04-10 15:04:13 -07:00
witgo a74fbbbca8 Fix SPARK-1413: Parquet messes up stdout and stdin when used in Spark REPL
Author: witgo <witgo@qq.com>

Closes #325 from witgo/SPARK-1413 and squashes the following commits:

e57cd8e [witgo] use scala reflection to access and call the SLF4JBridgeHandler  methods
45c8f40 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
5e35d87 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
0d5f819 [witgo] review commit
45e5b70 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
fa69dcf [witgo] Merge branch 'master' into SPARK-1413
3c98dc4 [witgo] Merge branch 'master' into SPARK-1413
38160cb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
ba09bcd [witgo] remove set the parquet log level
a63d574 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
5231ecd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
3feb635 [witgo] parquet logger use parent handler
fa00d5d [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
8bb6ffd [witgo] enableLogForwarding note fix
edd9630 [witgo]  move to
f447f50 [witgo] merging master
5ad52bd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413
76670c1 [witgo] review commit
70f3c64 [witgo] Fix SPARK-1413
2014-04-10 10:36:20 -07:00
Patrick Wendell 87bd1f9ef7 SPARK-1093: Annotate developer and experimental API's
This patch marks some existing classes as private[spark] and adds two types of API annotations:
- `EXPERIMENTAL API` = experimental user-facing module
- `DEVELOPER API - UNSTABLE` = developer-facing API that might change

There is some discussion of the different mechanisms for doing this here:
https://issues.apache.org/jira/browse/SPARK-1081

I was pretty aggressive with marking things private. Keep in mind that if we want to open something up in the future we can, but we can never reduce visibility.

A few notes here:
- In the past we've been inconsistent with the visiblity of the X-RDD classes. This patch marks them private whenever there is an existing function in RDD that can directly creat them (e.g. CoalescedRDD and rdd.coalesce()). One trade-off here is users can't subclass them.
- Noted that compression and serialization formats don't have to be wire compatible across versions.
- Compression codecs and serialization formats are semi-private as users typically don't instantiate them directly.
- Metrics sources are made private - user only interacts with them through Spark's reflection

Author: Patrick Wendell <pwendell@gmail.com>
Author: Andrew Or <andrewor14@gmail.com>

Closes #274 from pwendell/private-apis and squashes the following commits:

44179e4 [Patrick Wendell] Merge remote-tracking branch 'apache-github/master' into private-apis
042c803 [Patrick Wendell] spark.annotations -> spark.annotation
bfe7b52 [Patrick Wendell] Adding experimental for approximate counts
8d0c873 [Patrick Wendell] Warning in SparkEnv
99b223a [Patrick Wendell] Cleaning up annotations
e849f64 [Patrick Wendell] Merge pull request #2 from andrewor14/annotations
982a473 [Andrew Or] Generalize jQuery matching for non Spark-core API docs
a01c076 [Patrick Wendell] Merge pull request #1 from andrewor14/annotations
c1bcb41 [Andrew Or] DeveloperAPI -> DeveloperApi
0d48908 [Andrew Or] Comments and new lines (minor)
f3954e0 [Andrew Or] Add identifier tags in comments to work around scaladocs bug
99192ef [Andrew Or] Dynamically add badges based on annotations
824011b [Andrew Or] Add support for injecting arbitrary JavaScript to API docs
037755c [Patrick Wendell] Some changes after working with andrew or
f7d124f [Patrick Wendell] Small fixes
c318b24 [Patrick Wendell] Use CSS styles
e4c76b9 [Patrick Wendell] Logging
f390b13 [Patrick Wendell] Better visibility for workaround constructors
d6b0afd [Patrick Wendell] Small chang to existing constructor
403ba52 [Patrick Wendell] Style fix
870a7ba [Patrick Wendell] Work around for SI-8479
7fb13b2 [Patrick Wendell] Changes to UnionRDD and EmptyRDD
4a9e90c [Patrick Wendell] EXPERIMENTAL API --> EXPERIMENTAL
c581dce [Patrick Wendell] Changes after building against Shark.
8452309 [Patrick Wendell] Style fixes
1ed27d2 [Patrick Wendell] Formatting and coloring of badges
cd7a465 [Patrick Wendell] Code review feedback
2f706f1 [Patrick Wendell] Don't use floats
542a736 [Patrick Wendell] Small fixes
cf23ec6 [Patrick Wendell] Marking GraphX as alpha
d86818e [Patrick Wendell] Another naming change
5a76ed6 [Patrick Wendell] More visiblity clean-up
42c1f09 [Patrick Wendell] Using better labels
9d48cbf [Patrick Wendell] Initial pass
2014-04-09 01:14:46 -07:00
Cheng Lian 0d0493fcf7 [SPARK-1402] Added 3 more compression schemes
JIRA issue: [SPARK-1402](https://issues.apache.org/jira/browse/SPARK-1402)

This PR provides 3 more compression schemes for Spark SQL in-memory columnar storage:

* `BooleanBitSet`
* `IntDelta`
* `LongDelta`

Now there are 6 compression schemes in total, including the no-op `PassThrough` scheme.

Also fixed a bug in PR #286: not all compression schemes are added as available schemes when accessing an in-memory column, and when a column is compressed with an unrecognised scheme, `ColumnAccessor` throws exception.

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

Closes #330 from liancheng/moreCompressionSchemes and squashes the following commits:

1d037b8 [Cheng Lian] Fixed SPARK-1436: in-memory column byte buffer must be able to be accessed multiple times
d7c0e8f [Cheng Lian] Added test suite for IntegralDelta (IntDelta & LongDelta)
3c1ad7a [Cheng Lian] Added test suite for BooleanBitSet, refactored other test suites
44fe4b2 [Cheng Lian] Refactored CompressionScheme, added 3 more compression schemes.
2014-04-07 22:24:12 -07:00
Reynold Xin f27e56aa61 Change timestamp cast semantics. When cast to numeric types, return the unix time in seconds (instead of millis).
@marmbrus @chenghao-intel

Author: Reynold Xin <rxin@apache.org>

Closes #352 from rxin/timestamp-cast and squashes the following commits:

18aacd3 [Reynold Xin] Fixed precision for double.
2adb235 [Reynold Xin] Change timestamp cast semantics. When cast to numeric types, return the unix time in seconds (instead of millis).
2014-04-07 19:28:24 -07:00
Reynold Xin 31e6fff037 Added eval for Rand (without any support for user-defined seed).
Author: Reynold Xin <rxin@apache.org>

Closes #349 from rxin/rand and squashes the following commits:

fd11322 [Reynold Xin] Added eval for Rand (without any support for user-defined seed).
2014-04-07 18:40:08 -07:00
Reynold Xin 55dfd5dcdb Removed the default eval implementation from Expression, and added a bunch of override's in classes I touched.
It is more robust to not provide a default implementation for Expression's.

Author: Reynold Xin <rxin@apache.org>

Closes #350 from rxin/eval-default and squashes the following commits:

0a83b8f [Reynold Xin] Removed the default eval implementation from Expression, and added a bunch of override's in classes I touched.
2014-04-07 18:39:18 -07:00
Reynold Xin 14c9238aa7 [sql] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables.
Author: Reynold Xin <rxin@apache.org>

Closes #348 from rxin/aggregate and squashes the following commits:

f4bc36f [Reynold Xin] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables.
2014-04-07 18:38:44 -07:00
Reynold Xin 83f2a2f14e [sql] Rename Expression.apply to eval for better readability.
Also used this opportunity to add a bunch of override's and made some members private.

Author: Reynold Xin <rxin@apache.org>

Closes #340 from rxin/eval and squashes the following commits:

a7c7ca7 [Reynold Xin] Fixed conflicts in merge.
9069de6 [Reynold Xin] Merge branch 'master' into eval
3ccc313 [Reynold Xin] Merge branch 'master' into eval
1a47e10 [Reynold Xin] Renamed apply to eval for generators and added a bunch of override's.
ea061de [Reynold Xin] Rename Expression.apply to eval for better readability.
2014-04-07 10:45:31 -07:00
Michael Armbrust b5bae849db [SQL] SPARK-1427 Fix toString for SchemaRDD NativeCommands.
Author: Michael Armbrust <michael@databricks.com>

Closes #343 from marmbrus/toStringFix and squashes the following commits:

37198fe [Michael Armbrust] Fix toString for SchemaRDD NativeCommands.
2014-04-07 01:46:50 -07:00
Michael Armbrust accd0999f9 [SQL] SPARK-1371 Hash Aggregation Improvements
Given:
```scala
case class Data(a: Int, b: Int)
val rdd =
  sparkContext
    .parallelize(1 to 200)
    .flatMap(_ => (1 to 50000).map(i => Data(i % 100, i)))
rdd.registerAsTable("data")
cacheTable("data")
```
Before:
```
SELECT COUNT(*) FROM data:[10000000]
16795.567ms
SELECT a, SUM(b) FROM data GROUP BY a
7536.436ms
SELECT SUM(b) FROM data
10954.1ms
```

After:
```
SELECT COUNT(*) FROM data:[10000000]
1372.175ms
SELECT a, SUM(b) FROM data GROUP BY a
2070.446ms
SELECT SUM(b) FROM data
958.969ms
```

Author: Michael Armbrust <michael@databricks.com>

Closes #295 from marmbrus/hashAgg and squashes the following commits:

ec63575 [Michael Armbrust] Add comment.
d0495a9 [Michael Armbrust] Use scaladoc instead.
b4a6887 [Michael Armbrust] Address review comments.
a2d90ba [Michael Armbrust] Capture child output statically to avoid issues with generators and serialization.
7c13112 [Michael Armbrust] Rewrite Aggregate operator to stream input and use projections.  Remove unused local RDD functions implicits.
5096f99 [Michael Armbrust] Make HiveUDAF fields transient since object inspectors are not serializable.
6a4b671 [Michael Armbrust] Add option to avoid binding operators expressions automatically.
92cca08 [Michael Armbrust] Always include serialization debug info when running tests.
1279df2 [Michael Armbrust] Increase default number of partitions.
2014-04-07 00:14:00 -07:00
Aaron Davidson 4106558435 SPARK-1314: Use SPARK_HIVE to determine if we include Hive in packaging
Previously, we based our decision regarding including datanucleus jars based on the existence of a spark-hive-assembly jar, which was incidentally built whenever "sbt assembly" is run. This means that a typical and previously supported pathway would start using hive jars.

This patch has the following features/bug fixes:

- Use of SPARK_HIVE (default false) to determine if we should include Hive in the assembly jar.
- Analagous feature in Maven with -Phive (previously, there was no support for adding Hive to any of our jars produced by Maven)
- assemble-deps fixed since we no longer use a different ASSEMBLY_DIR
- avoid adding log message in compute-classpath.sh to the classpath :)

Still TODO before mergeable:
- We need to download the datanucleus jars outside of sbt. Perhaps we can have spark-class download them if SPARK_HIVE is set similar to how sbt downloads itself.
- Spark SQL documentation updates.

Author: Aaron Davidson <aaron@databricks.com>

Closes #237 from aarondav/master and squashes the following commits:

5dc4329 [Aaron Davidson] Typo fixes
dd4f298 [Aaron Davidson] Doc update
dd1a365 [Aaron Davidson] Eliminate need for SPARK_HIVE at runtime by d/ling datanucleus from Maven
a9269b5 [Aaron Davidson] [WIP] Use SPARK_HIVE to determine if we include Hive in packaging
2014-04-06 17:48:41 -07:00
witgo 7012ffafad Fix SPARK-1420 The maven build error for Spark Catalyst
Author: witgo <witgo@qq.com>

Closes #333 from witgo/SPARK-1420 and squashes the following commits:

902519e [witgo] add dependency scala-reflect to catalyst
2014-04-06 16:03:06 -07:00