Commit graph

1404 commits

Author SHA1 Message Date
Davies Liu a8332098ce [SPARK-6216] [PYSPARK] check python version of worker with driver
This PR revert #5404, change to pass the version of python in driver into JVM, check it in worker before deserializing closure, then it can works with different major version of Python.

Author: Davies Liu <davies@databricks.com>

Closes #6203 from davies/py_version and squashes the following commits:

b8fb76e [Davies Liu] fix test
6ce5096 [Davies Liu] use string for version
47c6278 [Davies Liu] check python version of worker with driver

(cherry picked from commit 32fbd297dd)
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
2015-05-18 12:55:37 -07:00
Cheng Lian 39623481fc [SPARK-7673] [SQL] WIP: HadoopFsRelation and ParquetRelation2 performance optimizations
This PR introduces several performance optimizations to `HadoopFsRelation` and `ParquetRelation2`:

1.  Moving `FileStatus` listing from `DataSourceStrategy` into a cache within `HadoopFsRelation`.

    This new cache generalizes and replaces the one used in `ParquetRelation2`.

    This also introduces an interface change: to reuse cached `FileStatus` objects, `HadoopFsRelation.buildScan` methods now receive `Array[FileStatus]` instead of `Array[String]`.

1.  When Parquet task side metadata reading is enabled, skip reading row group information when reading Parquet footers.

    This is basically what PR #5334 does. Also, now we uses `ParquetFileReader.readAllFootersInParallel` to read footers in parallel.

Another optimization in question is, instead of asking `HadoopFsRelation.buildScan` to return an `RDD[Row]` for a single selected partition and then union them all, we ask it to return an `RDD[Row]` for all selected partitions. This optimization is based on the fact that Hadoop configuration broadcasting used in `NewHadoopRDD` takes 34% time in the following microbenchmark.  However, this complicates data source user code because user code must merge partition values manually.

To check the cost of broadcasting in `NewHadoopRDD`, I also did microbenchmark after removing the `broadcast` call in `NewHadoopRDD`.  All results are shown below.

### Microbenchmark

#### Preparation code

Generating a partitioned table with 50k partitions, 1k rows per partition:

```scala
import sqlContext._
import sqlContext.implicits._

for (n <- 0 until 500) {
  val data = for {
    p <- (n * 10) until ((n + 1) * 10)
    i <- 0 until 1000
  } yield (i, f"val_$i%04d", f"$p%04d")

  data.
    toDF("a", "b", "p").
    write.
    partitionBy("p").
    mode("append").
    parquet(path)
}
```

#### Benchmarking code

```scala
import sqlContext._
import sqlContext.implicits._

import org.apache.spark.sql.types._
import com.google.common.base.Stopwatch

val path = "hdfs://localhost:9000/user/lian/5k"

def benchmark(n: Int)(f: => Unit) {
  val stopwatch = new Stopwatch()

  def run() = {
    stopwatch.reset()
    stopwatch.start()
    f
    stopwatch.stop()
    stopwatch.elapsedMillis()
  }

  val records = (0 until n).map(_ => run())

  (0 until n).foreach(i => println(s"Round $i: ${records(i)} ms"))
  println(s"Average: ${records.sum / n.toDouble} ms")
}

benchmark(3) { read.parquet(path).explain(extended = true) }
```

#### Results

Before:

```
Round 0: 72528 ms
Round 1: 68938 ms
Round 2: 65372 ms
Average: 68946.0 ms
```

After:

```
Round 0: 59499 ms
Round 1: 53645 ms
Round 2: 53844 ms
Round 3: 49093 ms
Round 4: 50555 ms
Average: 53327.2 ms
```

Also removing Hadoop configuration broadcasting:

(Note that I was testing on a local laptop, thus network cost is pretty low.)

```
Round 0: 15806 ms
Round 1: 14394 ms
Round 2: 14699 ms
Round 3: 15334 ms
Round 4: 14123 ms
Average: 14871.2 ms
```

Author: Cheng Lian <lian@databricks.com>

Closes #6225 from liancheng/spark-7673 and squashes the following commits:

2d58a2b [Cheng Lian] Skips reading row group information when using task side metadata reading
7aa3748 [Cheng Lian] Optimizes FileStatusCache by introducing a map from parent directories to child files
ba41250 [Cheng Lian] Reuses HadoopFsRelation FileStatusCache in ParquetRelation2
3d278f7 [Cheng Lian] Fixes a bug when reading a single Parquet data file
b84612a [Cheng Lian] Fixes Scala style issue
6a08b02 [Cheng Lian] WIP: Moves file status cache into HadoopFSRelation

(cherry picked from commit 9dadf019b9)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-18 12:47:19 -07:00
Yin Huai a385f4b8dd [SPARK-7567] [SQL] [follow-up] Use a new flag to set output committer based on mapreduce apis
cc liancheng marmbrus

Author: Yin Huai <yhuai@databricks.com>

Closes #6130 from yhuai/directOutput and squashes the following commits:

312b07d [Yin Huai] A data source can use spark.sql.sources.outputCommitterClass to override the output committer.

(cherry picked from commit 530397ba2f)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-18 12:17:22 -07:00
Wenchen Fan d6f5f37911 [SPARK-7269] [SQL] Incorrect analysis for aggregation(use semanticEquals)
A modified version of https://github.com/apache/spark/pull/6110, use `semanticEquals` to make it more efficient.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6173 from cloud-fan/7269 and squashes the following commits:

e4a3cc7 [Wenchen Fan] address comments
cc02045 [Wenchen Fan] consider elements length equal
d7ff8f4 [Wenchen Fan] fix 7269

(cherry picked from commit 103c863c2e)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-18 12:13:09 -07:00
scwf dbd4ec807b [SPARK-7631] [SQL] treenode argString should not print children
spark-sql>
> explain extended
> select * from (
> select key from src union all
> select key from src) t;

now the spark plan will print children in argString
```
== Physical Plan ==
Union[ HiveTableScan key#1, (MetastoreRelation default, src, None), None,
HiveTableScan key#3, (MetastoreRelation default, src, None), None]
HiveTableScan key#1, (MetastoreRelation default, src, None), None
HiveTableScan key#3, (MetastoreRelation default, src, None), None
```

after this patch:
```
== Physical Plan ==
Union
 HiveTableScan [key#1], (MetastoreRelation default, src, None), None
 HiveTableScan [key#3], (MetastoreRelation default, src, None), None
```

I have tested this locally

Author: scwf <wangfei1@huawei.com>

Closes #6144 from scwf/fix-argString and squashes the following commits:

1a642e0 [scwf] fix treenode argString

(cherry picked from commit fc2480ed13)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-18 12:05:32 -07:00
Zhan Zhang 65d71bd9fb [SPARK-2883] [SQL] ORC data source for Spark SQL
This PR updates PR #6135 authored by zhzhan from Hortonworks.

----

This PR implements a Spark SQL data source for accessing ORC files.

> **NOTE**
>
> Although ORC is now an Apache TLP, the codebase is still tightly coupled with Hive.  That's why the new ORC data source is under `org.apache.spark.sql.hive` package, and must be used with `HiveContext`.  However, it doesn't require existing Hive installation to access ORC files.

1.  Saving/loading ORC files without contacting Hive metastore

1.  Support for complex data types (i.e. array, map, and struct)

1.  Aware of common optimizations provided by Spark SQL:

    - Column pruning
    - Partitioning pruning
    - Filter push-down

1.  Schema evolution support
1.  Hive metastore table conversion

This PR also include initial work done by scwf from Huawei (PR #3753).

Author: Zhan Zhang <zhazhan@gmail.com>
Author: Cheng Lian <lian@databricks.com>

Closes #6194 from liancheng/polishing-orc and squashes the following commits:

55ecd96 [Cheng Lian] Reorganizes ORC test suites
d4afeed [Cheng Lian] Addresses comments
21ada22 [Cheng Lian] Adds @since and @Experimental annotations
128bd3b [Cheng Lian] ORC filter bug fix
d734496 [Cheng Lian] Polishes the ORC data source
2650a42 [Zhan Zhang] resolve review comments
3c9038e [Zhan Zhang] resolve review comments
7b3c7c5 [Zhan Zhang] save mode fix
f95abfd [Zhan Zhang] reuse test suite
7cc2c64 [Zhan Zhang] predicate fix
4e61c16 [Zhan Zhang] minor change
305418c [Zhan Zhang] orc data source support

(cherry picked from commit aa31e431fc)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-18 12:03:59 -07:00
Wenchen Fan 7d44c01c5e [SQL] [MINOR] [THIS] use private for internal field in ScalaUdf
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6235 from cloud-fan/tmp and squashes the following commits:

8f16367 [Wenchen Fan] use private[this]

(cherry picked from commit 56ede88485)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-18 12:02:10 -07:00
Cheng Lian c7623a20a9 [SPARK-7570] [SQL] Ignores _temporary during partition discovery
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/6091)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #6091 from liancheng/spark-7570 and squashes the following commits:

8ff07e8 [Cheng Lian] Ignores _temporary during partition discovery

(cherry picked from commit 010a1c2780)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-18 11:59:57 -07:00
Rene Treffer b41301a13f [SPARK-6888] [SQL] Make the jdbc driver handling user-definable
Replace the DriverQuirks with JdbcDialect(s) (and MySQLDialect/PostgresDialect)
and allow developers to change the dialects on the fly (for new JDBCRRDs only).

Some types (like an unsigned 64bit number) can be trivially mapped to java.
The status quo is that the RRD will fail to load.
This patch makes it possible to overwrite the type mapping to read e.g.
64Bit numbers as strings and handle them afterwards in software.

JDBCSuite has an example that maps all types to String, which should always
work (at the cost of extra code afterwards).

As a side effect it should now be possible to develop simple dialects
out-of-tree and even with spark-shell.

Author: Rene Treffer <treffer@measite.de>

Closes #5555 from rtreffer/jdbc-dialects and squashes the following commits:

3cbafd7 [Rene Treffer] [SPARK-6888] ignore classes belonging to changed API in MIMA report
fe7e2e8 [Rene Treffer] [SPARK-6888] Make the jdbc driver handling user-definable
2015-05-18 11:57:06 -07:00
Liang-Chi Hsieh 0e7cd8ff82 [SPARK-7299][SQL] Set precision and scale for Decimal according to JDBC metadata instead of returned BigDecimal
JIRA: https://issues.apache.org/jira/browse/SPARK-7299

When connecting with oracle db through jdbc, the precision and scale of `BigDecimal` object returned by `ResultSet.getBigDecimal` is not correctly matched to the table schema reported by `ResultSetMetaData.getPrecision` and `ResultSetMetaData.getScale`.

So in case you insert a value like `19999` into a column with `NUMBER(12, 2)` type, you get through a `BigDecimal` object with scale as 0. But the dataframe schema has correct type as `DecimalType(12, 2)`. Thus, after you save the dataframe into parquet file and then retrieve it, you will get wrong result `199.99`.

Because it is reported to be problematic on jdbc connection with oracle db. It might be difficult to add test case for it. But according to the user's test on JIRA, it solves this problem.

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

Closes #5833 from viirya/jdbc_decimal_precision and squashes the following commits:

69bc2b5 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into jdbc_decimal_precision
928f864 [Liang-Chi Hsieh] Add comments.
5f9da94 [Liang-Chi Hsieh] Set up Decimal's precision and scale according to table schema instead of returned BigDecimal.

(cherry picked from commit e32c0f69f3)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-18 01:11:10 -07:00
zsxwing 2a42d2d8f2 [SPARK-7693][Core] Remove "import scala.concurrent.ExecutionContext.Implicits.global"
Learnt a lesson from SPARK-7655: Spark should avoid to use `scala.concurrent.ExecutionContext.Implicits.global` because the user may submit blocking actions to `scala.concurrent.ExecutionContext.Implicits.global` and exhaust all threads in it. This could crash Spark. So Spark should always use its own thread pools for safety.

This PR removes all usages of `scala.concurrent.ExecutionContext.Implicits.global` and uses proper thread pools to replace them.

Author: zsxwing <zsxwing@gmail.com>

Closes #6223 from zsxwing/SPARK-7693 and squashes the following commits:

a33ff06 [zsxwing] Decrease the max thread number from 1024 to 128
cf4b3fc [zsxwing] Remove "import scala.concurrent.ExecutionContext.Implicits.global"

(cherry picked from commit ff71d34e00)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-17 20:37:27 -07:00
Wenchen Fan be66d1924e [SQL] [MINOR] use catalyst type converter in ScalaUdf
It's a follow-up of https://github.com/apache/spark/pull/5154, we can speed up scala udf evaluation by create type converter in advance.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6182 from cloud-fan/tmp and squashes the following commits:

241cfe9 [Wenchen Fan] use converter in ScalaUdf

(cherry picked from commit 2f22424e9f)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-17 16:52:21 -07:00
Michael Armbrust a8556086d3 [SPARK-7491] [SQL] Allow configuration of classloader isolation for hive
Author: Michael Armbrust <michael@databricks.com>

Closes #6167 from marmbrus/configureIsolation and squashes the following commits:

6147cbe [Michael Armbrust] filter other conf
22cc3bc7 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into configureIsolation
07476ee [Michael Armbrust] filter empty prefixes
dfdf19c [Michael Armbrust] [SPARK-6906][SQL] Allow configuration of classloader isolation for hive

(cherry picked from commit 2ca60ace8f)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-17 12:43:26 -07:00
Josh Rosen 53d6ab51b2 [SPARK-7686] [SQL] DescribeCommand is assigned wrong output attributes in SparkStrategies
In `SparkStrategies`, `RunnableDescribeCommand` is called with the output attributes of the table being described rather than the attributes for the `describe` command's output.  I discovered this issue because it caused type conversion errors in some UnsafeRow conversion code that I'm writing.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #6217 from JoshRosen/SPARK-7686 and squashes the following commits:

953a344 [Josh Rosen] Fix SPARK-7686 with a simple change in SparkStrategies.
a4eec9f [Josh Rosen] Add failing regression test for SPARK-7686

(cherry picked from commit 564562874f)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-17 11:59:35 -07:00
Liang-Chi Hsieh 898be62489 [SPARK-7447] [SQL] Don't re-merge Parquet schema when the relation is deserialized
JIRA: https://issues.apache.org/jira/browse/SPARK-7447

`MetadataCache` in `ParquetRelation2` is annotated as `transient`. When `ParquetRelation2` is deserialized, we ask `MetadataCache` to refresh and perform schema merging again. It is time-consuming especially for very many parquet files.

With the new `FSBasedParquetRelation`, although `MetadataCache` is not `transient` now, `MetadataCache.refresh()` still performs schema merging again when the relation is deserialized.

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

Closes #6012 from viirya/without_remerge_schema and squashes the following commits:

2663957 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into without_remerge_schema
6ac7d93 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into without_remerge_schema
b0fc09b [Liang-Chi Hsieh] Don't generate and merge parquetSchema multiple times.

(cherry picked from commit 3399055787)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-17 15:42:40 +08:00
Cheng Lian 671a6bca5f [MINOR] [SQL] Removes an unreachable case clause
This case clause is already covered by the one above, and generates a compilation warning.

Author: Cheng Lian <lian@databricks.com>

Closes #6214 from liancheng/remove-unreachable-code and squashes the following commits:

c38ca7c [Cheng Lian] Removes an unreachable case clause

(cherry picked from commit ba4f8ca0d9)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-16 23:20:19 -07:00
Reynold Xin 17e078671e [SPARK-7654][SQL] Move JDBC into DataFrame's reader/writer interface.
Also moved all the deprecated functions into one place for SQLContext and DataFrame, and updated tests to use the new API.

Author: Reynold Xin <rxin@databricks.com>

Closes #6210 from rxin/df-writer-reader-jdbc and squashes the following commits:

7465c2c [Reynold Xin] Fixed unit test.
118e609 [Reynold Xin] Updated tests.
3441b57 [Reynold Xin] Updated javadoc.
13cdd1c [Reynold Xin] [SPARK-7654][SQL] Move JDBC into DataFrame's reader/writer interface.

(cherry picked from commit 517eb37a85)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-16 22:02:00 -07:00
Cheng Lian 856619d485 [HOTFIX] [SQL] Fixes DataFrameWriter.mode(String)
We forgot an assignment there.

/cc rxin

Author: Cheng Lian <lian@databricks.com>

Closes #6212 from liancheng/fix-df-writer and squashes the following commits:

711fbb0 [Cheng Lian] Adds a test case
3b72d78 [Cheng Lian] Fixes DataFrameWriter.mode(String)

(cherry picked from commit ce6391296a)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-16 20:57:26 +08:00
zsxwing ad5b0b1ce2 [SPARK-7655][Core][SQL] Remove 'scala.concurrent.ExecutionContext.Implicits.global' in 'ask' and 'BroadcastHashJoin'
Because both `AkkaRpcEndpointRef.ask` and `BroadcastHashJoin` uses `scala.concurrent.ExecutionContext.Implicits.global`. However, because the tasks in `BroadcastHashJoin` are usually long-running tasks, which will occupy all threads in `global`. Then `ask` cannot get a chance to process the replies.

For `ask`, actually the tasks are very simple, so we can use `MoreExecutors.sameThreadExecutor()`. For `BroadcastHashJoin`, it's better to use `ThreadUtils.newDaemonCachedThreadPool`.

Author: zsxwing <zsxwing@gmail.com>

Closes #6200 from zsxwing/SPARK-7655-2 and squashes the following commits:

cfdc605 [zsxwing] Remove redundant imort and minor doc fix
cf83153 [zsxwing] Add "sameThread" and "newDaemonCachedThreadPool with maxThreadNumber" to ThreadUtils
08ad0ee [zsxwing] Remove 'scala.concurrent.ExecutionContext.Implicits.global' in 'ask' and 'BroadcastHashJoin'

(cherry picked from commit 47e7ffe36b)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-16 00:44:36 -07:00
Reynold Xin 9da55b5706 [SPARK-7654][SQL] DataFrameReader and DataFrameWriter for input/output API
This patch introduces DataFrameWriter and DataFrameReader.

DataFrameReader interface, accessible through SQLContext.read, contains methods that create DataFrames. These methods used to reside in SQLContext. Example usage:
```scala
sqlContext.read.json("...")
sqlContext.read.parquet("...")
```

DataFrameWriter interface, accessible through DataFrame.write, implements a builder pattern to avoid the proliferation of options in writing DataFrame out. It currently implements:
- mode
- format (e.g. "parquet", "json")
- options (generic options passed down into data sources)
- partitionBy (partitioning columns)
Example usage:
```scala
df.write.mode("append").format("json").partitionBy("date").saveAsTable("myJsonTable")
```

TODO:

- [ ] Documentation update
- [ ] Move JDBC into reader / writer?
- [ ] Deprecate the old interfaces
- [ ] Move the generic load interface into reader.
- [ ] Update example code and documentation

Author: Reynold Xin <rxin@databricks.com>

Closes #6175 from rxin/reader-writer and squashes the following commits:

b146c95 [Reynold Xin] Deprecation of old APIs.
bd8abdf [Reynold Xin] Fixed merge conflict.
26abea2 [Reynold Xin] Added general load methods.
244fbec [Reynold Xin] Added equivalent to example.
4f15d92 [Reynold Xin] Added documentation for partitionBy.
7e91611 [Reynold Xin] [SPARK-7654][SQL] DataFrameReader and DataFrameWriter for input/output API.

(cherry picked from commit 578bfeeff5)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-15 22:09:52 -07:00
Cheng Lian bcb2c5d169 [SPARK-7591] [SQL] Partitioning support API tweaks
Please see [SPARK-7591] [1] for the details.

/cc rxin marmbrus yhuai

[1]: https://issues.apache.org/jira/browse/SPARK-7591

Author: Cheng Lian <lian@databricks.com>

Closes #6150 from liancheng/spark-7591 and squashes the following commits:

af422e7 [Cheng Lian] Addresses @rxin's comments
37d1738 [Cheng Lian] Fixes HadoopFsRelation partition columns initialization
2fc680a [Cheng Lian] Fixes Scala style issue
189ad23 [Cheng Lian] Removes HadoopFsRelation constructor arguments
522c24e [Cheng Lian] Adds OutputWriterFactory
047d40d [Cheng Lian] Renames FSBased* to HadoopFs*, also renamed FSBasedParquetRelation back to ParquetRelation2

(cherry picked from commit fdf5bba35d)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-15 16:21:22 +08:00
Yin Huai 7aa269f4bb [SQL] When creating partitioned table scan, explicitly create UnionRDD.
Otherwise, it will cause stack overflow when there are many partitions.

Author: Yin Huai <yhuai@databricks.com>

Closes #6162 from yhuai/partitionUnionedRDD and squashes the following commits:

fa016d8 [Yin Huai] Explicitly create UnionRDD.

(cherry picked from commit e8f0e016ea)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-15 12:04:39 +08:00
Liang-Chi Hsieh bac45229aa [SPARK-7098][SQL] Make the WHERE clause with timestamp show consistent result
JIRA: https://issues.apache.org/jira/browse/SPARK-7098

The WHERE clause with timstamp shows inconsistent results. This pr fixes it.

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

Closes #5682 from viirya/consistent_timestamp and squashes the following commits:

