Commit graph

1094 commits

Author SHA1 Message Date
Dongjoon Hyun b5f02d6743 [SPARK-13583][CORE][STREAMING] Remove unused imports and add checkstyle rule
## What changes were proposed in this pull request?

After SPARK-6990, `dev/lint-java` keeps Java code healthy and helps PR review by saving much time.
This issue aims remove unused imports from Java/Scala code and add `UnusedImports` checkstyle rule to help developers.

## How was this patch tested?
```
./dev/lint-java
./build/sbt compile
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11438 from dongjoon-hyun/SPARK-13583.
2016-03-03 10:12:32 +00:00
Sean Owen e97fc7f176 [SPARK-13423][WIP][CORE][SQL][STREAMING] Static analysis fixes for 2.x
## What changes were proposed in this pull request?

Make some cross-cutting code improvements according to static analysis. These are individually up for discussion since they exist in separate commits that can be reverted. The changes are broadly:

- Inner class should be static
- Mismatched hashCode/equals
- Overflow in compareTo
- Unchecked warnings
- Misuse of assert, vs junit.assert
- get(a) + getOrElse(b) -> getOrElse(a,b)
- Array/String .size -> .length (occasionally, -> .isEmpty / .nonEmpty) to avoid implicit conversions
- Dead code
- tailrec
- exists(_ == ) -> contains find + nonEmpty -> exists filter + size -> count
- reduce(_+_) -> sum map + flatten -> map

The most controversial may be .size -> .length simply because of its size. It is intended to avoid implicits that might be expensive in some places.

## How was the this patch tested?

Existing Jenkins unit tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #11292 from srowen/SPARK-13423.
2016-03-03 09:54:09 +00:00
Liang-Chi Hsieh f87ce0504e [SPARK-13616][SQL] Let SQLBuilder convert logical plan without a project on top of it
JIRA: https://issues.apache.org/jira/browse/SPARK-13616

## What changes were proposed in this pull request?

It is possibly that a logical plan has been removed `Project` from the top of it. Or the plan doesn't has a top `Project` from the beginning because it is not necessary. Currently the `SQLBuilder` can't convert such plans back to SQL. This change is to add this feature.

## How was this patch tested?

A test is added to `LogicalPlanToSQLSuite`.

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

Closes #11466 from viirya/sqlbuilder-notopselect.
2016-03-02 22:21:49 -08:00
Davies Liu b5a59a0fe2 [SPARK-13601] call failure callbacks before writer.close()
## What changes were proposed in this pull request?

In order to tell OutputStream that the task has failed or not, we should call the failure callbacks BEFORE calling writer.close().

## How was this patch tested?

Added new unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11450 from davies/callback.
2016-03-02 14:35:44 -08:00
gatorsmile 9e01fe2ed1 [SPARK-13535][SQL] Fix Analysis Exceptions when Using Backticks in Transform Clause
#### What changes were proposed in this pull request?
```SQL
FROM
(FROM test SELECT TRANSFORM(key, value) USING 'cat' AS (`thing1` int, thing2 string)) t
SELECT thing1 + 1
```
This query returns an analysis error, like:
```
Failed to analyze query: org.apache.spark.sql.AnalysisException: cannot resolve '`thing1`' given input columns: [`thing1`, thing2]; line 3 pos 7
'Project [unresolvedalias(('thing1 + 1), None)]
+- SubqueryAlias t
   +- ScriptTransformation [key#2,value#3], cat, [`thing1`#6,thing2#7], HiveScriptIOSchema(List(),List(),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),Some(org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe),List((field.delim,	)),List((field.delim,	)),Some(org.apache.hadoop.hive.ql.exec.TextRecordReader),Some(org.apache.hadoop.hive.ql.exec.TextRecordWriter),false)
      +- SubqueryAlias test
         +- Project [_1#0 AS key#2,_2#1 AS value#3]
            +- LocalRelation [_1#0,_2#1], [[1,1],[2,2],[3,3],[4,4],[5,5]]
```

The backpacks of \`thing1\` should be cleaned before entering Parser/Analyzer. This PR fixes this issue.

#### How was this patch tested?

Added a test case and modified an existing test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11415 from gatorsmile/scriptTransform.
2016-03-02 23:07:48 +01:00
Liang-Chi Hsieh 6dfc4a764c [SPARK-13537][SQL] Fix readBytes in VectorizedPlainValuesReader
JIRA: https://issues.apache.org/jira/browse/SPARK-13537

## What changes were proposed in this pull request?

In readBytes of VectorizedPlainValuesReader, we use buffer[offset] to access bytes in buffer. It is incorrect because offset is added with Platform.BYTE_ARRAY_OFFSET when initialization. We should fix it.

## How was this patch tested?

`ParquetHadoopFsRelationSuite` sometimes (depending on the randomly generated data) will be [failed](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/52136/consoleFull) by this bug. After applying this, the test can be passed.

I added a test to `ParquetHadoopFsRelationSuite` with the data which will fail without this patch.

The error exception:

    [info] ParquetHadoopFsRelationSuite:
    [info] - test all data types - StringType (440 milliseconds)
    [info] - test all data types - BinaryType (434 milliseconds)
    [info] - test all data types - BooleanType (406 milliseconds)
    20:59:38.618 ERROR org.apache.spark.executor.Executor: Exception in task 0.0 in stage 2597.0 (TID 67966)
    java.lang.ArrayIndexOutOfBoundsException: 46
	at org.apache.spark.sql.execution.datasources.parquet.VectorizedPlainValuesReader.readBytes(VectorizedPlainValuesReader.java:88)

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

Closes #11418 from viirya/fix-readbytes.
2016-02-28 21:16:06 -08:00
Andrew Or cca79fad66 [SPARK-13526][SQL] Move SQLContext per-session states to new class
## What changes were proposed in this pull request?

This creates a `SessionState`, which groups a few fields that existed in `SQLContext`. Because `HiveContext` extends `SQLContext` we also need to make changes there. This is mainly a cleanup task that will soon pave the way for merging the two contexts.

## How was this patch tested?

Existing unit tests; this patch introduces no change in behavior.

Author: Andrew Or <andrew@databricks.com>

Closes #11405 from andrewor14/refactor-session.
2016-02-27 19:51:28 -08:00
Cheng Lian 99dfcedbfd [SPARK-13457][SQL] Removes DataFrame RDD operations
## What changes were proposed in this pull request?

This is another try of PR #11323.

This PR removes DataFrame RDD operations except for `foreach` and `foreachPartitions` (they are actions rather than transformations). Original calls are now replaced by calls to methods of `DataFrame.rdd`.

PR #11323 was reverted because it introduced a regression: both `DataFrame.foreach` and `DataFrame.foreachPartitions` wrap underlying RDD operations with `withNewExecutionId` to track Spark jobs. But they are removed in #11323.

## How was the this patch tested?

No extra tests are added. Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #11388 from liancheng/remove-df-rdd-ops.
2016-02-27 00:28:30 +08:00
Josh Rosen 633d63a48a [SPARK-12757] Add block-level read/write locks to BlockManager
## Motivation

As a pre-requisite to off-heap caching of blocks, we need a mechanism to prevent pages / blocks from being evicted while they are being read. With on-heap objects, evicting a block while it is being read merely leads to memory-accounting problems (because we assume that an evicted block is a candidate for garbage-collection, which will not be true during a read), but with off-heap memory this will lead to either data corruption or segmentation faults.

## Changes

### BlockInfoManager and reader/writer locks

This patch adds block-level read/write locks to the BlockManager. It introduces a new `BlockInfoManager` component, which is contained within the `BlockManager`, holds the `BlockInfo` objects that the `BlockManager` uses for tracking block metadata, and exposes APIs for locking blocks in either shared read or exclusive write modes.

`BlockManager`'s `get*()` and `put*()` methods now implicitly acquire the necessary locks. After a `get()` call successfully retrieves a block, that block is locked in a shared read mode. A `put()` call will block until it acquires an exclusive write lock. If the write succeeds, the write lock will be downgraded to a shared read lock before returning to the caller. This `put()` locking behavior allows us store a block and then immediately turn around and read it without having to worry about it having been evicted between the write and the read, which will allow us to significantly simplify `CacheManager` in the future (see #10748).

See `BlockInfoManagerSuite`'s test cases for a more detailed specification of the locking semantics.

### Auto-release of locks at the end of tasks

Our locking APIs support explicit release of locks (by calling `unlock()`), but it's not always possible to guarantee that locks will be released prior to the end of the task. One reason for this is our iterator interface: since our iterators don't support an explicit `close()` operator to signal that no more records will be consumed, operations like `take()` or `limit()` don't have a good means to release locks on their input iterators' blocks. Another example is broadcast variables, whose block locks can only be released at the end of the task.

To address this, `BlockInfoManager` uses a pair of maps to track the set of locks acquired by each task. Lock acquisitions automatically record the current task attempt id by obtaining it from `TaskContext`. When a task finishes, code in `Executor` calls `BlockInfoManager.unlockAllLocksForTask(taskAttemptId)` to free locks.

### Locking and the MemoryStore

In order to prevent in-memory blocks from being evicted while they are being read, the `MemoryStore`'s `evictBlocksToFreeSpace()` method acquires write locks on blocks which it is considering as candidates for eviction. These lock acquisitions are non-blocking, so a block which is being read will not be evicted. By holding write locks until the eviction is performed or skipped (in case evicting the blocks would not free enough memory), we avoid a race where a new reader starts to read a block after the block has been marked as an eviction candidate but before it has been removed.

### Locking and remote block transfer

This patch makes small changes to to block transfer and network layer code so that locks acquired by the BlockTransferService are released as soon as block transfer messages are consumed and released by Netty. This builds on top of #11193, a bug fix related to freeing of network layer ManagedBuffers.

## FAQ

- **Why not use Java's built-in [`ReadWriteLock`](https://docs.oracle.com/javase/7/docs/api/java/util/concurrent/locks/ReadWriteLock.html)?**

  Our locks operate on a per-task rather than per-thread level. Under certain circumstances a task may consist of multiple threads, so using `ReadWriteLock` would mean that we might call `unlock()` from a thread which didn't hold the lock in question, an operation which has undefined semantics. If we could rely on Java 8 classes, we might be able to use [`StampedLock`](https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/locks/StampedLock.html) to work around this issue.

- **Why not detect "leaked" locks in tests?**:

  See above notes about `take()` and `limit`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10705 from JoshRosen/pin-pages.
2016-02-25 17:17:56 -08:00
Davies Liu 751724b132 Revert "[SPARK-13457][SQL] Removes DataFrame RDD operations"
This reverts commit 157fe64f3e.
2016-02-25 11:53:48 -08:00
Cheng Lian 157fe64f3e [SPARK-13457][SQL] Removes DataFrame RDD operations
## What changes were proposed in this pull request?

This PR removes DataFrame RDD operations. Original calls are now replaced by calls to methods of `DataFrame.rdd`.

## How was the this patch tested?

No extra tests are added. Existing tests should do the work.

Author: Cheng Lian <lian@databricks.com>

Closes #11323 from liancheng/remove-df-rdd-ops.
2016-02-25 23:07:59 +08:00
Reynold Xin 2b2c8c3323 [SPARK-13486][SQL] Move SQLConf into an internal package
## What changes were proposed in this pull request?
This patch moves SQLConf into org.apache.spark.sql.internal package to make it very explicit that it is internal. Soon I will also submit more API work that creates implementations of interfaces in this internal package.

## How was this patch tested?
If it compiles, then the refactoring should work.

Author: Reynold Xin <rxin@databricks.com>

Closes #11363 from rxin/SPARK-13486.
2016-02-25 17:49:50 +08:00
Nong Li 5a7af9e7ac [SPARK-13250] [SQL] Update PhysicallRDD to convert to UnsafeRow if using the vectorized scanner.
Some parts of the engine rely on UnsafeRow which the vectorized parquet scanner does not want
to produce. This add a conversion in Physical RDD. In the case where codegen is used (and the
scan is the start of the pipeline), there is no requirement to use UnsafeRow. This patch adds
update PhysicallRDD to support codegen, which eliminates the need for the UnsafeRow conversion
in all cases.

The result of these changes for TPCDS-Q19 at the 10gb sf reduces the query time from 9.5 seconds
to 6.5 seconds.

Author: Nong Li <nong@databricks.com>

Closes #11141 from nongli/spark-13250.
2016-02-24 17:16:45 -08:00
Yin Huai bc353805bd [SPARK-13475][TESTS][SQL] HiveCompatibilitySuite should still run in PR builder even if a PR only changes sql/core
## What changes were proposed in this pull request?

`HiveCompatibilitySuite` should still run in PR build even if a PR only changes sql/core. So, I am going to remove `ExtendedHiveTest` annotation from `HiveCompatibilitySuite`.

https://issues.apache.org/jira/browse/SPARK-13475

Author: Yin Huai <yhuai@databricks.com>

Closes #11351 from yhuai/SPARK-13475.
2016-02-24 13:34:53 -08:00
gatorsmile 5289837a72 [HOT][TEST] Disable a Test that Requires Nested Union Support.
## What changes were proposed in this pull request?
Since "[SPARK-13321][SQL] Support nested UNION in parser" is reverted, we need to disable the test case that requires this PR. Thanks!

rxin yhuai marmbrus

## How was this patch tested?

N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11352 from gatorsmile/disableTestCase.
2016-02-24 13:30:23 -08:00
Davies Liu 9cdd867da9 [SPARK-13373] [SQL] generate sort merge join
## What changes were proposed in this pull request?

Generates code for SortMergeJoin.

## How was the this patch tested?

Unit tests and manually tested with TPCDS Q72, which showed 70% performance improvements (from 42s to 25s), but micro benchmark only show minor improvements, it may depends the distribution of data and number of columns.

Author: Davies Liu <davies@databricks.com>

Closes #11248 from davies/gen_smj.
2016-02-23 15:00:10 -08:00
gatorsmile 87250580f2 [SPARK-13263][SQL] SQL Generation Support for Tablesample
In the parser, tableSample clause is part of tableSource.
```
tableSource
init { gParent.pushMsg("table source", state); }
after { gParent.popMsg(state); }
    : tabname=tableName
    ((tableProperties) => props=tableProperties)?
    ((tableSample) => ts=tableSample)?
    ((KW_AS) => (KW_AS alias=Identifier)
    |
    (Identifier) => (alias=Identifier))?
    -> ^(TOK_TABREF $tabname $props? $ts? $alias?)
    ;
```

Two typical query samples using TABLESAMPLE are:
```
    "SELECT s.id FROM t0 TABLESAMPLE(10 PERCENT) s"
    "SELECT * FROM t0 TABLESAMPLE(0.1 PERCENT)"
```

FYI, the logical plan of a TABLESAMPLE query:
```
sql("SELECT * FROM t0 TABLESAMPLE(0.1 PERCENT)").explain(true)

== Analyzed Logical Plan ==
id: bigint
Project [id#16L]
+- Sample 0.0, 0.001, false, 381
   +- Subquery t0
      +- Relation[id#16L] ParquetRelation
```

Thanks! cc liancheng

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

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

Closes #11148 from gatorsmile/tablesplitsample.
2016-02-23 16:13:09 +08:00
gatorsmile 01e10c9fef [SPARK-13236] SQL Generation for Set Operations
This PR is to implement SQL generation for the following three set operations:
- Union Distinct
- Intersect
- Except

liancheng Thanks!

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #11195 from gatorsmile/setOpSQLGen.
2016-02-23 15:16:59 +08:00
gatorsmile 9dd5399d78 [SPARK-12723][SQL] Comprehensive Verification and Fixing of SQL Generation Support for Expressions
#### What changes were proposed in this pull request?

Ensure that all built-in expressions can be mapped to its SQL representation if there is one (e.g. ScalaUDF doesn't have a SQL representation). The function lists are from the expression list in `FunctionRegistry`.

window functions, grouping sets functions (`cube`, `rollup`, `grouping`, `grouping_id`), generator functions (`explode` and `json_tuple`) are covered by separate JIRA and PRs. Thus, this PR does not cover them. Except these functions, all the built-in expressions are covered. For details, see the list in `ExpressionToSQLSuite`.

Fixed a few issues. For example, the `prettyName` of `approx_count_distinct` is not right. The `sql` of `hash` function is not right, since the `hash` function does not accept `seed`.

Additionally, also correct the order of expressions in `FunctionRegistry` so that people are easier to find which functions are missing.

cc liancheng

#### How was the this patch tested?
Added two test cases in LogicalPlanToSQLSuite for covering `not like` and `not in`.

Added a new test suite `ExpressionToSQLSuite` to cover the functions:

1. misc non-aggregate functions + complex type creators + null expressions
2. math functions
3. aggregate functions
4. string functions
5. date time functions + calendar interval
6. collection functions
7. misc functions

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11314 from gatorsmile/expressionToSQL.
2016-02-22 22:17:56 -08:00
Daoyuan Wang 5d80fac58f [SPARK-11624][SPARK-11972][SQL] fix commands that need hive to exec
In SparkSQLCLI, we have created a `CliSessionState`, but then we call `SparkSQLEnv.init()`, which will start another `SessionState`. This would lead to exception because `processCmd` need to get the `CliSessionState` instance by calling `SessionState.get()`, but the return value would be a instance of `SessionState`. See the exception below.

spark-sql> !echo "test";
Exception in thread "main" java.lang.ClassCastException: org.apache.hadoop.hive.ql.session.SessionState cannot be cast to org.apache.hadoop.hive.cli.CliSessionState
	at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:112)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:301)
	at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:376)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:242)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:606)
	at org.apache.spark.deploy.SparkSubmit$.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:691)
	at org.apache.spark.deploy.SparkSubmit$.doRunMain$1(SparkSubmit.scala:180)
	at org.apache.spark.deploy.SparkSubmit$.submit(SparkSubmit.scala:205)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:120)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)

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

Closes #9589 from adrian-wang/clicommand.
2016-02-22 18:13:32 -08:00
Shixiong Zhu 0cbadf28c9 [SPARK-13271][SQL] Better error message if 'path' is not specified
Improved the error message as per discussion in https://github.com/apache/spark/pull/11034#discussion_r52111238. Also made `path` and `metadataPath` in FileStreamSource case insensitive.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11154 from zsxwing/path.
2016-02-21 15:34:39 -08:00
Andrew Or 6c3832b26e [SPARK-13080][SQL] Implement new Catalog API using Hive
## What changes were proposed in this pull request?

This is a step towards merging `SQLContext` and `HiveContext`. A new internal Catalog API was introduced in #10982 and extended in #11069. This patch introduces an implementation of this API using `HiveClient`, an existing interface to Hive. It also extends `HiveClient` with additional calls to Hive that are needed to complete the catalog implementation.

*Where should I start reviewing?* The new catalog introduced is `HiveCatalog`. This class is relatively simple because it just calls `HiveClientImpl`, where most of the new logic is. I would not start with `HiveClient`, `HiveQl`, or `HiveMetastoreCatalog`, which are modified mainly because of a refactor.

*Why is this patch so big?* I had to refactor HiveClient to remove an intermediate representation of databases, tables, partitions etc. After this refactor `CatalogTable` convert directly to and from `HiveTable` (etc.). Otherwise we would have to first convert `CatalogTable` to the intermediate representation and then convert that to HiveTable, which is messy.

The new class hierarchy is as follows:
```
org.apache.spark.sql.catalyst.catalog.Catalog
  - org.apache.spark.sql.catalyst.catalog.InMemoryCatalog
  - org.apache.spark.sql.hive.HiveCatalog
```

Note that, as of this patch, none of these classes are currently used anywhere yet. This will come in the future before the Spark 2.0 release.

## How was the this patch tested?
All existing unit tests, and HiveCatalogSuite that extends CatalogTestCases.

Author: Andrew Or <andrew@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #11293 from rxin/hive-catalog.
2016-02-21 15:00:24 -08:00
Herman van Hovell b6a873d6d4 [SPARK-13136][SQL] Create a dedicated Broadcast exchange operator
Quite a few Spark SQL join operators broadcast one side of the join to all nodes. The are a few problems with this:

- This conflates broadcasting (a data exchange) with joining. Data exchanges should be managed by a different operator.
- All these nodes implement their own (duplicate) broadcasting logic.
- Re-use of indices is quite hard.

This PR defines both a ```BroadcastDistribution``` and ```BroadcastPartitioning```, these contain a `BroadcastMode`. The `BroadcastMode` defines the way in which we transform the Array of `InternalRow`'s into an index. We currently support the following `BroadcastMode`'s:

- IdentityBroadcastMode: This broadcasts the rows in their original form.
- HashSetBroadcastMode: This applies a projection to the input rows, deduplicates these rows and broadcasts the resulting `Set`.
- HashedRelationBroadcastMode: This transforms the input rows into a `HashedRelation`, and broadcasts this index.

To match this distribution we implement a ```BroadcastExchange``` operator which will perform the broadcast for us, and have ```EnsureRequirements``` plan this operator. The old Exchange operator has been renamed into ShuffleExchange in order to clearly separate between Shuffled and Broadcasted exchanges. Finally the classes in Exchange.scala have been moved to a dedicated package.

cc rxin davies

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #11083 from hvanhovell/SPARK-13136.
2016-02-21 12:32:31 -08:00
Reynold Xin 0947f0989b [SPARK-13420][SQL] Rename Subquery logical plan to SubqueryAlias
## What changes were proposed in this pull request?
This patch renames logical.Subquery to logical.SubqueryAlias, which is a more appropriate name for this operator (versus subqueries as expressions).

## How was the this patch tested?
Unit tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #11288 from rxin/SPARK-13420.
2016-02-21 11:31:46 -08:00
Cheng Lian d9efe63ecd [SPARK-12799] Simplify various string output for expressions
This PR introduces several major changes:

1. Replacing `Expression.prettyString` with `Expression.sql`

   The `prettyString` method is mostly an internal, developer faced facility for debugging purposes, and shouldn't be exposed to users.

1. Using SQL-like representation as column names for selected fields that are not named expression (back-ticks and double quotes should be removed)

   Before, we were using `prettyString` as column names when possible, and sometimes the result column names can be weird.  Here are several examples:

   Expression         | `prettyString` | `sql`      | Note
   ------------------ | -------------- | ---------- | ---------------
   `a && b`           | `a && b`       | `a AND b`  |
   `a.getField("f")`  | `a[f]`         | `a.f`      | `a` is a struct

1. Adding trait `NonSQLExpression` extending from `Expression` for expressions that don't have a SQL representation (e.g. Scala UDF/UDAF and Java/Scala object expressions used for encoders)

   `NonSQLExpression.sql` may return an arbitrary user facing string representation of the expression.

Author: Cheng Lian <lian@databricks.com>

Closes #10757 from liancheng/spark-12799.simplify-expression-string-methods.
2016-02-21 22:53:15 +08:00
gatorsmile f88c641bc8 [SPARK-13310] [SQL] Resolve Missing Sorting Columns in Generate
```scala
// case 1: missing sort columns are resolvable if join is true
sql("SELECT explode(a) AS val, b FROM data WHERE b < 2 order by val, c")
// case 2: missing sort columns are not resolvable if join is false. Thus, issue an error message in this case
sql("SELECT explode(a) AS val FROM data order by val, c")
```

When sort columns are not in `Generate`, we can resolve them when `join` is equal to `true`. Still trying to add more test cases for the other `UnaryNode` types.

Could you review the changes? davies cloud-fan Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11198 from gatorsmile/missingInSort.
2016-02-20 13:53:23 -08:00
gatorsmile fee739f07b [SPARK-13221] [SQL] Fixing GroupingSets when Aggregate Functions Containing GroupBy Columns
Using GroupingSets will generate a wrong result when Aggregate Functions containing GroupBy columns.

This PR is to fix it. Since the code changes are very small. Maybe we also can merge it to 1.6

For example, the following query returns a wrong result:
```scala
sql("select course, sum(earnings) as sum from courseSales group by course, earnings" +
     " grouping sets((), (course), (course, earnings))" +
     " order by course, sum").show()
```
Before the fix, the results are like
```
[null,null]
[Java,null]
[Java,20000.0]
[Java,30000.0]
[dotNET,null]
[dotNET,5000.0]
[dotNET,10000.0]
[dotNET,48000.0]
```
After the fix, the results become correct:
```
[null,113000.0]
[Java,20000.0]
[Java,30000.0]
[Java,50000.0]
[dotNET,5000.0]
[dotNET,10000.0]
[dotNET,48000.0]
[dotNET,63000.0]
```

UPDATE:  This PR also deprecated the external column: GROUPING__ID.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11100 from gatorsmile/groupingSets.
2016-02-15 23:16:58 -08:00
Josh Rosen a8bbc4f50e [SPARK-12503][SPARK-12505] Limit pushdown in UNION ALL and OUTER JOIN
This patch adds a new optimizer rule for performing limit pushdown. Limits will now be pushed down in two cases:

- If a limit is on top of a `UNION ALL` operator, then a partition-local limit operator will be pushed to each of the union operator's children.
- If a limit is on top of an `OUTER JOIN` then a partition-local limit will be pushed to one side of the join. For `LEFT OUTER` and `RIGHT OUTER` joins, the limit will be pushed to the left and right side, respectively. For `FULL OUTER` join, we will only push limits when at most one of the inputs is already limited: if one input is limited we will push a smaller limit on top of it and if neither input is limited then we will limit the input which is estimated to be larger.

These optimizations were proposed previously by gatorsmile in #10451 and #10454, but those earlier PRs were closed and deferred for later because at that time Spark's physical `Limit` operator would trigger a full shuffle to perform global limits so there was a chance that pushdowns could actually harm performance by causing additional shuffles/stages. In #7334, we split the `Limit` operator into separate `LocalLimit` and `GlobalLimit` operators, so we can now push down only local limits (which don't require extra shuffles). This patch is based on both of gatorsmile's patches, with changes and simplifications due to partition-local-limiting.

When we push down the limit, we still keep the original limit in place, so we need a mechanism to ensure that the optimizer rule doesn't keep pattern-matching once the limit has been pushed down. In order to handle this, this patch adds a `maxRows` method to `SparkPlan` which returns the maximum number of rows that the plan can compute, then defines the pushdown rules to only push limits to children if the children's maxRows are greater than the limit's maxRows. This idea is carried over from #10451; see that patch for additional discussion.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11121 from JoshRosen/limit-pushdown-2.
2016-02-14 17:32:21 -08:00
Reynold Xin 354d4c24be [SPARK-13296][SQL] Move UserDefinedFunction into sql.expressions.
This pull request has the following changes:

1. Moved UserDefinedFunction into expressions package. This is more consistent with how we structure the packages for window functions and UDAFs.

2. Moved UserDefinedPythonFunction into execution.python package, so we don't have a random private class in the top level sql package.

3. Move everything in execution/python.scala into the newly created execution.python package.

Most of the diffs are just straight copy-paste.

Author: Reynold Xin <rxin@databricks.com>

Closes #11181 from rxin/SPARK-13296.
2016-02-13 21:06:31 -08:00
Sean Owen 388cd9ea8d [SPARK-13172][CORE][SQL] Stop using RichException.getStackTrace it is deprecated
Replace `getStackTraceString` with `Utils.exceptionString`

Author: Sean Owen <sowen@cloudera.com>

Closes #11182 from srowen/SPARK-13172.
2016-02-13 21:05:48 -08:00
Reynold Xin c4d5ad80c8 [SPARK-13282][SQL] LogicalPlan toSql should just return a String
Previously we were using Option[String] and None to indicate the case when Spark fails to generate SQL. It is easier to just use exceptions to propagate error cases, rather than having for comprehension everywhere. I also introduced a "build" function that simplifies string concatenation (i.e. no need to reason about whether we have an extra space or not).

Author: Reynold Xin <rxin@databricks.com>

Closes #11171 from rxin/SPARK-13282.
2016-02-12 10:08:19 -08:00
Davies Liu 5b805df279 [SPARK-12705] [SQL] push missing attributes for Sort
The current implementation of ResolveSortReferences can only push one missing attributes into it's child, it failed to analyze TPCDS Q98, because of there are two missing attributes in that (one from Window, another from Aggregate).

Author: Davies Liu <davies@databricks.com>

Closes #11153 from davies/resolve_sort.
2016-02-12 09:34:18 -08:00
Davies Liu 8f744fe3d9 [SPARK-13234] [SQL] remove duplicated SQL metrics
For lots of SQL operators, we have metrics for both of input and output, the number of input rows should be exactly the number of output rows of child, we could only have metrics for output rows.

After we improved the performance using whole stage codegen, the overhead of SQL metrics are not trivial anymore, we should avoid that if it's not necessary.

This PR remove all the SQL metrics for number of input rows, add SQL metric of number of output rows for all LeafNode. All remove the SQL metrics from those operators that have the same number of rows from input and output (for example, Projection, we may don't need that).

The new SQL UI will looks like:

![metrics](https://cloud.githubusercontent.com/assets/40902/12965227/63614e5e-d009-11e5-88b3-84fea04f9c20.png)

Author: Davies Liu <davies@databricks.com>

Closes #11163 from davies/remove_metrics.
2016-02-10 23:23:01 -08:00
Davies Liu b5761d150b [SPARK-12706] [SQL] grouping() and grouping_id()
Grouping() returns a column is aggregated or not, grouping_id() returns the aggregation levels.

grouping()/grouping_id() could be used with window function, but does not work in having/sort clause, will be fixed by another PR.

The GROUPING__ID/grouping_id() in Hive is wrong (according to docs), we also did it wrongly, this PR change that to match the behavior in most databases (also the docs of Hive).

Author: Davies Liu <davies@databricks.com>

Closes #10677 from davies/grouping.
2016-02-10 20:13:38 -08:00
gatorsmile 0f09f02269 [SPARK-13205][SQL] SQL Generation Support for Self Join
This PR addresses two issues:
  - Self join does not work in SQL Generation
  - When creating new instances for `LogicalRelation`, `metastoreTableIdentifier` is lost.

liancheng Could you please review the code changes? Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11084 from gatorsmile/selfJoinInSQLGen.
2016-02-11 11:08:21 +08:00
gatorsmile 663cc400f3 [SPARK-12725][SQL] Resolving Name Conflicts in SQL Generation and Name Ambiguity Caused by Internally Generated Expressions
Some analysis rules generate aliases or auxiliary attribute references with the same name but different expression IDs. For example, `ResolveAggregateFunctions` introduces `havingCondition` and `aggOrder`, and `DistinctAggregationRewriter` introduces `gid`.

This is OK for normal query execution since these attribute references get expression IDs. However, it's troublesome when converting resolved query plans back to SQL query strings since expression IDs are erased.

Here's an example Spark 1.6.0 snippet for illustration:
```scala
sqlContext.range(10).select('id as 'a, 'id as 'b).registerTempTable("t")
sqlContext.sql("SELECT SUM(a) FROM t GROUP BY a, b ORDER BY COUNT(a), COUNT(b)").explain(true)
```
The above code produces the following resolved plan:
```
== Analyzed Logical Plan ==
_c0: bigint
Project [_c0#101L]
+- Sort [aggOrder#102L ASC,aggOrder#103L ASC], true
   +- Aggregate [a#47L,b#48L], [(sum(a#47L),mode=Complete,isDistinct=false) AS _c0#101L,(count(a#47L),mode=Complete,isDistinct=false) AS aggOrder#102L,(count(b#48L),mode=Complete,isDistinct=false) AS aggOrder#103L]
      +- Subquery t
         +- Project [id#46L AS a#47L,id#46L AS b#48L]
            +- LogicalRDD [id#46L], MapPartitionsRDD[44] at range at <console>:26
```
Here we can see that both aggregate expressions in `ORDER BY` are extracted into an `Aggregate` operator, and both of them are named `aggOrder` with different expression IDs.

The solution is to automatically add the expression IDs into the attribute name for the Alias and AttributeReferences that are generated by Analyzer in SQL Generation.

In this PR, it also resolves another issue. Users could use the same name as the internally generated names. The duplicate names should not cause name ambiguity. When resolving the column, Catalyst should not pick the column that is internally generated.

Could you review the solution? marmbrus liancheng

I did not set the newly added flag for all the alias and attribute reference generated by Analyzers. Please let me know if I should do it? Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #11050 from gatorsmile/namingConflicts.
2016-02-11 10:44:39 +08:00
Josh Rosen 5cf20598ce [SPARK-13254][SQL] Fix planning of TakeOrderedAndProject operator
The patch for SPARK-8964 ("use Exchange to perform shuffle in Limit" / #7334) inadvertently broke the planning of the TakeOrderedAndProject operator: because ReturnAnswer was the new root of the query plan, the TakeOrderedAndProject rule was unable to match before BasicOperators.

This patch fixes this by moving the `TakeOrderedAndCollect` and `CollectLimit` rules into the same strategy.

In addition, I made changes to the TakeOrderedAndProject operator in order to make its `doExecute()` method lazy and added a new TakeOrderedAndProjectSuite which tests the new code path.

/cc davies and marmbrus for review.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11145 from JoshRosen/take-ordered-and-project-fix.
2016-02-10 11:00:38 -08:00
Nong Li 3708d13f1a [SPARK-12992] [SQL] Support vectorized decoding in UnsafeRowParquetRecordReader.
WIP: running tests. Code needs a bit of clean up.

This patch completes the vectorized decoding with the goal of passing the existing
tests. There is still more patches to support the rest of the format spec, even
just for flat schemas.

This patch adds a new flag to enable the vectorized decoding. Tests were updated
to try with both modes where applicable.

Once this is working well, we can remove the previous code path.

Author: Nong Li <nong@databricks.com>

Closes #11055 from nongli/spark-12992-2.
2016-02-08 22:21:26 -08:00
Jakob Odersky 352102ed0b [SPARK-13208][CORE] Replace use of Pairs with Tuple2s
Another trivial deprecation fix for Scala 2.11

Author: Jakob Odersky <jakob@odersky.com>

Closes #11089 from jodersky/SPARK-13208.
2016-02-04 22:22:41 -08:00
gatorsmile e3c75c6398 [SPARK-12850][SQL] Support Bucket Pruning (Predicate Pushdown for Bucketed Tables)
JIRA: https://issues.apache.org/jira/browse/SPARK-12850

This PR is to support bucket pruning when the predicates are `EqualTo`, `EqualNullSafe`, `IsNull`, `In`, and `InSet`.

Like HIVE, in this PR, the bucket pruning works when the bucketing key has one and only one column.

So far, I do not find a way to verify how many buckets are actually scanned. However, I did verify it when doing the debug. Could you provide a suggestion how to do it properly? Thank you! cloud-fan yhuai rxin marmbrus

BTW, we can add more cases to support complex predicate including `Or` and `And`. Please let me know if I should do it in this PR.

Maybe we also need to add test cases to verify if bucket pruning works well for each data type.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10942 from gatorsmile/pruningBuckets.
2016-02-04 18:37:58 -08:00
Josh Rosen 33212cb9a1 [SPARK-13168][SQL] Collapse adjacent repartition operators
Spark SQL should collapse adjacent `Repartition` operators and only keep the last one.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11064 from JoshRosen/collapse-repartition.
2016-02-04 11:08:50 -08:00
Davies Liu be5dd881f1 [SPARK-12913] [SQL] Improve performance of stat functions
As benchmarked and discussed here: https://github.com/apache/spark/pull/10786/files#r50038294, benefits from codegen, the declarative aggregate function could be much faster than imperative one.

Author: Davies Liu <davies@databricks.com>

Closes #10960 from davies/stddev.
2016-02-02 11:50:14 -08:00
Michael Armbrust 22ba21348b [SPARK-13087][SQL] Fix group by function for sort based aggregation
It is not valid to call `toAttribute` on a `NamedExpression` unless we know for sure that the child produced that `NamedExpression`.  The current code worked fine when the grouping expressions were simple, but when they were a derived value this blew up at execution time.

Author: Michael Armbrust <michael@databricks.com>

Closes #11013 from marmbrus/groupByFunction-master.
2016-02-02 16:48:59 +08:00
gatorsmile 8f26eb5ef6 [SPARK-12705][SPARK-10777][SQL] Analyzer Rule ResolveSortReferences
JIRA: https://issues.apache.org/jira/browse/SPARK-12705

**Scope:**
This PR is a general fix for sorting reference resolution when the child's `outputSet` does not have the order-by attributes (called, *missing attributes*):
  - UnaryNode support is limited to `Project`, `Window`, `Aggregate`, `Distinct`, `Filter`, `RepartitionByExpression`.
  - We will not try to resolve the missing references inside a subquery, unless the outputSet of this subquery contains it.

**General Reference Resolution Rules:**
  - Jump over the nodes with the following types: `Distinct`, `Filter`, `RepartitionByExpression`. Do not need to add missing attributes. The reason is their `outputSet` is decided by their `inputSet`, which is the `outputSet` of their children.
  - Group-by expressions in `Aggregate`: missing order-by attributes are not allowed to be added into group-by expressions since it will change the query result. Thus, in RDBMS, it is not allowed.
  - Aggregate expressions in `Aggregate`: if the group-by expressions in `Aggregate` contains the missing attributes but aggregate expressions do not have it, just add them into the aggregate expressions. This can resolve the analysisExceptions thrown by the three TCPDS queries.
  - `Project` and `Window` are special. We just need to add the missing attributes to their `projectList`.

**Implementation:**
  1. Traverse the whole tree in a pre-order manner to find all the resolvable missing order-by attributes.
  2. Traverse the whole tree in a post-order manner to add the found missing order-by attributes to the node if their `inputSet` contains the attributes.
  3. If the origins of the missing order-by attributes are different nodes, each pass only resolves the missing attributes that are from the same node.

**Risk:**
Low. This rule will be trigger iff ```!s.resolved && child.resolved``` is true. Thus, very few cases are affected.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10678 from gatorsmile/sortWindows.
2016-02-01 11:57:13 -08:00
Josh Rosen 289373b28c [SPARK-6363][BUILD] Make Scala 2.11 the default Scala version
This patch changes Spark's build to make Scala 2.11 the default Scala version. To be clear, this does not mean that Spark will stop supporting Scala 2.10: users will still be able to compile Spark for Scala 2.10 by following the instructions on the "Building Spark" page; however, it does mean that Scala 2.11 will be the default Scala version used by our CI builds (including pull request builds).

The Scala 2.11 compiler is faster than 2.10, so I think we'll be able to look forward to a slight speedup in our CI builds (it looks like it's about 2X faster for the Maven compile-only builds, for instance).

After this patch is merged, I'll update Jenkins to add new compile-only jobs to ensure that Scala 2.10 compilation doesn't break.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10608 from JoshRosen/SPARK-6363.
2016-01-30 00:20:28 -08:00
Reynold Xin 2cbc412821 [SPARK-13076][SQL] Rename ClientInterface -> HiveClient
And ClientWrapper -> HiveClientImpl.

I have some followup pull requests to introduce a new internal catalog, and I think this new naming reflects better the functionality of the two classes.

Author: Reynold Xin <rxin@databricks.com>

Closes #10981 from rxin/SPARK-13076.
2016-01-29 16:57:34 -08:00
Liang-Chi Hsieh 66449b8dcd [SPARK-12968][SQL] Implement command to set current database
JIRA: https://issues.apache.org/jira/browse/SPARK-12968

Implement command to set current database.

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

Closes #10916 from viirya/ddl-use-database.
2016-01-28 22:20:52 -08:00
Herman van Hovell ef96cd3c52 [SPARK-12865][SPARK-12866][SQL] Migrate SparkSQLParser/ExtendedHiveQlParser commands to new Parser
This PR moves all the functionality provided by the SparkSQLParser/ExtendedHiveQlParser to the new Parser hierarchy (SparkQl/HiveQl). This also improves the current SET command parsing: the current implementation swallows ```set role ...``` and ```set autocommit ...``` commands, this PR respects these commands (and passes them on to Hive).

This PR and https://github.com/apache/spark/pull/10723 end the use of Parser-Combinator parsers for SQL parsing. As a result we can also remove the ```AbstractSQLParser``` in Catalyst.

The PR is marked WIP as long as it doesn't pass all tests.

cc rxin viirya winningsix (this touches https://github.com/apache/spark/pull/10144)

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10905 from hvanhovell/SPARK-12866.
2016-01-27 13:45:00 -08:00
Cheng Lian 58f5d8c1da [SPARK-12728][SQL] Integrates SQL generation with native view
This PR is a follow-up of PR #10541. It integrates the newly introduced SQL generation feature with native view to make native view canonical.

In this PR, a new SQL option `spark.sql.nativeView.canonical` is added.  When this option and `spark.sql.nativeView` are both `true`, Spark SQL tries to handle `CREATE VIEW` DDL statements using SQL query strings generated from view definition logical plans. If we failed to map the plan to SQL, we fallback to the original native view approach.

One important issue this PR fixes is that, now we can use CTE when defining a view.  Originally, when native view is turned on, we wrap the view definition text with an extra `SELECT`.  However, HiveQL parser doesn't allow CTE appearing as a subquery.  Namely, something like this is disallowed:

```sql
SELECT n
FROM (
  WITH w AS (SELECT 1 AS n)
  SELECT * FROM w
) v
```

This PR fixes this issue because the extra `SELECT` is no longer needed (also, CTE expressions are inlined as subqueries during analysis phase, thus there won't be CTE expressions in the generated SQL query string).

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

Closes #10733 from liancheng/spark-12728.integrate-sql-gen-with-native-view.
2016-01-26 20:30:13 -08:00
Nong Li 555127387a [SPARK-12854][SQL] Implement complex types support in ColumnarBatch
This patch adds support for complex types for ColumnarBatch. ColumnarBatch supports structs
and arrays. There is a simple mapping between the richer catalyst types to these two. Strings
are treated as an array of bytes.

ColumnarBatch will contain a column for each node of the schema. Non-complex schemas consists
of just leaf nodes. Structs represent an internal node with one child for each field. Arrays
are internal nodes with one child. Structs just contain nullability. Arrays contain offsets
and lengths into the child array. This structure is able to handle arbitrary nesting. It has
the key property that we maintain columnar throughout and that primitive types are only stored
in the leaf nodes and contiguous across rows. For example, if the schema is
```
array<array<int>>
```
There are three columns in the schema. The internal nodes each have one children. The leaf node contains all the int data stored consecutively.

As part of this, this patch adds append APIs in addition to the Put APIs (e.g. putLong(rowid, v)
vs appendLong(v)). These APIs are necessary when the batch contains variable length elements.
The vectors are not fixed length and will grow as necessary. This should make the usage a lot
simpler for the writer.

Author: Nong Li <nong@databricks.com>

Closes #10820 from nongli/spark-12854.
2016-01-26 17:34:01 -08:00
Sameer Agarwal 08c781ca67 [SPARK-12682][SQL] Add support for (optionally) not storing tables in hive metadata format
This PR adds a new table option (`skip_hive_metadata`) that'd allow the user to skip storing the table metadata in hive metadata format. While this could be useful in general, the specific use-case for this change is that Hive doesn't handle wide schemas well (see https://issues.apache.org/jira/browse/SPARK-12682 and https://issues.apache.org/jira/browse/SPARK-6024) which in turn prevents such tables from being queried in SparkSQL.

Author: Sameer Agarwal <sameer@databricks.com>

Closes #10826 from sameeragarwal/skip-hive-metadata.
2016-01-26 07:50:37 -08:00
gatorsmile 9348431da2 [SPARK-12975][SQL] Throwing Exception when Bucketing Columns are part of Partitioning Columns
When users are using `partitionBy` and `bucketBy` at the same time, some bucketing columns might be part of partitioning columns. For example,
```
        df.write
          .format(source)
          .partitionBy("i")
          .bucketBy(8, "i", "k")
          .saveAsTable("bucketed_table")
```
However, in the above case, adding column `i` into `bucketBy` is useless. It is just wasting extra CPU when reading or writing bucket tables. Thus, like Hive, we can issue an exception and let users do the change.

Also added a test case for checking if the information of `sortBy` and `bucketBy` columns are correctly saved in the metastore table.

Could you check if my understanding is correct? cloud-fan rxin marmbrus Thanks!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10891 from gatorsmile/commonKeysInPartitionByBucketBy.
2016-01-25 13:38:09 -08:00
Josh Rosen f4004601b0 [SPARK-12971] Fix Hive tests which fail in Hadoop-2.3 SBT build
ErrorPositionSuite and one of the HiveComparisonTest tests have been consistently failing on the Hadoop 2.3 SBT build (but on no other builds). I believe that this is due to test isolation issues (e.g. tests sharing state via the sets of temporary tables that are registered to TestHive).

This patch attempts to improve the isolation of these tests in order to address this issue.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10884 from JoshRosen/fix-failing-hadoop-2.3-hive-tests.
2016-01-24 11:29:27 -08:00
gatorsmile e13c147e74 [SPARK-12959][SQL] Writing Bucketed Data with Disabled Bucketing in SQLConf
When users turn off bucketing in SQLConf, we should issue some messages to tell users these operations will be converted to normal way.

Also added a test case for this scenario and fixed the helper function.

Do you think this PR is helpful when using bucket tables? cloud-fan Thank you!

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10870 from gatorsmile/bucketTableWritingTestcases.
2016-01-22 01:03:41 -08:00
Yin Huai d60f8d74ac [SPARK-8968] [SQL] [HOT-FIX] Fix scala 2.11 build. 2016-01-20 17:48:18 -08:00
wangfei 015c8efb37 [SPARK-8968][SQL] external sort by the partition clomns when dynamic partitioning to optimize the memory overhead
Now the hash based writer dynamic partitioning show the bad performance for big data and cause many small files and high GC. This patch we do external sort first so that each time we only need open one writer.

before this patch:
![gc](https://cloud.githubusercontent.com/assets/7018048/9149788/edc48c6e-3dec-11e5-828c-9995b56e4d65.PNG)

after this patch:
![gc-optimize-externalsort](https://cloud.githubusercontent.com/assets/7018048/9149794/60f80c9c-3ded-11e5-8a56-7ae18ddc7a2f.png)

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

Closes #7336 from scwf/dynamic-optimize-basedon-apachespark.
2016-01-20 17:11:52 -08:00
Herman van Hovell 1017327930 [SPARK-12848][SQL] Change parsed decimal literal datatype from Double to Decimal
The current parser turns a decimal literal, for example ```12.1```, into a Double. The problem with this approach is that we convert an exact literal into a non-exact ```Double```. The PR changes this behavior, a Decimal literal is now converted into an extact ```BigDecimal```.

The behavior for scientific decimals, for example ```12.1e01```, is unchanged. This will be converted into a Double.

This PR replaces the ```BigDecimal``` literal by a ```Double``` literal, because the ```BigDecimal``` is the default now. You can use the double literal by appending a 'D' to the value, for instance: ```3.141527D```

cc davies rxin

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10796 from hvanhovell/SPARK-12848.
2016-01-20 15:13:01 -08:00
gatorsmile 8f90c15187 [SPARK-12616][SQL] Making Logical Operator Union Support Arbitrary Number of Children
The existing `Union` logical operator only supports two children. Thus, adding a new logical operator `Unions` which can have arbitrary number of children to replace the existing one.

`Union` logical plan is a binary node. However, a typical use case for union is to union a very large number of input sources (DataFrames, RDDs, or files). It is not uncommon to union hundreds of thousands of files. In this case, our optimizer can become very slow due to the large number of logical unions. We should change the Union logical plan to support an arbitrary number of children, and add a single rule in the optimizer to collapse all adjacent `Unions` into a single `Unions`. Note that this problem doesn't exist in physical plan, because the physical `Unions` already supports arbitrary number of children.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10577 from gatorsmile/unionAllMultiChildren.
2016-01-20 14:59:30 -08:00
Rajesh Balamohan ab4a6bfd11 [SPARK-12898] Consider having dummyCallSite for HiveTableScan
Currently, HiveTableScan runs with getCallSite which is really expensive and shows up when scanning through large table with partitions (e.g TPC-DS) which slows down the overall runtime of the job. It would be good to consider having dummyCallSite in HiveTableScan.

Author: Rajesh Balamohan <rbalamohan@apache.org>

Closes #10825 from rajeshbalamohan/SPARK-12898.
2016-01-20 11:30:03 -08:00
Rajesh Balamohan e75e340a40 [SPARK-12925][SQL] Improve HiveInspectors.unwrap for StringObjectIns…
Text is in UTF-8 and converting it via "UTF8String.fromString" incurs decoding and encoding, which turns out to be expensive and redundant.  Profiler snapshot details is attached in the JIRA (ref:https://issues.apache.org/jira/secure/attachment/12783331/SPARK-12925_profiler_cpu_samples.png)

Author: Rajesh Balamohan <rbalamohan@apache.org>

Closes #10848 from rajeshbalamohan/SPARK-12925.
2016-01-20 11:20:26 -08:00
Wenchen Fan e14817b528 [SPARK-12870][SQL] better format bucket id in file name
for normal parquet file without bucket, it's file name ends with a jobUUID which maybe all numbers and mistakeny regarded as bucket id. This PR improves the format of bucket id in file name by using a different seperator, `_`, so that the regex is more robust.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10799 from cloud-fan/fix-bucket.
2016-01-19 10:44:51 -08:00
Reynold Xin 39ac56fc60 [SPARK-12889][SQL] Rename ParserDialect -> ParserInterface.
Based on discussions in #10801, I'm submitting a pull request to rename ParserDialect to ParserInterface.

Author: Reynold Xin <rxin@databricks.com>

Closes #10817 from rxin/SPARK-12889.
2016-01-18 17:10:32 -08:00
Wenchen Fan 404190221a [SPARK-12882][SQL] simplify bucket tests and add more comments
Right now, the bucket tests are kind of hard to understand, this PR simplifies them and add more commetns.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10813 from cloud-fan/bucket-comment.
2016-01-18 15:10:04 -08:00
Reynold Xin 38c3c0e31a [SPARK-12855][SQL] Remove parser dialect developer API
This pull request removes the public developer parser API for external parsers. Given everything a parser depends on (e.g. logical plans and expressions) are internal and not stable, external parsers will break with every release of Spark. It is a bad idea to create the illusion that Spark actually supports pluggable parsers. In addition, this also reduces incentives for 3rd party projects to contribute parse improvements back to Spark.

Author: Reynold Xin <rxin@databricks.com>

Closes #10801 from rxin/SPARK-12855.
2016-01-18 13:55:42 -08:00
Dilip Biswal db9a860589 [SPARK-12558][FOLLOW-UP] AnalysisException when multiple functions applied in GROUP BY clause
Addresses the comments from Yin.
https://github.com/apache/spark/pull/10520

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #10758 from dilipbiswal/spark-12558-followup.
2016-01-18 10:28:01 -08:00
Davies Liu 3c0d2365d5 [SPARK-12796] [SQL] Whole stage codegen
This is the initial work for whole stage codegen, it support Projection/Filter/Range, we will continue work on this to support more physical operators.

A micro benchmark show that a query with range, filter and projection could be 3X faster then before.

It's turned on by default. For a tree that have at least two chained plans, a WholeStageCodegen will be inserted into it, for example, the following plan
```
Limit 10
+- Project [(id#5L + 1) AS (id + 1)#6L]
   +- Filter ((id#5L & 1) = 1)
      +- Range 0, 1, 4, 10, [id#5L]
```
will be translated into
```
Limit 10
+- WholeStageCodegen
      +- Project [(id#1L + 1) AS (id + 1)#2L]
         +- Filter ((id#1L & 1) = 1)
            +- Range 0, 1, 4, 10, [id#1L]
```

Here is the call graph to generate Java source for A and B (A  support codegen, but B does not):

```
  *   WholeStageCodegen       Plan A               FakeInput        Plan B
  * =========================================================================
  *
  * -> execute()
  *     |
  *  doExecute() -------->   produce()
  *                             |
  *                          doProduce()  -------> produce()
  *                                                   |
  *                                                doProduce() ---> execute()
  *                                                   |
  *                                                consume()
  *                          doConsume()  ------------|
  *                             |
  *  doConsume()  <-----    consume()
```

A SparkPlan that support codegen need to implement doProduce() and doConsume():

```
def doProduce(ctx: CodegenContext): (RDD[InternalRow], String)
def doConsume(ctx: CodegenContext, child: SparkPlan, input: Seq[ExprCode]): String
```

Author: Davies Liu <davies@databricks.com>

Closes #10735 from davies/whole2.
2016-01-16 10:29:27 -08:00
Wenchen Fan 3b5ccb12b8 [SPARK-12649][SQL] support reading bucketed table
This PR adds the support to read bucketed tables, and correctly populate `outputPartitioning`, so that we can avoid shuffle for some cases.

TODO(follow-up PRs):

* bucket pruning
* avoid shuffle for bucketed table join when use any super-set of the bucketing key.
 (we should re-visit it after https://issues.apache.org/jira/browse/SPARK-12704 is fixed)
* recognize hive bucketed table

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10604 from cloud-fan/bucket-read.
2016-01-15 17:20:01 -08:00
Herman van Hovell 7cd7f22025 [SPARK-12575][SQL] Grammar parity with existing SQL parser
In this PR the new CatalystQl parser stack reaches grammar parity with the old Parser-Combinator based SQL Parser. This PR also replaces all uses of the old Parser, and removes it from the code base.

Although the existing Hive and SQL parser dialects were mostly the same, some kinks had to be worked out:
- The SQL Parser allowed syntax like ```APPROXIMATE(0.01) COUNT(DISTINCT a)```. In order to make this work we needed to hardcode approximate operators in the parser, or we would have to create an approximate expression. ```APPROXIMATE_COUNT_DISTINCT(a, 0.01)``` would also do the job and is much easier to maintain. So, this PR **removes** this keyword.
- The old SQL Parser supports ```LIMIT``` clauses in nested queries. This is **not supported** anymore. See https://github.com/apache/spark/pull/10689 for the rationale for this.
- Hive has a charset name char set literal combination it supports, for instance the following expression ```_ISO-8859-1 0x4341464562616265``` would yield this string: ```CAFEbabe```. Hive will only allow charset names to start with an underscore. This is quite annoying in spark because as soon as you use a tuple names will start with an underscore. In this PR we **remove** this feature from the parser. It would be quite easy to implement such a feature as an Expression later on.
- Hive and the SQL Parser treat decimal literals differently. Hive will turn any decimal into a ```Double``` whereas the SQL Parser would convert a non-scientific decimal into a ```BigDecimal```, and would turn a scientific decimal into a Double. We follow Hive's behavior here. The new parser supports a big decimal literal, for instance: ```81923801.42BD```, which can be used when a big decimal is needed.

cc rxin viirya marmbrus yhuai cloud-fan

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10745 from hvanhovell/SPARK-12575-2.
2016-01-15 15:19:10 -08:00
Reynold Xin fe7246fea6 [SPARK-12830] Java style: disallow trailing whitespaces.
Author: Reynold Xin <rxin@databricks.com>

Closes #10764 from rxin/SPARK-12830.
2016-01-14 23:33:45 -08:00
Wenchen Fan 962e9bcf94 [SPARK-12756][SQL] use hash expression in Exchange
This PR makes bucketing and exchange share one common hash algorithm, so that we can guarantee the data distribution is same between shuffle and bucketed data source, which enables us to only shuffle one side when join a bucketed table and a normal one.

This PR also fixes the tests that are broken by the new hash behaviour in shuffle.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10703 from cloud-fan/use-hash-expr-in-shuffle.
2016-01-13 22:43:28 -08:00
Wenchen Fan c2ea79f96a [SPARK-12642][SQL] improve the hash expression to be decoupled from unsafe row
https://issues.apache.org/jira/browse/SPARK-12642

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10694 from cloud-fan/hash-expr.
2016-01-13 12:29:02 -08:00
Liang-Chi Hsieh 63eee86cc6 [SPARK-9297] [SQL] Add covar_pop and covar_samp
JIRA: https://issues.apache.org/jira/browse/SPARK-9297

Add two aggregation functions: covar_pop and covar_samp.

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

Closes #10029 from viirya/covar-funcs.
2016-01-13 10:26:55 -08:00
Kousuke Saruta cb7b864a24 [SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before ",")
Fix the style violation (space before , and :).
This PR is a followup for #10643 and rework of #10685 .

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

Closes #10732 from sarutak/SPARK-12692-followup-sql.
2016-01-12 22:25:20 -08:00
Dilip Biswal dc7b3870fc [SPARK-12558][SQL] AnalysisException when multiple functions applied in GROUP BY clause
cloud-fan Can you please take a look ?

In this case, we are failing during check analysis while validating the aggregation expression. I have added a semanticEquals for HiveGenericUDF to fix this. Please let me know if this is the right way to address this issue.

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #10520 from dilipbiswal/spark-12558.
2016-01-12 21:41:46 -08:00
Cheng Lian 8ed5f12d2b [SPARK-12724] SQL generation support for persisted data source tables
This PR implements SQL generation support for persisted data source tables.  A new field `metastoreTableIdentifier: Option[TableIdentifier]` is added to `LogicalRelation`.  When a `LogicalRelation` representing a persisted data source relation is created, this field holds the database name and table name of the relation.

Author: Cheng Lian <lian@databricks.com>

Closes #10712 from liancheng/spark-12724-datasources-sql-gen.
2016-01-12 14:19:53 -08:00
Reynold Xin 0d543b98f3 Revert "[SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":")"
This reverts commit 8cfa218f4f.
2016-01-12 12:56:52 -08:00
Kousuke Saruta 8cfa218f4f [SPARK-12692][BUILD][SQL] Scala style: Fix the style violation (Space before "," or ":")
Fix the style violation (space before , and :).
This PR is a followup for #10643.

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

Closes #10718 from sarutak/SPARK-12692-followup-sql.
2016-01-12 00:51:00 -08:00
wangfei 473907adf6 [SPARK-12742][SQL] org.apache.spark.sql.hive.LogicalPlanToSQLSuite failure due to Table already exists exception
```
[info] Exception encountered when attempting to run a suite with class name:
org.apache.spark.sql.hive.LogicalPlanToSQLSuite *** ABORTED *** (325 milliseconds)
[info]   org.apache.spark.sql.AnalysisException: Table `t1` already exists.;
[info]   at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:296)
[info]   at org.apache.spark.sql.DataFrameWriter.saveAsTable(DataFrameWriter.scala:285)
[info]   at org.apache.spark.sql.hive.LogicalPlanToSQLSuite.beforeAll(LogicalPlanToSQLSuite.scala:33)
[info]   at org.scalatest.BeforeAndAfterAll$class.beforeAll(BeforeAndAfterAll.scala:187)
[info]   at org.apache.spark.sql.hive.LogicalPlanToSQLSuite.beforeAll(LogicalPlanToSQLSuite.scala:23)
[info]   at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:253)
[info]   at org.apache.spark.sql.hive.LogicalPlanToSQLSuite.run(LogicalPlanToSQLSuite.scala:23)
[info]   at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
[info]   at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:296)
[info]   at sbt.ForkMain$Run$2.call(ForkMain.java:286)
[info]   at java.util.concurrent.FutureTask.run(FutureTask.java:266)
[info]   at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
[info]   at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
[info]   at java.lang.Thread.run(Thread.java:745)
```

/cc liancheng

Author: wangfei <wangfei_hello@126.com>

Closes #10682 from scwf/fix-test.
2016-01-11 18:18:44 -08:00
Herman van Hovell fe9eb0b0ce [SPARK-12576][SQL] Enable expression parsing in CatalystQl
The PR allows us to use the new SQL parser to parse SQL expressions such as: ```1 + sin(x*x)```

We enable this functionality in this PR, but we will not start using this actively yet. This will be done as soon as we have reached grammar parity with the existing parser stack.

cc rxin

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10649 from hvanhovell/SPARK-12576.
2016-01-11 16:29:37 -08:00
Marcelo Vanzin 6439a82503 [SPARK-3873][BUILD] Enable import ordering error checking.
Turn import ordering violations into build errors, plus a few adjustments
to account for how the checker behaves. I'm a little on the fence about
whether the existing code is right, but it's easier to appease the checker
than to discuss what's the more correct order here.

Plus a few fixes to imports that cropped in since my recent cleanups.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10612 from vanzin/SPARK-3873-enable.
2016-01-10 20:04:50 -08:00
Cheng Lian d9447cac74 [SPARK-12593][SQL] Converts resolved logical plan back to SQL
This PR tries to enable Spark SQL to convert resolved logical plans back to SQL query strings.  For now, the major use case is to canonicalize Spark SQL native view support.  The major entry point is `SQLBuilder.toSQL`, which returns an `Option[String]` if the logical plan is recognized.

The current version is still in WIP status, and is quite limited.  Known limitations include:

1.  The logical plan must be analyzed but not optimized

    The optimizer erases `Subquery` operators, which contain necessary scope information for SQL generation.  Future versions should be able to recover erased scope information by inserting subqueries when necessary.

1.  The logical plan must be created using HiveQL query string

    Query plans generated by composing arbitrary DataFrame API combinations are not supported yet.  Operators within these query plans need to be rearranged into a canonical form that is more suitable for direct SQL generation.  For example, the following query plan

    ```
    Filter (a#1 < 10)
     +- MetastoreRelation default, src, None
    ```

    need to be canonicalized into the following form before SQL generation:

    ```
    Project [a#1, b#2, c#3]
     +- Filter (a#1 < 10)
         +- MetastoreRelation default, src, None
    ```

    Otherwise, the SQL generation process will have to handle a large number of special cases.

1.  Only a fraction of expressions and basic logical plan operators are supported in this PR

    Currently, 95.7% (1720 out of 1798) query plans in `HiveCompatibilitySuite` can be successfully converted to SQL query strings.

    Known unsupported components are:

    - Expressions
      - Part of math expressions
      - Part of string expressions (buggy?)
      - Null expressions
      - Calendar interval literal
      - Part of date time expressions
      - Complex type creators
      - Special `NOT` expressions, e.g. `NOT LIKE` and `NOT IN`
    - Logical plan operators/patterns
      - Cube, rollup, and grouping set
      - Script transformation
      - Generator
      - Distinct aggregation patterns that fit `DistinctAggregationRewriter` analysis rule
      - Window functions

    Support for window functions, generators, and cubes etc. will be added in follow-up PRs.

This PR leverages `HiveCompatibilitySuite` for testing SQL generation in a "round-trip" manner:

*   For all select queries, we try to convert it back to SQL
*   If the query plan is convertible, we parse the generated SQL into a new logical plan
*   Run the new logical plan instead of the original one

If the query plan is inconvertible, the test case simply falls back to the original logic.

TODO

- [x] Fix failed test cases
- [x] Support for more basic expressions and logical plan operators (e.g. distinct aggregation etc.)
- [x] Comments and documentation

Author: Cheng Lian <lian@databricks.com>

Closes #10541 from liancheng/sql-generation.
2016-01-08 14:08:13 -08:00
Davies Liu fd1dcfaf26 [SPARK-12542][SQL] support except/intersect in HiveQl
Parse the SQL query with except/intersect in FROM clause for HivQL.

Author: Davies Liu <davies@databricks.com>

Closes #10622 from davies/intersect.
2016-01-06 23:46:12 -08:00
Wenchen Fan 917d3fc069 [SPARK-12539][SQL] support writing bucketed table
This PR adds bucket write support to Spark SQL. User can specify bucketing columns, numBuckets and sorting columns with or without partition columns. For example:
```
df.write.partitionBy("year").bucketBy(8, "country").sortBy("amount").saveAsTable("sales")
```

When bucketing is used, we will calculate bucket id for each record, and group the records by bucket id. For each group, we will create a file with bucket id in its name, and write data into it. For each bucket file, if sorting columns are specified, the data will be sorted before write.

Note that there may be multiply files for one bucket, as the data is distributed.

Currently we store the bucket metadata at hive metastore in a non-hive-compatible way. We use different bucketing hash function compared to hive, so we can't be compatible anyway.

Limitations:

* Can't write bucketed data without hive metastore.
* Can't insert bucketed data into existing hive tables.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10498 from cloud-fan/bucket-write.
2016-01-06 16:58:10 -08:00
Herman van Hovell ea489f14f1 [SPARK-12573][SPARK-12574][SQL] Move SQL Parser from Hive to Catalyst
This PR moves a major part of the new SQL parser to Catalyst. This is a prelude to start using this parser for all of our SQL parsing. The following key changes have been made:

The ANTLR Parser & Supporting classes have been moved to the Catalyst project. They are now part of the ```org.apache.spark.sql.catalyst.parser``` package. These classes contained quite a bit of code that was originally from the Hive project, I have added aknowledgements whenever this applied. All Hive dependencies have been factored out. I have also taken this chance to clean-up the ```ASTNode``` class, and to improve the error handling.

The HiveQl object that provides the functionality to convert an AST into a LogicalPlan has been refactored into three different classes, one for every SQL sub-project:
- ```CatalystQl```: This implements Query and Expression parsing functionality.
- ```SparkQl```: This is a subclass of CatalystQL and provides SQL/Core only functionality such as Explain and Describe.
- ```HiveQl```: This is a subclass of ```SparkQl``` and this adds Hive-only functionality to the parser such as Analyze, Drop, Views, CTAS & Transforms. This class still depends on Hive.

cc rxin

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10583 from hvanhovell/SPARK-12575.
2016-01-06 11:16:53 -08:00
Liang-Chi Hsieh b2467b3810 [SPARK-12578][SQL] Distinct should not be silently ignored when used in an aggregate function with OVER clause
JIRA: https://issues.apache.org/jira/browse/SPARK-12578

Slightly update to Hive parser. We should keep the distinct keyword when used in an aggregate function with OVER clause. So the CheckAnalysis will detect it and throw exception later.

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

Closes #10557 from viirya/keep-distinct-hivesql.
2016-01-06 00:40:14 -08:00
Marcelo Vanzin b3ba1be3b7 [SPARK-3873][TESTS] Import ordering fixes.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10582 from vanzin/SPARK-3873-tests.
2016-01-05 19:07:39 -08:00
Marcelo Vanzin df8bd97520 [SPARK-3873][SQL] Import ordering fixes.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #10573 from vanzin/SPARK-3873-sql.
2016-01-05 16:48:59 -08:00
Reynold Xin 8ce645d4ee [SPARK-12615] Remove some deprecated APIs in RDD/SparkContext
I looked at each case individually and it looks like they can all be removed. The only one that I had to think twice was toArray (I even thought about un-deprecating it, until I realized it was a problem in Java to have toArray returning java.util.List).

Author: Reynold Xin <rxin@databricks.com>

Closes #10569 from rxin/SPARK-12615.
2016-01-05 11:10:14 -08:00
Wenchen Fan b1a771231e [SPARK-12480][SQL] add Hash expression that can calculate hash value for a group of expressions
just write the arguments into unsafe row and use murmur3 to calculate hash code

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10435 from cloud-fan/hash-expr.
2016-01-04 18:49:41 -08:00
Reynold Xin 77ab49b857 [SPARK-12600][SQL] Remove deprecated methods in Spark SQL
Author: Reynold Xin <rxin@databricks.com>

Closes #10559 from rxin/remove-deprecated-sql.
2016-01-04 18:02:38 -08:00
Davies Liu d084a2de32 [SPARK-12541] [SQL] support cube/rollup as function
This PR enable cube/rollup as function, so they can be used as this:
```
select a, b, sum(c) from t group by rollup(a, b)
```

Author: Davies Liu <davies@databricks.com>

Closes #10522 from davies/rollup.
2016-01-04 14:26:56 -08:00
Reynold Xin 6c5bbd628a Revert "Revert "[SPARK-12286][SPARK-12290][SPARK-12294][SPARK-12284][SQL] always output UnsafeRow""
This reverts commit 44ee920fd4.
2016-01-02 22:39:25 -08:00
Sean Owen 15bd73627e [SPARK-12481][CORE][STREAMING][SQL] Remove usage of Hadoop deprecated APIs and reflection that supported 1.x
Remove use of deprecated Hadoop APIs now that 2.2+ is required

Author: Sean Owen <sowen@cloudera.com>

Closes #10446 from srowen/SPARK-12481.
2016-01-02 13:15:53 +00:00
Herman van Hovell 970635a9f8 [SPARK-12362][SQL][WIP] Inline Hive Parser
This PR inlines the Hive SQL parser in Spark SQL.

The previous (merged) incarnation of this PR passed all tests, but had and still has problems with the build. These problems are caused by a the fact that - for some reason - in some cases the ANTLR generated code is not included in the compilation fase.

This PR is a WIP and should not be merged until we have sorted out the build issues.

Author: Herman van Hovell <hvanhovell@questtec.nl>
Author: Nong Li <nong@databricks.com>
Author: Nong Li <nongli@gmail.com>

Closes #10525 from hvanhovell/SPARK-12362.
2016-01-01 23:22:50 -08:00
Reynold Xin 44ee920fd4 Revert "[SPARK-12286][SPARK-12290][SPARK-12294][SPARK-12284][SQL] always output UnsafeRow"
This reverts commit 0da7bd50dd.
2016-01-01 19:23:06 -08:00
Davies Liu 0da7bd50dd [SPARK-12286][SPARK-12290][SPARK-12294][SPARK-12284][SQL] always output UnsafeRow
It's confusing that some operator output UnsafeRow but some not, easy to make mistake.

This PR change to only output UnsafeRow for all the operators (SparkPlan), removed the rule to insert Unsafe/Safe conversions. For those that can't output UnsafeRow directly, added UnsafeProjection into them.

Closes #10330

cc JoshRosen rxin

Author: Davies Liu <davies@databricks.com>

Closes #10511 from davies/unsafe_row.
2016-01-01 13:39:20 -08:00
Cheng Lian 01a29866b1 [SPARK-12592][SQL][TEST] Don't mute Spark loggers in TestHive.reset()
There's a hack done in `TestHive.reset()`, which intended to mute noisy Hive loggers. However, Spark testing loggers are also muted.

Author: Cheng Lian <lian@databricks.com>

Closes #10540 from liancheng/spark-12592.dont-mute-spark-loggers.
2016-01-01 13:24:09 -08:00
Yin Huai 5cdecb1841 [SPARK-12039][SQL] Re-enable HiveSparkSubmitSuite's SPARK-9757 Persist Parquet relation with decimal column
https://issues.apache.org/jira/browse/SPARK-12039

since we do not support hadoop1, we can re-enable this test in master.

Author: Yin Huai <yhuai@databricks.com>

Closes #10533 from yhuai/SPARK-12039-enable.
2015-12-31 01:33:21 -08:00
Herman van Hovell f76ee109d8 [SPARK-8641][SPARK-12455][SQL] Native Spark Window functions - Follow-up (docs & tests)
This PR is a follow-up for PR https://github.com/apache/spark/pull/9819. It adds documentation for the window functions and a couple of NULL tests.

The documentation was largely based on the documentation in (the source of)  Hive and Presto:
* https://prestodb.io/docs/current/functions/window.html
* https://cwiki.apache.org/confluence/display/Hive/LanguageManual+WindowingAndAnalytics

I am not sure if we need to add the licenses of these two projects to the licenses directory. They are both under the ASL. srowen any thoughts?

cc yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #10402 from hvanhovell/SPARK-8641-docs.
2015-12-30 16:51:07 -08:00
Reynold Xin 27af6157f9 Revert "[SPARK-12362][SQL][WIP] Inline Hive Parser"
This reverts commit b600bccf41 due to non-deterministic build breaks.
2015-12-30 00:08:44 -08:00
Nong Li b600bccf41 [SPARK-12362][SQL][WIP] Inline Hive Parser
This is a WIP. The PR has been taken over from nongli (see https://github.com/apache/spark/pull/10420). I have removed some additional dead code, and fixed a few issues which were caused by the fact that the inlined Hive parser is newer than the Hive parser we currently use in Spark.

I am submitting this PR in order to get some feedback and testing done. There is quite a bit of work to do:
- [ ] Get it to pass jenkins build/test.
- [ ] Aknowledge Hive-project for using their parser.
- [ ] Refactorings between HiveQl and the java classes.
  - [ ] Create our own ASTNode and integrate the current implicit extentions.
  - [ ] Move remaining ```SemanticAnalyzer``` and ```ParseUtils``` functionality to ```HiveQl```.
- [ ] Removing Hive dependencies from the parser. This will require some edits in the grammar files.
  - [ ] Introduce our own context which needs to contain a ```TokenRewriteStream```.
  - [ ] Add ```useSQL11ReservedKeywordsForIdentifier``` and ```allowQuotedId``` to the catalyst or sql configuration.
  - [ ] Remove ```HiveConf``` from grammar files &HiveQl, and pass in our own configuration.
- [ ] Moving the parser into sql/core.

cc nongli rxin

Author: Herman van Hovell <hvanhovell@questtec.nl>
Author: Nong Li <nong@databricks.com>
Author: Nong Li <nongli@gmail.com>

Closes #10509 from hvanhovell/SPARK-12362.
2015-12-29 18:47:41 -08:00
gatorsmile 043135819c [SPARK-12522][SQL][MINOR] Add the missing document strings for the SQL configuration
Fixing the missing the document for the configuration. We can see the missing messages "TODO" when issuing the command "SET -V".
```
spark.sql.columnNameOfCorruptRecord
spark.sql.hive.verifyPartitionPath
spark.sql.sources.parallelPartitionDiscovery.threshold
spark.sql.hive.convertMetastoreParquet.mergeSchema
spark.sql.hive.convertCTAS
spark.sql.hive.thriftServer.async
```

Author: gatorsmile <gatorsmile@gmail.com>

Closes #10471 from gatorsmile/commandDesc.
2015-12-28 17:22:18 -08:00
gatorsmile 01ba95d8bf [SPARK-12441][SQL] Fixing missingInput in Generate/MapPartitions/AppendColumns/MapGroups/CoGroup
When explain any plan with Generate, we will see an exclamation mark in the plan. Normally, when we see this mark, it means the plan has an error. This PR is to correct the `missingInput` in `Generate`.

For example,
```scala
val df = Seq((1, "a b c"), (2, "a b"), (3, "a")).toDF("number", "letters")
val df2 =
  df.explode('letters) {
    case Row(letters: String) => letters.split(" ").map(Tuple1(_)).toSeq
  }

df2.explain(true)
```
Before the fix, the plan is like
```
== Parsed Logical Plan ==
'Generate UserDefinedGenerator('letters), true, false, None
+- Project [_1#0 AS number#2,_2#1 AS letters#3]
   +- LocalRelation [_1#0,_2#1], [[1,a b c],[2,a b],[3,a]]

== Analyzed Logical Plan ==
number: int, letters: string, _1: string
Generate UserDefinedGenerator(letters#3), true, false, None, [_1#8]
+- Project [_1#0 AS number#2,_2#1 AS letters#3]
   +- LocalRelation [_1#0,_2#1], [[1,a b c],[2,a b],[3,a]]

== Optimized Logical Plan ==
Generate UserDefinedGenerator(letters#3), true, false, None, [_1#8]
+- LocalRelation [number#2,letters#3], [[1,a b c],[2,a b],[3,a]]

== Physical Plan ==
!Generate UserDefinedGenerator(letters#3), true, false, [number#2,letters#3,_1#8]
+- LocalTableScan [number#2,letters#3], [[1,a b c],[2,a b],[3,a]]
```

**Updates**: The same issues are also found in the other four Dataset operators: `MapPartitions`/`AppendColumns`/`MapGroups`/`CoGroup`. Fixed all these four.

Author: gatorsmile <gatorsmile@gmail.com>
Author: xiaoli <lixiao1983@gmail.com>
Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local>

Closes #10393 from gatorsmile/generateExplain.
2015-12-28 12:48:30 -08:00
Cheng Lian 8e23d8db7f [SPARK-12218] Fixes ORC conjunction predicate push down
This PR is a follow-up of PR #10362.

Two major changes:

1.  The fix introduced in #10362 is OK for Parquet, but may disable ORC PPD in many cases

    PR #10362 stops converting an `AND` predicate if any branch is inconvertible.  On the other hand, `OrcFilters` combines all filters into a single big conjunction first and then tries to convert it into ORC `SearchArgument`.  This means, if any filter is inconvertible, no filters can be pushed down.  This PR fixes this issue by finding out all convertible filters first before doing the actual conversion.

    The reason behind the current implementation is mostly due to the limitation of ORC `SearchArgument` builder, which is documented in this PR in detail.

1.  Copied the `AND` predicate fix for ORC from #10362 to avoid merge conflict.

Same as #10362, this PR targets master (2.0.0-SNAPSHOT), branch-1.6, and branch-1.5.

Author: Cheng Lian <lian@databricks.com>

Closes #10377 from liancheng/spark-12218.fix-orc-conjunction-ppd.
2015-12-28 08:48:44 -08:00
hyukjinkwon 364d244a50 [SPARK-11677][SQL][FOLLOW-UP] Add tests for checking the ORC filter creation against pushed down filters.
https://issues.apache.org/jira/browse/SPARK-11677
Although it checks correctly the filters by the number of results if ORC filter-push-down is enabled, the filters themselves are not being tested.
So, this PR includes the test similarly with `ParquetFilterSuite`.
Since the results are checked by `OrcQuerySuite`, this `OrcFilterSuite` only checks if the appropriate filters are created.

One thing different with `ParquetFilterSuite` here is, it does not check the results because that is checked in `OrcQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #10341 from HyukjinKwon/SPARK-11677-followup.
2015-12-23 00:39:49 +08:00
Wenchen Fan 7634fe9511 [SPARK-12321][SQL] JSON format for TreeNode (use reflection)
An alternative solution for https://github.com/apache/spark/pull/10295 , instead of implementing json format for all logical/physical plans and expressions, use reflection to implement it in `TreeNode`.

Here I use pre-order traversal to flattern a plan tree to a plan list, and add an extra field `num-children` to each plan node, so that we can reconstruct the tree from the list.

example json:

logical plan tree:
```
[ {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.Sort",
  "num-children" : 1,
  "order" : [ [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.SortOrder",
    "num-children" : 1,
    "child" : 0,
    "direction" : "Ascending"
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "i",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 10,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  } ] ],
  "global" : false,
  "child" : 0
}, {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.Project",
  "num-children" : 1,
  "projectList" : [ [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.Alias",
    "num-children" : 1,
    "child" : 0,
    "name" : "i",
    "exprId" : {
      "id" : 10,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Add",
    "num-children" : 2,
    "left" : 0,
    "right" : 1
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "a",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 0,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Literal",
    "num-children" : 0,
    "value" : "1",
    "dataType" : "integer"
  } ], [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.Alias",
    "num-children" : 1,
    "child" : 0,
    "name" : "j",
    "exprId" : {
      "id" : 11,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Multiply",
    "num-children" : 2,
    "left" : 0,
    "right" : 1
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "a",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 0,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  }, {
    "class" : "org.apache.spark.sql.catalyst.expressions.Literal",
    "num-children" : 0,
    "value" : "2",
    "dataType" : "integer"
  } ] ],
  "child" : 0
}, {
  "class" : "org.apache.spark.sql.catalyst.plans.logical.LocalRelation",
  "num-children" : 0,
  "output" : [ [ {
    "class" : "org.apache.spark.sql.catalyst.expressions.AttributeReference",
    "num-children" : 0,
    "name" : "a",
    "dataType" : "integer",
    "nullable" : true,
    "metadata" : { },
    "exprId" : {
      "id" : 0,
      "jvmId" : "cd1313c7-3f66-4ed7-a320-7d91e4633ac6"
    },
    "qualifiers" : [ ]
  } ] ],
  "data" : [ ]
} ]
```

Author: Wenchen Fan <wenchen@databricks.com>

Closes #10311 from cloud-fan/toJson-reflection.
2015-12-21 12:47:07 -08:00
Reynold Xin f496031bd2 Bump master version to 2.0.0-SNAPSHOT.
Author: Reynold Xin <rxin@databricks.com>

Closes #10387 from rxin/version-bump.
2015-12-19 15:13:05 -08:00
Yin Huai 41ee7c57ab [SPARK-12218][SQL] Invalid splitting of nested AND expressions in Data Source filter API
JIRA: https://issues.apache.org/jira/browse/SPARK-12218

When creating filters for Parquet/ORC, we should not push nested AND expressions partially.

Author: Yin Huai <yhuai@databricks.com>

Closes #10362 from yhuai/SPARK-12218.
2015-12-18 10:53:13 -08:00
Dilip Biswal ee444fe4b8 [SPARK-11619][SQL] cannot use UDTF in DataFrame.selectExpr
Description of the problem from cloud-fan

Actually this line: https://github.com/apache/spark/blob/branch-1.5/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala#L689
When we use `selectExpr`, we pass in `UnresolvedFunction` to `DataFrame.select` and fall in the last case. A workaround is to do special handling for UDTF like we did for `explode`(and `json_tuple` in 1.6), wrap it with `MultiAlias`.
Another workaround is using `expr`, for example, `df.select(expr("explode(a)").as(Nil))`, I think `selectExpr` is no longer needed after we have the `expr` function....

Author: Dilip Biswal <dbiswal@us.ibm.com>

Closes #9981 from dilipbiswal/spark-11619.
2015-12-18 09:54:30 -08:00
Herman van Hovell 658f66e620 [SPARK-8641][SQL] Native Spark Window functions
This PR removes Hive windows functions from Spark and replaces them with (native) Spark ones. The PR is on par with Hive in terms of features.

This has the following advantages:
* Better memory management.
* The ability to use spark UDAFs in Window functions.

cc rxin / yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #9819 from hvanhovell/SPARK-8641-2.
2015-12-17 15:16:35 -08:00
hyukjinkwon 9657ee8788 [SPARK-11677][SQL] ORC filter tests all pass if filters are actually not pushed down.
Currently ORC filters are not tested properly. All the tests pass even if the filters are not pushed down or disabled. In this PR, I add some logics for this.
Since ORC does not filter record by record fully, this checks the count of the result and if it contains the expected values.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9687 from HyukjinKwon/SPARK-11677.
2015-12-16 13:24:49 -08:00
Davies Liu 834e71489b [SPARK-12213][SQL] use multiple partitions for single distinct query
Currently, we could generate different plans for query with single distinct (depends on spark.sql.specializeSingleDistinctAggPlanning), one works better on low cardinality columns, the other
works better for high cardinality column (default one).

This PR change to generate a single plan (three aggregations and two exchanges), which work better in both cases, then we could safely remove the flag `spark.sql.specializeSingleDistinctAggPlanning` (introduced in 1.6).

For a query like `SELECT COUNT(DISTINCT a) FROM table` will be
```
AGG-4 (count distinct)
  Shuffle to a single reducer
    Partial-AGG-3 (count distinct, no grouping)
      Partial-AGG-2 (grouping on a)
        Shuffle by a
          Partial-AGG-1 (grouping on a)
```

This PR also includes large refactor for aggregation (reduce 500+ lines of code)

cc yhuai nongli marmbrus

Author: Davies Liu <davies@databricks.com>

Closes #10228 from davies/single_distinct.
2015-12-13 22:57:01 -08:00
Yin Huai ec5f9ed5de [SPARK-12228][SQL] Try to run execution hive's derby in memory.
This PR tries to make execution hive's derby run in memory since it is a fake metastore and every time we create a HiveContext, we will switch to a new one. It is possible that it can reduce the flakyness of our tests that need to create HiveContext (e.g. HiveSparkSubmitSuite). I will test it more.

https://issues.apache.org/jira/browse/SPARK-12228

Author: Yin Huai <yhuai@databricks.com>

Closes #10204 from yhuai/derbyInMemory.
2015-12-10 12:04:20 -08:00
Yin Huai bc5f56aa60 [SPARK-12250][SQL] Allow users to define a UDAF without providing details of its inputSchema
https://issues.apache.org/jira/browse/SPARK-12250

Author: Yin Huai <yhuai@databricks.com>

Closes #10236 from yhuai/SPARK-12250.
2015-12-10 12:03:29 -08:00
Cheng Lian 6e1c55eac4 [SPARK-12012][SQL] Show more comprehensive PhysicalRDD metadata when visualizing SQL query plan
This PR adds a `private[sql]` method `metadata` to `SparkPlan`, which can be used to describe detail information about a physical plan during visualization. Specifically, this PR uses this method to provide details of `PhysicalRDD`s translated from a data source relation. For example, a `ParquetRelation` converted from Hive metastore table `default.psrc` is now shown as the following screenshot:

![image](https://cloud.githubusercontent.com/assets/230655/11526657/e10cb7e6-9916-11e5-9afa-f108932ec890.png)

And here is the screenshot for a regular `ParquetRelation` (not converted from Hive metastore table) loaded from a really long path:

![output](https://cloud.githubusercontent.com/assets/230655/11680582/37c66460-9e94-11e5-8f50-842db5309d5a.png)

Author: Cheng Lian <lian@databricks.com>

Closes #10004 from liancheng/spark-12012.physical-rdd-metadata.
2015-12-09 23:30:42 +08:00
Dmitry Erastov d0d8222778 [SPARK-6990][BUILD] Add Java linting script; fix minor warnings
This replaces https://github.com/apache/spark/pull/9696

Invoke Checkstyle and print any errors to the console, failing the step.
Use Google's style rules modified according to
https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide
Some important checks are disabled (see TODOs in `checkstyle.xml`) due to
multiple violations being present in the codebase.

Suggest fixing those TODOs in a separate PR(s).

More on Checkstyle can be found on the [official website](http://checkstyle.sourceforge.net/).

Sample output (from [build 46345](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/46345/consoleFull)) (duplicated because I run the build twice with different profiles):

> Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [error] running /home/jenkins/workspace/SparkPullRequestBuilder2/dev/lint-java ; received return code 1

Also fix some of the minor violations that didn't require sweeping changes.

Apologies for the previous botched PRs - I finally figured out the issue.

cr: JoshRosen, pwendell

> I state that the contribution is my original work, and I license the work to the project under the project's open source license.

Author: Dmitry Erastov <derastov@gmail.com>

Closes #9867 from dskrvk/master.
2015-12-04 12:03:45 -08:00
Josh Rosen ef6790fdc3 [SPARK-12075][SQL] Speed up HiveComparisionTest by avoiding / speeding up TestHive.reset()
When profiling HiveCompatibilitySuite, I noticed that most of the time seems to be spent in expensive `TestHive.reset()` calls. This patch speeds up suites based on HiveComparisionTest, such as HiveCompatibilitySuite, with the following changes:

- Avoid `TestHive.reset()` whenever possible:
  - Use a simple set of heuristics to guess whether we need to call `reset()` in between tests.
  - As a safety-net, automatically re-run failed tests by calling `reset()` before the re-attempt.
- Speed up the expensive parts of `TestHive.reset()`: loading the `src` and `srcpart` tables took roughly 600ms per test, so we now avoid this by using a simple heuristic which only loads those tables by tests that reference them. This is based on simple string matching over the test queries which errs on the side of loading in more situations than might be strictly necessary.

After these changes, HiveCompatibilitySuite seems to run in about 10 minutes.

This PR is a revival of #6663, an earlier experimental PR from June, where I played around with several possible speedups for this suite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10055 from JoshRosen/speculative-testhive-reset.
2015-12-02 07:29:45 +08:00
Yin Huai 0ddfe78689 [SPARK-12039] [SQL] Ignore HiveSparkSubmitSuite's "SPARK-9757 Persist Parquet relation with decimal column".
https://issues.apache.org/jira/browse/SPARK-12039

Since it is pretty flaky in hadoop 1 tests, we can disable it while we are investigating the cause.

Author: Yin Huai <yhuai@databricks.com>

Closes #10035 from yhuai/SPARK-12039-ignore.
2015-11-29 19:02:15 -08:00
Huaxin Gao 4d4cbc034b [SPARK-11778][SQL] add regression test
Fix regression test for SPARK-11778.
 marmbrus
Could you please take a look?
Thank you very much!!

Author: Huaxin Gao <huaxing@oc0558782468.ibm.com>

Closes #9890 from huaxingao/spark-11778-regression-test.
2015-11-26 19:17:46 -08:00
Yin Huai ad76562390 [SPARK-11998][SQL][TEST-HADOOP2.0] When downloading Hadoop artifacts from maven, we need to try to download the version that is used by Spark
If we need to download Hive/Hadoop artifacts, try to download a Hadoop that matches the Hadoop used by Spark. If the Hadoop artifact cannot be resolved (e.g. Hadoop version is a vendor specific version like 2.0.0-cdh4.1.1), we will use Hadoop 2.4.0 (we used to hard code this version as the hadoop that we will download from maven) and we will not share Hadoop classes.

I tested this match in my laptop with the following confs (these confs are used by our builds). All tests are good.
```
build/sbt -Phadoop-1 -Dhadoop.version=1.2.1 -Pkinesis-asl -Phive-thriftserver -Phive
build/sbt -Phadoop-1 -Dhadoop.version=2.0.0-mr1-cdh4.1.1 -Pkinesis-asl -Phive-thriftserver -Phive
build/sbt -Pyarn -Phadoop-2.2 -Pkinesis-asl -Phive-thriftserver -Phive
build/sbt -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Pkinesis-asl -Phive-thriftserver -Phive
```

Author: Yin Huai <yhuai@databricks.com>

Closes #9979 from yhuai/versionsSuite.
2015-11-26 16:20:08 -08:00
Reynold Xin 4d6bbbc03d [SPARK-11947][SQL] Mark deprecated methods with "This will be removed in Spark 2.0."
Also fixed some documentation as I saw them.

Author: Reynold Xin <rxin@databricks.com>

Closes #9930 from rxin/SPARK-11947.
2015-11-24 18:58:55 -08:00
Cheng Lian c7f95df5c6 [SPARK-11783][SQL] Fixes execution Hive client when using remote Hive metastore
When using remote Hive metastore, `hive.metastore.uris` is set to the metastore URI.  However, it overrides `javax.jdo.option.ConnectionURL` unexpectedly, thus the execution Hive client connects to the actual remote Hive metastore instead of the Derby metastore created in the temporary directory.  Cleaning this configuration for the execution Hive client fixes this issue.

Author: Cheng Lian <lian@databricks.com>

Closes #9895 from liancheng/spark-11783.clean-remote-metastore-config.
2015-11-24 15:08:53 -08:00
Josh Rosen 9db5f601fa [SPARK-9866][SQL] Speed up VersionsSuite by using persistent Ivy cache
This patch attempts to speed up VersionsSuite by storing fetched Hive JARs in an Ivy cache that persists across tests runs. If `SPARK_VERSIONS_SUITE_IVY_PATH` is set, that path will be used for the cache; if it is not set, VersionsSuite will create a temporary Ivy cache which is deleted after the test completes.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9624 from JoshRosen/SPARK-9866.
2015-11-23 16:33:26 -08:00
Xiu Guo 94ce65dfcb [SPARK-11628][SQL] support column datatype of char(x) to recognize HiveChar
Can someone review my code to make sure I'm not missing anything? Thanks!

Author: Xiu Guo <xguo27@gmail.com>
Author: Xiu Guo <guoxi@us.ibm.com>

Closes #9612 from xguo27/SPARK-11628.
2015-11-23 08:53:40 -08:00
Nong Li 9ed4ad4265 [SPARK-11724][SQL] Change casting between int and timestamp to consistently treat int in seconds.
Hive has since changed this behavior as well. https://issues.apache.org/jira/browse/HIVE-3454

Author: Nong Li <nong@databricks.com>
Author: Nong Li <nongli@gmail.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #9685 from nongli/spark-11724.
2015-11-20 14:19:34 -08:00
Josh Rosen a66142dece [SPARK-11877] Prevent agg. fallback conf. from leaking across test suites
This patch fixes an issue where the `spark.sql.TungstenAggregate.testFallbackStartsAt` SQLConf setting was not properly reset / cleared at the end of `TungstenAggregationQueryWithControlledFallbackSuite`. This ended up causing test failures in HiveCompatibilitySuite in Maven builds by causing spilling to occur way too frequently.

This configuration leak was inadvertently introduced during test cleanup in #9618.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9857 from JoshRosen/clear-fallback-prop-in-test-teardown.
2015-11-20 00:46:29 -08:00
Reynold Xin 014c0f7a9d [SPARK-11858][SQL] Move sql.columnar into sql.execution.
In addition, tightened visibility of a lot of classes in the columnar package from private[sql] to private[columnar].

Author: Reynold Xin <rxin@databricks.com>

Closes #9842 from rxin/SPARK-11858.
2015-11-19 14:48:18 -08:00
Huaxin Gao 4700074530 [SPARK-11778][SQL] parse table name before it is passed to lookupRelation
Fix a bug in DataFrameReader.table (table with schema name such as "db_name.table" doesn't work)
Use SqlParser.parseTableIdentifier to parse the table name before lookupRelation.

Author: Huaxin Gao <huaxing@oc0558782468.ibm.com>

Closes #9773 from huaxingao/spark-11778.
2015-11-19 13:08:01 -08:00
navis.ryu fc3f77b42d [SPARK-11614][SQL] serde parameters should be set only when all params are ready
see HIVE-7975 and HIVE-12373

With changed semantic of setters in thrift objects in hive, setter should be called only after all parameters are set. It's not problem of current state but will be a problem in some day.

Author: navis.ryu <navis@apache.org>

Closes #9580 from navis/SPARK-11614.
2015-11-18 19:37:14 -08:00
Cheng Lian 7b1407c7b9 [SPARK-11089][SQL] Adds option for disabling multi-session in Thrift server
This PR adds a new option `spark.sql.hive.thriftServer.singleSession` for disabling multi-session support in the Thrift server.

Note that this option is added as a Spark configuration (retrieved from `SparkConf`) rather than Spark SQL configuration (retrieved from `SQLConf`). This is because all SQL configurations are session-ized. Since multi-session support is by default on, no JDBC connection can modify global configurations like the newly added one.

Author: Cheng Lian <lian@databricks.com>

Closes #9740 from liancheng/spark-11089.single-session-option.
2015-11-17 11:17:52 -08:00
Cheng Lian fa13301ae4 [SPARK-11191][SQL][FOLLOW-UP] Cleans up unnecessary anonymous HiveFunctionRegistry
According to discussion in PR #9664, the anonymous `HiveFunctionRegistry` in `HiveContext` can be removed now.

Author: Cheng Lian <lian@databricks.com>

Closes #9737 from liancheng/spark-11191.follow-up.
2015-11-17 18:11:08 +08:00
xin Wu 0e79604aed [SPARK-11522][SQL] input_file_name() returns "" for external tables
When computing partition for non-parquet relation, `HadoopRDD.compute` is used. but it does not set the thread local variable `inputFileName` in `NewSqlHadoopRDD`, like `NewSqlHadoopRDD.compute` does.. Yet, when getting the `inputFileName`, `NewSqlHadoopRDD.inputFileName` is exptected, which is empty now.
Adding the setting inputFileName in HadoopRDD.compute resolves this issue.

Author: xin Wu <xinwu@us.ibm.com>

Closes #9542 from xwu0226/SPARK-11522.
2015-11-16 08:10:48 -08:00
Yu Gao 72c1d68b4a [SPARK-10181][SQL] Do kerberos login for credentials during hive client initialization
On driver process start up, UserGroupInformation.loginUserFromKeytab is called with the principal and keytab passed in, and therefore static var UserGroupInfomation,loginUser is set to that principal with kerberos credentials saved in its private credential set, and all threads within the driver process are supposed to see and use this login credentials to authenticate with Hive and Hadoop. However, because of IsolatedClientLoader, UserGroupInformation class is not shared for hive metastore clients, and instead it is loaded separately and of course not able to see the prepared kerberos login credentials in the main thread.

The first proposed fix would cause other classloader conflict errors, and is not an appropriate solution. This new change does kerberos login during hive client initialization, which will make credentials ready for the particular hive client instance.

 yhuai Please take a look and let me know. If you are not the right person to talk to, could you point me to someone responsible for this?

Author: Yu Gao <ygao@us.ibm.com>
Author: gaoyu <gaoyu@gaoyu-macbookpro.roam.corp.google.com>
Author: Yu Gao <crystalgaoyu@gmail.com>

Closes #9272 from yolandagao/master.
2015-11-15 14:53:59 -08:00
Yin Huai 3e2e1873b2 [SPARK-11738] [SQL] Making ArrayType orderable
https://issues.apache.org/jira/browse/SPARK-11738

Author: Yin Huai <yhuai@databricks.com>

Closes #9718 from yhuai/makingArrayOrderable.
2015-11-15 13:59:59 -08:00
Reynold Xin d22fc10887 [SPARK-11734][SQL] Rename TungstenProject -> Project, TungstenSort -> Sort
I didn't remove the old Sort operator, since we still use it in randomized tests. I moved it into test module and renamed it ReferenceSort.

Author: Reynold Xin <rxin@databricks.com>

Closes #9700 from rxin/SPARK-11734.
2015-11-15 10:33:53 -08:00
Yin Huai 7b5d9051cf [SPARK-11678][SQL] Partition discovery should stop at the root path of the table.
https://issues.apache.org/jira/browse/SPARK-11678

The change of this PR is to pass root paths of table to the partition discovery logic. So, the process of partition discovery stops at those root paths instead of going all the way to the root path of the file system.

Author: Yin Huai <yhuai@databricks.com>

Closes #9651 from yhuai/SPARK-11678.
2015-11-13 18:36:56 +08:00
Cheng Lian 4fe99c72c6 [SPARK-11191][SQL] Looks up temporary function using execution Hive client
When looking up Hive temporary functions, we should always use the `SessionState` within the execution Hive client, since temporary functions are registered there.

Author: Cheng Lian <lian@databricks.com>

Closes #9664 from liancheng/spark-11191.fix-temp-function.
2015-11-12 12:17:51 -08:00
Reynold Xin 30e7433643 [SPARK-11673][SQL] Remove the normal Project physical operator (and keep TungstenProject)
Also make full outer join being able to produce UnsafeRows.

Author: Reynold Xin <rxin@databricks.com>

Closes #9643 from rxin/SPARK-11673.
2015-11-12 08:14:08 -08:00
Yin Huai 14cf753704 [SPARK-11661][SQL] Still pushdown filters returned by unhandledFilters.
https://issues.apache.org/jira/browse/SPARK-11661

Author: Yin Huai <yhuai@databricks.com>

Closes #9634 from yhuai/unhandledFilters.
2015-11-12 16:47:00 +08:00
Reynold Xin e49e723392 [SPARK-11675][SQL] Remove shuffle hash joins.
Author: Reynold Xin <rxin@databricks.com>

Closes #9645 from rxin/SPARK-11675.
2015-11-11 19:32:52 -08:00
Josh Rosen 2d76e44b1a [SPARK-11647] Attempt to reduce time/flakiness of Thriftserver CLI and SparkSubmit tests
This patch aims to reduce the test time and flakiness of HiveSparkSubmitSuite, SparkSubmitSuite, and CliSuite.

Key changes:

- Disable IO synchronization calls for Derby writes, since durability doesn't matter for tests. This was done for HiveCompatibilitySuite in #6651 and resulted in huge test speedups.
- Add a few missing `--conf`s to disable various Spark UIs. The CliSuite, in particular, never disabled these UIs, leaving it prone to port-contention-related flakiness.
- Fix two instances where tests defined `beforeAll()` methods which were never called because the appropriate traits were not mixed in. I updated these tests suites to extend `BeforeAndAfterEach` so that they play nicely with our `ResetSystemProperties` trait.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9623 from JoshRosen/SPARK-11647.
2015-11-11 14:30:38 -08:00
Reynold Xin df97df2b39 [SPARK-11644][SQL] Remove the option to turn off unsafe and codegen.
Author: Reynold Xin <rxin@databricks.com>

Closes #9618 from rxin/SPARK-11644.
2015-11-11 12:47:02 -08:00
hyukjinkwon 1bc41125ee [SPARK-11500][SQL] Not deterministic order of columns when using merging schemas.
https://issues.apache.org/jira/browse/SPARK-11500

As filed in SPARK-11500, if merging schemas is enabled, the order of files to touch is a matter which might affect the ordering of the output columns.

This was mostly because of the use of `Set` and `Map` so I replaced them to `LinkedHashSet` and `LinkedHashMap` to keep the insertion order.

Also, I changed `reduceOption` to `reduceLeftOption`, and replaced the order of `filesToTouch` from `metadataStatuses ++ commonMetadataStatuses ++ needMerged` to  `needMerged ++ metadataStatuses ++ commonMetadataStatuses` in order to touch the part-files first which always have the schema in footers whereas the others might not exist.

One nit is, If merging schemas is not enabled, but when multiple files are given, there is no guarantee of the output order, since there might not be a summary file for the first file, which ends up putting ahead the columns of the other files.

However, I thought this should be okay since disabling merging schemas means (assumes) all the files have the same schemas.

In addition, in the test code for this, I only checked the names of fields.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9517 from HyukjinKwon/SPARK-11500.
2015-11-11 16:46:04 +08:00
Forest Fang 12c7635dc0 [MINOR] Fix typo in AggregationQuerySuite.scala
Author: Forest Fang <saurfang@users.noreply.github.com>

Closes #9357 from saurfang/patch-1.
2015-11-10 16:56:06 -08:00
Herman van Hovell 21c562fa03 [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)
This PR is a 2nd follow-up for [SPARK-9241](https://issues.apache.org/jira/browse/SPARK-9241). It contains the following improvements:
* Fix for a potential bug in distinct child expression and attribute alignment.
* Improved handling of duplicate distinct child expressions.
* Added test for distinct UDAF with multiple children.

cc yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #9566 from hvanhovell/SPARK-9241-followup-2.
2015-11-10 16:28:21 -08:00
Wenchen Fan 53600854c2 [SPARK-11590][SQL] use native json_tuple in lateral view
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9562 from cloud-fan/json-tuple.
2015-11-10 11:21:31 -08:00
Yin Huai e0701c7560 [SPARK-9830][SQL] Remove AggregateExpression1 and Aggregate Operator used to evaluate AggregateExpression1s
https://issues.apache.org/jira/browse/SPARK-9830

This PR contains the following main changes.
* Removing `AggregateExpression1`.
* Removing `Aggregate` operator, which is used to evaluate `AggregateExpression1`.
* Removing planner rule used to plan `Aggregate`.
* Linking `MultipleDistinctRewriter` to analyzer.
* Renaming `AggregateExpression2` to `AggregateExpression` and `AggregateFunction2` to `AggregateFunction`.
* Updating places where we create aggregate expression. The way to create aggregate expressions is `AggregateExpression(aggregateFunction, mode, isDistinct)`.
* Changing `val`s in `DeclarativeAggregate`s that touch children of this function to `lazy val`s (when we create aggregate expression in DataFrame API, children of an aggregate function can be unresolved).

Author: Yin Huai <yhuai@databricks.com>

Closes #9556 from yhuai/removeAgg1.
2015-11-10 11:06:29 -08:00
Davies Liu d6cd3a18e7 [SPARK-11599] [SQL] fix NPE when resolve Hive UDF in SQLParser
The DataFrame APIs that takes a SQL expression always use SQLParser, then the HiveFunctionRegistry will called outside of Hive state, cause NPE if there is not a active Session State for current thread (in PySpark).

cc rxin yhuai

Author: Davies Liu <davies@databricks.com>

Closes #9576 from davies/hive_udf.
2015-11-09 23:27:36 -08:00
Cheng Lian 150f6a89b7 [SPARK-11595] [SQL] Fixes ADD JAR when the input path contains URL scheme
Author: Cheng Lian <lian@databricks.com>

Closes #9569 from liancheng/spark-11595.fix-add-jar.
2015-11-09 14:32:52 -08:00
Nick Buroojy f138cb8733 [SPARK-9301][SQL] Add collect_set and collect_list aggregate functions
For now they are thin wrappers around the corresponding Hive UDAFs.

One limitation with these in Hive 0.13.0 is they only support aggregating primitive types.

I chose snake_case here instead of camelCase because it seems to be used in the majority of the multi-word fns.

Do we also want to add these to `functions.py`?

This approach was recommended here: https://github.com/apache/spark/pull/8592#issuecomment-154247089

marmbrus rxin

Author: Nick Buroojy <nick.buroojy@civitaslearning.com>

Closes #9526 from nburoojy/nick/udaf-alias.

(cherry picked from commit a6ee4f989d)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-11-09 14:30:52 -08:00
Wenchen Fan d8b50f7029 [SPARK-11453][SQL] append data to partitioned table will messes up the result
The reason is that:

1. For partitioned hive table, we will move the partitioned columns after data columns. (e.g. `<a: Int, b: Int>` partition by `a` will become `<b: Int, a: Int>`)
2. When append data to table, we use position to figure out how to match input columns to table's columns.

So when we append data to partitioned table, we will match wrong columns between input and table. A solution is reordering the input columns before match by position, like what we did for [`InsertIntoHadoopFsRelation`](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelation.scala#L101-L105)

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9408 from cloud-fan/append.
2015-11-08 21:01:53 -08:00
Herman van Hovell 30c8ba71a7 [SPARK-11451][SQL] Support single distinct count on multiple columns.
This PR adds support for multiple column in a single count distinct aggregate to the new aggregation path.

cc yhuai

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #9409 from hvanhovell/SPARK-11451.
2015-11-08 11:06:10 -08:00
Herman van Hovell ef362846eb [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up
This PR is a follow up for PR https://github.com/apache/spark/pull/9406. It adds more documentation to the rewriting rule, removes a redundant if expression in the non-distinct aggregation path and adds a multiple distinct test to the AggregationQuerySuite.

cc yhuai marmbrus

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #9541 from hvanhovell/SPARK-9241-followup.
2015-11-07 13:37:37 -08:00
Cheng Lian c048929c6a [SPARK-10978][SQL][FOLLOW-UP] More comprehensive tests for PR #9399
This PR adds test cases that test various column pruning and filter push-down cases.

Author: Cheng Lian <lian@databricks.com>

Closes #9468 from liancheng/spark-10978.follow-up.
2015-11-06 11:11:36 -08:00
Cheng Lian 77488fb8e5 [MINOR][SQL] A minor log line fix
`jars` in the log line is an array, so `$jars` doesn't print its content.

Author: Cheng Lian <lian@databricks.com>

Closes #9494 from liancheng/minor.log-fix.
2015-11-05 23:49:44 +08:00
Sean Owen 6f81eae24f [SPARK-11440][CORE][STREAMING][BUILD] Declare rest of @Experimental items non-experimental if they've existed since 1.2.0
Remove `Experimental` annotations in core, streaming for items that existed in 1.2.0 or before. The changes are:

* SparkContext
  * binary{Files,Records} : 1.2.0
  * submitJob : 1.0.0
* JavaSparkContext
  * binary{Files,Records} : 1.2.0
* DoubleRDDFunctions, JavaDoubleRDD
  * {mean,sum}Approx : 1.0.0
* PairRDDFunctions, JavaPairRDD
  * sampleByKeyExact : 1.2.0
  * countByKeyApprox : 1.0.0
* PairRDDFunctions
  * countApproxDistinctByKey : 1.1.0
* RDD
  * countApprox, countByValueApprox, countApproxDistinct : 1.0.0
* JavaRDDLike
  * countApprox : 1.0.0
* PythonHadoopUtil.Converter : 1.1.0
* PortableDataStream : 1.2.0 (related to binaryFiles)
* BoundedDouble : 1.0.0
* PartialResult : 1.0.0
* StreamingContext, JavaStreamingContext
  * binaryRecordsStream : 1.2.0
* HiveContext
  * analyze : 1.2.0

Author: Sean Owen <sowen@cloudera.com>

Closes #9396 from srowen/SPARK-11440.
2015-11-05 09:08:53 +00:00
Davies Liu 81498dd5c8 [SPARK-11425] [SPARK-11486] Improve hybrid aggregation
After aggregation, the dataset could be smaller than inputs, so it's better to do hash based aggregation for all inputs, then using sort based aggregation to merge them.

Author: Davies Liu <davies@databricks.com>

Closes #9383 from davies/fix_switch.
2015-11-04 21:30:21 -08:00
Zhenhua Wang a752ddad7f [SPARK-11398] [SQL] unnecessary def dialectClassName in HiveContext, and misleading dialect conf at the start of spark-sql
1. def dialectClassName in HiveContext is unnecessary.
In HiveContext, if conf.dialect == "hiveql", getSQLDialect() will return new HiveQLDialect(this);
else it will use super.getSQLDialect(). Then in super.getSQLDialect(), it calls dialectClassName, which is overriden in HiveContext and still return super.dialectClassName.
So we'll never reach the code "classOf[HiveQLDialect].getCanonicalName" of def dialectClassName in HiveContext.

2. When we start bin/spark-sql, the default context is HiveContext, and the corresponding dialect is hiveql.
However, if we type "set spark.sql.dialect;", the result is "sql", which is inconsistent with the actual dialect and is misleading. For example, we can use sql like "create table" which is only allowed in hiveql, but this dialect conf shows it's "sql".
Although this problem will not cause any execution error, it's misleading to spark sql users. Therefore I think we should fix it.
In this pr, while procesing “set spark.sql.dialect” in SetCommand, I use "conf.dialect" instead of "getConf()" for the case of key == SQLConf.DIALECT.key, so that it will return the right dialect conf.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #9349 from wzhfy/dialect.
2015-11-04 17:16:00 -08:00
Cheng Lian ebf8b0b48d [SPARK-10978][SQL] Allow data sources to eliminate filters
This PR adds a new method `unhandledFilters` to `BaseRelation`. Data sources which implement this method properly may avoid the overhead of defensive filtering done by Spark SQL.

Author: Cheng Lian <lian@databricks.com>

Closes #9399 from liancheng/spark-10978.unhandled-filters.
2015-11-03 10:07:45 -08:00
navis.ryu c34c27fe92 [SPARK-9034][SQL] Reflect field names defined in GenericUDTF
Hive GenericUDTF#initialize() defines field names in a returned schema though,
the current HiveGenericUDTF drops these names.
We might need to reflect these in a logical plan tree.

Author: navis.ryu <navis@apache.org>

Closes #8456 from navis/SPARK-9034.
2015-11-02 23:52:36 -08:00
Nong Li 9cb5c731da [SPARK-11329][SQL] Support star expansion for structs.
1. Supporting expanding structs in Projections. i.e.
  "SELECT s.*" where s is a struct type.
  This is fixed by allowing the expand function to handle structs in addition to tables.

2. Supporting expanding * inside aggregate functions of structs.
   "SELECT max(struct(col1, structCol.*))"
   This requires recursively expanding the expressions. In this case, it it the aggregate
   expression "max(...)" and we need to recursively expand its children inputs.

Author: Nong Li <nongli@gmail.com>

Closes #9343 from nongli/spark-11329.
2015-11-02 20:32:08 -08:00
tedyu db11ee5e56 [SPARK-11371] Make "mean" an alias for "avg" operator
From Reynold in the thread 'Exception when using some aggregate operators' (http://search-hadoop.com/m/q3RTt0xFr22nXB4/):

I don't think these are bugs. The SQL standard for average is "avg", not "mean". Similarly, a distinct count is supposed to be written as "count(distinct col)", not "countDistinct(col)".
We can, however, make "mean" an alias for "avg" to improve compatibility between DataFrame and SQL.

Author: tedyu <yuzhihong@gmail.com>

Closes #9332 from ted-yu/master.
2015-11-02 13:51:53 -08:00
Daoyuan Wang 74ba95228d [SPARK-11311][SQL] spark cannot describe temporary functions
When describe temporary function, spark would return 'Unable to find function', this is not right.

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

Closes #9277 from adrian-wang/functionreg.
2015-11-02 23:07:30 +08:00
Liang-Chi Hsieh 3e770a64a4 [SPARK-9298][SQL] Add pearson correlation aggregation function
JIRA: https://issues.apache.org/jira/browse/SPARK-9298

This patch adds pearson correlation aggregation function based on `AggregateExpression2`.

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

Closes #8587 from viirya/corr_aggregation.
2015-11-01 18:37:27 -08:00
Cheng Lian aa494a9c2e [SPARK-11117] [SPARK-11345] [SQL] Makes all HadoopFsRelation data sources produce UnsafeRow
This PR fixes two issues:

1.  `PhysicalRDD.outputsUnsafeRows` is always `false`

    Thus a `ConvertToUnsafe` operator is often required even if the underlying data source relation does output `UnsafeRow`.

1.  Internal/external row conversion for `HadoopFsRelation` is kinda messy

    Currently we're using `HadoopFsRelation.needConversion` and [dirty type erasure hacks][1] to indicate whether the relation outputs external row or internal row and apply external-to-internal conversion when necessary.  Basically, all builtin `HadoopFsRelation` data sources, i.e. Parquet, JSON, ORC, and Text output `InternalRow`, while typical external `HadoopFsRelation` data sources, e.g. spark-avro and spark-csv, output `Row`.

This PR adds a `private[sql]` interface method `HadoopFsRelation.buildInternalScan`, which by default invokes `HadoopFsRelation.buildScan` and converts `Row`s to `UnsafeRow`s (which are also `InternalRow`s).  All builtin `HadoopFsRelation` data sources override this method and directly output `UnsafeRow`s.  In this way, now `HadoopFsRelation` always produces `UnsafeRow`s. Thus `PhysicalRDD.outputsUnsafeRows` can be properly set by checking whether the underlying data source is a `HadoopFsRelation`.

A remaining question is that, can we assume that all non-builtin `HadoopFsRelation` data sources output external rows?  At least all well known ones do so.  However it's possible that some users implemented their own `HadoopFsRelation` data sources that leverages `InternalRow` and thus all those unstable internal data representations.  If this assumption is safe, we can deprecate `HadoopFsRelation.needConversion` and cleanup some more conversion code (like [here][2] and [here][3]).

This PR supersedes #9125.

Follow-ups:

1.  Makes JSON and ORC data sources output `UnsafeRow` directly

1.  Makes `HiveTableScan` output `UnsafeRow` directly

    This is related to 1 since ORC data source shares the same `Writable` unwrapping code with `HiveTableScan`.

[1]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala#L353
[2]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala#L331-L335
[3]: https://github.com/apache/spark/blob/v1.5.1/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala#L630-L669

Author: Cheng Lian <lian@databricks.com>

Closes #9305 from liancheng/spark-11345.unsafe-hadoop-fs-relation.
2015-10-31 21:16:09 -07:00
sethah a01cbf5daa [SPARK-10641][SQL] Add Skewness and Kurtosis Support
Implementing skewness and kurtosis support based on following algorithm:
https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics

Author: sethah <seth.hendrickson16@gmail.com>

Closes #9003 from sethah/SPARK-10641.
2015-10-29 11:58:39 -07:00
xin Wu f7a51deeba [SPARK-11246] [SQL] Table cache for Parquet broken in 1.5
The root cause is that when spark.sql.hive.convertMetastoreParquet=true by default, the cached InMemoryRelation of the ParquetRelation can not be looked up from the cachedData of CacheManager because the key comparison fails even though it is the same LogicalPlan representing the Subquery that wraps the ParquetRelation.
The solution in this PR is overriding the LogicalPlan.sameResult function in Subquery case class to eliminate subquery node first before directly comparing the child (ParquetRelation), which will find the key  to the cached InMemoryRelation.

Author: xin Wu <xinwu@us.ibm.com>

Closes #9326 from xwu0226/spark-11246-commit.
2015-10-29 07:42:46 -07:00
Cheng Hao d9c6039897 [SPARK-10484] [SQL] Optimize the cartesian join with broadcast join for some cases
In some cases, we can broadcast the smaller relation in cartesian join, which improve the performance significantly.

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

Closes #8652 from chenghao-intel/cartesian.
2015-10-27 20:26:38 -07:00
Wenchen Fan a150e6c1b0 [SPARK-10562] [SQL] support mixed case partitionBy column names for tables stored in metastore
https://issues.apache.org/jira/browse/SPARK-10562

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9226 from cloud-fan/par.
2015-10-26 21:14:26 -07:00
Yin Huai 4725cb988b [SPARK-11194] [SQL] Use MutableURLClassLoader for the classLoader in IsolatedClientLoader.
https://issues.apache.org/jira/browse/SPARK-11194

Author: Yin Huai <yhuai@databricks.com>

Closes #9170 from yhuai/SPARK-11194.
2015-10-23 17:15:13 -07:00
Reynold Xin cdea0174e3 [SPARK-11273][SQL] Move ArrayData/MapData/DataTypeParser to catalyst.util package
Author: Reynold Xin <rxin@databricks.com>

Closes #9239 from rxin/types-private.
2015-10-23 00:00:21 -07:00
Cheng Hao d4950e6be4 [SPARK-9735][SQL] Respect the user specified schema than the infer partition schema for HadoopFsRelation
To enable the unit test of `hadoopFsRelationSuite.Partition column type casting`. It previously threw exception like below, as we treat the auto infer partition schema with higher priority than the user specified one.

```
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getUTF8String(rows.scala:220)
	at org.apache.spark.sql.catalyst.expressions.JoinedRow.getUTF8String(JoinedRow.scala:102)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(generated.java:62)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
	at scala.collection.AbstractIterator.to(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
07:44:01.344 ERROR org.apache.spark.executor.Executor: Exception in task 14.0 in stage 3.0 (TID 206)
java.lang.ClassCastException: java.lang.Integer cannot be cast to org.apache.spark.unsafe.types.UTF8String
	at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
	at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getUTF8String(rows.scala:220)
	at org.apache.spark.sql.catalyst.expressions.JoinedRow.getUTF8String(JoinedRow.scala:102)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.apply(generated.java:62)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at org.apache.spark.sql.execution.datasources.DataSourceStrategy$$anonfun$17$$anonfun$apply$9.apply(DataSourceStrategy.scala:212)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
	at scala.collection.Iterator$class.foreach(Iterator.scala:727)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
	at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
	at scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
	at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
	at scala.collection.AbstractIterator.to(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
	at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
	at scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
	at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:903)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1846)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
```

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

Closes #8026 from chenghao-intel/partition_discovery.
2015-10-22 13:11:37 -07:00
navis.ryu f481090a71 [SPARK-10151][SQL] Support invocation of hive macro
Macro in hive (which is GenericUDFMacro) contains real function inside of it but it's not conveyed to tasks, resulting null-pointer exception.

Author: navis.ryu <navis@apache.org>

Closes #8354 from navis/SPARK-10151.
2015-10-21 15:07:08 -07:00
Artem Aliev a37cd87048 [SPARK-11208][SQL] Filter out 'hive.metastore.rawstore.impl' from executionHive temporary config
The executionHive assumed to be a standard meta store located in temporary directory as a derby db. But hive.metastore.rawstore.impl was not filtered out so any custom implementation of the metastore with other storage properties (not JDO) will persist that temporary functions. CassandraHiveMetaStore from DataStax Enterprise is one of examples.

Author: Artem Aliev <artem.aliev@datastax.com>

Closes #9178 from artem-aliev/SPARK-11208.
2015-10-21 14:12:07 -07:00
Yin Huai 3afe448d39 [SPARK-9740][SPARK-9592][SPARK-9210][SQL] Change the default behavior of First/Last to RESPECT NULLS.
I am changing the default behavior of `First`/`Last` to respect null values (the SQL standard default behavior).

https://issues.apache.org/jira/browse/SPARK-9740

Author: Yin Huai <yhuai@databricks.com>

Closes #8113 from yhuai/firstLast.
2015-10-21 13:43:17 -07:00
Davies Liu f8c6bec657 [SPARK-11197][SQL] run SQL on files directly
This PR introduce a new feature to run SQL directly on files without create a table, for example:

```
select id from json.`path/to/json/files` as j
```

Author: Davies Liu <davies@databricks.com>

Closes #9173 from davies/source.
2015-10-21 13:38:30 -07:00
Jakob Odersky 60851bc7bf [SPARK-11110][BUILD] Remove transient annotation for parameters.
`transient` annotations on class parameters (not case class parameters or vals) causes compilation errors during compilation with Scala 2.11.
I understand that transient *parameters* make no sense, however I don't quite understand why the 2.10 compiler accepted them.

Note: in case it is preferred to keep the annotations in case someone would in the future want to redefine them as vals, it would also be possible to just add `val` after the annotation, e.g. `class Foo(transient x: Int)` becomes `class Foo(transient private val x: Int)`.

I chose to remove the annotation as it also reduces needles clutter, however please feel free to tell me if you prefer the second option and I'll update the PR

Author: Jakob Odersky <jodersky@gmail.com>

Closes #9126 from jodersky/sbt-scala-2.11.
2015-10-20 08:54:34 +01:00
Josh Rosen 4ace4f8a9c [SPARK-11017] [SQL] Support ImperativeAggregates in TungstenAggregate
This patch extends TungstenAggregate to support ImperativeAggregate functions. The existing TungstenAggregate operator only supported DeclarativeAggregate functions, which are defined in terms of Catalyst expressions and can be evaluated via generated projections. ImperativeAggregate functions, on the other hand, are evaluated by calling their `initialize`, `update`, `merge`, and `eval` methods.

The basic strategy here is similar to how SortBasedAggregate evaluates both types of aggregate functions: use a generated projection to evaluate the expression-based declarative aggregates with dummy placeholder expressions inserted in place of the imperative aggregate function output, then invoke the imperative aggregate functions and target them against the aggregation buffer. The bulk of the diff here consists of code that was copied and adapted from SortBasedAggregate, with some key changes to handle TungstenAggregate's sort fallback path.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9038 from JoshRosen/support-interpreted-in-tungsten-agg-final.
2015-10-14 17:27:50 -07:00
Wenchen Fan 56d7da14ab [SPARK-10104] [SQL] Consolidate different forms of table identifiers
Right now, we have QualifiedTableName, TableIdentifier, and Seq[String] to represent table identifiers. We should only have one form and TableIdentifier is the best one because it provides methods to get table name, database name, return unquoted string, and return quoted string.

Author: Wenchen Fan <wenchen@databricks.com>
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8453 from cloud-fan/table-name.
2015-10-14 16:05:37 -07:00
Yin Huai ce3f9a8065 [SPARK-11091] [SQL] Change spark.sql.canonicalizeView to spark.sql.nativeView.
https://issues.apache.org/jira/browse/SPARK-11091

Author: Yin Huai <yhuai@databricks.com>

Closes #9103 from yhuai/SPARK-11091.
2015-10-13 18:21:24 -07:00
Davies Liu d0cc79ccd0 [SPARK-11030] [SQL] share the SQLTab across sessions
The SQLTab will be shared by multiple sessions.

If we create multiple independent SQLContexts (not using newSession()), will still see multiple SQLTabs in the Spark UI.

Author: Davies Liu <davies@databricks.com>

Closes #9048 from davies/sqlui.
2015-10-13 09:57:53 -07:00
Davies Liu 6987c06793 [SPARK-11009] [SQL] fix wrong result of Window function in cluster mode
Currently, All windows function could generate wrong result in cluster sometimes.

The root cause is that AttributeReference is called in executor, then id of it may not be unique than others created in driver.

Here is the script that could reproduce the problem (run in local cluster):
```
from pyspark import SparkContext, HiveContext
from pyspark.sql.window import Window
from pyspark.sql.functions import rowNumber

sqlContext = HiveContext(SparkContext())
sqlContext.setConf("spark.sql.shuffle.partitions", "3")
df =  sqlContext.range(1<<20)
df2 = df.select((df.id % 1000).alias("A"), (df.id / 1000).alias('B'))
ws = Window.partitionBy(df2.A).orderBy(df2.B)
df3 = df2.select("client", "date", rowNumber().over(ws).alias("rn")).filter("rn < 0")
assert df3.count() == 0
```

Author: Davies Liu <davies@databricks.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #9050 from davies/wrong_window.
2015-10-13 09:43:33 -07:00
Yin Huai 8a354bef55 [SPARK-11042] [SQL] Add a mechanism to ban creating multiple root SQLContexts/HiveContexts in a JVM
https://issues.apache.org/jira/browse/SPARK-11042

Author: Yin Huai <yhuai@databricks.com>

Closes #9058 from yhuai/SPARK-11042.
2015-10-12 13:50:34 -07:00
Liang-Chi Hsieh fcb37a0417 [SPARK-10960] [SQL] SQL with windowing function should be able to refer column in inner select
JIRA: https://issues.apache.org/jira/browse/SPARK-10960

When accessing a column in inner select from a select with window function, `AnalysisException` will be thrown. For example, an query like this:

     select area, rank() over (partition by area order by tmp.month) + tmp.tmp1 as c1 from (select month, area, product, 1 as tmp1 from windowData) tmp

Currently, the rule `ExtractWindowExpressions` in `Analyzer` only extracts regular expressions from `WindowFunction`, `WindowSpecDefinition` and `AggregateExpression`. We need to also extract other attributes as the one in `Alias` as shown in the above query.

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

Closes #9011 from viirya/fix-window-inner-column.
2015-10-12 09:16:14 -07:00
Davies Liu 3390b400d0 [SPARK-10810] [SPARK-10902] [SQL] Improve session management in SQL
This PR improve the sessions management by replacing the thread-local based to one SQLContext per session approach, introduce separated temporary tables and UDFs/UDAFs for each session.

A new session of SQLContext could be created by:

1) create an new SQLContext
2) call newSession() on existing SQLContext

For HiveContext, in order to reduce the cost for each session, the classloader and Hive client are shared across multiple sessions (created by newSession).

CacheManager is also shared by multiple sessions, so cache a table multiple times in different sessions will not cause multiple copies of in-memory cache.

Added jars are still shared by all the sessions, because SparkContext does not support sessions.

cc marmbrus yhuai rxin

Author: Davies Liu <davies@databricks.com>

Closes #8909 from davies/sessions.
2015-10-08 17:34:24 -07:00
Cheng Lian 02149ff08e [SPARK-8848] [SQL] Refactors Parquet write path to follow parquet-format
This PR refactors Parquet write path to follow parquet-format spec.  It's a successor of PR #7679, but with less non-essential changes.

Major changes include:

1.  Replaces `RowWriteSupport` and `MutableRowWriteSupport` with `CatalystWriteSupport`

    - Writes Parquet data using standard layout defined in parquet-format

      Specifically, we are now writing ...

      - ... arrays and maps in standard 3-level structure with proper annotations and field names
      - ... decimals as `INT32` and `INT64` whenever possible, and taking `FIXED_LEN_BYTE_ARRAY` as the final fallback

    - Supports legacy mode which is compatible with Spark 1.4 and prior versions

      The legacy mode is by default off, and can be turned on by flipping SQL option `spark.sql.parquet.writeLegacyFormat` to `true`.

    - Eliminates per value data type dispatching costs via prebuilt composed writer functions

1.  Cleans up the last pieces of old Parquet support code

As pointed out by rxin previously, we probably want to rename all those `Catalyst*` Parquet classes to `Parquet*` for clarity.  But I'd like to do this in a follow-up PR to minimize code review noises in this one.

Author: Cheng Lian <lian@databricks.com>

Closes #8988 from liancheng/spark-8848/standard-parquet-write-path.
2015-10-08 16:18:35 -07:00
Wenchen Fan af2a554487 [SPARK-10337] [SQL] fix hive views on non-hive-compatible tables.
add a new config to deal with this special case.

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8990 from cloud-fan/view-master.
2015-10-08 12:42:10 -07:00
Cheng Lian 2df882ef14 [SPARK-5775] [SPARK-5508] [SQL] Re-enable Hive Parquet array reading tests
Since SPARK-5508 has already been fixed.

Author: Cheng Lian <lian@databricks.com>

Closes #8999 from liancheng/spark-5775.enable-array-tests.
2015-10-08 09:22:42 -07:00
navis.ryu 713e4f44e9 [SPARK-10679] [CORE] javax.jdo.JDOFatalUserException in executor
HadoopRDD throws exception in executor, something like below.
{noformat}
5/09/17 18:51:21 INFO metastore.HiveMetaStore: 0: Opening raw store with implemenation class:org.apache.hadoop.hive.metastore.ObjectStore
15/09/17 18:51:21 INFO metastore.ObjectStore: ObjectStore, initialize called
15/09/17 18:51:21 WARN metastore.HiveMetaStore: Retrying creating default database after error: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
javax.jdo.JDOFatalUserException: Class org.datanucleus.api.jdo.JDOPersistenceManagerFactory was not found.
	at javax.jdo.JDOHelper.invokeGetPersistenceManagerFactoryOnImplementation(JDOHelper.java:1175)
	at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:808)
	at javax.jdo.JDOHelper.getPersistenceManagerFactory(JDOHelper.java:701)
	at org.apache.hadoop.hive.metastore.ObjectStore.getPMF(ObjectStore.java:365)
	at org.apache.hadoop.hive.metastore.ObjectStore.getPersistenceManager(ObjectStore.java:394)
	at org.apache.hadoop.hive.metastore.ObjectStore.initialize(ObjectStore.java:291)
	at org.apache.hadoop.hive.metastore.ObjectStore.setConf(ObjectStore.java:258)
	at org.apache.hadoop.util.ReflectionUtils.setConf(ReflectionUtils.java:73)
	at org.apache.hadoop.util.ReflectionUtils.newInstance(ReflectionUtils.java:133)
	at org.apache.hadoop.hive.metastore.RawStoreProxy.<init>(RawStoreProxy.java:57)
	at org.apache.hadoop.hive.metastore.RawStoreProxy.getProxy(RawStoreProxy.java:66)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.newRawStore(HiveMetaStore.java:593)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.getMS(HiveMetaStore.java:571)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.createDefaultDB(HiveMetaStore.java:620)
	at org.apache.hadoop.hive.metastore.HiveMetaStore$HMSHandler.init(HiveMetaStore.java:461)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.<init>(RetryingHMSHandler.java:66)
	at org.apache.hadoop.hive.metastore.RetryingHMSHandler.getProxy(RetryingHMSHandler.java:72)
	at org.apache.hadoop.hive.metastore.HiveMetaStore.newRetryingHMSHandler(HiveMetaStore.java:5762)
	at org.apache.hadoop.hive.metastore.HiveMetaStoreClient.<init>(HiveMetaStoreClient.java:199)
	at org.apache.hadoop.hive.ql.metadata.SessionHiveMetaStoreClient.<init>(SessionHiveMetaStoreClient.java:74)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:526)
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.newInstance(MetaStoreUtils.java:1521)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.<init>(RetryingMetaStoreClient.java:86)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:132)
	at org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.getProxy(RetryingMetaStoreClient.java:104)
	at org.apache.hadoop.hive.ql.metadata.Hive.createMetaStoreClient(Hive.java:3005)
	at org.apache.hadoop.hive.ql.metadata.Hive.getMSC(Hive.java:3024)
	at org.apache.hadoop.hive.ql.metadata.Hive.getAllDatabases(Hive.java:1234)
	at org.apache.hadoop.hive.ql.metadata.Hive.reloadFunctions(Hive.java:174)
	at org.apache.hadoop.hive.ql.metadata.Hive.<clinit>(Hive.java:166)
	at org.apache.hadoop.hive.ql.plan.PlanUtils.configureJobPropertiesForStorageHandler(PlanUtils.java:803)
	at org.apache.hadoop.hive.ql.plan.PlanUtils.configureInputJobPropertiesForStorageHandler(PlanUtils.java:782)
	at org.apache.spark.sql.hive.HadoopTableReader$.initializeLocalJobConfFunc(TableReader.scala:298)
	at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
	at org.apache.spark.sql.hive.HadoopTableReader$$anonfun$12.apply(TableReader.scala:274)
	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
	at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:176)
	at scala.Option.map(Option.scala:145)
	at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:176)
	at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:220)
	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:216)
	at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:101)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.UnionRDD.compute(UnionRDD.scala:87)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:297)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:264)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
	at org.apache.spark.scheduler.Task.run(Task.scala:88)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
{noformat}

Author: navis.ryu <navis@apache.org>

Closes #8804 from navis/SPARK-10679.
2015-10-07 14:56:02 -07:00
Marcelo Vanzin 94fc57afdf [SPARK-10300] [BUILD] [TESTS] Add support for test tags in run-tests.py.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8775 from vanzin/SPARK-10300.
2015-10-07 14:11:21 -07:00
Josh Rosen a9ecd06149 [SPARK-10941] [SQL] Refactor AggregateFunction2 and AlgebraicAggregate interfaces to improve code clarity
This patch refactors several of the Aggregate2 interfaces in order to improve code clarity.

The biggest change is a refactoring of the `AggregateFunction2` class hierarchy. In the old code, we had a class named `AlgebraicAggregate` that inherited from `AggregateFunction2`, added a new set of methods, then banned the use of the inherited methods. I found this to be fairly confusing because.

If you look carefully at the existing code, you'll see that subclasses of `AggregateFunction2` fall into two disjoint categories: imperative aggregation functions which directly extended `AggregateFunction2` and declarative, expression-based aggregate functions which extended `AlgebraicAggregate`. In order to make this more explicit, this patch refactors things so that `AggregateFunction2` is a sealed abstract class with two subclasses, `ImperativeAggregateFunction` and `ExpressionAggregateFunction`. The superclass, `AggregateFunction2`, now only contains methods and fields that are common to both subclasses.

After making this change, I updated the various AggregationIterator classes to comply with this new naming scheme. I also performed several small renamings in the aggregate interfaces themselves in order to improve clarity and rewrote or expanded a number of comments.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8973 from JoshRosen/tungsten-agg-comments.
2015-10-07 13:19:49 -07:00
Cheng Lian 01cd688f52 [SPARK-10400] [SQL] Renames SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC
We introduced SQL option `spark.sql.parquet.followParquetFormatSpec` while working on implementing Parquet backwards-compatibility rules in SPARK-6777. It indicates whether we should use legacy Parquet format adopted by Spark 1.4 and prior versions or the standard format defined in parquet-format spec to write Parquet files.

This option defaults to `false` and is marked as a non-public option (`isPublic = false`) because we haven't finished refactored Parquet write path. The problem is, the name of this option is somewhat confusing, because it's not super intuitive why we shouldn't follow the spec. Would be nice to rename it to `spark.sql.parquet.writeLegacyFormat`, and invert its default value (the two option names have opposite meanings).

Although this option is private in 1.5, we'll make it public in 1.6 after refactoring Parquet write path. So that users can decide whether to write Parquet files in standard format or legacy format.

Author: Cheng Lian <lian@databricks.com>

Closes #8566 from liancheng/spark-10400/deprecate-follow-parquet-format-spec.
2015-10-01 17:23:27 -07:00
Wenchen Fan 02026a8132 [SPARK-10671] [SQL] Throws an analysis exception if we cannot find Hive UDFs
Takes over https://github.com/apache/spark/pull/8800

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8941 from cloud-fan/hive-udf.
2015-10-01 13:23:59 -07:00
Reynold Xin 03cca5dce2 [SPARK-10770] [SQL] SparkPlan.executeCollect/executeTake should return InternalRow rather than external Row.
Author: Reynold Xin <rxin@databricks.com>

Closes #8900 from rxin/SPARK-10770-1.
2015-09-30 14:36:54 -04:00
Wenchen Fan 418e5e4cbd [SPARK-10741] [SQL] Hive Query Having/OrderBy against Parquet table is not working
https://issues.apache.org/jira/browse/SPARK-10741
I choose the second approach: do not change output exprIds when convert MetastoreRelation to LogicalRelation

Author: Wenchen Fan <cloud0fan@163.com>

Closes #8889 from cloud-fan/hot-bug.
2015-09-27 09:08:38 -07:00
Cheng Lian 6f94d56a95 [SPARK-10845] [SQL] Makes spark.sql.hive.version a SQLConfEntry
When refactoring SQL options from plain strings to the strongly typed `SQLConfEntry`, `spark.sql.hive.version` wasn't migrated, and doesn't show up in the result of `SET -v`, as `SET -v` only shows public `SQLConfEntry` instances. This affects compatibility with Simba ODBC driver.

This PR migrates this SQL option as a `SQLConfEntry` to fix this issue.

Author: Cheng Lian <lian@databricks.com>

Closes #8925 from liancheng/spark-10845/hive-version-conf.
2015-09-26 19:08:55 -07:00
Wenchen Fan 341b13f8f5 [SPARK-10765] [SQL] use new aggregate interface for hive UDAF
Author: Wenchen Fan <cloud0fan@163.com>

Closes #8874 from cloud-fan/hive-agg.
2015-09-24 09:54:07 -07:00
Zhichao Li 84f81e035e [SPARK-10310] [SQL] Fixes script transformation field/line delimiters
**Please attribute this PR to `Zhichao Li <zhichao.liintel.com>`.**

This PR is based on PR #8476 authored by zhichao-li. It fixes SPARK-10310 by adding field delimiter SerDe property to the default `LazySimpleSerDe`, and enabling default record reader/writer classes.

Currently, we only support `LazySimpleSerDe`, used together with `TextRecordReader` and `TextRecordWriter`, and don't support customizing record reader/writer using `RECORDREADER`/`RECORDWRITER` clauses. This should be addressed in separate PR(s).

Author: Cheng Lian <lian@databricks.com>

Closes #8860 from liancheng/spark-10310/fix-script-trans-delimiters.
2015-09-22 19:41:57 -07:00
Yin Huai 2204cdb284 [SPARK-10672] [SQL] Do not fail when we cannot save the metadata of a data source table in a hive compatible way
https://issues.apache.org/jira/browse/SPARK-10672

With changes in this PR, we will fallback to same the metadata of a table in Spark SQL specific way if we fail to save it in a hive compatible way (Hive throws an exception because of its internal restrictions, e.g. binary and decimal types cannot be saved to parquet if the metastore is running Hive 0.13). I manually tested the fix with the following test in `DataSourceWithHiveMetastoreCatalogSuite` (`spark.sql.hive.metastore.version=0.13` and `spark.sql.hive.metastore.jars`=`maven`).

```
    test(s"fail to save metadata of a parquet table in hive 0.13") {
      withTempPath { dir =>
        withTable("t") {
          val path = dir.getCanonicalPath

          sql(
            s"""CREATE TABLE t USING $provider
               |OPTIONS (path '$path')
               |AS SELECT 1 AS d1, cast("val_1" as binary) AS d2
             """.stripMargin)

          sql(
            s"""describe formatted t
             """.stripMargin).collect.foreach(println)

          sqlContext.table("t").show
        }
      }
    }
  }
```

Without this fix, we will fail with the following error.
```
org.apache.hadoop.hive.ql.metadata.HiveException: java.lang.UnsupportedOperationException: Unknown field type: binary
	at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:619)
	at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:576)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply$mcV$sp(ClientWrapper.scala:359)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply(ClientWrapper.scala:357)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$createTable$1.apply(ClientWrapper.scala:357)
	at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$withHiveState$1.apply(ClientWrapper.scala:256)
	at org.apache.spark.sql.hive.client.ClientWrapper.retryLocked(ClientWrapper.scala:211)
	at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:248)
	at org.apache.spark.sql.hive.client.ClientWrapper.createTable(ClientWrapper.scala:357)
	at org.apache.spark.sql.hive.HiveMetastoreCatalog.createDataSourceTable(HiveMetastoreCatalog.scala:358)
	at org.apache.spark.sql.hive.execution.CreateMetastoreDataSourceAsSelect.run(commands.scala:285)
	at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57)
	at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57)
	at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:69)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:140)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$5.apply(SparkPlan.scala:138)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:150)
	at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:138)
	at org.apache.spark.sql.execution.QueryExecution.toRdd$lzycompute(QueryExecution.scala:58)
	at org.apache.spark.sql.execution.QueryExecution.toRdd(QueryExecution.scala:58)
	at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:144)
	at org.apache.spark.sql.DataFrame.<init>(DataFrame.scala:129)
	at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51)
	at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:725)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$sql$1.apply(SQLTestUtils.scala:56)
	at org.apache.spark.sql.test.SQLTestUtils$$anonfun$sql$1.apply(SQLTestUtils.scala:56)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2$$anonfun$apply$2.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:165)
	at org.apache.spark.sql.test.SQLTestUtils$class.withTable(SQLTestUtils.scala:150)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.withTable(HiveMetastoreCatalogSuite.scala:52)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2.apply(HiveMetastoreCatalogSuite.scala:162)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1$$anonfun$apply$mcV$sp$2.apply(HiveMetastoreCatalogSuite.scala:161)
	at org.apache.spark.sql.test.SQLTestUtils$class.withTempPath(SQLTestUtils.scala:125)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.withTempPath(HiveMetastoreCatalogSuite.scala:52)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply$mcV$sp(HiveMetastoreCatalogSuite.scala:161)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:161)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite$$anonfun$4$$anonfun$apply$1.apply(HiveMetastoreCatalogSuite.scala:161)
	at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22)
	at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42)
	at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306)
	at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175)
	at org.scalatest.FunSuite.runTest(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413)
	at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401)
	at scala.collection.immutable.List.foreach(List.scala:318)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401)
	at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483)
	at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1555)
	at org.scalatest.Suite$class.run(Suite.scala:1424)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:545)
	at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.org$scalatest$BeforeAndAfterAll$$super$run(HiveMetastoreCatalogSuite.scala:52)
	at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257)
	at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256)
	at org.apache.spark.sql.hive.DataSourceWithHiveMetastoreCatalogSuite.run(HiveMetastoreCatalogSuite.scala:52)
	at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462)
	at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671)
	at sbt.ForkMain$Run$2.call(ForkMain.java:294)
	at sbt.ForkMain$Run$2.call(ForkMain.java:284)
	at java.util.concurrent.FutureTask.run(FutureTask.java:262)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
	at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.UnsupportedOperationException: Unknown field type: binary
	at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.getObjectInspector(ArrayWritableObjectInspector.java:108)
	at org.apache.hadoop.hive.ql.io.parquet.serde.ArrayWritableObjectInspector.<init>(ArrayWritableObjectInspector.java:60)
	at org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe.initialize(ParquetHiveSerDe.java:113)
	at org.apache.hadoop.hive.metastore.MetaStoreUtils.getDeserializer(MetaStoreUtils.java:339)
	at org.apache.hadoop.hive.ql.metadata.Table.getDeserializerFromMetaStore(Table.java:288)
	at org.apache.hadoop.hive.ql.metadata.Table.checkValidity(Table.java:194)
	at org.apache.hadoop.hive.ql.metadata.Hive.createTable(Hive.java:597)
	... 76 more
```

Author: Yin Huai <yhuai@databricks.com>

Closes #8824 from yhuai/datasourceMetadata.
2015-09-22 13:29:39 -07:00
Davies Liu 22d40159e6 [SPARK-10593] [SQL] fix resolve output of Generate
The output of Generate should not be resolved as Reference.

Author: Davies Liu <davies@databricks.com>

Closes #8755 from davies/view.
2015-09-22 11:07:10 -07:00