Commit graph

546 commits

Author SHA1 Message Date
Yin Huai 8d6d8a538c [SPARK-7654] [SQL] Move insertInto into reader/writer interface.
This one continues the work of https://github.com/apache/spark/pull/6216.

Author: Yin Huai <yhuai@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #6366 from yhuai/insert and squashes the following commits:

3d717fb [Yin Huai] Use insertInto to handle the casue when table exists and Append is used for saveAsTable.
56d2540 [Yin Huai] Add PreWriteCheck to HiveContext's analyzer.
c636e35 [Yin Huai] Remove unnecessary empty lines.
cf83837 [Yin Huai] Move insertInto to write. Also, remove the partition columns from InsertIntoHadoopFsRelation.
0841a54 [Reynold Xin] Removed experimental tag for deprecated methods.
33ed8ef [Reynold Xin] [SPARK-7654][SQL] Move insertInto into reader/writer interface.

(cherry picked from commit 2b7e63585d)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-23 09:48:30 -07:00
Davies Liu d1515381cb [SPARK-7322, SPARK-7836, SPARK-7822][SQL] DataFrame window function related updates
1. ntile should take an integer as parameter.
2. Added Python API (based on #6364)
3. Update documentation of various DataFrame Python functions.

Author: Davies Liu <davies@databricks.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #6374 from rxin/window-final and squashes the following commits:

69004c7 [Reynold Xin] Style fix.
288cea9 [Reynold Xin] Update documentaiton.
7cb8985 [Reynold Xin] Merge pull request #6364 from davies/window
66092b4 [Davies Liu] update docs
ed73cb4 [Reynold Xin] [SPARK-7322][SQL] Improve DataFrame window function documentation.
ef55132 [Davies Liu] Merge branch 'master' of github.com:apache/spark into window4
8936ade [Davies Liu] fix maxint in python 3
2649358 [Davies Liu] update docs
778e2c0 [Davies Liu] SPARK-7836 and SPARK-7822: Python API of window functions

(cherry picked from commit efe3bfdf49)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-23 08:30:18 -07:00
Liang-Chi Hsieh d6cb044630 [SPARK-7270] [SQL] Consider dynamic partition when inserting into hive table
JIRA: https://issues.apache.org/jira/browse/SPARK-7270

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

Closes #5864 from viirya/dyn_partition_insert and squashes the following commits:

b5627df [Liang-Chi Hsieh] For comments.
3b21e4b [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into dyn_partition_insert
8a4352d [Liang-Chi Hsieh] Consider dynamic partition when inserting into hive table.

(cherry picked from commit 126d7235de)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-22 15:40:32 -07:00
WangTaoTheTonic 40989cea0d [SPARK-7758] [SQL] Override more configs to avoid failure when connect to a postgre sql
https://issues.apache.org/jira/browse/SPARK-7758

When initializing `executionHive`, we only masks
`javax.jdo.option.ConnectionURL` to override metastore location.  However,
other properties that relates to the actual Hive metastore data source are not
masked.  For example, when using Spark SQL with a PostgreSQL backed Hive
metastore, `executionHive` actually tries to use settings read from
`hive-site.xml`, which talks about PostgreSQL, to connect to the temporary
Derby metastore, thus causes error.

To fix this, we need to mask all metastore data source properties.
Specifically, according to the code of [Hive `ObjectStore.getDataSourceProps()`
method] [1], all properties whose name mentions "jdo" and "datanucleus" must be
included.

[1]: https://github.com/apache/hive/blob/release-0.13.1/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java#L288

Have tested using postgre sql as metastore, it worked fine.

Author: WangTaoTheTonic <wangtao111@huawei.com>

Closes #6314 from WangTaoTheTonic/SPARK-7758 and squashes the following commits:

ca7ae7c [WangTaoTheTonic] add comments
86caf2c [WangTaoTheTonic] delete unused import
e4f0feb [WangTaoTheTonic] block more data source related property
92a81fa [WangTaoTheTonic] fix style check
e3e683d [WangTaoTheTonic] override more configs to avoid failuer connecting to postgre sql

(cherry picked from commit 31d5d463e7)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-22 14:44:29 -07:00
Cheng Hao bfaf6a094a [SPARK-7322][SQL] Window functions in DataFrame
This closes #6104.

Author: Cheng Hao <hao.cheng@intel.com>
Author: Reynold Xin <rxin@databricks.com>

Closes #6343 from rxin/window-df and squashes the following commits:

026d587 [Reynold Xin] Address code review feedback.
dc448fe [Reynold Xin] Fixed Hive tests.
9794d9d [Reynold Xin] Moved Java test package.
9331605 [Reynold Xin] Refactored API.
3313e2a [Reynold Xin] Merge pull request #6104 from chenghao-intel/df_window
d625a64 [Cheng Hao] Update the dataframe window API as suggsted
c141fb1 [Cheng Hao] hide all of properties of the WindowFunctionDefinition
3b1865f [Cheng Hao] scaladoc typos
f3fd2d0 [Cheng Hao] polish the unit test
6847825 [Cheng Hao] Add additional analystcs functions
57e3bc0 [Cheng Hao] typos
24a08ec [Cheng Hao] scaladoc
28222ed [Cheng Hao] fix bug of range/row Frame
1d91865 [Cheng Hao] style issue
53f89f2 [Cheng Hao] remove the over from the functions.scala
964c013 [Cheng Hao] add more unit tests and window functions
64e18a7 [Cheng Hao] Add Window Function support for DataFrame

(cherry picked from commit f6f2eeb179)
Signed-off-by: Reynold Xin <rxin@databricks.com>
2015-05-22 01:00:37 -07:00
Yin Huai 96c82515b8 [SPARK-7763] [SPARK-7616] [SQL] Persists partition columns into metastore
Author: Yin Huai <yhuai@databricks.com>
Author: Cheng Lian <lian@databricks.com>

Closes #6285 from liancheng/spark-7763 and squashes the following commits:

bb2829d [Yin Huai] Fix hashCode.
d677f7d [Cheng Lian] Fixes Scala style issue
44b283f [Cheng Lian] Adds test case for SPARK-7616
6733276 [Yin Huai] Fix a bug that potentially causes https://issues.apache.org/jira/browse/SPARK-7616.
6cabf3c [Yin Huai] Update unit test.
7e02910 [Yin Huai] Use metastore partition columns and do not hijack maybePartitionSpec.
e9a03ec [Cheng Lian] Persists partition columns into metastore

(cherry picked from commit 30f3f556f7)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-21 13:51:49 -07:00
scwf e79ecc7dc3 [SQL] [TEST] udf_java_method failed due to jdk version
java.lang.Math.exp(1.0) has different result between jdk versions. so do not use createQueryTest, write a separate test for it.
```
jdk version   	result
1.7.0_11		2.7182818284590455
1.7.0_05        2.7182818284590455
1.7.0_71		2.718281828459045
```

Author: scwf <wangfei1@huawei.com>

Closes #6274 from scwf/java_method and squashes the following commits:

3dd2516 [scwf] address comments
5fa1459 [scwf] style
df46445 [scwf] fix test error
fcb6d22 [scwf] fix udf_java_method

(cherry picked from commit f6c486aa4b)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-21 12:32:10 -07:00
Cheng Lian 70d9839cf3 [SPARK-7749] [SQL] Fixes partition discovery for non-partitioned tables
When no partition columns can be found, we should have an empty `PartitionSpec`, rather than a `PartitionSpec` with empty partition columns.

This PR together with #6285 should fix SPARK-7749.

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

Closes #6287 from liancheng/spark-7749 and squashes the following commits:

a799ff3 [Cheng Lian] Adds test cases for SPARK-7749
c4949be [Cheng Lian] Minor refactoring, and tolerant _TEMPORARY directory name
5aa87ea [Yin Huai] Make parsePartitions more robust.
fc56656 [Cheng Lian] Returns empty PartitionSpec if no partition columns can be inferred
19ae41e [Cheng Lian] Don't list base directory as leaf directory

(cherry picked from commit 8730fbb47b)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-21 10:56:26 -07:00
Cheng Hao f0e4213517 [SPARK-7320] [SQL] [Minor] Move the testData into beforeAll()
Follow up of #6340, to avoid the test report missing once it fails.

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

Closes #6312 from chenghao-intel/rollup_minor and squashes the following commits:

b03a25f [Cheng Hao] simplify the testData instantiation
09b7e8b [Cheng Hao] move the testData into beforeAll()

(cherry picked from commit feb3a9d3f8)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-21 09:28:10 -07:00
Cheng Hao 4fd674336c [SPARK-7320] [SQL] Add Cube / Rollup for dataframe
This is a follow up for #6257, which broke the maven test.

Add cube & rollup for DataFrame
For example:
```scala
testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b"))
testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b"))
```

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

Closes #6304 from chenghao-intel/rollup and squashes the following commits:

04bb1de [Cheng Hao] move the table register/unregister into beforeAll/afterAll
a6069f1 [Cheng Hao] cancel the implicit keyword
ced4b8f [Cheng Hao] remove the unnecessary code changes
9959dfa [Cheng Hao] update the code as comments
e1d88aa [Cheng Hao] update the code as suggested
03bc3d9 [Cheng Hao] Remove the CubedData & RollupedData
5fd62d0 [Cheng Hao] hiden the CubedData & RollupedData
5ffb196 [Cheng Hao] Add Cube / Rollup for dataframe

(cherry picked from commit 42c592adb3)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-20 19:58:33 -07:00
Patrick Wendell 9b37e32c55 Preparing development version 1.4.0-SNAPSHOT 2015-05-20 17:29:00 -07:00
Patrick Wendell 1e458e3553 Preparing Spark release rc-test 2015-05-20 17:28:55 -07:00
pwendell 8d66849862 Preparing development version 1.4.0-SNAPSHOT 2015-05-20 17:26:15 -07:00
pwendell ae29aeaf8e Preparing Spark release rc-test 2015-05-20 17:26:10 -07:00
jenkins 534c787b9f Preparing development version 1.4.0-SNAPSHOT 2015-05-20 16:49:59 -07:00
jenkins 5f4d87f608 Preparing Spark release rc-test 2015-05-20 16:49:54 -07:00
Patrick Wendell 205ed15f29 Preparing development version 1.4.0-SNAPSHOT 2015-05-20 16:30:01 -07:00
Patrick Wendell 09a1c6231e Preparing Spark release rc-test 2015-05-20 16:29:52 -07:00
Patrick Wendell f84bdbce8c Revert "[SPARK-7320] [SQL] Add Cube / Rollup for dataframe"
This reverts commit 10698e1131.
2015-05-20 13:39:22 -07:00
Cheng Hao 10698e1131 [SPARK-7320] [SQL] Add Cube / Rollup for dataframe
Add `cube` & `rollup` for DataFrame
For example:
```scala
testData.rollup($"a" + $"b", $"b").agg(sum($"a" - $"b"))
testData.cube($"a" + $"b", $"b").agg(sum($"a" - $"b"))
```

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

Closes #6257 from chenghao-intel/rollup and squashes the following commits:

7302319 [Cheng Hao] cancel the implicit keyword
a66e38f [Cheng Hao] remove the unnecessary code changes
a2869d4 [Cheng Hao] update the code as comments
c441777 [Cheng Hao] update the code as suggested
84c9564 [Cheng Hao] Remove the CubedData & RollupedData
279584c [Cheng Hao] hiden the CubedData & RollupedData
ef357e1 [Cheng Hao] Add Cube / Rollup for dataframe

(cherry picked from commit 09265ad7c8)
Signed-off-by: Cheng Lian <lian@databricks.com>
2015-05-20 19:12:12 +08:00
scwf 86893390cf [SPARK-7656] [SQL] use CatalystConf in FunctionRegistry
follow up for #5806

Author: scwf <wangfei1@huawei.com>

Closes #6164 from scwf/FunctionRegistry and squashes the following commits:

15e6697 [scwf] use catalogconf in FunctionRegistry

(cherry picked from commit 60336e3bc0)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-19 17:36:33 -07:00
Cheng Hao 62b4c7392a [SPARK-7662] [SQL] Resolve correct names for generator in projection
```
select explode(map(value, key)) from src;
```
Throws exception
```
org.apache.spark.sql.AnalysisException: The number of aliases supplied in the AS clause does not match the number of columns output by the UDTF expected 2 aliases but got _c0 ;
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$class.failAnalysis(CheckAnalysis.scala:38)
at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:43)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate$.org$apache$spark$sql$catalyst$analysis$Analyzer$ResolveGenerate$$makeGeneratorOutput(Analyzer.scala:605)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate$$anonfun$apply$16$$anonfun$22.apply(Analyzer.scala:562)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate$$anonfun$apply$16$$anonfun$22.apply(Analyzer.scala:548)
at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:251)
at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:251)
at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47)
at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:251)
at scala.collection.AbstractTraversable.flatMap(Traversable.scala:105)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate$$anonfun$apply$16.applyOrElse(Analyzer.scala:548)
at org.apache.spark.sql.catalyst.analysis.Analyzer$ResolveGenerate$$anonfun$apply$16.applyOrElse(Analyzer.scala:538)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:222)
```

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

Closes #6178 from chenghao-intel/explode and squashes the following commits:

916fbe9 [Cheng Hao] add more strict rules for TGF alias
5c3f2c5 [Cheng Hao] fix bug in unit test
e1d93ab [Cheng Hao] Add more unit test
19db09e [Cheng Hao] resolve names for generator in projection

(cherry picked from commit bcb1ff8146)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-05-19 15:21:03 -07:00
Patrick Wendell ac3197e1b9 Preparing development version 1.4.1-SNAPSHOT 2015-05-19 09:35:12 +00:00
Patrick Wendell 777a08166f Preparing Spark release v1.4.0-rc1 2015-05-19 09:35:12 +00:00
Patrick Wendell 586ede6b32 Revert "Preparing Spark release v1.4.0-rc1"
This reverts commit 79fb01a3be.
2015-05-19 02:27:14 -07:00
Patrick Wendell e7309ec729 Revert "Preparing development version 1.4.1-SNAPSHOT"
This reverts commit a1d896b85b.
2015-05-19 02:27:07 -07:00
Patrick Wendell a1d896b85b Preparing development version 1.4.1-SNAPSHOT 2015-05-19 07:13:24 +00:00
Patrick Wendell 79fb01a3be Preparing Spark release v1.4.0-rc1 2015-05-19 07:13:24 +00:00
Patrick Wendell b0c63d2413 Revert "Preparing Spark release v1.4.0-rc1"
This reverts commit 38ccef36c1.
2015-05-19 00:10:39 -07:00
Patrick Wendell 198a186ad3 Revert "Preparing development version 1.4.1-SNAPSHOT"
This reverts commit 40190ce226.
2015-05-19 00:10:37 -07:00
Patrick Wendell 40190ce226 Preparing development version 1.4.1-SNAPSHOT 2015-05-19 06:06:41 +00:00
Patrick Wendell 38ccef36c1 Preparing Spark release v1.4.0-rc1 2015-05-19 06:06:40 +00:00
Patrick Wendell 152b0291c0 Revert "Preparing Spark release v1.4.0-rc1"
This reverts commit e8e97e3a63.
2015-05-18 23:06:15 -07:00
Patrick Wendell 4d098bc049 Revert "Preparing development version 1.4.1-SNAPSHOT"
This reverts commit 758ca74bab.
2015-05-18 23:06:13 -07:00
Patrick Wendell be1fc938f8 [HOTFIX]: Java 6 Build Breaks
These were blocking RC1 so I fixed them manually.
2015-05-19 06:01:39 +00:00
Patrick Wendell 758ca74bab Preparing development version 1.4.1-SNAPSHOT 2015-05-19 05:01:11 +00:00
Patrick Wendell e8e97e3a63 Preparing Spark release v1.4.0-rc1 2015-05-19 05:01:11 +00:00
Michael Armbrust f8f23c48a2 [SQL] Fix serializability of ORC table scan
A follow-up to #6244.

Author: Michael Armbrust <michael@databricks.com>

Closes #6247 from marmbrus/fixOrcTests and squashes the following commits:

e39ee1b [Michael Armbrust] [SQL] Fix serializability of ORC table scan

(cherry picked from commit eb4632f282)
Signed-off-by: Yin Huai <yhuai@databricks.com>
2015-05-18 15:24:39 -07:00
Michael Armbrust ba502ab206 [HOTFIX] Fix ORC build break
Fix break caused by merging #6225 and #6194.

Author: Michael Armbrust <michael@databricks.com>

Closes #6244 from marmbrus/fixOrcBuildBreak and squashes the following commits:

b10e47b [Michael Armbrust] [HOTFIX] Fix ORC Build break

(cherry picked from commit fcf90b75cc)
Signed-off-by: Andrew Or <andrew@databricks.com>
2015-05-18 14:04:11 -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
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
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
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
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
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
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
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
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