171445a [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into consistent_timestamp
4e98520 [Liang-Chi Hsieh] Make the WHERE clause with timestamp show consistent result.

(cherry picked from commit f9705d4613)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-14 20:49:32 -07:00
Michael Armbrust 778a0548cc [SPARK-7548] [SQL] Add explode function for DataFrames
Add an `explode` function for dataframes and modify the analyzer so that single table generating functions can be present in a select clause along with other expressions.   There are currently the following restrictions:
 - only top level TGFs are allowed (i.e. no `select(explode('list) + 1)`)
 - only one may be present in a single select to avoid potentially confusing implicit Cartesian products.

TODO:
 - [ ] Python

Author: Michael Armbrust <michael@databricks.com>

Closes #6107 from marmbrus/explodeFunction and squashes the following commits:

7ee2c87 [Michael Armbrust] whitespace
6f80ba3 [Michael Armbrust] Update dataframe.py
c176c89 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explodeFunction
81b5da3 [Michael Armbrust] style
d3faa05 [Michael Armbrust] fix self join case
f9e1e3e [Michael Armbrust] fix python, add since
4f0d0a9 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into explodeFunction
e710fe4 [Michael Armbrust] add java and python
52ca0dc [Michael Armbrust] [SPARK-7548][SQL] Add explode function for dataframes.

(cherry picked from commit 6d0633e3ec)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-14 19:51:00 -07:00
Wenchen Fan aa8a0f9637 [SQL][minor] rename apply for QueryPlanner
A follow-up of https://github.com/apache/spark/pull/5624

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6142 from cloud-fan/tmp and squashes the following commits:

971a92b [Wenchen Fan] use plan instead of execute
24c5ffe [Wenchen Fan] rename apply

(cherry picked from commit f2cd00be35)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-14 10:25:32 -07:00
linweizhong c80e0cff25 [SPARK-7595] [SQL] Window will cause resolve failed with self join
for example:
table: src(key string, value string)
sql: with v1 as(select key, count(value) over (partition by key) cnt_val from src), v2 as(select v1.key, v1_lag.cnt_val from v1, v1 v1_lag where v1.key = v1_lag.key) select * from v2 limit 5;
then will analyze fail when resolving conflicting references in Join:
'Limit 5
 'Project [*]
  'Subquery v2
   'Project ['v1.key,'v1_lag.cnt_val]
    'Filter ('v1.key = 'v1_lag.key)
     'Join Inner, None
      Subquery v1
       Project [key#95,cnt_val#94L]
        Window [key#95,value#96], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount(value#96) WindowSpecDefinition [key#95], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS cnt_val#94L], WindowSpecDefinition [key#95], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
         Project [key#95,value#96]
          MetastoreRelation default, src, None
      Subquery v1_lag
       Subquery v1
        Project [key#97,cnt_val#94L]
         Window [key#97,value#98], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFCount(value#98) WindowSpecDefinition [key#97], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS cnt_val#94L], WindowSpecDefinition [key#97], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
          Project [key#97,value#98]
           MetastoreRelation default, src, None

Conflicting attributes: cnt_val#94L

Author: linweizhong <linweizhong@huawei.com>

Closes #6114 from Sephiroth-Lin/spark-7595 and squashes the following commits:

f8f2637 [linweizhong] Add unit test
dfe9169 [linweizhong] Handle windowExpression with self join

(cherry picked from commit 13e652b61a)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-14 00:23:41 -07:00
zsxwing d518c0369f [HOTFIX] Use 'new Job' in fsBasedParquet.scala
Same issue as #6095

cc liancheng

Author: zsxwing <zsxwing@gmail.com>

Closes #6136 from zsxwing/hotfix and squashes the following commits:

4beea54 [zsxwing] Use 'new Job' in fsBasedParquet.scala

(cherry picked from commit 728af88cf6)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-13 17:58:54 -07:00
Venkata Ramana Gollamudi 820aaa6b9a [SPARK-7601] [SQL] Support Insert into JDBC Datasource
Supported InsertableRelation for JDBC Datasource JDBCRelation.
Example usage:
sqlContext.sql(
      s"""
        |CREATE TEMPORARY TABLE testram1
        |USING org.apache.spark.sql.jdbc
        |OPTIONS (url '$url', dbtable 'testram1', user 'xx', password 'xx', driver 'com.h2.Driver')
      """.stripMargin.replaceAll("\n", " "))

sqlContext.sql("insert into table testram1 select * from testsrc")
sqlContext.sql("insert overwrite table testram1 select * from testsrc")

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

Closes #6121 from gvramana/JDBCDatasource_insert and squashes the following commits:

f3fb5f1 [Venkata Ramana Gollamudi] Support for JDBC Datasource InsertableRelation

(cherry picked from commit 59aaa1dad6)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-13 17:24:31 -07:00
Josh Rosen c53ebea9db [SPARK-7081] Faster sort-based shuffle path using binary processing cache-aware sort
This patch introduces a new shuffle manager that enhances the existing sort-based shuffle with a new cache-friendly sort algorithm that operates directly on binary data. The goals of this patch are to lower memory usage and Java object overheads during shuffle and to speed up sorting. It also lays groundwork for follow-up patches that will enable end-to-end processing of serialized records.

The new shuffle manager, `UnsafeShuffleManager`, can be enabled by setting `spark.shuffle.manager=tungsten-sort` in SparkConf.

The new shuffle manager uses directly-managed memory to implement several performance optimizations for certain types of shuffles. In cases where the new performance optimizations cannot be applied, the new shuffle manager delegates to SortShuffleManager to handle those shuffles.

UnsafeShuffleManager's optimizations will apply when _all_ of the following conditions hold:

 - The shuffle dependency specifies no aggregation or output ordering.
 - The shuffle serializer supports relocation of serialized values (this is currently supported
   by KryoSerializer and Spark SQL's custom serializers).
 - The shuffle produces fewer than 16777216 output partitions.
 - No individual record is larger than 128 MB when serialized.

In addition, extra spill-merging optimizations are automatically applied when the shuffle compression codec supports concatenation of serialized streams. This is currently supported by Spark's LZF serializer.

At a high-level, UnsafeShuffleManager's design is similar to Spark's existing SortShuffleManager.  In sort-based shuffle, incoming records are sorted according to their target partition ids, then written to a single map output file. Reducers fetch contiguous regions of this file in order to read their portion of the map output. In cases where the map output data is too large to fit in memory, sorted subsets of the output can are spilled to disk and those on-disk files are merged to produce the final output file.

UnsafeShuffleManager optimizes this process in several ways:

 - Its sort operates on serialized binary data rather than Java objects, which reduces memory consumption and GC overheads. This optimization requires the record serializer to have certain properties to allow serialized records to be re-ordered without requiring deserialization.  See SPARK-4550, where this optimization was first proposed and implemented, for more details.

 - It uses a specialized cache-efficient sorter (UnsafeShuffleExternalSorter) that sorts arrays of compressed record pointers and partition ids. By using only 8 bytes of space per record in the sorting array, this fits more of the array into cache.

 - The spill merging procedure operates on blocks of serialized records that belong to the same partition and does not need to deserialize records during the merge.

 - When the spill compression codec supports concatenation of compressed data, the spill merge simply concatenates the serialized and compressed spill partitions to produce the final output partition.  This allows efficient data copying methods, like NIO's `transferTo`, to be used and avoids the need to allocate decompression or copying buffers during the merge.

The shuffle read path is unchanged.

This patch is similar to [SPARK-4550](http://issues.apache.org/jira/browse/SPARK-4550) / #4450 but uses a slightly different implementation. The `unsafe`-based implementation featured in this patch lays the groundwork for followup patches that will enable sorting to operate on serialized data pages that will be prepared by Spark SQL's new `unsafe` operators (such as the new aggregation operator introduced in #5725).

### Future work

There are several tasks that build upon this patch, which will be left to future work:

- [SPARK-7271](https://issues.apache.org/jira/browse/SPARK-7271) Redesign / extend the shuffle interfaces to accept binary data as input. The goal here is to let us bypass serialization steps in cases where the sort input is produced by an operator that operates directly on binary data.
- Extension / redesign of the `Serializer` API. We can add new methods which allow serializers to determine the size requirements for serializing objects and for serializing objects directly to a specified memory address (similar to how `UnsafeRowConverter` works in Spark SQL).

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5868 from JoshRosen/unsafe-sort and squashes the following commits:

ef0a86e [Josh Rosen] Fix scalastyle errors
7610f2f [Josh Rosen] Add tests for proper cleanup of shuffle data.
d494ffe [Josh Rosen] Fix deserialization of JavaSerializer instances.
52a9981 [Josh Rosen] Fix some bugs in the address packing code.
51812a7 [Josh Rosen] Change shuffle manager sort name to tungsten-sort
4023fa4 [Josh Rosen] Add @Private annotation to some Java classes.
de40b9d [Josh Rosen] More comments to try to explain metrics code
df07699 [Josh Rosen] Attempt to clarify confusing metrics update code
5e189c6 [Josh Rosen] Track time spend closing / flushing files; split TimeTrackingOutputStream into separate file.
d5779c6 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
c2ce78e [Josh Rosen] Fix a missed usage of MAX_PARTITION_ID
e3b8855 [Josh Rosen] Cleanup in UnsafeShuffleWriter
4a2c785 [Josh Rosen] rename 'sort buffer' to 'pointer array'
6276168 [Josh Rosen] Remove ability to disable spilling in UnsafeShuffleExternalSorter.
57312c9 [Josh Rosen] Clarify fileBufferSize units
2d4e4f4 [Josh Rosen] Address some minor comments in UnsafeShuffleExternalSorter.
fdcac08 [Josh Rosen] Guard against overflow when expanding sort buffer.
85da63f [Josh Rosen] Cleanup in UnsafeShuffleSorterIterator.
0ad34da [Josh Rosen] Fix off-by-one in nextInt() call
56781a1 [Josh Rosen] Rename UnsafeShuffleSorter to UnsafeShuffleInMemorySorter
e995d1a [Josh Rosen] Introduce MAX_SHUFFLE_OUTPUT_PARTITIONS.
e58a6b4 [Josh Rosen] Add more tests for PackedRecordPointer encoding.
4f0b770 [Josh Rosen] Attempt to implement proper shuffle write metrics.
d4e6d89 [Josh Rosen] Update to bit shifting constants
69d5899 [Josh Rosen] Remove some unnecessary override vals
8531286 [Josh Rosen] Add tests that automatically trigger spills.
7c953f9 [Josh Rosen] Add test that covers UnsafeShuffleSortDataFormat.swap().
e1855e5 [Josh Rosen] Fix a handful of misc. IntelliJ inspections
39434f9 [Josh Rosen] Avoid integer multiplication overflow in getMemoryUsage (thanks FindBugs!)
1e3ad52 [Josh Rosen] Delete unused ByteBufferOutputStream class.
ea4f85f [Josh Rosen] Roll back an unnecessary change in Spillable.
ae538dc [Josh Rosen] Document UnsafeShuffleManager.
ec6d626 [Josh Rosen] Add notes on maximum # of supported shuffle partitions.
0d4d199 [Josh Rosen] Bump up shuffle.memoryFraction to make tests pass.
b3b1924 [Josh Rosen] Properly implement close() and flush() in DummySerializerInstance.
1ef56c7 [Josh Rosen] Revise compression codec support in merger; test cross product of configurations.
b57c17f [Josh Rosen] Disable some overly-verbose logs that rendered DEBUG useless.
f780fb1 [Josh Rosen] Add test demonstrating which compression codecs support concatenation.
4a01c45 [Josh Rosen] Remove unnecessary log message
27b18b0 [Josh Rosen] That for inserting records AT the max record size.
fcd9a3c [Josh Rosen] Add notes + tests for maximum record / page sizes.
9d1ee7c [Josh Rosen] Fix MiMa excludes for ShuffleWriter change
fd4bb9e [Josh Rosen] Use own ByteBufferOutputStream rather than Kryo's
67d25ba [Josh Rosen] Update Exchange operator's copying logic to account for new shuffle manager
8f5061a [Josh Rosen] Strengthen assertion to check partitioning
01afc74 [Josh Rosen] Actually read data in UnsafeShuffleWriterSuite
1929a74 [Josh Rosen] Update to reflect upstream ShuffleBlockManager -> ShuffleBlockResolver rename.
e8718dd [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
9b7ebed [Josh Rosen] More defensive programming RE: cleaning up spill files and memory after errors
7cd013b [Josh Rosen] Begin refactoring to enable proper tests for spilling.
722849b [Josh Rosen] Add workaround for transferTo() bug in merging code; refactor tests.
9883e30 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
b95e642 [Josh Rosen] Refactor and document logic that decides when to spill.
1ce1300 [Josh Rosen] More minor cleanup
5e8cf75 [Josh Rosen] More minor cleanup
e67f1ea [Josh Rosen] Remove upper type bound in ShuffleWriter interface.
cfe0ec4 [Josh Rosen] Address a number of minor review comments:
8a6fe52 [Josh Rosen] Rename UnsafeShuffleSpillWriter to UnsafeShuffleExternalSorter
11feeb6 [Josh Rosen] Update TODOs related to shuffle write metrics.
b674412 [Josh Rosen] Merge remote-tracking branch 'origin/master' into unsafe-sort
aaea17b [Josh Rosen] Add comments to UnsafeShuffleSpillWriter.
4f70141 [Josh Rosen] Fix merging; now passes UnsafeShuffleSuite tests.
133c8c9 [Josh Rosen] WIP towards testing UnsafeShuffleWriter.
f480fb2 [Josh Rosen] WIP in mega-refactoring towards shuffle-specific sort.
57f1ec0 [Josh Rosen] WIP towards packed record pointers for use in optimized shuffle sort.
69232fd [Josh Rosen] Enable compressible address encoding for off-heap mode.
7ee918e [Josh Rosen] Re-order imports in tests
3aeaff7 [Josh Rosen] More refactoring and cleanup; begin cleaning iterator interfaces
3490512 [Josh Rosen] Misc. cleanup
f156a8f [Josh Rosen] Hacky metrics integration; refactor some interfaces.
2776aca [Josh Rosen] First passing test for ExternalSorter.
5e100b2 [Josh Rosen] Super-messy WIP on external sort
595923a [Josh Rosen] Remove some unused variables.
8958584 [Josh Rosen] Fix bug in calculating free space in current page.
f17fa8f [Josh Rosen] Add missing newline
c2fca17 [Josh Rosen] Small refactoring of SerializerPropertiesSuite to enable test re-use:
b8a09fe [Josh Rosen] Back out accidental log4j.properties change
bfc12d3 [Josh Rosen] Add tests for serializer relocation property.
240864c [Josh Rosen] Remove PrefixComputer and require prefix to be specified as part of insert()
1433b42 [Josh Rosen] Store record length as int instead of long.
026b497 [Josh Rosen] Re-use a buffer in UnsafeShuffleWriter
0748458 [Josh Rosen] Port UnsafeShuffleWriter to Java.
87e721b [Josh Rosen] Renaming and comments
d3cc310 [Josh Rosen] Flag that SparkSqlSerializer2 supports relocation
e2d96ca [Josh Rosen] Expand serializer API and use new function to help control when new UnsafeShuffle path is used.
e267cee [Josh Rosen] Fix compilation of UnsafeSorterSuite
9c6cf58 [Josh Rosen] Refactor to use DiskBlockObjectWriter.
253f13e [Josh Rosen] More cleanup
8e3ec20 [Josh Rosen] Begin code cleanup.
4d2f5e1 [Josh Rosen] WIP
3db12de [Josh Rosen] Minor simplification and sanity checks in UnsafeSorter
767d3ca [Josh Rosen] Fix invalid range in UnsafeSorter.
e900152 [Josh Rosen] Add test for empty iterator in UnsafeSorter
57a4ea0 [Josh Rosen] Make initialSize configurable in UnsafeSorter
abf7bfe [Josh Rosen] Add basic test case.
81d52c5 [Josh Rosen] WIP on UnsafeSorter

(cherry picked from commit 73bed408fb)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-13 17:07:39 -07:00
Reynold Xin acd872bbdb [SQL] Move some classes into packages that are more appropriate.
JavaTypeInference into catalyst
types.DateUtils into catalyst
CacheManager into execution
DefaultParserDialect into catalyst

Author: Reynold Xin <rxin@databricks.com>

Closes #6108 from rxin/sql-rename and squashes the following commits:

3fc9613 [Reynold Xin] Fixed import ordering.
83d9ff4 [Reynold Xin] Fixed codegen tests.
e271e86 [Reynold Xin] mima
f4e24a6 [Reynold Xin] [SQL] Move some classes into packages that are more appropriate.

(cherry picked from commit e683182c3e)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-13 16:15:43 -07:00
scwf d5c52d9ac1 [SPARK-7303] [SQL] push down project if possible when the child is sort
Optimize the case of `project(_, sort)` , a example is:

`select key from (select * from testData order by key) t`

before this PR:
```
== Parsed Logical Plan ==
'Project ['key]
 'Subquery t
  'Sort ['key ASC], true
   'Project [*]
    'UnresolvedRelation [testData], None

== Analyzed Logical Plan ==
Project [key#0]
 Subquery t
  Sort [key#0 ASC], true
   Project [key#0,value#1]
    Subquery testData
     LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Optimized Logical Plan ==
Project [key#0]
 Sort [key#0 ASC], true
  LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Physical Plan ==
Project [key#0]
 Sort [key#0 ASC], true
  Exchange (RangePartitioning [key#0 ASC], 5), []
   PhysicalRDD [key#0,value#1], MapPartitionsRDD[1]
```

after this PR
```
== Parsed Logical Plan ==
'Project ['key]
 'Subquery t
  'Sort ['key ASC], true
   'Project [*]
    'UnresolvedRelation [testData], None

== Analyzed Logical Plan ==
Project [key#0]
 Subquery t
  Sort [key#0 ASC], true
   Project [key#0,value#1]
    Subquery testData
     LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Optimized Logical Plan ==
Sort [key#0 ASC], true
 Project [key#0]
  LogicalRDD [key#0,value#1], MapPartitionsRDD[1]

== Physical Plan ==
Sort [key#0 ASC], true
 Exchange (RangePartitioning [key#0 ASC], 5), []
  Project [key#0]
   PhysicalRDD [key#0,value#1], MapPartitionsRDD[1]
```

with this rule we will first do column pruning on the table and then do sorting.

Author: scwf <wangfei1@huawei.com>

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

Closes #5838 from scwf/pruning and squashes the following commits:

b00d833 [scwf] address michael's comment
e230155 [scwf] fix tests failure
b09b895 [scwf] improve column pruning

(cherry picked from commit 59250fe514)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-13 16:14:01 -07:00
Wenchen Fan 3a60bcb80d [SPARK-7551][DataFrame] support backticks for DataFrame attribute resolution
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6074 from cloud-fan/7551 and squashes the following commits:

e6f579e [Wenchen Fan] allow space
2b86699 [Wenchen Fan] handle blank
e218d99 [Wenchen Fan] address comments
54c4209 [Wenchen Fan] fix 7551

(cherry picked from commit 213a6f30fe)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-13 12:48:01 -07:00
Cheng Lian 90f304b0c9 [SPARK-7567] [SQL] Migrating Parquet data source to FSBasedRelation
This PR migrates Parquet data source to the newly introduced `FSBasedRelation`. `FSBasedParquetRelation` is created to replace `ParquetRelation2`. Major differences are:

1. Partition discovery code has been factored out to `FSBasedRelation`
1. `AppendingParquetOutputFormat` is not used now. Instead, an anonymous subclass of `ParquetOutputFormat` is used to handle appending and writing dynamic partitions
1. When scanning partitioned tables, `FSBasedParquetRelation.buildScan` only builds an `RDD[Row]` for a single selected partition
1. `FSBasedParquetRelation` doesn't rely on Catalyst expressions for filter push down, thus it doesn't extend `CatalystScan` anymore

   After migrating `JSONRelation` (which extends `CatalystScan`), we can remove `CatalystScan`.

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

Author: Cheng Lian <lian@databricks.com>

Closes #6090 from liancheng/parquet-migration and squashes the following commits:

6063f87 [Cheng Lian] Casts to OutputCommitter rather than FileOutputCommtter
bfd1cf0 [Cheng Lian] Fixes compilation error introduced while rebasing
f9ea56e [Cheng Lian] Adds ParquetRelation2 related classes to MiMa check whitelist
261d8c1 [Cheng Lian] Minor bug fix and more tests
db65660 [Cheng Lian] Migrates Parquet data source to FSBasedRelation

(cherry picked from commit 7ff16e8abe)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-13 11:04:21 -07:00
Cheng Hao 42cf4a2a5e [SPARK-6734] [SQL] Add UDTF.close support in Generate
Some third-party UDTF extensions generate additional rows in the "GenericUDTF.close()" method, which is supported / documented by Hive.
https://cwiki.apache.org/confluence/display/Hive/DeveloperGuide+UDTF
However, Spark SQL ignores the "GenericUDTF.close()", and it causes bug while porting job from Hive to Spark SQL.

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

Closes #5383 from chenghao-intel/udtf_close and squashes the following commits:

98b4e4b [Cheng Hao] Support UDTF.close

(cherry picked from commit 0da254fb29)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-14 00:35:22 +08:00
Cheng Lian d78f0e1b48 [MINOR] [SQL] Removes debugging println
Author: Cheng Lian <lian@databricks.com>

Closes #6123 from liancheng/remove-println and squashes the following commits:

03356b6 [Cheng Lian] Removes debugging println

(cherry picked from commit aa6ba3f216)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-13 23:40:30 +08:00
Yin Huai 9ca28d9de6 [SQL] In InsertIntoFSBasedRelation.insert, log cause before abort job/task.
We need to add a log entry before calling `abortTask`/`abortJob`. Otherwise, an exception from `abortTask`/`abortJob` will shadow the real cause.

cc liancheng

Author: Yin Huai <yhuai@databricks.com>

Closes #6105 from yhuai/logCause and squashes the following commits:

8dfe0d8 [Yin Huai] Log cause.

(cherry picked from commit b061bd517a)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-13 23:36:36 +08:00
Cheng Lian cb1fe81339 [SPARK-7599] [SQL] Don't restrict customized output committers to be subclasses of FileOutputCommitter
Author: Cheng Lian <lian@databricks.com>

Closes #6118 from liancheng/spark-7599 and squashes the following commits:

31e1bd6 [Cheng Lian] Don't restrict customized output committers to be subclasses of FileOutputCommitter

(cherry picked from commit 10c546e9d4)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-13 07:36:04 -07:00
Santiago M. Mola 6ff3379a19 [SPARK-7566][SQL] Add type to HiveContext.analyzer
This makes HiveContext.analyzer overrideable.

Author: Santiago M. Mola <santi@mola.io>

Closes #6086 from smola/patch-3 and squashes the following commits:

8ece136 [Santiago M. Mola] [SPARK-7566][SQL] Add type to HiveContext.analyzer

(cherry picked from commit 208b902257)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-12 23:44:43 -07:00
Reynold Xin 219a9043ef [SPARK-7321][SQL] Add Column expression for conditional statements (when/otherwise)
This builds on https://github.com/apache/spark/pull/5932 and should close https://github.com/apache/spark/pull/5932 as well.

As an example:
```python
df.select(when(df['age'] == 2, 3).otherwise(4).alias("age")).collect()
```

Author: Reynold Xin <rxin@databricks.com>
Author: kaka1992 <kaka_1992@163.com>

Closes #6072 from rxin/when-expr and squashes the following commits:

8f49201 [Reynold Xin] Throw exception if otherwise is applied twice.
0455eda [Reynold Xin] Reset run-tests.
bfb9d9f [Reynold Xin] Updated documentation and test cases.
762f6a5 [Reynold Xin] Merge pull request #5932 from kaka1992/IFCASE
95724c6 [kaka1992] Update
8218d0a [kaka1992] Update
801009e [kaka1992] Update
76d6346 [kaka1992] [SPARK-7321][SQL] Add Column expression for conditional statements (if, case)

(cherry picked from commit 97dee313f2)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-12 21:44:16 -07:00
Reynold Xin bdd5db9f16 [SPARK-7588] Document all SQL/DataFrame public methods with @since tag
This pull request adds since tag to all public methods/classes in SQL/DataFrame to indicate which version the methods/classes were first added.

Author: Reynold Xin <rxin@databricks.com>

Closes #6101 from rxin/tbc and squashes the following commits:

ed55e11 [Reynold Xin] Add since version to all DataFrame methods.

(cherry picked from commit 8fd55358b7)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-12 18:37:10 -07:00
zsxwing 2cc3301812 [HOTFIX] Use the old Job API to support old Hadoop versions
#5526 uses `Job.getInstance`, which does not exist in the old Hadoop versions. Just use `new Job` to replace it.

cc liancheng

Author: zsxwing <zsxwing@gmail.com>

Closes #6095 from zsxwing/hotfix and squashes the following commits:

b0c2049 [zsxwing] Use the old Job API to support old Hadoop versions

(cherry picked from commit 247b70349c)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-13 08:33:43 +08:00
Venkata Ramana Gollamudi 32819fcb7c [SPARK-7484][SQL]Support jdbc connection properties
Few jdbc drivers like SybaseIQ support passing username and password only through connection properties. So the same needs to be supported for
SQLContext.jdbc, dataframe.createJDBCTable and dataframe.insertIntoJDBC.
Added as default arguments or overrided function to support backward compatability.

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

Closes #6009 from gvramana/add_jdbc_conn_properties and squashes the following commits:

396a0d0 [Venkata Ramana Gollamudi] fixed comments
d66dd8c [Venkata Ramana Gollamudi] fixed comments
1b8cd8c [Venkata Ramana Gollamudi] Support jdbc connection properties

(cherry picked from commit 455551d1c6)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-12 14:38:53 -07:00
Michael Armbrust c68485e7a7 [SPARK-7569][SQL] Better error for invalid binary expressions
`scala> Seq((1,1)).toDF("a", "b").select(lit(1) + new java.sql.Date(1)) `

Before:

```
org.apache.spark.sql.AnalysisException: invalid expression (1 + 0) between Literal 1, IntegerType and Literal 0, DateType;
```

After:
```
org.apache.spark.sql.AnalysisException: invalid expression (1 + 0) between int and date;
```

Author: Michael Armbrust <michael@databricks.com>

Closes #6089 from marmbrus/betterBinaryError and squashes the following commits:

23b68ad [Michael Armbrust] [SPARK-7569][SQL] Better error for invalid binary expressions

(cherry picked from commit 2a41c0d71a)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-12 13:37:01 -07:00
Wenchen Fan 8be43f897f [SPARK-7276] [DATAFRAME] speed up DataFrame.select by collapsing Project
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5831 from cloud-fan/7276 and squashes the following commits:

ee4a1e1 [Wenchen Fan] fix rebase mistake
a3b565d [Wenchen Fan] refactor
99deb5d [Wenchen Fan] add test
f1f67ad [Wenchen Fan] fix 7276

(cherry picked from commit 4e290522c2)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-12 11:58:50 -07:00
Wenchen Fan ec8928604d [DataFrame][minor] support column in field accessor
Minor improvement, now we can use `Column` as extraction expression.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6080 from cloud-fan/tmp and squashes the following commits:

0fdefb7 [Wenchen Fan] support column in field accessor

(cherry picked from commit bfcaf8adcd)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-12 10:38:19 -07:00
Cheng Lian d2328137f7 [SPARK-3928] [SPARK-5182] [SQL] Partitioning support for the data sources API
This PR adds partitioning support for the external data sources API. It aims to simplify development of file system based data sources, and provide first class partitioning support for both read path and write path.  Existing data sources like JSON and Parquet can be simplified with this work.

## New features provided

1. Hive compatible partition discovery

   This actually generalizes the partition discovery strategy used in Parquet data source in Spark 1.3.0.

1. Generalized partition pruning optimization

   Now partition pruning is handled during physical planning phase.  Specific data sources don't need to worry about this harness anymore.

   (This also implies that we can remove `CatalystScan` after migrating the Parquet data source, since now we don't need to pass Catalyst expressions to data source implementations.)

1. Insertion with dynamic partitions

   When inserting data to a `FSBasedRelation`, data can be partitioned dynamically by specified partition columns.

## New structures provided

### Developer API

1. `FSBasedRelation`

   Base abstract class for file system based data sources.

1. `OutputWriter`

   Base abstract class for output row writers, responsible for writing a single row object.

1. `FSBasedRelationProvider`

   A new relation provider for `FSBasedRelation` subclasses. Note that data sources extending `FSBasedRelation` don't need to extend `RelationProvider` and `SchemaRelationProvider`.

### User API

New overloaded versions of

1. `DataFrame.save()`
1. `DataFrame.saveAsTable()`
1. `SQLContext.load()`

are provided to allow users to save/load DataFrames with user defined dynamic partition columns.

### Spark SQL query planning

1. `InsertIntoFSBasedRelation`

   Used to implement write path for `FSBasedRelation`s.

1. New rules for `FSBasedRelation` in `DataSourceStrategy`

   These are added to hook `FSBasedRelation` into physical query plan in read path, and perform partition pruning.

## TODO

- [ ] Use scratch directories when overwriting a table with data selected from itself.

      Currently, this is not supported, because the table been overwritten is always deleted before writing any data to it.

- [ ] When inserting with dynamic partition columns, use external sorter to group the data first.

      This ensures that we only need to open a single `OutputWriter` at a time.  For data sources like Parquet, `OutputWriter`s can be quite memory consuming.  One issue is that, this approach breaks the row distribution in the original DataFrame.  However, we did't promise to preserve data distribution when writing a DataFrame.

- [x] More tests.  Specifically, test cases for

      - [x] Self-join
      - [x] Loading partitioned relations with a subset of partition columns stored in data files.
      - [x] `SQLContext.load()` with user defined dynamic partition columns.

## Parquet data source migration

Parquet data source migration is covered in PR https://github.com/liancheng/spark/pull/6, which is against this PR branch and for preview only. A formal PR need to be made after this one is merged.

Author: Cheng Lian <lian@databricks.com>

Closes #5526 from liancheng/partitioning-support and squashes the following commits:

5351a1b [Cheng Lian] Fixes compilation error introduced while rebasing
1f9b1a5 [Cheng Lian] Tweaks data schema passed to FSBasedRelations
43ba50e [Cheng Lian] Avoids serializing generated projection code
edf49e7 [Cheng Lian] Removed commented stale code block
348a922 [Cheng Lian] Adds projection in FSBasedRelation.buildScan(requiredColumns, inputPaths)
ad4d4de [Cheng Lian] Enables HDFS style globbing
8d12e69 [Cheng Lian] Fixes compilation error
c71ac6c [Cheng Lian] Addresses comments from @marmbrus
7552168 [Cheng Lian] Fixes typo in MimaExclude.scala
0349e09 [Cheng Lian] Fixes compilation error introduced while rebasing
52b0c9b [Cheng Lian] Adjusts project/MimaExclude.scala
c466de6 [Cheng Lian] Addresses comments
bc3f9b4 [Cheng Lian] Uses projection to separate partition columns and data columns while inserting rows
795920a [Cheng Lian] Fixes compilation error after rebasing
0b8cd70 [Cheng Lian] Adds Scala/Catalyst row conversion when writing non-partitioned tables
fa543f3 [Cheng Lian] Addresses comments
5849dd0 [Cheng Lian] Fixes doc typos.  Fixes partition discovery refresh.
51be443 [Cheng Lian] Replaces FSBasedRelation.outputCommitterClass with FSBasedRelation.prepareForWrite
c4ed4fe [Cheng Lian] Bug fixes and a new test suite
a29e663 [Cheng Lian] Bug fix: should only pass actuall data files to FSBaseRelation.buildScan
5f423d3 [Cheng Lian] Bug fixes. Lets data source to customize OutputCommitter rather than OutputFormat
54c3d7b [Cheng Lian] Enforces that FileOutputFormat must be used
be0c268 [Cheng Lian] Uses TaskAttempContext rather than Configuration in OutputWriter.init
0bc6ad1 [Cheng Lian] Resorts to new Hadoop API, and now FSBasedRelation can customize output format class
f320766 [Cheng Lian] Adds prepareForWrite() hook, refactored writer containers
422ff4a [Cheng Lian] Fixes style issue
ce52353 [Cheng Lian] Adds new SQLContext.load() overload with user defined dynamic partition columns
8d2ff71 [Cheng Lian] Merges partition columns when reading partitioned relations
ca1805b [Cheng Lian] Removes duplicated partition discovery code in new Parquet
f18dec2 [Cheng Lian] More strict schema checking
b746ab5 [Cheng Lian] More tests
9b487bf [Cheng Lian] Fixes compilation errors introduced while rebasing
ea6c8dd [Cheng Lian] Removes remote debugging stuff
327bb1d [Cheng Lian] Implements partitioning support for data sources API
3c5073a [Cheng Lian] Fixes SaveModes used in test cases
fb5a607 [Cheng Lian] Fixes compilation error
9d17607 [Cheng Lian] Adds the contract that OutputWriter should have zero-arg constructor
5de194a [Cheng Lian] Forgot Apache licence header
95d0b4d [Cheng Lian] Renames PartitionedSchemaRelationProvider to FSBasedRelationProvider
770b5ba [Cheng Lian] Adds tests for FSBasedRelation
3ba9bbf [Cheng Lian] Adds DataFrame.saveAsTable() overrides which support partitioning
1b8231f [Cheng Lian] Renames FSBasedPrunedFilteredScan to FSBasedRelation
aa8ba9a [Cheng Lian] Javadoc fix
012ed2d [Cheng Lian] Adds PartitioningOptions
7dd8dd5 [Cheng Lian] Adds new interfaces and stub methods for data sources API partitioning support

(cherry picked from commit 0595b6de8f)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-13 01:32:55 +08:00
Wenchen Fan a9d84a9bf7 [DataFrame][minor] cleanup unapply methods in DataTypes
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #6079 from cloud-fan/unapply and squashes the following commits:

40da442 [Wenchen Fan] one more
7d90a05 [Wenchen Fan] cleanup unapply in DataTypes

(cherry picked from commit 831504cf6b)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-12 10:29:05 -07:00
Marcelo Vanzin afe54b76a6 [SPARK-7485] [BUILD] Remove pyspark files from assembly.
The sbt part of the build is hacky; it basically tricks sbt
into generating the zip by using a generator, but returns
an empty list for the generated files so that nothing is
actually added to the assembly.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #6022 from vanzin/SPARK-7485 and squashes the following commits:

22c1e04 [Marcelo Vanzin] Remove unneeded code.
4893622 [Marcelo Vanzin] [SPARK-7485] [build] Remove pyspark files from assembly.

(cherry picked from commit 82e890fb19)
Signed-off-by: Andrew Or <andrew@databricks.com>
2015-05-12 01:39:28 -07:00
Reynold Xin c6b8148458 [SQL] Rename Dialect -> ParserDialect.
Author: Reynold Xin <rxin@databricks.com>

Closes #6071 from rxin/parserdialect and squashes the following commits:

ca2eb31 [Reynold Xin] Rename Dialect -> ParserDialect.

(cherry picked from commit 16696759e9)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-11 22:07:02 -07:00
Zhongshuai Pei c30982dd86 [SPARK-7437] [SQL] Fold "literal in (item1, item2, ..., literal, ...)" into true or false directly
SQL
```
select key from src where 3 in (4, 5);
```
Before
```
== Optimized Logical Plan ==
Project [key#12]
 Filter 3 INSET (5,4)
  MetastoreRelation default, src, None
```

After
```
== Optimized Logical Plan ==
LocalRelation [key#228], []
```

Author: Zhongshuai Pei <799203320@qq.com>
Author: DoingDone9 <799203320@qq.com>

Closes #5972 from DoingDone9/InToFalse and squashes the following commits:

4c722a2 [Zhongshuai Pei] Update predicates.scala
abe2bbb [Zhongshuai Pei] Update Optimizer.scala
fa461a5 [Zhongshuai Pei] Update Optimizer.scala
e34c28a [Zhongshuai Pei] Update predicates.scala
24739bd [Zhongshuai Pei] Update ConstantFoldingSuite.scala
f4dbf50 [Zhongshuai Pei] Update ConstantFoldingSuite.scala
35ceb7a [Zhongshuai Pei] Update Optimizer.scala
36c194e [Zhongshuai Pei] Update Optimizer.scala
2e8f6ca [Zhongshuai Pei] Update Optimizer.scala
14952e2 [Zhongshuai Pei] Merge pull request #13 from apache/master
f03fe7f [Zhongshuai Pei] Merge pull request #12 from apache/master
f12fa50 [Zhongshuai Pei] Merge pull request #10 from apache/master
f61210c [Zhongshuai Pei] Merge pull request #9 from apache/master
34b1a9a [Zhongshuai Pei] Merge pull request #8 from apache/master
802261c [DoingDone9] Merge pull request #7 from apache/master
d00303b [DoingDone9] Merge pull request #6 from apache/master
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master

(cherry picked from commit 4b5e1fe94c)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-11 19:23:09 -07:00
Cheng Hao 1a664a0d54 [SPARK-7411] [SQL] Support SerDe for HiveQl in CTAS
This is a follow up of #5876 and should be merged after #5876.

Let's wait for unit testing result from Jenkins.

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

Closes #5963 from chenghao-intel/useIsolatedClient and squashes the following commits:

f87ace6 [Cheng Hao] remove the TODO and add `resolved condition` for HiveTable
a8260e8 [Cheng Hao] Update code as feedback
f4e243f [Cheng Hao] remove the serde setting for SequenceFile
d166afa [Cheng Hao] style issue
d25a4aa [Cheng Hao] Add SerDe support for CTAS

(cherry picked from commit e35d878be3)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-11 19:21:29 -07:00
Reynold Xin 8a9d2348e9 [SPARK-7324] [SQL] DataFrame.dropDuplicates
This should also close https://github.com/apache/spark/pull/5870

Author: Reynold Xin <rxin@databricks.com>

Closes #6066 from rxin/dropDups and squashes the following commits:

130692f [Reynold Xin] [SPARK-7324][SQL] DataFrame.dropDuplicates

(cherry picked from commit b6bf4f76c7)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-11 19:15:32 -07:00
Reynold Xin e1e599d58c Updated DataFrame.saveAsTable Hive warning to include SPARK-7550 ticket.
So users that are interested in this can track it easily.

Author: Reynold Xin <rxin@databricks.com>

Closes #6067 from rxin/SPARK-7550 and squashes the following commits:

ee0e34c [Reynold Xin] Updated DataFrame.saveAsTable Hive warning to include SPARK-7550 ticket.

(cherry picked from commit 87229c95c6)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-11 18:10:52 -07:00
Reynold Xin eaa6116200 [SPARK-7462][SQL] Update documentation for retaining grouping columns in DataFrames.
Author: Reynold Xin <rxin@databricks.com>

Closes #6062 from rxin/agg-retain-doc and squashes the following commits:

43e511e [Reynold Xin] [SPARK-7462][SQL] Update documentation for retaining grouping columns in DataFrames.

(cherry picked from commit 3a9b6997df)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-11 18:07:19 -07:00
madhukar 0dbfe16814 [SPARK-7084] improve saveAsTable documentation
Author: madhukar <phatak.dev@gmail.com>

Closes #5654 from phatak-dev/master and squashes the following commits:

386f407 [madhukar] #5654 updated for all the methods
2c997c5 [madhukar] Merge branch 'master' of https://github.com/apache/spark
00bc819 [madhukar] Merge branch 'master' of https://github.com/apache/spark
2a802c6 [madhukar] #5654 updated the doc according to comments
866e8df [madhukar] [SPARK-7084] improve saveAsTable documentation

(cherry picked from commit 57255dcd79)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-11 17:06:09 -07:00
Reynold Xin 0ff34f804f [SQL] Show better error messages for incorrect join types in DataFrames.
As a follow-up to https://github.com/apache/spark/pull/5944

Author: Reynold Xin <rxin@databricks.com>

Closes #6064 from rxin/jointype-better-error and squashes the following commits:

7629bf7 [Reynold Xin] [SQL] Show better error messages for incorrect join types in DataFrames.

(cherry picked from commit 4f4dbb030c)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-11 17:02:17 -07:00
LCY Vincent 788503a402 Update Documentation: leftsemi instead of semijoin
should sync up with here?
119f45d61d/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala (L26)

Author: LCY Vincent <lauchunyin@gmail.com>

Closes #5944 from vincentlaucy/master and squashes the following commits:

fc0e454 [LCY Vincent] Update DataFrame.scala

(cherry picked from commit a8ea09683a)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-11 14:48:18 -07:00
Reynold Xin 9c35f02b35 [SPARK-7462] By default retain group by columns in aggregate
Updated Java, Scala, Python, and R.

Author: Reynold Xin <rxin@databricks.com>
Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #5996 from rxin/groupby-retain and squashes the following commits:

aac7119 [Reynold Xin] Merge branch 'groupby-retain' of github.com:rxin/spark into groupby-retain
f6858f6 [Reynold Xin] Merge branch 'master' into groupby-retain
5f923c0 [Reynold Xin] Merge pull request #15 from shivaram/sparkr-groupby-retrain
c1de670 [Shivaram Venkataraman] Revert workaround in SparkR to retain grouped cols Based on reverting code added in commit 9a6be746ef
b8b87e1 [Reynold Xin] Fixed DataFrameJoinSuite.
d910141 [Reynold Xin] Updated rest of the files
1e6e666 [Reynold Xin] [SPARK-7462] By default retain group by columns in aggregate

(cherry picked from commit 0a4844f90a)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-11 11:35:35 -07:00
tianyi fff3c86b06 [SPARK-7519] [SQL] fix minor bugs in thrift server UI
Bugs description:

1. There are extra commas on the top of session list.
2. The format of time in "Start at:" part is not the same as others.
3. The total number of online sessions is wrong.

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

Closes #6048 from tianyi/SPARK-7519 and squashes the following commits:

ed366b7 [tianyi] fix bug

(cherry picked from commit 2242ab31e9)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-11 14:08:34 +08:00
Cheng Lian fd87b2aec3 [MINOR] [SQL] Fixes variable name typo
<!-- Reviewable:start -->
[<img src="https://reviewable.io/review_button.png" height=40 alt="Review on Reviewable"/>](https://reviewable.io/reviews/apache/spark/6038)
<!-- Reviewable:end -->

Author: Cheng Lian <lian@databricks.com>

Closes #6038 from liancheng/fix-typo and squashes the following commits:

572c2a4 [Cheng Lian] Fixes variable name typo

(cherry picked from commit 6bf9352fa5)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-10 21:40:59 +08:00
Oleg Sidorkin 5c4040312b [SPARK-7345][SQL] Spark cannot detect renamed columns using JDBC connector
Issue appears when one tries to create DataFrame using sqlContext.load("jdbc"...) statement when "dbtable" contains query with renamed columns.
If original column is used in SQL query once the resulting DataFrame will contain non-renamed column.
If original column is used in SQL query several times with different aliases, sqlContext.load will fail.
Original implementation of JDBCRDD.resolveTable uses getColumnName to detect column names in RDD schema.
Suggested implementation uses getColumnLabel to handle column renames in SQL statement which is aware of SQL "AS" statement.

Readings:
http://stackoverflow.com/questions/4271152/getcolumnlabel-vs-getcolumnname
http://stackoverflow.com/questions/12259829/jdbc-getcolumnname-getcolumnlabel-db2

Official documentation unfortunately a bit misleading in definition of "suggested title" purpose however clearly defines behavior of AS keyword in SQL statement.
http://docs.oracle.com/javase/7/docs/api/java/sql/ResultSetMetaData.html
getColumnLabel - Gets the designated column's suggested title for use in printouts and displays. The suggested title is usually specified by the SQL AS clause. If a SQL AS is not specified, the value returned from getColumnLabel will be the same as the value returned by the getColumnName method.

Author: Oleg Sidorkin <oleg.sidorkin@gmail.com>

Closes #6032 from osidorkin/master and squashes the following commits:

10fc44b [Oleg Sidorkin] [SPARK-7345][SQL] Regression test for JDBCSuite (resolved scala style test error)
2aaf6f7 [Oleg Sidorkin] [SPARK-7345][SQL] Regression test for JDBCSuite (renamed fields in JDBC query)
b7d5b22 [Oleg Sidorkin] [SPARK-7345][SQL] Regression test for JDBCSuite
09559a0 [Oleg Sidorkin] [SPARK-7345][SQL] Spark cannot detect renamed columns using JDBC connector

(cherry picked from commit d7a37bcaf1)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-10 01:31:44 -07:00
tedyu 5110f3efe5 [BUILD] Reference fasterxml.jackson.version in sql/core/pom.xml
Author: tedyu <yuzhihong@gmail.com>

Closes #6031 from tedyu/master and squashes the following commits:

5c2580c [tedyu] Reference fasterxml.jackson.version in sql/core/pom.xml
ff2a44f [tedyu] Merge branch 'master' of github.com:apache/spark
28c8394 [tedyu] Upgrade version of jackson-databind in sql/core/pom.xml

(cherry picked from commit bd74301ff8)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-09 13:19:33 -07:00
tedyu 6c5b9ffda8 Upgrade version of jackson-databind in sql/core/pom.xml
Currently version of jackson-databind in sql/core/pom.xml is 2.3.0

This is older than the version specified in root pom.xml

This PR upgrades the version in sql/core/pom.xml so that they're consistent.

Author: tedyu <yuzhihong@gmail.com>

Closes #6028 from tedyu/master and squashes the following commits:

28c8394 [tedyu] Upgrade version of jackson-databind in sql/core/pom.xml

(cherry picked from commit 3071aac387)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-09 10:42:11 -07:00
Josh Rosen 21212a27cb [SPARK-7375] [SQL] Avoid row copying in exchange when sort.serializeMapOutputs takes effect
This patch refactors the SQL `Exchange` operator's logic for determining whether map outputs need to be copied before being shuffled. As part of this change, we'll now avoid unnecessary copies in cases where sort-based shuffle operates on serialized map outputs (as in #4450 /
SPARK-4550).

This patch also includes a change to copy the input to RangePartitioner partition bounds calculation, which is necessary because this calculation buffers mutable Java objects.

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5948 from JoshRosen/SPARK-7375 and squashes the following commits:

f305ff3 [Josh Rosen] Reduce scope of some variables in Exchange
899e1d7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-7375
6a6bfce [Josh Rosen] Fix issue related to RangePartitioning:
ad006a4 [Josh Rosen] [SPARK-7375] Avoid defensive copying in exchange operator when sort.serializeMapOutputs takes effect.

(cherry picked from commit cde5483884)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-08 22:10:10 -04:00
Shivaram Venkataraman 448ff333fa [SPARK-7231] [SPARKR] Changes to make SparkR DataFrame dplyr friendly.
Changes include
1. Rename sortDF to arrange
2. Add new aliases `group_by` and `sample_frac`, `summarize`
3. Add more user friendly column addition (mutate), rename
4. Support mean as an alias for avg in Scala and also support n_distinct, n as in dplyr

Using these changes we can pretty much run the examples as described in http://cran.rstudio.com/web/packages/dplyr/vignettes/introduction.html with the same syntax

The only thing missing in SparkR is auto resolving column names when used in an expression i.e. making something like `select(flights, delay)` works in dply but we right now need `select(flights, flights$delay)` or `select(flights, "delay")`. But this is a complicated change and I'll file a new issue for it

cc sun-rui rxin

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #6005 from shivaram/sparkr-df-api and squashes the following commits:

5e0716a [Shivaram Venkataraman] Fix some roxygen bugs
1254953 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/apache/spark into sparkr-df-api
0521149 [Shivaram Venkataraman] Changes to make SparkR DataFrame dplyr friendly. Changes include 1. Rename sortDF to arrange 2. Add new aliases `group_by` and `sample_frac`, `summarize` 3. Add more user friendly column addition (mutate), rename 4. Support mean as an alias for avg in Scala and also support n_distinct, n as in dplyr

(cherry picked from commit 0a901dd3a1)
Signed-off-by: Shivaram Venkataraman <shivaram@cs.berkeley.edu>
2015-05-08 18:30:10 -07:00
Andrew Or cafffd0c29 [SPARK-7469] [SQL] DAG visualization: show SQL query operators
The DAG visualization currently displays only low-level Spark primitives (e.g. `map`, `reduceByKey`, `filter` etc.). For SQL, these aren't particularly useful. Instead, we should display higher level physical operators (e.g. `Filter`, `Exchange`, `ShuffleHashJoin`). cc marmbrus

-----------------
**Before**
<img src="https://issues.apache.org/jira/secure/attachment/12731586/before.png" width="600px"/>
-----------------
**After** (Pay attention to the words)
<img src="https://issues.apache.org/jira/secure/attachment/12731587/after.png" width="600px"/>
-----------------

Author: Andrew Or <andrew@databricks.com>

Closes #5999 from andrewor14/dag-viz-sql and squashes the following commits:

0db23a4 [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-viz-sql
1e211db [Andrew Or] Update comment
0d49fd6 [Andrew Or] Merge branch 'master' of github.com:apache/spark into dag-viz-sql
ffd237a [Andrew Or] Fix style
202dac1 [Andrew Or] Make ignoreParent false by default
e61b1ab [Andrew Or] Visualize SQL operators, not low-level Spark primitives
569034a [Andrew Or] Add a flag to ignore parent settings and scopes

(cherry picked from commit bd61f07039)
Signed-off-by: Andrew Or <andrew@databricks.com>
2015-05-08 17:15:17 -07:00
Jacky Li 21bd7222e5 [SPARK-4699] [SQL] Make caseSensitive configurable in spark sql analyzer
based on #3558

Author: Jacky Li <jacky.likun@huawei.com>
Author: wangfei <wangfei1@huawei.com>
Author: scwf <wangfei1@huawei.com>

Closes #5806 from scwf/case and squashes the following commits:

cd51712 [wangfei] fix compile
d4b724f [wangfei] address michael's comment
af512c7 [wangfei] fix conflicts
4ef1be7 [wangfei] fix conflicts
269cf21 [scwf] fix conflicts
b73df6c [scwf] style issue
9e11752 [scwf] improve SimpleCatalystConf
b35529e [scwf] minor style
a3f7659 [scwf] remove unsed imports
2a56515 [scwf] fix conflicts
6db4bf5 [scwf] also fix for HiveContext
7fc4a98 [scwf] fix test case
d5a9933 [wangfei] fix style
eee75ba [wangfei] fix EmptyConf
6ef31cf [wangfei] revert pom changes
5d7c456 [wangfei] set CASE_SENSITIVE false in TestHive
966e719 [wangfei] set CASE_SENSITIVE false in hivecontext
fd30e25 [wangfei] added override
69b3b70 [wangfei] fix AnalysisSuite
5472b08 [wangfei] fix compile issue
56034ca [wangfei] fix conflicts and improve for catalystconf
664d1e9 [Jacky Li] Merge branch 'master' of https://github.com/apache/spark into case
12eca9a [Jacky Li] solve conflict with master
39e369c [Jacky Li] fix confilct after DataFrame PR
dee56e9 [Jacky Li] fix test case failure
05b09a3 [Jacky Li] fix conflict base on the latest master branch
73c16b1 [Jacky Li] fix bug in sql/hive
9bf4cc7 [Jacky Li] fix bug in catalyst
005c56d [Jacky Li] make SQLContext caseSensitivity configurable
6332e0f [Jacky Li] fix bug
fcbf0d9 [Jacky Li] fix scalastyle check
e7bca31 [Jacky Li] make caseSensitive configuration in Analyzer and Catalog
91b1b96 [Jacky Li] make caseSensitive configurable in Analyzer
f57f15c [Jacky Li] add testcase
578d167 [Jacky Li] make caseSensitive configurable

(cherry picked from commit 6dad76e5eb)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-08 15:26:04 -07:00
Liang-Chi Hsieh 5205eb4c29 [SPARK-7390] [SQL] Only merge other CovarianceCounter when its count is greater than zero
JIRA: https://issues.apache.org/jira/browse/SPARK-7390

Also fix a minor typo.

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

Closes #5931 from viirya/fix_covariancecounter and squashes the following commits:

352eda6 [Liang-Chi Hsieh] Only merge other CovarianceCounter when its count is greater than zero.

(cherry picked from commit 90527f5604)
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2015-05-08 14:41:28 -07:00
Wenchen Fan f8468c4511 [SPARK-7133] [SQL] Implement struct, array, and map field accessor
It's the first step: generalize UnresolvedGetField to support all map, struct, and array
TODO: add `apply` in Scala and `__getitem__` in Python, and unify the `getItem` and `getField` methods to one single API(or should we keep them for compatibility?).

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5744 from cloud-fan/generalize and squashes the following commits:

715c589 [Wenchen Fan] address comments
7ea5b31 [Wenchen Fan] fix python test
4f0833a [Wenchen Fan] add python test
f515d69 [Wenchen Fan] add apply method and test cases
8df6199 [Wenchen Fan] fix python test
239730c [Wenchen Fan] fix test compile
2a70526 [Wenchen Fan] use _bin_op in dataframe.py
6bf72bc [Wenchen Fan] address comments
3f880c3 [Wenchen Fan] add java doc
ab35ab5 [Wenchen Fan] fix python test
b5961a9 [Wenchen Fan] fix style
c9d85f5 [Wenchen Fan] generalize UnresolvedGetField to support all map, struct, and array

(cherry picked from commit 2d05f325dc)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-08 11:49:49 -07:00
wangfei bb5872f2d0 [SPARK-7232] [SQL] Add a Substitution batch for spark sql analyzer
Added a new batch named `Substitution` before `Resolution` batch. The motivation for this is there are kind of cases we want to do some substitution on the parsed logical plan before resolve it.
Consider this two cases:
1 CTE, for cte we first build a row logical plan
```
'With Map(q1 -> 'Subquery q1
                   'Project ['key]
                      'UnresolvedRelation [src], None)
 'Project [*]
  'Filter ('key = 5)
   'UnresolvedRelation [q1], None
```
In `With` logicalplan here is a map stored the (`q1-> subquery`), we want first take off the with command and substitute the  `q1` of `UnresolvedRelation` by the `subquery`

2 Another example is Window function, in window function user may define some windows, we also need substitute the window name of child by the concrete window. this should also done in the Substitution batch.

Author: wangfei <wangfei1@huawei.com>

Closes #5776 from scwf/addbatch and squashes the following commits:

d4b962f [wangfei] added WindowsSubstitution
70f6932 [wangfei] Merge branch 'master' of https://github.com/apache/spark into addbatch
ecaeafb [wangfei] address yhuai's comments
553005a [wangfei] fix test case
0c54798 [wangfei] address comments
29aaaaf [wangfei] fix compile
1c9a092 [wangfei] added Substitution bastch

(cherry picked from commit f496bf3c53)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-07 22:57:15 -07:00
Yin Huai 9d0d28940f [SPARK-6986] [SQL] Use Serializer2 in more cases.
With 0a2b15ce43, the serialization stream and deserialization stream has enough information to determine it is handling a key-value pari, a key, or a value. It is safe to use `SparkSqlSerializer2` in more cases.

Author: Yin Huai <yhuai@databricks.com>

Closes #5849 from yhuai/serializer2MoreCases and squashes the following commits:

53a5eaa [Yin Huai] Josh's comments.
487f540 [Yin Huai] Use BufferedOutputStream.
8385f95 [Yin Huai] Always create a new row at the deserialization side to work with sort merge join.
c7e2129 [Yin Huai] Update tests.
4513d13 [Yin Huai] Use Serializer2 in more places.

(cherry picked from commit 3af423c92f)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-07 20:59:53 -07:00
Michael Armbrust 05454fd8ae [SPARK-6908] [SQL] Use isolated Hive client
This PR switches Spark SQL's Hive support to use the isolated hive client interface introduced by #5851, instead of directly interacting with the client.  By using this isolated client we can now allow users to dynamically configure the version of Hive that they are connecting to by setting `spark.sql.hive.metastore.version` without the need recompile.  This also greatly reduces the surface area for our interaction with the hive libraries, hopefully making it easier to support other versions in the future.

Jars for the desired hive version can be configured using `spark.sql.hive.metastore.jars`, which accepts the following options:
 - a colon-separated list of jar files or directories for hive and hadoop.
 - `builtin` - attempt to discover the jars that were used to load Spark SQL and use those. This
            option is only valid when using the execution version of Hive.
 - `maven` - download the correct version of hive on demand from maven.

By default, `builtin` is used for Hive 13.

This PR also removes the test step for building against Hive 12, as this will no longer be required to talk to Hive 12 metastores.  However, the full removal of the Shim is deferred until a later PR.

Remaining TODOs:
 - Remove the Hive Shims and inline code for Hive 13.
 - Several HiveCompatibility tests are not yet passing.
  - `nullformatCTAS` - As detailed below, we now are handling CTAS parsing ourselves instead of hacking into the Hive semantic analyzer.  However, we currently only handle the common cases and not things like CTAS where the null format is specified.
  - `combine1` now leaks state about compression somehow, breaking all subsequent tests.  As such we currently add it to the blacklist
  - `part_inherit_tbl_props` and `part_inherit_tbl_props_with_star` do not work anymore.  We are correctly propagating the information
  - "load_dyn_part14.*" - These tests pass when run on their own, but fail when run with all other tests.  It seems our `RESET` mechanism may not be as robust as it used to be?

Other required changes:
 -  `CreateTableAsSelect` no longer carries parts of the HiveQL AST with it through the query execution pipeline.  Instead, we parse CTAS during the HiveQL conversion and construct a `HiveTable`.  The full parsing here is not yet complete as detailed above in the remaining TODOs.  Since the operator is Hive specific, it is moved to the hive package.
 - `Command` is simplified to be a trait that simply acts as a marker for a LogicalPlan that should be eagerly evaluated.

Author: Michael Armbrust <michael@databricks.com>

Closes #5876 from marmbrus/useIsolatedClient and squashes the following commits:

258d000 [Michael Armbrust] really really correct path handling
e56fd4a [Michael Armbrust] getAbsolutePath
5a259f5 [Michael Armbrust] fix typos
81bb366 [Michael Armbrust] comments from vanzin
5f3945e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient
4b5cd41 [Michael Armbrust] yin's comments
f5de7de [Michael Armbrust] cleanup
11e9c72 [Michael Armbrust] better coverage in versions suite
7e8f010 [Michael Armbrust] better error messages and jar handling
e7b3941 [Michael Armbrust] more permisive checking for function registration
da91ba7 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient
5fe5894 [Michael Armbrust] fix serialization suite
81711c4 [Michael Armbrust] Initial support for running without maven
1d8ae44 [Michael Armbrust] fix final tests?
1c50813 [Michael Armbrust] more comments
a3bee70 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient
a6f5df1 [Michael Armbrust] style
ab07f7e [Michael Armbrust] WIP
4d8bf02 [Michael Armbrust] Remove hive 12 compilation
8843a25 [Michael Armbrust] [SPARK-6908] [SQL] Use isolated Hive client

(cherry picked from commit cd1d4110cf)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-07 19:36:41 -07:00
Wenchen Fan 622a0c51c7 [SPARK-2155] [SQL] [WHEN D THEN E] [ELSE F] add CaseKeyWhen for "CASE a WHEN b THEN c * END"
Avoid translating to CaseWhen and evaluate the key expression many times.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5979 from cloud-fan/condition and squashes the following commits:

3ce54e1 [Wenchen Fan] add CaseKeyWhen

(cherry picked from commit 35f0173b8f)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-07 16:27:06 -07:00
Iulian Dragos 9fd25f7a33 [SPARK-5281] [SQL] Registering table on RDD is giving MissingRequirementError
Go through the context classloader when reflecting on user types in ScalaReflection.

Replaced calls to `typeOf` with `typeTag[T].in(mirror)`. The convenience method assumes
all types can be found in the classloader that loaded scala-reflect (the primordial
classloader). This assumption is not valid in all contexts (sbt console, Eclipse launchers).

Fixed SPARK-5281

Author: Iulian Dragos <jaguarul@gmail.com>

Closes #5981 from dragos/issue/mirrors-missing-requirement-error and squashes the following commits:

d103e70 [Iulian Dragos] Go through the context classloader when reflecting on user types in ScalaReflection

(cherry picked from commit 937ba798c5)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-07 16:24:21 -07:00
Liang-Chi Hsieh 7064ea0cdc [SPARK-7277] [SQL] Throw exception if the property mapred.reduce.tasks is set to -1
JIRA: https://issues.apache.org/jira/browse/SPARK-7277

As automatically determining the number of reducers is not supported (`mapred.reduce.tasks` is set to `-1`), we should throw exception to users.

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

Closes #5811 from viirya/no_neg_reduce_tasks and squashes the following commits:

e518f96 [Liang-Chi Hsieh] Consider other wrong setting values.
fd9c817 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_neg_reduce_tasks
4ede705 [Liang-Chi Hsieh] Throw exception instead of warning message.
68a1c70 [Liang-Chi Hsieh] Show warning message if mapred.reduce.tasks is set to -1.

(cherry picked from commit ea3077f19c)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-07 16:22:55 -07:00
scwf 2425e4df13 [SQL] [MINOR] make star and multialias extend NamedExpression
`Star` and `MultiAlias` just used in `analyzer` and them will be substituted after analyze,  So just like `Alias` they do not need extend `Attribute`

Author: scwf <wangfei1@huawei.com>

Closes #5928 from scwf/attribute and squashes the following commits:

73a0560 [scwf] star and multialias do not need extend attribute

(cherry picked from commit 97d1182af6)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-07 16:21:37 -07:00
Cheng Hao 2b0c423856 [SPARK-5213] [SQL] Remove the duplicated SparkSQLParser
This is a follow up of #5827 to remove the additional `SparkSQLParser`

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

Closes #5965 from chenghao-intel/remove_sparksqlparser and squashes the following commits:

509a233 [Cheng Hao] Remove the HiveQlQueryExecution
a5f9e3b [Cheng Hao] Remove the duplicated SparkSQLParser

(cherry picked from commit 074d75d4c8)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-07 12:10:04 -07:00
ksonj 86f141c90a [SPARK-7116] [SQL] [PYSPARK] Remove cache() causing memory leak
This patch simply removes a `cache()` on an intermediate RDD when evaluating Python UDFs.

Author: ksonj <kson@siberie.de>

Closes #5973 from ksonj/udf and squashes the following commits:

db5b564 [ksonj] removed TODO about cleaning up
fe70c54 [ksonj] Remove cache() causing memory leak

(cherry picked from commit dec8f53719)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-07 12:05:03 -07:00
Yin Huai 9dcf4f78f4 [SPARK-1442] [SQL] [FOLLOW-UP] Address minor comments in Window Function PR (#5604).
Address marmbrus and scwf's comments in #5604.

Author: Yin Huai <yhuai@databricks.com>

Closes #5945 from yhuai/windowFollowup and squashes the following commits:

0ef879d [Yin Huai] Add collectFirst to TreeNode.
2373968 [Yin Huai] wip
4a16df9 [Yin Huai] Address minor comments for [SPARK-1442].

(cherry picked from commit 5784c8d955)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-07 11:47:01 -07:00
Daoyuan Wang 84ee348bce [SPARK-7330] [SQL] avoid NPE at jdbc rdd
Thank nadavoosh point this out in #5590

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

Closes #5877 from adrian-wang/jdbcrdd and squashes the following commits:

cc11900 [Daoyuan Wang] avoid NPE in jdbcrdd

(cherry picked from commit ed9be06a47)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-07 10:05:19 -07:00
Shiti 703211b970 [SPARK-7295][SQL] bitwise operations for DataFrame DSL
Author: Shiti <ssaxena.ece@gmail.com>

Closes #5867 from Shiti/spark-7295 and squashes the following commits:

71a9913 [Shiti] implementation for bitwise and,or, not and xor on Column with tests and docs

(cherry picked from commit fa8fddffd5)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-07 01:00:39 -07:00
Nathan Howell 2337ccc15d [SPARK-5938] [SPARK-5443] [SQL] Improve JsonRDD performance
This patch comprises of a few related pieces of work:

* Schema inference is performed directly on the JSON token stream
* `String => Row` conversion populate Spark SQL structures without intermediate types
* Projection pushdown is implemented via CatalystScan for DataFrame queries
* Support for the legacy parser by setting `spark.sql.json.useJacksonStreamingAPI` to `false`

Performance improvements depend on the schema and queries being executed, but it should be faster across the board. Below are benchmarks using the last.fm Million Song dataset:

```
Command                                            | Baseline | Patched
---------------------------------------------------|----------|--------
import sqlContext.implicits._                      |          |
val df = sqlContext.jsonFile("/tmp/lastfm.json")   |    70.0s |   14.6s
df.count()                                         |    28.8s |    6.2s
df.rdd.count()                                     |    35.3s |   21.5s
df.where($"artist" === "Robert Hood").collect()    |    28.3s |   16.9s
```

To prepare this dataset for benchmarking, follow these steps:

```
# Fetch the datasets from http://labrosa.ee.columbia.edu/millionsong/lastfm
wget http://labrosa.ee.columbia.edu/millionsong/sites/default/files/lastfm/lastfm_test.zip \
     http://labrosa.ee.columbia.edu/millionsong/sites/default/files/lastfm/lastfm_train.zip

# Decompress and combine, pipe through `jq -c` to ensure there is one record per line
unzip -p lastfm_test.zip lastfm_train.zip  | jq -c . > lastfm.json
```

Author: Nathan Howell <nhowell@godaddy.com>

Closes #5801 from NathanHowell/json-performance and squashes the following commits:

26fea31 [Nathan Howell] Recreate the baseRDD each for each scan operation
a7ebeb2 [Nathan Howell] Increase coverage of inserts into a JSONRelation
e06a1dd [Nathan Howell] Add comments to the `useJacksonStreamingAPI` config flag
6822712 [Nathan Howell] Split up JsonRDD2 into multiple objects
fa8234f [Nathan Howell] Wrap long lines
b31917b [Nathan Howell] Rename `useJsonRDD2` to `useJacksonStreamingAPI`
15c5d1b [Nathan Howell] JSONRelation's baseRDD need not be lazy
f8add6e [Nathan Howell] Add comments on lack of support for precision and scale DecimalTypes
fa0be47 [Nathan Howell] Remove unused default case in the field parser
80dba17 [Nathan Howell] Add comments regarding null handling and empty strings
842846d [Nathan Howell] Point the empty schema inference test at JsonRDD2
ab6ee87 [Nathan Howell] Add projection pushdown support to JsonRDD/JsonRDD2
f636c14 [Nathan Howell] Enable JsonRDD2 by default, add a flag to switch back to JsonRDD
0bbc445 [Nathan Howell] Improve JSON parsing and type inference performance
7ca70c1 [Nathan Howell] Eliminate arrow pattern, replace with pattern matches

(cherry picked from commit 2d6612cc8b)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-06 22:57:09 -07:00
Yin Huai 14bcb84e84 [HOT-FIX] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir.
Author: Yin Huai <yhuai@databricks.com>

Closes #5951 from yhuai/fixBuildMaven and squashes the following commits:

fdde183 [Yin Huai] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir.

(cherry picked from commit 7740996700)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-06 14:49:13 -07:00
Josh Rosen d651e28383 [SPARK-7311] Introduce internal Serializer API for determining if serializers support object relocation
This patch extends the `Serializer` interface with a new `Private` API which allows serializers to indicate whether they support relocation of serialized objects in serializer stream output.

This relocatibilty property is described in more detail in `Serializer.scala`, but in a nutshell a serializer supports relocation if reordering the bytes of serialized objects in serialization stream output is equivalent to having re-ordered those elements prior to serializing them.  The optimized shuffle path introduced in #4450 and #5868 both rely on serializers having this property; this patch just centralizes the logic for determining whether a serializer has this property.  I also added tests and comments clarifying when this works for KryoSerializer.

This change allows the optimizations in #4450 to be applied for shuffles that use `SqlSerializer2`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5924 from JoshRosen/SPARK-7311 and squashes the following commits:

50a68ca [Josh Rosen] Address minor nits
0a7ebd7 [Josh Rosen] Clarify reason why SqlSerializer2 supports this serializer
123b992 [Josh Rosen] Cleanup for submitting as standalone patch.
4aa61b2 [Josh Rosen] Add missing newline
2c1233a [Josh Rosen] Small refactoring of SerializerPropertiesSuite to enable test re-use:
0ba75e6 [Josh Rosen] Add tests for serializer relocation property.
450fa21 [Josh Rosen] Back out accidental log4j.properties change
86d4dcd [Josh Rosen] Flag that SparkSqlSerializer2 supports relocation
b9624ee [Josh Rosen] Expand serializer API and use new function to help control when new UnsafeShuffle path is used.

(cherry picked from commit 002c12384d)
Signed-off-by: Josh Rosen <joshrosen@databricks.com>
2015-05-06 10:53:19 -07:00
Yin Huai b521a3b030 [SPARK-1442] [SQL] Window Function Support for Spark SQL
Adding more information about the implementation...

This PR is adding the support of window functions to Spark SQL (specifically OVER and WINDOW clause). For every expression having a OVER clause, we use a WindowExpression as the container of a WindowFunction and the corresponding WindowSpecDefinition (the definition of a window frame, i.e. partition specification, order specification, and frame specification appearing in a OVER clause).
# Implementation #
The high level work flow of the implementation is described as follows.

*	Query parsing: In the query parse process, all WindowExpressions are originally placed in the projectList of a Project operator or the aggregateExpressions of an Aggregate operator. It makes our changes to simple and keep all of parsing rules for window functions at a single place (nodesToWindowSpecification). For the WINDOWclause in a query, we use a WithWindowDefinition as the container as the mapping from the name of a window specification to a WindowSpecDefinition. This changes is similar with our common table expression support.

*	Analysis: The query analysis process has three steps for window functions.

 *	Resolve all WindowSpecReferences by replacing them with WindowSpecReferences according to the mapping table stored in the node of WithWindowDefinition.
 *	Resolve WindowFunctions in the projectList of a Project operator or the aggregateExpressions of an Aggregate operator. For this PR, we use Hive's functions for window functions because we will have a major refactoring of our internal UDAFs and it is better to switch our UDAFs after that refactoring work.
 *	Once we have resolved all WindowFunctions, we will use ResolveWindowFunction to extract WindowExpressions from projectList and aggregateExpressions and then create a Window operator for every distinct WindowSpecDefinition. With this choice, at the execution time, we can rely on the Exchange operator to do all of work on reorganizing the table and we do not need to worry about it in the physical Window operator. An example analyzed plan is shown as follows

```
sql("""
SELECT
  year, country, product, sales,
  avg(sales) over(partition by product) avg_product,
  sum(sales) over(partition by country) sum_country
FROM sales
ORDER BY year, country, product
""").explain(true)

== Analyzed Logical Plan ==
Sort [year#34 ASC,country#35 ASC,product#36 ASC], true
 Project [year#34,country#35,product#36,sales#37,avg_product#27,sum_country#28]
  Window [year#34,country#35,product#36,sales#37,avg_product#27], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum(sales#37) WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_country#28], WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
   Window [year#34,country#35,product#36,sales#37], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage(sales#37) WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS avg_product#27], WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
    Project [year#34,country#35,product#36,sales#37]
     MetastoreRelation default, sales, None
```

*	Query planning: In the process of query planning, we simple generate the physical Window operator based on the logical Window operator. Then, to prepare the executedPlan, the EnsureRequirements rule will add Exchange and Sort operators if necessary. The EnsureRequirements rule will analyze the data properties and try to not add unnecessary shuffle and sort. The physical plan for the above example query is shown below.

```
== Physical Plan ==
Sort [year#34 ASC,country#35 ASC,product#36 ASC], true
 Exchange (RangePartitioning [year#34 ASC,country#35 ASC,product#36 ASC], 200), []
  Window [year#34,country#35,product#36,sales#37,avg_product#27], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFSum(sales#37) WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS sum_country#28], WindowSpecDefinition [country#35], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
   Exchange (HashPartitioning [country#35], 200), [country#35 ASC]
    Window [year#34,country#35,product#36,sales#37], [HiveWindowFunction#org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage(sales#37) WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING AS avg_product#27], WindowSpecDefinition [product#36], [], ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
     Exchange (HashPartitioning [product#36], 200), [product#36 ASC]
      HiveTableScan [year#34,country#35,product#36,sales#37], (MetastoreRelation default, sales, None), None
```

*	Execution time: At execution time, a physical Window operator buffers all rows in a partition specified in the partition spec of a OVER clause. If necessary, it also maintains a sliding window frame. The current implementation tries to buffer the input parameters of a window function according to the window frame to avoid evaluating a row multiple times.

# Future work #

Here are three improvements that are not hard to add:
*	Taking advantage of the window frame specification to reduce the number of rows buffered in the physical Window operator. For some cases, we only need to buffer the rows appearing in the sliding window. But for other cases, we will not be able to reduce the number of rows buffered (e.g. ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING).

*	When aRAGEN frame is used, for <value> PRECEDING and <value> FOLLOWING, it will be great if the <value> part is an expression (we can start with Literal). So, when the data type of ORDER BY expression is a FractionalType, we can support FractionalType as the type <value> (<value> still needs to be evaluated as a positive value).

*	When aRAGEN frame is used, we need to support DateType and TimestampType as the data type of the expression appearing in the order specification. Then, the <value> part of <value> PRECEDING and <value> FOLLOWING can support interval types (once we support them).

This is a joint work with guowei2 and yhuai
Thanks hbutani hvanhovell for his comments
Thanks scwf for his comments and unit tests

Author: Yin Huai <yhuai@databricks.com>

Closes #5604 from guowei2/windowImplement and squashes the following commits:

76fe1c8 [Yin Huai] Implementation.
aa2b0ae [Yin Huai] Tests.

(cherry picked from commit f2c47082c3)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-06 10:43:47 -07:00
Daoyuan Wang 7212897dc6 [SPARK-6201] [SQL] promote string and do widen types for IN
huangjs
Acutally spark sql will first go through analysis period, in which we do widen types and promote strings, and then optimization, where constant IN will be converted into INSET.

So it turn out that we only need to fix this for IN.

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

Closes #4945 from adrian-wang/inset and squashes the following commits:

71e05cc [Daoyuan Wang] minor fix
581fa1c [Daoyuan Wang] mysql way
f3f7baf [Daoyuan Wang] address comments
5eed4bc [Daoyuan Wang] promote string and do widen types for IN

(cherry picked from commit c3eb441f54)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-06 10:31:48 -07:00
Daoyuan Wang f1a5cafb76 [SPARK-5456] [SQL] fix decimal compare for jdbc rdd
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #5803 from adrian-wang/decimalcompare and squashes the following commits:

aef0e96 [Daoyuan Wang] add null handle
ec455b9 [Daoyuan Wang] fix decimal compare for jdbc rdd

(cherry picked from commit 150f671c28)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-06 10:05:18 -07:00
Reynold Xin 389b755db4 [SQL] JavaDoc update for various DataFrame functions.
Author: Reynold Xin <rxin@databricks.com>

Closes #5935 from rxin/df-doc1 and squashes the following commits:

aaeaadb [Reynold Xin] [SQL] JavaDoc update for various DataFrame functions.

(cherry picked from commit 322e7e7f68)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-06 08:51:26 -07:00
Burak Yavuz 8aa6681d5f [SPARK-7358][SQL] Move DataFrame mathfunctions into functions
After a discussion on the user mailing list, it was decided to put all UDF's under `o.a.s.sql.functions`

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5923 from brkyvz/move-math-funcs and squashes the following commits:

a8dc3f7 [Burak Yavuz] address comments
cf7a7bb [Burak Yavuz] [SPARK-7358] Move DataFrame mathfunctions into functions

(cherry picked from commit ba2b56614d)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-05 22:56:23 -07:00
Reynold Xin e61083ccab [SPARK-6231][SQL/DF] Automatically resolve join condition ambiguity for self-joins.
See the comment in join function for more information.

Author: Reynold Xin <rxin@databricks.com>

Closes #5919 from rxin/self-join-resolve and squashes the following commits:

e2fb0da [Reynold Xin] Updated SQLConf comment.
7233a86 [Reynold Xin] Updated comment.
6be2b4d [Reynold Xin] Removed println
9f6b72f [Reynold Xin] [SPARK-6231][SQL/DF] Automatically resolve ambiguity in join condition for self-joins.
2015-05-05 19:04:16 -07:00
wangfei b6566a22ce [SQL][Minor] make StringComparison extends ExpectsInputTypes
make StringComparison extends ExpectsInputTypes and added expectedChildTypes, so do not need override expectedChildTypes in each subclass

Author: wangfei <wangfei1@huawei.com>

Closes #5905 from scwf/ExpectsInputTypes and squashes the following commits:

b374ddf [wangfei] make stringcomparison extends ExpectsInputTypes

(cherry picked from commit 3059291e20)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-05 14:25:08 -07:00
云峤 c68d0e2352 [SPARK-7294][SQL] ADD BETWEEN
Author: 云峤 <chensong.cs@alibaba-inc.com>
Author: kaka1992 <kaka_1992@163.com>

Closes #5839 from kaka1992/master and squashes the following commits:

b15360d [kaka1992] Fix python unit test in sql/test. =_= I forget to commit this file last time.
f928816 [kaka1992] Fix python style in sql/test.
d2e7f72 [kaka1992] Fix python style in sql/test.
c54d904 [kaka1992] Fix empty map bug.
7e64d1e [云峤] Update
7b9b858 [云峤] undo
f080f8d [云峤] update pep8
76f0c51 [云峤] Merge remote-tracking branch 'remotes/upstream/master'
7d62368 [云峤] [SPARK-7294] ADD BETWEEN
baf839b [云峤] [SPARK-7294] ADD BETWEEN
d11d5b9 [云峤] [SPARK-7294] ADD BETWEEN

(cherry picked from commit 735bc3d042)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-05 13:24:01 -07:00
Burak Yavuz 598902b549 [SPARK-7243][SQL] Reduce size for Contingency Tables in DataFrames
Reduced take size from 1e8 to 1e6.

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5900 from brkyvz/df-cont-followup and squashes the following commits:

c11e762 [Burak Yavuz] fix grammar
b30ace2 [Burak Yavuz] address comments
a417ba5 [Burak Yavuz] [SPARK-7243][SQL] Reduce  size for Contingency Tables in DataFrames

(cherry picked from commit 18340d7be5)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-05 11:01:34 -07:00
Liang-Chi Hsieh d28832299a [MINOR] Minor update for document
Two minor doc errors in `BytesToBytesMap` and `UnsafeRow`.

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

Closes #5906 from viirya/minor_doc and squashes the following commits:

27f9089 [Liang-Chi Hsieh] Minor update for doc.

(cherry picked from commit b83091ae45)
Signed-off-by: Sean Owen <sowen@cloudera.com>
2015-05-05 14:44:10 +01:00
Reynold Xin 1388a469b1 [SPARK-7266] Add ExpectsInputTypes to expressions when possible.
This should gives us better analysis time error messages (rather than runtime) and automatic type casting.

Author: Reynold Xin <rxin@databricks.com>

Closes #5796 from rxin/expected-input-types and squashes the following commits:

c900760 [Reynold Xin] [SPARK-7266] Add ExpectsInputTypes to expressions when possible.

(cherry picked from commit 678c4da0fa)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-04 18:03:33 -07:00
Burak Yavuz ecf0d8a9f1 [SPARK-7243][SQL] Contingency Tables for DataFrames
Computes a pair-wise frequency table of the given columns. Also known as cross-tabulation.
cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5842 from brkyvz/df-cont and squashes the following commits:

a07c01e [Burak Yavuz] addressed comments v4.1
ae9e01d [Burak Yavuz] fix test
9106585 [Burak Yavuz] addressed comments v4.0
bced829 [Burak Yavuz] fix merge conflicts
a63ad00 [Burak Yavuz] addressed comments v3.0
a0cad97 [Burak Yavuz] addressed comments v3.0
6805df8 [Burak Yavuz] addressed comments and fixed test
939b7c4 [Burak Yavuz] lint python
7f098bc [Burak Yavuz] add crosstab pyTest
fd53b00 [Burak Yavuz] added python support for crosstab
27a5a81 [Burak Yavuz] implemented crosstab

(cherry picked from commit 8055411170)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-04 17:03:03 -07:00
云峤 34edaa8ac2 [SPARK-7319][SQL] Improve the output from DataFrame.show()
Author: 云峤 <chensong.cs@alibaba-inc.com>

Closes #5865 from kaka1992/df.show and squashes the following commits:

c79204b [云峤] Update
a1338f6 [云峤] Update python dataFrame show test and add empty df unit test.
734369c [云峤] Update python dataFrame show test and add empty df unit test.
84aec3e [云峤] Update python dataFrame show test and add empty df unit test.
159b3d5 [云峤] update
03ef434 [云峤] update
7394fd5 [云峤] update test show
ced487a [云峤] update pep8
b6e690b [云峤] Merge remote-tracking branch 'upstream/master' into df.show
30ac311 [云峤] [SPARK-7294] ADD BETWEEN
7d62368 [云峤] [SPARK-7294] ADD BETWEEN
baf839b [云峤] [SPARK-7294] ADD BETWEEN
d11d5b9 [云峤] [SPARK-7294] ADD BETWEEN

(cherry picked from commit f32e69ecc3)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-04 13:24:52 -07:00
tianyi 343d3bfafd [SPARK-5100] [SQL] add webui for thriftserver
This PR is a rebased version of #3946 , and mainly focused on creating an independent tab for the thrift server in spark web UI.

Features:

1. Session related statistics ( username and IP are only supported in hive-0.13.1 )
2. List all the SQL executing or executed on this server
3. Provide links to the job generated by SQL
4. Provide link to show all SQL executing or executed in a specified session

Prototype snapshots:

This is the main page for thrift server

![image](https://cloud.githubusercontent.com/assets/1411869/7361379/df7dcc64-ed89-11e4-9964-4df0b32f475e.png)

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

Closes #5730 from tianyi/SPARK-5100 and squashes the following commits:

cfd14c7 [tianyi] style fix
0efe3d5 [tianyi] revert part of pom change
c0f2fa0 [tianyi] extends HiveThriftJdbcTest to start/stop thriftserver for UI test
aa20408 [tianyi] fix style problem
c9df6f9 [tianyi] add testsuite for thriftserver ui and fix some style issue
9830199 [tianyi] add webui for thriftserver
2015-05-04 16:59:34 +08:00
Burak Yavuz 9646018bb4 [SPARK-7241] Pearson correlation for DataFrames
submitting this PR from a phone, excuse the brevity.
adds Pearson correlation to Dataframes, reusing the covariance calculation code

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5858 from brkyvz/df-corr and squashes the following commits:

285b838 [Burak Yavuz] addressed comments v2.0
d10babb [Burak Yavuz] addressed comments v0.2
4b74b24 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into df-corr
4fe693b [Burak Yavuz] addressed comments v0.1
a682d06 [Burak Yavuz] ready for PR
2015-05-03 21:44:39 -07:00
Michael Armbrust daa70bf135 [SPARK-6907] [SQL] Isolated client for HiveMetastore
This PR adds initial support for loading multiple versions of Hive in a single JVM and provides a common interface for extracting metadata from the `HiveMetastoreClient` for a given version.  This is accomplished by creating an isolated `ClassLoader` that operates according to the following rules:

 - __Shared Classes__: Java, Scala, logging, and Spark classes are delegated to `baseClassLoader`
  allowing the results of calls to the `ClientInterface` to be visible externally.
 - __Hive Classes__: new instances are loaded from `execJars`.  These classes are not
  accessible externally due to their custom loading.
 - __Barrier Classes__: Classes such as `ClientWrapper` are defined in Spark but must link to a specific version of Hive.  As a result, the bytecode is acquired from the Spark `ClassLoader` but a new copy is created for each instance of `IsolatedClientLoader`.
  This new instance is able to see a specific version of hive without using reflection where ever hive is consistent across versions. Since
  this is a unique instance, it is not visible externally other than as a generic
  `ClientInterface`, unless `isolationOn` is set to `false`.

In addition to the unit tests, I have also tested this locally against mysql instances of the Hive Metastore.  I've also successfully ported Spark SQL to run with this client, but due to the size of the changes, that will come in a follow-up PR.

By default, Hive jars are currently downloaded from Maven automatically for a given version to ease packaging and testing.  However, there is also support for specifying their location manually for deployments without internet.

Author: Michael Armbrust <michael@databricks.com>

Closes #5851 from marmbrus/isolatedClient and squashes the following commits:

c72f6ac [Michael Armbrust] rxins comments
1e271fa [Michael Armbrust] [SPARK-6907][SQL] Isolated client for HiveMetastore
2015-05-03 13:12:50 -07:00
Cheng Hao 5d6b90d939 [SPARK-5213] [SQL] Pluggable SQL Parser Support
based on #4015, we should not delete `sqlParser` from sqlcontext, that leads to mima failed. Users implement dialect to give a fallback for `sqlParser`  and we should construct `sqlParser` in sqlcontext according to the dialect
`protected[sql] val sqlParser = new SparkSQLParser(getSQLDialect().parse(_))`

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

Closes #5827 from scwf/sqlparser1 and squashes the following commits:

81b9737 [scwf] comment fix
0878bd1 [scwf] remove comments
c19780b [scwf] fix mima tests
c2895cf [scwf] Merge branch 'master' of https://github.com/apache/spark into sqlparser1
493775c [Cheng Hao] update the code as feedback
81a731f [Cheng Hao] remove the unecessary comment
aab0b0b [Cheng Hao] polish the code a little bit
49b9d81 [Cheng Hao] shrink the comment for rebasing
2015-05-02 15:20:07 -07:00
Marcelo Vanzin 82c8c37c09 [MINOR] [HIVE] Fix QueryPartitionSuite.
At least in the version of Hive I tested on, the test was deleting
a temp directory generated by Hive instead of one containing partition
data. So fix the filter to only consider partition directories when
deciding what to delete.

Author: Marcelo Vanzin <vanzin@cloudera.com>

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

7594ae9 [Marcelo Vanzin] Fix typo.
729fa80 [Marcelo Vanzin] [minor] [hive] Fix QueryPartitionSuite.
2015-05-02 23:10:35 +01:00
Burak Yavuz 2e0f3579f1 [SPARK-7242] added python api for freqItems in DataFrames
The python api for DataFrame's plus addressed your comments from previous PR.
rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5859 from brkyvz/df-freq-py2 and squashes the following commits:

f9aa9ce [Burak Yavuz] addressed comments v0.1
4b25056 [Burak Yavuz] added python api for freqItems
2015-05-01 23:43:24 -07:00
Cheng Hao 98e7045805 [SPARK-6999] [SQL] Remove the infinite recursive method (useless)
Remove the method, since it causes infinite recursive calls. And seems it's a dummy method, since we have the API:
`def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame`

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

Closes #5804 from chenghao-intel/spark_6999 and squashes the following commits:

63220a8 [Cheng Hao] remove the infinite recursive method (useless)
2015-05-01 19:39:30 -05:00
Yin Huai 41c6a44b1a [SPARK-7312][SQL] SPARK-6913 broke jdk6 build
JIRA: https://issues.apache.org/jira/browse/SPARK-7312

Author: Yin Huai <yhuai@databricks.com>

Closes #5847 from yhuai/jdbcJava6 and squashes the following commits:

68433a2 [Yin Huai] compile with Java 6
2015-05-01 16:47:00 -07:00
Burak Yavuz 4dc8d74491 [SPARK-7240][SQL] Single pass covariance calculation for dataframes
Added the calculation of covariance between two columns to DataFrames.

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5825 from brkyvz/df-cov and squashes the following commits:

cb18046 [Burak Yavuz] changed to sample covariance
f2e862b [Burak Yavuz] fixed failed test
51e39b8 [Burak Yavuz] moved implementation
0c6a759 [Burak Yavuz] addressed math comments
8456eca [Burak Yavuz] fix pyStyle3
aa2ad29 [Burak Yavuz] fix pyStyle2
4e97a50 [Burak Yavuz] Merge branch 'master' of github.com:apache/spark into df-cov
e3b0b85 [Burak Yavuz] addressed comments v0.1
a7115f1 [Burak Yavuz] fix python style
7dc6dbc [Burak Yavuz] reorder imports
408cb77 [Burak Yavuz] initial commit
2015-05-01 13:29:17 -07:00
Reynold Xin 37537760d1 [SPARK-7274] [SQL] Create Column expression for array/struct creation.
Author: Reynold Xin <rxin@databricks.com>

Closes #5802 from rxin/SPARK-7274 and squashes the following commits:

19aecaa [Reynold Xin] Fixed unicode tests.
bfc1538 [Reynold Xin] Export all Python functions.
2517b8c [Reynold Xin] Code review.
23da335 [Reynold Xin] Fixed Python bug.
132002e [Reynold Xin] Fixed tests.
56fce26 [Reynold Xin] Added Python support.
b0d591a [Reynold Xin] Fixed debug error.
86926a6 [Reynold Xin] Added test suite.
7dbb9ab [Reynold Xin] Ok one more.
470e2f5 [Reynold Xin] One more MLlib ...
e2d14f0 [Reynold Xin] [SPARK-7274][SQL] Create Column expression for array/struct creation.
2015-05-01 12:49:02 -07:00
Sandy Ryza 0a2b15ce43 [SPARK-4550] In sort-based shuffle, store map outputs in serialized form
Refer to the JIRA for the design doc and some perf results.

I wanted to call out some of the more possibly controversial changes up front:
* Map outputs are only stored in serialized form when Kryo is in use.  I'm still unsure whether Java-serialized objects can be relocated.  At the very least, Java serialization writes out a stream header which causes problems with the current approach, so I decided to leave investigating this to future work.
* The shuffle now explicitly operates on key-value pairs instead of any object.  Data is written to shuffle files in alternating keys and values instead of key-value tuples.  `BlockObjectWriter.write` now accepts a key argument and a value argument instead of any object.
* The map output buffer can hold a max of Integer.MAX_VALUE bytes.  Though this wouldn't be terribly difficult to change.
* When spilling occurs, the objects that still in memory at merge time end up serialized and deserialized an extra time.

Author: Sandy Ryza <sandy@cloudera.com>

Closes #4450 from sryza/sandy-spark-4550 and squashes the following commits:

8c70dd9 [Sandy Ryza] Fix serialization
9c16fe6 [Sandy Ryza] Fix a couple tests and move getAutoReset to KryoSerializerInstance
6c54e06 [Sandy Ryza] Fix scalastyle
d8462d8 [Sandy Ryza] SPARK-4550
2015-04-30 23:14:14 -07:00
Burak Yavuz b5347a4664 [SPARK-7248] implemented random number generators for DataFrames
Adds the functions `rand` (Uniform Dist) and `randn` (Normal Dist.) as expressions to DataFrames.

cc mengxr rxin

Author: Burak Yavuz <brkyvz@gmail.com>

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

50d69d4 [Burak Yavuz] add seed for test that failed
4234c3a [Burak Yavuz] fix Rand expression
13cad5c [Burak Yavuz] couple fixes
7d53953 [Burak Yavuz] waiting for hive tests
b453716 [Burak Yavuz] move radn with seed down
03637f0 [Burak Yavuz] fix broken hive func
c5909eb [Burak Yavuz] deleted old implementation of Rand
6d43895 [Burak Yavuz] implemented random generators
2015-04-30 21:56:03 -07:00
Patrick Wendell beeafcfd6e Revert "[SPARK-5213] [SQL] Pluggable SQL Parser Support"
This reverts commit 3ba5aaab82.
2015-04-30 20:33:36 -07:00
scwf 473552fa5d [SPARK-7123] [SQL] support table.star in sqlcontext
Run following sql get error
`SELECT r.*
FROM testData l join testData2 r on (l.key = r.a)`

Author: scwf <wangfei1@huawei.com>

Closes #5690 from scwf/tablestar and squashes the following commits:

3b2e2b6 [scwf] support table.star
2015-04-30 18:50:14 -07:00
Cheng Hao 3ba5aaab82 [SPARK-5213] [SQL] Pluggable SQL Parser Support
This PR aims to make the SQL Parser Pluggable, and user can register it's own parser via Spark SQL CLI.

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

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

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

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

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

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

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

493775c [Cheng Hao] update the code as feedback
81a731f [Cheng Hao] remove the unecessary comment
aab0b0b [Cheng Hao] polish the code a little bit
49b9d81 [Cheng Hao] shrink the comment for rebasing
2015-04-30 18:49:06 -07:00
Vyacheslav Baranov e991255e72 [SPARK-6913][SQL] Fixed "java.sql.SQLException: No suitable driver found"
Fixed `java.sql.SQLException: No suitable driver found` when loading DataFrame into Spark SQL if the driver is supplied with `--jars` argument.

The problem is in `java.sql.DriverManager` class that can't access drivers loaded by Spark ClassLoader.

Wrappers that forward requests are created for these drivers.

Also, it's not necessary any more to include JDBC drivers in `--driver-class-path` in local mode, specifying in `--jars` argument is sufficient.

Author: Vyacheslav Baranov <slavik.baranov@gmail.com>

Closes #5782 from SlavikBaranov/SPARK-6913 and squashes the following commits:

510c43f [Vyacheslav Baranov] [SPARK-6913] Fixed review comments
b2a727c [Vyacheslav Baranov] [SPARK-6913] Fixed thread race on driver registration
c8294ae [Vyacheslav Baranov] [SPARK-6913] Fixed "No suitable driver found" when using using JDBC driver added with SparkContext.addJar
2015-04-30 18:45:14 -07:00
wangfei a0d8a61ab1 [SPARK-7109] [SQL] Push down left side filter for left semi join
Now in spark sql optimizer we only push down right side filter for left semi join, actually we can push down left side filter because left semi join is doing filter on left table essentially.

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

Closes #5677 from scwf/leftsemi and squashes the following commits:

483d205 [wangfei] update with master to fix compile issue
82df0e1 [wangfei] Merge branch 'master' of https://github.com/apache/spark into leftsemi
d68a053 [wangfei] added apply
8f48a3d [scwf] added test
ebadaa9 [wangfei] left filter push down for left semi join
2015-04-30 18:18:54 -07:00
scwf 079733817f [SPARK-7093] [SQL] Using newPredicate in NestedLoopJoin to enable code generation
Using newPredicate in NestedLoopJoin instead of InterpretedPredicate to make it can make use of code generation

Author: scwf <wangfei1@huawei.com>

Closes #5665 from scwf/NLP and squashes the following commits:

d19dd31 [scwf] improvement
a887c02 [scwf] improve for NLP boundCondition
2015-04-30 18:15:56 -07:00
rakeshchalasani ee04413935 [SPARK-7280][SQL] Add "drop" column/s on a data frame
Takes a column name/s and returns a new DataFrame that drops a column/s.

Author: rakeshchalasani <vnit.rakesh@gmail.com>

Closes #5818 from rakeshchalasani/SPARK-7280 and squashes the following commits:

ce2ec09 [rakeshchalasani] Minor edit
45c06f1 [rakeshchalasani] Change withColumnRename and format changes
f68945a [rakeshchalasani] Minor fix
0b9104d [rakeshchalasani] Drop one column at a time
289afd2 [rakeshchalasani] [SPARK-7280][SQL] Add "drop" column/s on a data frame
2015-04-30 17:42:50 -07:00
Burak Yavuz 149b3ee2da [SPARK-7242][SQL][MLLIB] Frequent items for DataFrames
Finding frequent items with possibly false positives, using the algorithm described in `http://www.cs.umd.edu/~samir/498/karp.pdf`.
public API under:
```
df.stat.freqItems(cols: Array[String], support: Double = 0.001): DataFrame
```

The output is a local DataFrame having the input column names with `-freqItems` appended to it. This is a single pass algorithm that may return false positives, but no false negatives.

cc mengxr rxin

Let's get the implementations in, I can add python API in a follow up PR.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5799 from brkyvz/freq-items and squashes the following commits:

a6ec82c [Burak Yavuz] addressed comments v?
39b1bba [Burak Yavuz] removed toSeq
0915e23 [Burak Yavuz] addressed comments v2.1
3a5c177 [Burak Yavuz] addressed comments v2.0
482e741 [Burak Yavuz] removed old import
38e784d [Burak Yavuz] addressed comments v1.0
8279d4d [Burak Yavuz] added default value for support
3d82168 [Burak Yavuz] made base implementation
2015-04-30 16:40:32 -07:00
Josh Rosen fa01bec484 [Build] Enable MiMa checks for SQL
Now that 1.3 has been released, we should enable MiMa checks for the `sql` subproject.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5727 from JoshRosen/enable-more-mima-checks and squashes the following commits:

3ad302b [Josh Rosen] Merge remote-tracking branch 'origin/master' into enable-more-mima-checks
0c48e4d [Josh Rosen] Merge remote-tracking branch 'origin/master' into enable-more-mima-checks
e276cee [Josh Rosen] Fix SQL MiMa checks via excludes and private[sql]
44d0d01 [Josh Rosen] Add back 'launcher' exclude
1aae027 [Josh Rosen] Enable MiMa checks for launcher and sql projects.
2015-04-30 16:23:01 -07:00
Zhongshuai Pei 77cc25fb74 [SPARK-7267][SQL]Push down Project when it's child is Limit
SQL
```
select key from (select key,value from t1 limit 100) t2 limit 10
```
Optimized Logical Plan before modifying
```
== Optimized Logical Plan ==
Limit 10
  Project key#228
    Limit 100
      MetastoreRelation default, t1, None
```
Optimized Logical Plan after modifying
```
== Optimized Logical Plan ==
Limit 10
  Limit 100
    Project key#228
      MetastoreRelation default, t1, None
```
After this, we can combine limits

Author: Zhongshuai Pei <799203320@qq.com>
Author: DoingDone9 <799203320@qq.com>

Closes #5797 from DoingDone9/ProjectLimit and squashes the following commits:

70d0fca [Zhongshuai Pei] Update FilterPushdownSuite.scala
dc83ae9 [Zhongshuai Pei] Update FilterPushdownSuite.scala
485c61c [Zhongshuai Pei] Update Optimizer.scala
f03fe7f [Zhongshuai Pei] Merge pull request #12 from apache/master
f12fa50 [Zhongshuai Pei] Merge pull request #10 from apache/master
f61210c [Zhongshuai Pei] Merge pull request #9 from apache/master
34b1a9a [Zhongshuai Pei] Merge pull request #8 from apache/master
802261c [DoingDone9] Merge pull request #7 from apache/master
d00303b [DoingDone9] Merge pull request #6 from apache/master
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-04-30 15:22:13 -07:00
Liang-Chi Hsieh 6702324b60 [SPARK-7196][SQL] Support precision and scale of decimal type for JDBC
JIRA: https://issues.apache.org/jira/browse/SPARK-7196

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

Closes #5777 from viirya/jdbc_precision and squashes the following commits:

f40f5e6 [Liang-Chi Hsieh] Support precision and scale for NUMERIC type.
49acbf9 [Liang-Chi Hsieh] Add unit test.
a509e19 [Liang-Chi Hsieh] Support precision and scale of decimal type for JDBC.
2015-04-30 15:13:43 -07:00
Zhongshuai Pei 4459514497 [SPARK-7225][SQL] CombineLimits optimizer does not work
SQL
```
select key from (select key from src limit 100) t2 limit 10
```
Optimized Logical Plan before modifying
```
== Optimized Logical Plan ==
Limit 10
Limit 100
Project key#3
MetastoreRelation default, src, None
```
Optimized Logical Plan after modifying
```
== Optimized Logical Plan ==
Limit 10
 Project [key#1]
  MetastoreRelation default, src, None
```

Author: Zhongshuai Pei <799203320@qq.com>
Author: DoingDone9 <799203320@qq.com>

Closes #5770 from DoingDone9/limitOptimizer and squashes the following commits:

c68eaa7 [Zhongshuai Pei] Update CombiningLimitsSuite.scala
97e18cf [Zhongshuai Pei] Update Optimizer.scala
19ab875 [Zhongshuai Pei] Update CombiningLimitsSuite.scala
7db4566 [Zhongshuai Pei] Update CombiningLimitsSuite.scala
e2a491d [Zhongshuai Pei] Update Optimizer.scala
f03fe7f [Zhongshuai Pei] Merge pull request #12 from apache/master
f12fa50 [Zhongshuai Pei] Merge pull request #10 from apache/master
f61210c [Zhongshuai Pei] Merge pull request #9 from apache/master
34b1a9a [Zhongshuai Pei] Merge pull request #8 from apache/master
802261c [DoingDone9] Merge pull request #7 from apache/master
d00303b [DoingDone9] Merge pull request #6 from apache/master
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-04-29 22:44:14 -07:00
Burak Yavuz 5553198fe5 [SPARK-7156][SQL] Addressed follow up comments for randomSplit
small fixes regarding comments in PR #5761

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5795 from brkyvz/split-followup and squashes the following commits:

369c522 [Burak Yavuz] changed wording a little
1ea456f [Burak Yavuz] Addressed follow up comments
2015-04-29 19:13:47 -07:00
云峤 7143f6e971 [SPARK-7234][SQL] Fix DateType mismatch when codegen on.
Author: 云峤 <chensong.cs@alibaba-inc.com>

Closes #5778 from kaka1992/fix_codegenon_datetype_mismatch and squashes the following commits:

1ad4cff [云峤] SPARK-7234 fix dateType mismatch
2015-04-29 18:23:42 -07:00
wangfei 1fdfdb47b4 [SQL] [Minor] Print detail query execution info when spark answer is not right
Print detail query execution info including parsed/analyzed/optimized/Physical plan for query when spak answer is not rignt.

```
Results do not match for query:
== Parsed Logical Plan ==
'Aggregate ['x.str], ['x.str,SUM('x.strCount) AS c1#46]
 'Join Inner, Some(('x.str = 'y.str))
  'UnresolvedRelation [df], Some(x)
  'UnresolvedRelation [df], Some(y)

== Analyzed Logical Plan ==
Aggregate [str#44], [str#44,SUM(strCount#45L) AS c1#46L]
 Join Inner, Some((str#44 = str#51))
  Subquery x
   Subquery df
    Aggregate [str#44], [str#44,COUNT(str#44) AS strCount#45L]
     Project [_1#41 AS int#43,_2#42 AS str#44]
      LocalRelation [_1#41,_2#42], [[1,1],[2,2],[3,3]]
  Subquery y
   Subquery df
    Aggregate [str#51], [str#51,COUNT(str#51) AS strCount#47L]
     Project [_1#41 AS int#50,_2#42 AS str#51]
      LocalRelation [_1#41,_2#42], [[1,1],[2,2],[3,3]]

== Optimized Logical Plan ==
Aggregate [str#44], [str#44,SUM(strCount#45L) AS c1#46L]
 Project [str#44,strCount#45L]
  Join Inner, Some((str#44 = str#51))
   Aggregate [str#44], [str#44,COUNT(str#44) AS strCount#45L]
    LocalRelation [str#44], [[1],[2],[3]]
   Aggregate [str#51], [str#51]
    LocalRelation [str#51], [[1],[2],[3]]

== Physical Plan ==
Aggregate false, [str#44], [str#44,CombineSum(PartialSum#53L) AS c1#46L]
 Aggregate true, [str#44], [str#44,SUM(strCount#45L) AS PartialSum#53L]
  Project [str#44,strCount#45L]
   BroadcastHashJoin [str#44], [str#51], BuildRight
    Aggregate false, [str#44], [str#44,Coalesce(SUM(PartialCount#55L),0) AS strCount#45L]
     Exchange (HashPartitioning [str#44], 5), []
      Aggregate true, [str#44], [str#44,COUNT(str#44) AS PartialCount#55L]
       LocalTableScan [str#44], [[1],[2],[3]]
    Aggregate false, [str#51], [str#51]
     Exchange (HashPartitioning [str#51], 5), []
      Aggregate true, [str#51], [str#51]
       LocalTableScan [str#51], [[1],[2],[3]]

Code Generation: false
== RDD ==
== Results ==
!== Correct Answer - 3 ==   == Spark Answer - 3 ==
 [1,1]                      [1,1]
![2,3]                      [2,1]
 [3,1]                      [3,1]
```

Author: wangfei <wangfei1@huawei.com>

Closes #5774 from scwf/checkanswer and squashes the following commits:

5be6f78 [wangfei] print detail query execution info when Spark Answer is not right
2015-04-29 17:00:24 -07:00
Cheng Hao f8cbb0a4b3 [SPARK-7229] [SQL] SpecificMutableRow should take integer type as internal representation for Date
Author: Cheng Hao <hao.cheng@intel.com>

Closes #5772 from chenghao-intel/specific_row and squashes the following commits:

2cd064d [Cheng Hao] scala style issue
60347a2 [Cheng Hao] SpecificMutableRow should take integer type as internal representation for DateType
2015-04-29 16:23:34 -07:00
Burak Yavuz d7dbce8f7d [SPARK-7156][SQL] support RandomSplit in DataFrames
This is built on top of kaka1992 's PR #5711 using Logical plans.

Author: Burak Yavuz <brkyvz@gmail.com>

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

a1fb0aa [Burak Yavuz] remove unrelated file
69669c3 [Burak Yavuz] fix broken test
1ddb3da [Burak Yavuz] copy base
6000328 [Burak Yavuz] added python api and fixed test
3c11d1b [Burak Yavuz] fixed broken test
f400ade [Burak Yavuz] fix build errors
2384266 [Burak Yavuz] addressed comments v0.1
e98ebac [Burak Yavuz] [SPARK-7156][SQL] support RandomSplit in DataFrames
2015-04-29 15:34:05 -07:00
Wenchen Fan 81ea42bf39 [SQL][Minor] fix java doc for DataFrame.agg
Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5712 from cloud-fan/minor and squashes the following commits:

be23064 [Wenchen Fan] fix java doc for DataFrame.agg
2015-04-29 09:49:24 -07:00
Josh Rosen f49284b5bf [SPARK-7076][SPARK-7077][SPARK-7080][SQL] Use managed memory for aggregations
This patch adds managed-memory-based aggregation to Spark SQL / DataFrames. Instead of working with Java objects, this new aggregation path uses `sun.misc.Unsafe` to manipulate raw memory.  This reduces the memory footprint for aggregations, resulting in fewer spills, OutOfMemoryErrors, and garbage collection pauses.  As a result, this allows for higher memory utilization.  It can also result in better cache locality since objects will be stored closer together in memory.

This feature can be eanbled by setting `spark.sql.unsafe.enabled=true`.  For now, this feature is only supported when codegen is enabled and only supports aggregations for which the grouping columns are primitive numeric types or strings and aggregated values are numeric.

### Managing memory with sun.misc.Unsafe

This patch supports both on- and off-heap managed memory.

- In on-heap mode, memory addresses are identified by the combination of a base Object and an offset within that object.
- In off-heap mode, memory is addressed directly with 64-bit long addresses.

To support both modes, functions that manipulate memory accept both `baseObject` and `baseOffset` fields.  In off-heap mode, we simply pass `null` as `baseObject`.

We allocate memory in large chunks, so memory fragmentation and allocation speed are not significant bottlenecks.

By default, we use on-heap mode.  To enable off-heap mode, set `spark.unsafe.offHeap=true`.

To track allocated memory, this patch extends `SparkEnv` with an `ExecutorMemoryManager` and supplies each `TaskContext` with a `TaskMemoryManager`.  These classes work together to track allocations and detect memory leaks.

### Compact tuple format

This patch introduces `UnsafeRow`, a compact row layout.  In this format, each tuple has three parts: a null bit set, fixed length values, and variable-length values:

![image](https://cloud.githubusercontent.com/assets/50748/7328538/2fdb65ce-ea8b-11e4-9743-6c0f02bb7d1f.png)

- Rows are always 8-byte word aligned (so their sizes will always be a multiple of 8 bytes)
- The bit set is used for null tracking:
	- Position _i_ is set if and only if field _i_ is null
	- The bit set is aligned to an 8-byte word boundary.
- Every field appears as an 8-byte word in the fixed-length values part:
	- If a field is null, we zero out the values.
	- If a field is variable-length, the word stores a relative offset (w.r.t. the base of the tuple) that points to the beginning of the field's data in the variable-length part.
- Each variable-length data type can have its own encoding:
	- For strings, the first word stores the length of the string and is followed by UTF-8 encoded bytes.  If necessary, the end of the string is padded with empty bytes in order to ensure word-alignment.

For example, a tuple that consists 3 fields of type (int, string, string), with value (null, “data”, “bricks”) would look like this:

![image](https://cloud.githubusercontent.com/assets/50748/7328526/1e21959c-ea8b-11e4-9a28-a4350fe4a7b5.png)

This format allows us to compare tuples for equality by directly comparing their raw bytes.  This also enables fast hashing of tuples.

### Hash map for performing aggregations

This patch introduces `UnsafeFixedWidthAggregationMap`, a hash map for performing aggregations where the aggregation result columns are fixed-with.  This map's keys and values are `Row` objects. `UnsafeFixedWidthAggregationMap` is implemented on top of `BytesToBytesMap`, an append-only map which supports byte-array keys and values.

`BytesToBytesMap` stores pointers to key and value tuples.  For each record with a new key, we copy the key and create the aggregation value buffer for that key and put them in a buffer. The hash table then simply stores pointers to the key and value. For each record with an existing key, we simply run the aggregation function to update the values in place.

This map is implemented using open hashing with triangular sequence probing.  Each entry stores two words in a long array: the first word stores the address of the key and the second word stores the relative offset from the key tuple to the value tuple, as well as the key's 32-bit hashcode.  By storing the full hashcode, we reduce the number of equality checks that need to be performed to handle position collisions ()since the chance of hashcode collision is much lower than position collision).

`UnsafeFixedWidthAggregationMap` allows regular Spark SQL `Row` objects to be used when probing the map.  Internally, it encodes these rows into `UnsafeRow` format using `UnsafeRowConverter`.  This conversion has a small overhead that can be eliminated in the future once we use UnsafeRows in other operators.

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

Author: Josh Rosen <joshrosen@databricks.com>

Closes #5725 from JoshRosen/unsafe and squashes the following commits:

eeee512 [Josh Rosen] Add converters for Null, Boolean, Byte, and Short columns.
81f34f8 [Josh Rosen] Follow 'place children last' convention for GeneratedAggregate
1bc36cc [Josh Rosen] Refactor UnsafeRowConverter to avoid unnecessary boxing.
017b2dc [Josh Rosen] Remove BytesToBytesMap.finalize()
50e9671 [Josh Rosen] Throw memory leak warning even in case of error; add warning about code duplication
70a39e4 [Josh Rosen] Split MemoryManager into ExecutorMemoryManager and TaskMemoryManager:
6e4b192 [Josh Rosen] Remove an unused method from ByteArrayMethods.
de5e001 [Josh Rosen] Fix debug vs. trace in logging message.
a19e066 [Josh Rosen] Rename unsafe Java test suites to match Scala test naming convention.
78a5b84 [Josh Rosen] Add logging to MemoryManager
ce3c565 [Josh Rosen] More comments, formatting, and code cleanup.
529e571 [Josh Rosen] Measure timeSpentResizing in nanoseconds instead of milliseconds.
3ca84b2 [Josh Rosen] Only zero the used portion of groupingKeyConversionScratchSpace
162caf7 [Josh Rosen] Fix test compilation
b45f070 [Josh Rosen] Don't redundantly store the offset from key to value, since we can compute this from the key size.
a8e4a3f [Josh Rosen] Introduce MemoryManager interface; add to SparkEnv.
0925847 [Josh Rosen] Disable MiMa checks for new unsafe module
cde4132 [Josh Rosen] Add missing pom.xml
9c19fc0 [Josh Rosen] Add configuration options for heap vs. offheap
6ffdaa1 [Josh Rosen] Null handling improvements in UnsafeRow.
31eaabc [Josh Rosen] Lots of TODO and doc cleanup.
a95291e [Josh Rosen] Cleanups to string handling code
afe8dca [Josh Rosen] Some Javadoc cleanup
f3dcbfe [Josh Rosen] More mod replacement
854201a [Josh Rosen] Import and comment cleanup
06e929d [Josh Rosen] More warning cleanup
ef6b3d3 [Josh Rosen] Fix a bunch of FindBugs and IntelliJ inspections
29a7575 [Josh Rosen] Remove debug logging
49aed30 [Josh Rosen] More long -> int conversion.
b26f1d3 [Josh Rosen] Fix bug in murmur hash implementation.
765243d [Josh Rosen] Enable optional performance metrics for hash map.
23a440a [Josh Rosen] Bump up default hash map size
628f936 [Josh Rosen] Use ints intead of longs for indexing.
92d5a06 [Josh Rosen] Address a number of minor code review comments.
1f4b716 [Josh Rosen] Merge Unsafe code into the regular GeneratedAggregate, guarded by a configuration flag; integrate planner support and re-enable all tests.
d85eeff [Josh Rosen] Add basic sanity test for UnsafeFixedWidthAggregationMap
bade966 [Josh Rosen] Comment update (bumping to refresh GitHub cache...)
b3eaccd [Josh Rosen] Extract aggregation map into its own class.
d2bb986 [Josh Rosen] Update to implement new Row methods added upstream
58ac393 [Josh Rosen] Use UNSAFE allocator in GeneratedAggregate (TODO: make this configurable)
7df6008 [Josh Rosen] Optimizations related to zeroing out memory:
c1b3813 [Josh Rosen] Fix bug in UnsafeMemoryAllocator.free():
738fa33 [Josh Rosen] Add feature flag to guard UnsafeGeneratedAggregate
c55bf66 [Josh Rosen] Free buffer once iterator has been fully consumed.
62ab054 [Josh Rosen] Optimize for fact that get() is only called on String columns.
c7f0b56 [Josh Rosen] Reuse UnsafeRow pointer in UnsafeRowConverter
ae39694 [Josh Rosen] Add finalizer as "cleanup method of last resort"
c754ae1 [Josh Rosen] Now that the store*() contract has been stregthened, we can remove an extra lookup
f764d13 [Josh Rosen] Simplify address + length calculation in Location.
079f1bf [Josh Rosen] Some clarification of the BytesToBytesMap.lookup() / set() contract.
1a483c5 [Josh Rosen] First version that passes some aggregation tests:
fc4c3a8 [Josh Rosen] Sketch how the converters will be used in UnsafeGeneratedAggregate
53ba9b7 [Josh Rosen] Start prototyping Java Row -> UnsafeRow converters
1ff814d [Josh Rosen] Add reminder to free memory on iterator completion
8a8f9df [Josh Rosen] Add skeleton for GeneratedAggregate integration.
5d55cef [Josh Rosen] Add skeleton for Row implementation.
f03e9c1 [Josh Rosen] Play around with Unsafe implementations of more string methods.
ab68e08 [Josh Rosen] Begin merging the UTF8String implementations.
480a74a [Josh Rosen] Initial import of code from Databricks unsafe utils repo.
2015-04-29 01:07:26 -07:00
Burak Yavuz fe917f5ec9 [SPARK-7188] added python support for math DataFrame functions
Adds support for the math functions for DataFrames in PySpark.

rxin I love Davies.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5750 from brkyvz/python-math-udfs and squashes the following commits:

7c4f563 [Burak Yavuz] removed is_math
3c4adde [Burak Yavuz] cleanup imports
d5dca3f [Burak Yavuz] moved math functions to mathfunctions
25e6534 [Burak Yavuz] addressed comments v2.0
d3f7e0f [Burak Yavuz] addressed comments and added tests
7b7d7c4 [Burak Yavuz] remove tests for removed methods
33c2c15 [Burak Yavuz] fixed python style
3ee0c05 [Burak Yavuz] added python functions
2015-04-29 00:09:24 -07:00
Burak Yavuz 271c4c621d [SPARK-7215] made coalesce and repartition a part of the query plan
Coalesce and repartition now show up as part of the query plan, rather than resulting in a new `DataFrame`.

cc rxin

Author: Burak Yavuz <brkyvz@gmail.com>

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

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

Author: Sean Owen <sowen@cloudera.com>

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

98a8947 [Sean Owen] Make install, deploy plugin versions explicit
4ecf3b2 [Sean Owen] Update Maven build plugin versions and centralize plugin version management
2015-04-28 07:48:34 -04:00
Pei-Lun Lee e13cd86567 [SPARK-6352] [SQL] Custom parquet output committer
Add new config "spark.sql.parquet.output.committer.class" to allow custom parquet output committer and an output committer class specific to use on s3.
Fix compilation error introduced by https://github.com/apache/spark/pull/5042.
Respect ParquetOutputFormat.ENABLE_JOB_SUMMARY flag.

Author: Pei-Lun Lee <pllee@appier.com>

Closes #5525 from ypcat/spark-6352 and squashes the following commits:

54c6b15 [Pei-Lun Lee] error handling
472870e [Pei-Lun Lee] add back custom parquet output committer
ddd0f69 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
9ece5c5 [Pei-Lun Lee] compatibility with hadoop 1.x
8413fcd [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
fe65915 [Pei-Lun Lee] add support for parquet config parquet.enable.summary-metadata
e17bf47 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
9ae7545 [Pei-Lun Lee] [SPARL-6352] [SQL] Change to allow custom parquet output committer.
0d540b9 [Pei-Lun Lee] [SPARK-6352] [SQL] add license
c42468c [Pei-Lun Lee] [SPARK-6352] [SQL] add test case
0fc03ca [Pei-Lun Lee] [SPARK-6532] [SQL] hide class DirectParquetOutputCommitter
769bd67 [Pei-Lun Lee] DirectParquetOutputCommitter
f75e261 [Pei-Lun Lee] DirectParquetOutputCommitter
2015-04-28 16:50:18 +08:00
Reynold Xin d94cd1a733 [SPARK-7135][SQL] DataFrame expression for monotonically increasing IDs.
Author: Reynold Xin <rxin@databricks.com>

Closes #5709 from rxin/inc-id and squashes the following commits:

7853611 [Reynold Xin] private sql.
a9fda0d [Reynold Xin] Missed a few numbers.
343d896 [Reynold Xin] Self review feedback.
a7136cb [Reynold Xin] [SPARK-7135][SQL] DataFrame expression for monotonically increasing IDs.
2015-04-28 00:39:08 -07:00
Burak Yavuz 29576e7860 [SPARK-6829] Added math functions for DataFrames
Implemented almost all math functions found in scala.math (max, min and abs were already present).

cc mengxr marmbrus

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #5616 from brkyvz/math-udfs and squashes the following commits:

fb27153 [Burak Yavuz] reverted exception message
836a098 [Burak Yavuz] fixed test and addressed small comment
e5f0d13 [Burak Yavuz] addressed code review v2.2
b26c5fb [Burak Yavuz] addressed review v2.1
2761f08 [Burak Yavuz] addressed review v2
6588a5b [Burak Yavuz] fixed merge conflicts
b084e10 [Burak Yavuz] Addressed code review
029e739 [Burak Yavuz] fixed atan2 test
534cc11 [Burak Yavuz] added more tests, addressed comments
fa68dbe [Burak Yavuz] added double specific test data
937d5a5 [Burak Yavuz] use doubles instead of ints
8e28fff [Burak Yavuz] Added apache header
7ec8f7f [Burak Yavuz] Added math functions for DataFrames
2015-04-27 23:10:14 -07:00
Sean Owen ab5adb7a97 [SPARK-7145] [CORE] commons-lang (2.x) classes used instead of commons-lang3 (3.x); commons-io used without dependency
Remove use of commons-lang in favor of commons-lang3 classes; remove commons-io use in favor of Guava

Author: Sean Owen <sowen@cloudera.com>

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

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

Author: baishuo <vc_java@hotmail.com>

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

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

Author: wangfei <wangfei1@huawei.com>

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

c319977 [wangfei] rename apply to parse
2015-04-26 21:08:47 -07:00
Reynold Xin ca55dc95b7 [SPARK-7152][SQL] Add a Column expression for partition ID.
Author: Reynold Xin <rxin@databricks.com>

Closes #5705 from rxin/df-pid and squashes the following commits:

401018f [Reynold Xin] [SPARK-7152][SQL] Add a Column expression for partition ID.
2015-04-26 11:46:58 -07:00
Yin Huai aa6966ff34 [SQL] Update SQL readme to include instructions on generating golden answer files based on Hive 0.13.1.
Author: Yin Huai <yhuai@databricks.com>

Closes #5702 from yhuai/howToGenerateGoldenFiles and squashes the following commits:

9c4a7f8 [Yin Huai] Update readme to include instructions on generating golden answer files based on Hive 0.13.1.
2015-04-25 13:43:39 -07:00
Calvin Jia 438859eb7c [SPARK-6122] [CORE] Upgrade tachyon-client version to 0.6.3
This is a reopening of #4867.
A short summary of the issues resolved from the previous PR:

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

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

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

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

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

0eefe4d [Calvin Jia] Handle httpclient version in maven dependency management. Remove httpclient version setting from profiles.
7c00dfa [Calvin Jia] Set httpclient version to 4.3.2 for selenium. Specify version of httpclient for sql/hive (previously 4.2.5 transitive dependency of libthrift).
9263097 [Calvin Jia] Merge master to test latest changes
dbfc1bd [Calvin Jia] Use Tachyon 0.6.4 for cleaner dependencies.
e2ff80a [Calvin Jia] Exclude the jetty and curator promoted dependencies from tachyon-client.
a3a29da [Calvin Jia] Update tachyon-client exclusions.
0ae6c97 [Calvin Jia] Change tachyon version to 0.6.3
a204df9 [Calvin Jia] Update make distribution tachyon version.
a93c94f [Calvin Jia] Exclude jackson-mapper-asl from hadoop client since it has a lower version than spark's expected version.
a8a923c [Calvin Jia] Exclude httpcomponents from Tachyon
910fabd [Calvin Jia] Update to master
eed9230 [Calvin Jia] Update tachyon version to 0.6.1.
11907b3 [Calvin Jia] Use TachyonURI for tachyon paths instead of strings.
71bf441 [Calvin Jia] Upgrade Tachyon client version to 0.6.0.
2015-04-24 17:57:41 -04:00
Reynold Xin 4c722d77ae Fixed a typo from the previous commit. 2015-04-23 22:39:00 -07:00
Reynold Xin d3a302defc [SQL] Fixed expression data type matching.
Also took the chance to improve documentation for various types.

Author: Reynold Xin <rxin@databricks.com>

Closes #5675 from rxin/data-type-matching-expr and squashes the following commits:

0f31856 [Reynold Xin] One more function documentation.
27c1973 [Reynold Xin] Added more documentation.
336a36d [Reynold Xin] [SQL] Fixed expression data type matching.
2015-04-23 21:21:03 -07:00
Reynold Xin 6220d933e5 [SQL] Break dataTypes.scala into multiple files.
It was over 1000 lines of code, making it harder to find all the types. Only moved code around, and didn't change any.

Author: Reynold Xin <rxin@databricks.com>

Closes #5670 from rxin/break-types and squashes the following commits:

8c59023 [Reynold Xin] Check in missing files.
dcd5193 [Reynold Xin] [SQL] Break dataTypes.scala into multiple files.
2015-04-23 14:48:19 -07:00
Vinod K C c1213e6a92 [SPARK-7055][SQL]Use correct ClassLoader for JDBC Driver in JDBCRDD.getConnector
Author: Vinod K C <vinod.kc@huawei.com>

Closes #5633 from vinodkc/use_correct_classloader_driverload and squashes the following commits:

73c5380 [Vinod K C] Use correct ClassLoader for JDBC Driver
2015-04-23 12:00:23 -07:00
Cheng Hao cc48e6387a [SPARK-7044] [SQL] Fix the deadlock in script transformation
Author: Cheng Hao <hao.cheng@intel.com>

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

5ec1dd2 [Cheng Hao] fix the deadlock issue in ScriptTransform
2015-04-23 10:35:22 -07:00
Prashant Sharma a7d65d38f9 [HOTFIX] [SQL] Fix compilation for scala 2.11.
Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #5652 from ScrapCodes/hf/compilation-fix-scala-2.11 and squashes the following commits:

819ff06 [Prashant Sharma] [HOTFIX] Fix compilation for scala 2.11.
2015-04-23 16:45:26 +05:30
Reynold Xin f60bece14f [SPARK-7069][SQL] Rename NativeType -> AtomicType.
Also renamed JvmType to InternalType.

Author: Reynold Xin <rxin@databricks.com>

Closes #5651 from rxin/native-to-atomic-type and squashes the following commits:

cbd4028 [Reynold Xin] [SPARK-7069][SQL] Rename NativeType -> AtomicType.
2015-04-23 01:43:40 -07:00
Reynold Xin 29163c5200 [SPARK-7068][SQL] Remove PrimitiveType
Author: Reynold Xin <rxin@databricks.com>

Closes #5646 from rxin/remove-primitive-type and squashes the following commits:

01b673d [Reynold Xin] [SPARK-7068][SQL] Remove PrimitiveType
2015-04-22 23:55:20 -07:00
Liang-Chi Hsieh d9e70f331f [HOTFIX][SQL] Fix broken cached test
Added in #5475. Pointed as broken in #5639.
/cc marmbrus

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

Closes #5640 from viirya/fix_cached_test and squashes the following commits:

c0cf69a [Liang-Chi Hsieh] Fix broken cached test.
2015-04-22 22:18:56 -07:00
Reynold Xin d20686066e [SPARK-7066][MLlib] VectorAssembler should use NumericType not NativeType.
Author: Reynold Xin <rxin@databricks.com>

Closes #5642 from rxin/mllib-native-type and squashes the following commits:

e23af5b [Reynold Xin] Remove StringType
7cbb205 [Reynold Xin] [SPARK-7066][MLlib] VectorAssembler should use NumericType and StringType, not NativeType.
2015-04-22 21:35:42 -07:00
Daoyuan Wang 04525c077c [SPARK-6967] [SQL] fix date type convertion in jdbcrdd
This pr convert java.sql.Date type into Int for JDBCRDD.

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

Closes #5590 from adrian-wang/datebug and squashes the following commits:

f897b81 [Daoyuan Wang] add a test case
3c9184c [Daoyuan Wang] fix date type convertion in jdbcrdd
2015-04-22 19:14:28 -07:00
Reynold Xin baf865ddc2 [SPARK-7059][SQL] Create a DataFrame join API to facilitate equijoin.
Author: Reynold Xin <rxin@databricks.com>

Closes #5638 from rxin/joinUsing and squashes the following commits:

13e9cc9 [Reynold Xin] Code review + Python.
b1bd914 [Reynold Xin] [SPARK-7059][SQL] Create a DataFrame join API to facilitate equijoin and self join.
2015-04-22 15:26:58 -07:00
szheng79 fbe7106d75 [SPARK-7039][SQL]JDBCRDD: Add support on type NVARCHAR
Issue:

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

Add support to column type NVARCHAR in Sql Server

java.sql.Types:
http://docs.oracle.com/javase/7/docs/api/java/sql/Types.html

Author: szheng79 <szheng.code@gmail.com>

Closes #5618 from szheng79/patch-1 and squashes the following commits:

10da99c [szheng79] Update JDBCRDD.scala
eab0bd8 [szheng79] Add support on type NVARCHAR
2015-04-22 13:02:55 -07:00
Reynold Xin cdf0328684 [SQL] Rename some apply functions.
I was looking at the code gen code and got confused by a few of use cases of apply, in particular apply on objects. So I went ahead and changed a few of them. Hopefully slightly more clear with a proper verb.

Author: Reynold Xin <rxin@databricks.com>

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

ee45034 [Reynold Xin] [SQL] Rename some apply functions.
2015-04-22 11:18:01 -07:00
Marcelo Vanzin e72c16e30d [SPARK-6014] [core] Revamp Spark shutdown hooks, fix shutdown races.
This change adds some new utility code to handle shutdown hooks in
Spark. The main goal is to take advantage of Hadoop 2.x's API for
shutdown hooks, which allows Spark to register a hook that will
run before the one that cleans up HDFS clients, and thus avoids
some races that would cause exceptions to show up and other issues
such as failure to properly close event logs.

Unfortunately, Hadoop 1.x does not have such APIs, so in that case
correctness is still left to chance.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #5560 from vanzin/SPARK-6014 and squashes the following commits:

edfafb1 [Marcelo Vanzin] Better scaladoc.
fcaeedd [Marcelo Vanzin] Merge branch 'master' into SPARK-6014
e7039dc [Marcelo Vanzin] [SPARK-6014] [core] Revamp Spark shutdown hooks, fix shutdown races.
2015-04-21 20:33:57 -04:00
Cheng Hao 7662ec23bb [SPARK-5817] [SQL] Fix bug of udtf with column names
It's a bug while do query like:
```sql
select d from (select explode(array(1,1)) d from src limit 1) t
```
And it will throws exception like:
```
org.apache.spark.sql.AnalysisException: cannot resolve 'd' given input columns _c0; line 1 pos 7
at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:48)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$apply$3$$anonfun$apply$1.applyOrElse(CheckAnalysis.scala:45)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:250)
at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:50)
at org.apache.spark.sql.catalyst.trees.TreeNode.transformUp(TreeNode.scala:249)
at org.apache.spark.sql.catalyst.plans.QueryPlan.org$apache$spark$sql$catalyst$plans$QueryPlan$$transformExpressionUp$1(QueryPlan.scala:103)
at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2$$anonfun$apply$2.apply(QueryPlan.scala:117)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:244)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:244)
at scala.collection.AbstractTraversable.map(Traversable.scala:105)
at org.apache.spark.sql.catalyst.plans.QueryPlan$$anonfun$2.apply(QueryPlan.scala:116)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
```

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

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

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

c2a5132 [Cheng Hao] add back resolved for Alias
556e982 [Cheng Hao] revert the unncessary change
002c361 [Cheng Hao] change the rule of resolved for Generate
04ae500 [Cheng Hao] add qualifier only for generator output
5ee5d2c [Cheng Hao] prepend the new qualifier
d2e8b43 [Cheng Hao] Update the code as feedback
ca5e7f4 [Cheng Hao] shrink the commits
2015-04-21 15:11:15 -07:00
Punya Biswal 2a24bf92e6 [SPARK-6996][SQL] Support map types in java beans
liancheng mengxr this is similar to #5146.

Author: Punya Biswal <pbiswal@palantir.com>

Closes #5578 from punya/feature/SPARK-6996 and squashes the following commits:

d56c3e0 [Punya Biswal] Fix imports
c7e308b [Punya Biswal] Support java iterable types in POJOs
5e00685 [Punya Biswal] Support map types in java beans
2015-04-21 14:50:02 -07:00
Yin Huai 6265cba00f [SPARK-6969][SQL] Refresh the cached table when REFRESH TABLE is used
https://issues.apache.org/jira/browse/SPARK-6969

Author: Yin Huai <yhuai@databricks.com>

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

1e5142b [Yin Huai] Add todo.
92b2498 [Yin Huai] Minor updates.
367df92 [Yin Huai] Recache data in the command of REFRESH TABLE.
2015-04-21 14:48:42 -07:00
Wenchen Fan 03fd921671 [SQL][minor] make it more clear that we only need to re-throw GetField exception for UnresolvedAttribute
For `GetField` outside `UnresolvedAttribute`, we will throw exception in `Analyzer`.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5588 from cloud-fan/tmp and squashes the following commits:

7ac74d2 [Wenchen Fan] small refactor
2015-04-21 14:48:02 -07:00
vidmantas zemleris 2e8c6ca47d [SPARK-6994] Allow to fetch field values by name in sql.Row
It looked weird that up to now there was no way in Spark's Scala API to access fields of `DataFrame/sql.Row` by name, only by their index.

This tries to solve this issue.

Author: vidmantas zemleris <vidmantas@vinted.com>

Closes #5573 from vidma/features/row-with-named-fields and squashes the following commits:

6145ae3 [vidmantas zemleris] [SPARK-6994][SQL] Allow to fetch field values by name on Row
9564ebb [vidmantas zemleris] [SPARK-6994][SQL] Add fieldIndex to schema (StructType)
2015-04-21 14:47:09 -07:00
Prashant Sharma 04bf34e34f [SPARK-7011] Build(compilation) fails with scala 2.11 option, because a protected[sql] type is accessed in ml package.
[This](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala#L58) is where it is used and fails compilations at.

Author: Prashant Sharma <prashant.s@imaginea.com>

Closes #5593 from ScrapCodes/SPARK-7011/build-fix and squashes the following commits:

e6d57a3 [Prashant Sharma] [SPARK-7011] Build fails with scala 2.11 option, because a protected[sql] type is accessed in ml package.
2015-04-21 14:43:46 -07:00
Liang-Chi Hsieh c736220dac [SPARK-6635][SQL] DataFrame.withColumn should replace columns with identical column names
JIRA https://issues.apache.org/jira/browse/SPARK-6635

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

Closes #5541 from viirya/replace_with_column and squashes the following commits:

b539c7b [Liang-Chi Hsieh] For comment.
72f35b1 [Liang-Chi Hsieh] DataFrame.withColumn can replace original column with identical column name.
2015-04-20 18:54:01 -07:00
Yin Huai ce7ddabbcd [SPARK-6368][SQL] Build a specialized serializer for Exchange operator.
JIRA: https://issues.apache.org/jira/browse/SPARK-6368

Author: Yin Huai <yhuai@databricks.com>

Closes #5497 from yhuai/serializer2 and squashes the following commits:

da562c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2
50e0c3d [Yin Huai] When no filed is emitted to shuffle, use SparkSqlSerializer for now.
9f1ed92 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2
6d07678 [Yin Huai] Address comments.
4273b8c [Yin Huai] Enabled SparkSqlSerializer2.
09e587a [Yin Huai] Remove TODO.
791b96a [Yin Huai] Use UTF8String.
60a1487 [Yin Huai] Merge remote-tracking branch 'upstream/master' into serializer2
3e09655 [Yin Huai] Use getAs for Date column.
43b9fb4 [Yin Huai] Test.
8297732 [Yin Huai] Fix test.
c9373c8 [Yin Huai] Support DecimalType.
2379eeb [Yin Huai] ASF header.
39704ab [Yin Huai] Specialized serializer for Exchange.
2015-04-20 18:42:50 -07:00
cafreeman 59e206deb7 [SPARK-6807] [SparkR] Merge recent SparkR-pkg changes
This PR pulls in recent changes in SparkR-pkg, including

cartesian, intersection, sampleByKey, subtract, subtractByKey, except, and some API for StructType and StructField.

Author: cafreeman <cfreeman@alteryx.com>
Author: Davies Liu <davies@databricks.com>
Author: Zongheng Yang <zongheng.y@gmail.com>
Author: Shivaram Venkataraman <shivaram.venkataraman@gmail.com>
Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>
Author: Sun Rui <rui.sun@intel.com>

Closes #5436 from davies/R3 and squashes the following commits:

c2b09be [Davies Liu] SQLTypes -> schema
a5a02f2 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R3
168b7fe [Davies Liu] sort generics
b1fe460 [Davies Liu] fix conflict in README.md
e74c04e [Davies Liu] fix schema.R
4f5ac09 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R5
41f8184 [Davies Liu] rm man
ae78312 [Davies Liu] Merge pull request #237 from sun-rui/SPARKR-154_3
1bdcb63 [Zongheng Yang] Updates to README.md.
5a553e7 [cafreeman] Use object attribute instead of argument
71372d9 [cafreeman] Update docs and examples
8526d2e71 [cafreeman] Remove `tojson` functions
6ef5f2d [cafreeman] Fix spacing
7741d66 [cafreeman] Rename the SQL DataType function
141efd8 [Shivaram Venkataraman] Merge pull request #245 from hqzizania/upstream
9387402 [Davies Liu] fix style
40199eb [Shivaram Venkataraman] Move except into sorted position
07d0dbc [Sun Rui] [SPARKR-244] Fix test failure after integration of subtract() and subtractByKey() for RDD.
7e8caa3 [Shivaram Venkataraman] Merge pull request #246 from hlin09/fixCombineByKey
ed66c81 [cafreeman] Update `subtract` to work with `generics.R`
f3ba785 [cafreeman] Fixed duplicate export
275deb4 [cafreeman] Update `NAMESPACE` and tests
1a3b63d [cafreeman] new version of `CreateDF`
836c4bf [cafreeman] Update `createDataFrame` and `toDF`
be5d5c1 [cafreeman] refactor schema functions
40338a4 [Zongheng Yang] Merge pull request #244 from sun-rui/SPARKR-154_5
20b97a6 [Zongheng Yang] Merge pull request #234 from hqzizania/assist
ba54e34 [Shivaram Venkataraman] Merge pull request #238 from sun-rui/SPARKR-154_4
c9497a3 [Shivaram Venkataraman] Merge pull request #208 from lythesia/master
b317aa7 [Zongheng Yang] Merge pull request #243 from hqzizania/master
136a07e [Zongheng Yang] Merge pull request #242 from hqzizania/stats
cd66603 [cafreeman] new line at EOF
8b76e81 [Shivaram Venkataraman] Merge pull request #233 from redbaron/fail-early-on-missing-dep
7dd81b7 [cafreeman] Documentation
0e2a94f [cafreeman] Define functions for schema and fields
2015-04-17 13:42:19 -07:00
Michael Armbrust 8220d5265f [SPARK-6972][SQL] Add Coalesce to DataFrame
Author: Michael Armbrust <michael@databricks.com>

Closes #5545 from marmbrus/addCoalesce and squashes the following commits:

9fdf3f6 [Michael Armbrust] [SPARK-6972][SQL] Add Coalesce to DataFrame
2015-04-16 21:49:26 -05:00
Michael Armbrust e5949c287e [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver
Otherwise we cannot add jars with drivers after the fact.

Author: Michael Armbrust <michael@databricks.com>

Closes #5543 from marmbrus/jdbcClassloader and squashes the following commits:

d9930f3 [Michael Armbrust] fix imports
73d0614 [Michael Armbrust] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver
2015-04-16 17:59:49 -07:00
Liang-Chi Hsieh 1e43851d64 [SPARK-6899][SQL] Fix type mismatch when using codegen with Average on DecimalType
JIRA https://issues.apache.org/jira/browse/SPARK-6899

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

Closes #5517 from viirya/fix_codegen_average and squashes the following commits:

8ae5f65 [Liang-Chi Hsieh] Add the case of DecimalType.Unlimited to Average.
2015-04-16 17:50:20 -07:00
scwf d96608674f [SQL][Minor] Fix foreachUp of treenode
`foreachUp` should runs the given function recursively on [[children]] then on this node(just like transformUp). The current implementation does not follow this.

This will leads to checkanalysis do not check from bottom of logical tree.

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

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

18e28b2 [scwf] added a test case
1ccbfa8 [Fei Wang] fix foreachUp
2015-04-16 17:35:51 -07:00
Davies Liu 6183b5e2ca [SPARK-6911] [SQL] improve accessor for nested types
Support access columns by index in Python:
```
>>> df[df[0] > 3].collect()
[Row(age=5, name=u'Bob')]
```

Access items in ArrayType or MapType
```
>>> df.select(df.l.getItem(0), df.d.getItem("key")).show()
>>> df.select(df.l[0], df.d["key"]).show()
```

Access field in StructType
```
>>> df.select(df.r.getField("b")).show()
>>> df.select(df.r.a).show()
```

Author: Davies Liu <davies@databricks.com>

Closes #5513 from davies/access and squashes the following commits:

e04d5a0 [Davies Liu] Update run-tests-jenkins
7ada9eb [Davies Liu] update timeout
d125ac4 [Davies Liu] check column name, improve scala tests
6b62540 [Davies Liu] fix test
db15b42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into access
6c32e79 [Davies Liu] add scala tests
11f1df3 [Davies Liu] improve accessor for nested types
2015-04-16 17:33:57 -07:00
云峤 5fe4343352 SPARK-6927 [SQL] Sorting Error when codegen on
Fix this error by adding BinaryType comparor in GenerateOrdering.
JIRA https://issues.apache.org/jira/browse/SPARK-6927

Author: 云峤 <chensong.cs@alibaba-inc.com>

Closes #5524 from kaka1992/fix-codegen-sort and squashes the following commits:

d7e2afe [云峤] fix codegen sorting error
2015-04-16 17:32:42 -07:00
Jin Adachi 3ae37b93a7 [SPARK-6694][SQL]SparkSQL CLI must be able to specify an option --database on the command line.
SparkSQL CLI has an option --database as follows.
But, the option --database is ignored.

```
$ spark-sql --help
:
CLI options:
    :
    --database <databasename>     Specify the database to use
```

Author: Jin Adachi <adachij2002@yahoo.co.jp>
Author: adachij <adachij@nttdata.co.jp>

Closes #5345 from adachij2002/SPARK-6694 and squashes the following commits:

8659084 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694
0301eb9 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694
df81086 [Jin Adachi] Modify code style.
846f83e [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694
dbe8c63 [Jin Adachi] Change file permission to 644.
7b58f42 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694
c581d06 [Jin Adachi] Add an option --database test
db56122 [Jin Adachi] Merge branch 'SPARK-6694' of https://github.com/adachij2002/spark into SPARK-6694
ee09fa5 [adachij] Merge branch 'master' into SPARK-6694
c804c03 [adachij] SparkSQL CLI must be able to specify an option --database on the command line.
2015-04-16 23:41:04 +08:00
Max Seiden 8a53de16fc [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators
[SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators

There were a few places where new SparkSqlSerializer instances were created with new, empty SparkConfs resulting in user specified registrators sometimes not getting initialized.

The fix is to try and pull a conf from the SparkEnv, and construct a new conf (that loads defaults) if one cannot be found.

The changes touched:
    1) SparkSqlSerializer's resource pool (this appears to fix the issue in the comment)
    2) execution.Exchange (for all of the partitioners)
    3) execution.Limit (for the HashPartitioner)

A few tests were added to ColumnTypeSuite, ensuring that a custom registrator and serde is initialized and used when in-memory columns are written.

Author: Max Seiden <max@platfora.com>

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

Closes #5237 from mhseiden/sql_udt_kryo and squashes the following commits:

3175c2f [Max Seiden] [SPARK-5277][SQL] - address code review comments
e5011fb [Max Seiden] [SPARK-5277][SQL] - SparkSqlSerializer does not register user specified KryoRegistrators
2015-04-15 16:15:11 -07:00
Daoyuan Wang 585638e81c [SPARK-2213] [SQL] sort merge join for spark sql
Thanks for the initial work from Ishiihara in #3173

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

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

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

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

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

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

Author: Wenchen Fan <cloud0fan@outlook.com>

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

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

48e3e57 [Wenchen Fan] more style fix
820dc45 [Wenchen Fan] do not ignore newName in UnresolvedAttribute
d81ad43 [Wenchen Fan] fix style
11699d6 [Wenchen Fan] completely support special chars in column names
2015-04-15 13:39:12 -07:00
Liang-Chi Hsieh cf38fe04f8 [SPARK-6844][SQL] Clean up accumulators used in InMemoryRelation when it is uncached
JIRA: https://issues.apache.org/jira/browse/SPARK-6844

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

Closes #5475 from viirya/cache_memory_leak and squashes the following commits:

0b41235 [Liang-Chi Hsieh] fix style.
dc1d5d5 [Liang-Chi Hsieh] For comments.
78af229 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cache_memory_leak
26c9bb6 [Liang-Chi Hsieh] Add configuration to enable in-memory table scan accumulators.
1c3b06e [Liang-Chi Hsieh] Clean up accumulators used in InMemoryRelation when it is uncached.
2015-04-15 13:15:58 -07:00
Davies Liu 85842760dc [SPARK-6638] [SQL] Improve performance of StringType in SQL
This PR change the internal representation for StringType from java.lang.String to UTF8String, which is implemented use ArrayByte.

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

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

cc rxin

Author: Davies Liu <davies@databricks.com>

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

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

Author: Yin Huai <yhuai@databricks.com>

Closes #5499 from yhuai/inMemFloat and squashes the following commits:

84cba38 [Yin Huai] Add test.
4b75ba6 [Yin Huai] Add FloatType back.
2015-04-15 13:04:03 -07:00
Liang-Chi Hsieh e3e4e9a38b [SPARK-6800][SQL] Update doc for JDBCRelation's columnPartition
JIRA https://issues.apache.org/jira/browse/SPARK-6800

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

Closes #5488 from viirya/fix_jdbc_where and squashes the following commits:

51386c8 [Liang-Chi Hsieh] Update code comment.
1dcc929 [Liang-Chi Hsieh] Update document.
3eb74d6 [Liang-Chi Hsieh] Revert and modify doc.
df11783 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_jdbc_where
3e7db15 [Liang-Chi Hsieh] Fix wrong logic to generate WHERE clause for JDBC.
2015-04-15 13:02:25 -07:00
Liang-Chi Hsieh b75b307074 [SPARK-6730][SQL] Allow using keyword as identifier in OPTIONS
JIRA: https://issues.apache.org/jira/browse/SPARK-6730

It is very possible that keyword will be used as identifier in `OPTIONS`, this pr makes it works.

However, another approach is that we can request that `OPTIONS` can't include keywords and has to use alternative identifier (e.g. table -> cassandraTable) if needed.

If so, please let me know to close this pr. Thanks.

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

Closes #5520 from viirya/relax_options and squashes the following commits:

339fd68 [Liang-Chi Hsieh] Use regex parser.
92be11c [Liang-Chi Hsieh] Allow using keyword as identifier in OPTIONS.
2015-04-15 13:00:19 -07:00
Daoyuan Wang 29aabdd6c2 [HOTFIX] [SPARK-6896] [SQL] fix compile error in hive-thriftserver
SPARK-6440 #5424 import guava but did not promote guava dependency to compile level.

[INFO] compiler plugin: BasicArtifact(org.scalamacros,paradise_2.10.4,2.0.1,null)
[info] Compiling 8 Scala sources to /root/projects/spark/sql/hive-thriftserver/target/scala-2.10/classes...
[error] bad symbolic reference. A signature in Utils.class refers to term util
[error] in package com.google.common which is not available.
[error] It may be completely missing from the current classpath, or the version on
[error] the classpath might be incompatible with the version used when compiling Utils.class.
[error]
[error] while compiling: /root/projects/spark/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala
[error] during phase: erasure
[error] library version: version 2.10.4
[error] compiler version: version 2.10.4
[error] reconstructed args: -deprecation -classpath

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

Closes #5507 from adrian-wang/guava and squashes the following commits:

c337dad [Daoyuan Wang] fix compile error
2015-04-15 10:23:53 +01:00
Liang-Chi Hsieh 6be918942c [SPARK-6871][SQL] WITH clause in CTE can not following another WITH clause
JIRA https://issues.apache.org/jira/browse/SPARK-6871

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

Closes #5480 from viirya/no_cte_after_cte and squashes the following commits:

4da3712 [Liang-Chi Hsieh] Create new test.
40b38ed [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_cte_after_cte
0edf568 [Liang-Chi Hsieh] for comments.
6591b79 [Liang-Chi Hsieh] WITH clause in CTE can not following another WITH clause.
2015-04-14 23:47:16 -07:00
Josh Rosen a76b921a92 Revert "[SPARK-6352] [SQL] Add DirectParquetOutputCommitter"
This reverts commit b29663eeea.

I'm reverting this because it broke test compilation for the Hadoop 1.x
profiles.
2015-04-14 14:10:15 -07:00
Daoyuan Wang b45059d0d7 [SPARK-5794] [SQL] fix add jar
Author: Daoyuan Wang <daoyuan.wang@intel.com>

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

efdd602 [Daoyuan Wang] move jar to another place
6c707e8 [Daoyuan Wang] restrict hive version for test
32c4fb8 [Daoyuan Wang] fix style and add a test
9957d87 [Daoyuan Wang] use sessionstate classloader in makeRDDforTable
0810e71 [Daoyuan Wang] remove variable substitution
1898309 [Daoyuan Wang] fix classnotfound
95a40da [Daoyuan Wang] support env argus in add jar, and set add jar ret to 0
2015-04-13 18:26:00 -07:00
Liang-Chi Hsieh 4898dfa464 [SPARK-6877][SQL] Add code generation support for Min
Currently `min` is not supported in code generation. This pr adds the support for it.

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

Closes #5487 from viirya/add_min_codegen and squashes the following commits:

0ddec23 [Liang-Chi Hsieh] Add code generation support for Min.
2015-04-13 18:16:33 -07:00
Liang-Chi Hsieh 5b8b324f33 [SPARK-6303][SQL] Remove unnecessary Average in GeneratedAggregate
Because `Average` is a `PartialAggregate`, we never get a `Average` node when reaching `HashAggregation` to prepare `GeneratedAggregate`.

That is why in SQLQuerySuite there is already a test for `avg` with codegen. And it works.

But we can find a case in `GeneratedAggregate` to deal with `Average`. Based on the above, we actually never execute this case.

So we can remove this case from `GeneratedAggregate`.

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

Closes #4996 from viirya/add_average_codegened and squashes the following commits:

621c12f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened
368cfbc [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened
74926d1 [Liang-Chi Hsieh] Add Average in canBeCodeGened lists.
2015-04-13 18:15:29 -07:00
Cheng Hao c5602bdc31 [SPARK-5941] [SQL] Unit Test loads the table src twice for leftsemijoin.q
In `leftsemijoin.q`, there is a data loading command for table `sales` already, but in `TestHive`, it also created the table `sales`, which causes duplicated records inserted into the `sales`.

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

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

0be05f7 [Cheng Hao] Remove the table `sales` creating from TestHive
2015-04-13 16:02:18 -07:00
Daoyuan Wang e63a86abe2 [SPARK-6872] [SQL] add copy in external sort
We need add copy before call externalsort.

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

Closes #5481 from adrian-wang/extsort and squashes the following commits:

9611586 [Daoyuan Wang] fix bug in external sort
2015-04-13 16:00:58 -07:00
Yash Datta 3a205bbd9e [SQL][SPARK-6742]: Don't push down predicates which reference partition column(s)
cc liancheng

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

Closes #5390 from saucam/fpush and squashes the following commits:

3f026d6 [Yash Datta] SPARK-6742: Fix scalastyle
ce3d702 [Yash Datta] SPARK-6742: Add test case, fix scalastyle
8592acc [Yash Datta] SPARK-6742: Don't push down predicates which reference partition column(s)
2015-04-13 14:43:07 -07:00
Daoyuan Wang 85ee0cabe8 [SPARK-6130] [SQL] support if not exists for insert overwrite into partition in hiveQl
Standard syntax:
INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 FROM from_statement;
INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 FROM from_statement;
 
Hive extension (multiple inserts):
FROM from_statement
INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1
[INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2]
[INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] ...;
FROM from_statement
INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1
[INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2]
[INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] ...;
 
Hive extension (dynamic partition inserts):
INSERT OVERWRITE TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement;
INSERT INTO TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement;

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

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

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

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

Author: Reynold Xin <rxin@databricks.com>

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

01683de [Reynold Xin] Fixed new code.
a4ab46e [Reynold Xin] Fixed tests.
20adbc8 [Reynold Xin] Missed one violation.
5e36521 [Reynold Xin] [SPARK-6765] Enable scalastyle on test code.
2015-04-13 09:29:04 -07:00
Pei-Lun Lee b29663eeea [SPARK-6352] [SQL] Add DirectParquetOutputCommitter
Add a DirectParquetOutputCommitter class that skips _temporary directory when saving to s3. Add new config value "spark.sql.parquet.useDirectParquetOutputCommitter" (default false) to choose between the default output committer.

Author: Pei-Lun Lee <pllee@appier.com>

Closes #5042 from ypcat/spark-6352 and squashes the following commits:

e17bf47 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352
9ae7545 [Pei-Lun Lee] [SPARL-6352] [SQL] Change to allow custom parquet output committer.
0d540b9 [Pei-Lun Lee] [SPARK-6352] [SQL] add license
c42468c [Pei-Lun Lee] [SPARK-6352] [SQL] add test case
0fc03ca [Pei-Lun Lee] [SPARK-6532] [SQL] hide class DirectParquetOutputCommitter
769bd67 [Pei-Lun Lee] DirectParquetOutputCommitter
f75e261 [Pei-Lun Lee] DirectParquetOutputCommitter
2015-04-13 21:52:00 +08:00
nyaapa 9d117cee0b [SPARK-6440][CORE]Handle IPv6 addresses properly when constructing URI
Author: nyaapa <nyaapa@gmail.com>

Closes #5424 from nyaapa/master and squashes the following commits:

6b717aa [nyaapa] [SPARK-6440][CORE] Remove Utils.localIpAddressHostname, Utils.localIpAddressURI and Utils.getAddressHostName; make Utils.localIpAddress private; rename Utils.localHostURI into Utils.localHostNameForURI; use Utils.localHostName in org.apache.spark.streaming.kinesis.KinesisReceiver and org.apache.spark.sql.hive.thriftserver.SparkSQLEnv
2098081 [nyaapa] [SPARK-6440][CORE] style fixes and use getHostAddress instead of getHostName
84763d7 [nyaapa] [SPARK-6440][CORE]Handle IPv6 addresses properly when constructing URI
2015-04-13 12:55:25 +01:00
Reynold Xin 68d1faa3c0 [SPARK-6562][SQL] DataFrame.replace
Supports replacing values with other values in DataFrames.

Python support should be in a separate pull request.

Author: Reynold Xin <rxin@databricks.com>

Closes #5282 from rxin/df-na-replace and squashes the following commits:

4b72434 [Reynold Xin] Removed println.
c8d9946 [Reynold Xin] col -> cols
fbb3c21 [Reynold Xin] [SPARK-6562][SQL] DataFrame.replace
2015-04-12 22:56:12 -07:00
Daoyuan Wang 04bcd67cfc [MINOR] a typo: coalesce
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #5482 from adrian-wang/typo and squashes the following commits:

e65ef6f [Daoyuan Wang] typo
2015-04-12 18:58:53 +01:00
Josh Rosen dea5dacc5d [HOTFIX] Add explicit return types to fix lint errors 2015-04-11 20:12:40 -07:00
Wenchen Fan 5c2844c51a [SQL][minor] move resolveGetField into a object
The method `resolveGetField` isn't belong to `LogicalPlan` logically and didn't access any members of it.

Author: Wenchen Fan <cloud0fan@outlook.com>

Closes #5435 from cloud-fan/tmp and squashes the following commits:

9a66c83 [Wenchen Fan] code clean up
2015-04-11 19:35:56 -07:00
Yin Huai 6d4e854ffb [SPARK-6367][SQL] Use the proper data type for those expressions that are hijacking existing data types.
This PR adds internal UDTs for expressions that are hijacking existing data types.
The following UDTs are added:
* `HyperLogLogUDT` (`BinaryType` as the SQL type) for `ApproxCountDistinctPartition`
* `OpenHashSetUDT` (`ArrayType` as the SQL type) for `CollectHashSet`, `NewSet`, `AddItemToSet`, and `CombineSets`.

I am also adding more unit tests for aggregation with code gen enabled.

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

Author: Yin Huai <yhuai@databricks.com>

Closes #5094 from yhuai/expressionType and squashes the following commits:

8bcd11a [Yin Huai] Return types.
61a1d66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType
e8b4599 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType
2753156 [Yin Huai] Ignore aggregations having sum functions for now.
b5eb259 [Yin Huai] Case object for HyperLogLog type.
00ebdbd [Yin Huai] deserialize/serialize.
54b87ae [Yin Huai] Add UDTs for expressions that return HyperLogLog and OpenHashSet.
2015-04-11 19:26:15 -07:00
Yin Huai d2383fb5ff [SQL] Handle special characters in the authority of a Path's URI.
Author: Yin Huai <yhuai@databricks.com>

Closes #5381 from yhuai/parquetPath2 and squashes the following commits:

fe296b4 [Yin Huai] Create new Path to take care special characters in the authority of a Path's URI.
2015-04-11 18:44:54 -07:00
Takeshi YAMAMURO 352a5da421 [SPARK-6379][SQL] Support a functon to call user-defined functions registered in SQLContext
This is useful for using pre-defined UDFs in SQLContext;

val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value")
val sqlctx = df.sqlContext
sqlctx.udf.register("simpleUdf", (v: Int) => v * v)
df.select($"id", sqlctx.callUdf("simpleUdf", $"value"))

Author: Takeshi YAMAMURO <linguin.m.s@gmail.com>

Closes #5061 from maropu/SupportUDFConversionInSparkContext and squashes the following commits:

f858aff [Takeshi YAMAMURO] Move the function into functions.scala
afd0380 [Takeshi YAMAMURO] Add a return type of callUDF
599b76c [Takeshi YAMAMURO] Remove the implicit conversion and add SqlContext#callUdf
8b56f10 [Takeshi YAMAMURO] Support an implicit conversion from udf"name" to an UDF defined in SQLContext
2015-04-11 18:41:12 -07:00
DoingDone9 48cc840021 [SPARK-6179][SQL] Add token for "SHOW PRINCIPALS role_name" and "SHOW TRANSACTIONS" and "SHOW COMPACTIONS"
[SHOW PRINCIPALS role_name]
Lists all roles and users who belong to this role.
Only the admin role has privilege for this.

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

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

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

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

4add42f [Zhongshuai Pei] for test
311f806 [Zhongshuai Pei] for test
0c7550a [DoingDone9] Update HiveQl.scala
c8aeb1c [Xu Tingjun] aa
802261c [DoingDone9] Merge pull request #7 from apache/master
d00303b [DoingDone9] Merge pull request #6 from apache/master
98b134f [DoingDone9] Merge pull request #5 from apache/master
161cae3 [DoingDone9] Merge pull request #4 from apache/master
c87e8b6 [DoingDone9] Merge pull request #3 from apache/master
cb1852d [DoingDone9] Merge pull request #2 from apache/master
c3f046f [DoingDone9] Merge pull request #1 from apache/master
2015-04-11 18:34:17 -07:00