Commit graph

342 commits

Author SHA1 Message Date
Daoyuan Wang 8161562eab [SPARK-7790] [SQL] date and decimal conversion for dynamic partition key
Author: Daoyuan Wang <daoyuan.wang@intel.com>

Closes #6318 from adrian-wang/dynpart and squashes the following commits:

ad73b61 [Daoyuan Wang] not use sqlTestUtils for try catch because dont have sqlcontext here
6c33b51 [Daoyuan Wang] fix according to liancheng
f0f8074 [Daoyuan Wang] some specific types as dynamic partition
2015-05-27 12:42:13 -07:00
Cheng Lian b463e6d618 [SPARK-7868] [SQL] Ignores _temporary directories in HadoopFsRelation
So that potential partial/corrupted data files left by failed tasks/jobs won't affect normal data scan.

Author: Cheng Lian <lian@databricks.com>

Closes #6411 from liancheng/spark-7868 and squashes the following commits:

273ea36 [Cheng Lian] Ignores _temporary directories
2015-05-26 20:48:56 -07:00
Josh Rosen 0c33c7b4a6 [SPARK-7858] [SQL] Use output schema, not relation schema, for data source input conversion
In `DataSourceStrategy.createPhysicalRDD`, we use the relation schema as the target schema for converting incoming rows into Catalyst rows.  However, we should be using the output schema instead, since our scan might return a subset of the relation's columns.

This patch incorporates #6414 by liancheng, which fixes an issue in `SimpleTestRelation` that prevented this bug from being caught by our old tests:

> In `SimpleTextRelation`, we specified `needsConversion` to `true`, indicating that values produced by this testing relation should be of Scala types, and need to be converted to Catalyst types when necessary. However, we also used `Cast` to convert strings to expected data types. And `Cast` always produces values of Catalyst types, thus no conversion is done at all. This PR makes `SimpleTextRelation` produce Scala values so that data conversion code paths can be properly tested.

Closes #5986.

Author: Josh Rosen <joshrosen@databricks.com>
Author: Cheng Lian <lian@databricks.com>
Author: Cheng Lian <liancheng@users.noreply.github.com>

Closes #6400 from JoshRosen/SPARK-7858 and squashes the following commits:

e71c866 [Josh Rosen] Re-fix bug so that the tests pass again
56b13e5 [Josh Rosen] Add regression test to hadoopFsRelationSuites
2169a0f [Josh Rosen] Remove use of SpecificMutableRow and BufferedIterator
6cd7366 [Josh Rosen] Fix SPARK-7858 by using output types for conversion.
5a00e66 [Josh Rosen] Add assertions in order to reproduce SPARK-7858
8ba195c [Cheng Lian] Merge 9968fba9979287aaa1f141ba18bfb9d4c116a3b3 into 61664732b2
9968fba [Cheng Lian] Tests the data type conversion code paths
2015-05-26 20:24:35 -07:00
Cheng Lian 8af1bf10b7 [SPARK-7842] [SQL] Makes task committing/aborting in InsertIntoHadoopFsRelation more robust
When committing/aborting a write task issued in `InsertIntoHadoopFsRelation`, if an exception is thrown from `OutputWriter.close()`, the committing/aborting process will be interrupted, and leaves messy stuff behind (e.g., the `_temporary` directory created by `FileOutputCommitter`).

This PR makes these two process more robust by catching potential exceptions and falling back to normal task committment/abort.

Author: Cheng Lian <lian@databricks.com>

Closes #6378 from liancheng/spark-7838 and squashes the following commits:

f18253a [Cheng Lian] Makes task committing/aborting in InsertIntoHadoopFsRelation more robust
2015-05-26 00:28:47 +08:00
Yin Huai 2b7e63585d [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.
2015-05-23 09:48:20 -07:00
Davies Liu efe3bfdf49 [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
2015-05-23 08:30:05 -07:00
Liang-Chi Hsieh 126d7235de [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.
2015-05-22 15:39:58 -07:00
Cheng Hao f6f2eeb179 [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
2015-05-22 01:00:16 -07:00
Yin Huai 30f3f556f7 [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
2015-05-21 13:51:40 -07:00
scwf f6c486aa4b [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
2015-05-21 12:31:58 -07:00
Cheng Lian 8730fbb47b [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
2015-05-21 10:56:17 -07:00
Cheng Hao feb3a9d3f8 [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()
2015-05-21 09:28:00 -07:00
Cheng Hao 42c592adb3 [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
2015-05-20 19:58:22 -07:00
Patrick Wendell 6338c40da6 Revert "[SPARK-7320] [SQL] Add Cube / Rollup for dataframe"
This reverts commit 10698e1131.
2015-05-20 13:39:04 -07:00
Cheng Hao 09265ad7c8 [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
2015-05-20 19:09:47 +08:00
Cheng Hao bcb1ff8146 [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
2015-05-19 15:20:46 -07:00
Cheng Lian 9dadf019b9 [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
2015-05-18 12:45:37 -07:00
Wenchen Fan 103c863c2e [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
2015-05-18 12:12:55 -07:00
Zhan Zhang aa31e431fc [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
2015-05-18 12:03:40 -07:00
Reynold Xin 517eb37a85 [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.
2015-05-16 22:01:53 -07:00
Reynold Xin 578bfeeff5 [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.
2015-05-15 22:00:31 -07:00
Cheng Lian fdf5bba35d [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
2015-05-15 16:20:49 +08:00
linweizhong 13e652b61a [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
2015-05-14 00:23:27 -07:00
Reynold Xin e683182c3e [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.
2015-05-13 16:15:31 -07:00
Cheng Lian 7ff16e8abe [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
2015-05-13 11:04:10 -07:00
Cheng Hao 0da254fb29 [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
2015-05-14 00:14:59 +08:00
Cheng Lian aa6ba3f216 [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
2015-05-13 23:40:13 +08:00
Cheng Lian 0595b6de8f [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
2015-05-13 01:32:28 +08:00
Reynold Xin 16696759e9 [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.
2015-05-11 22:06:56 -07:00
Cheng Hao e35d878be3 [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
2015-05-11 19:21:16 -07:00
Jacky Li 6dad76e5eb [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
2015-05-08 15:25:54 -07:00
Michael Armbrust cd1d4110cf [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
2015-05-07 19:36:24 -07:00
Wenchen Fan 35f0173b8f [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
2015-05-07 16:26:49 -07:00
Cheng Hao 074d75d4c8 [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
2015-05-07 12:09:54 -07:00
Yin Huai 7740996700 [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.
2015-05-06 14:48:25 -07:00
Yin Huai f2c47082c3 [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.
2015-05-06 10:43:00 -07:00
Reynold Xin 1fd31ba089 [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 18:59:46 -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
Patrick Wendell beeafcfd6e Revert "[SPARK-5213] [SQL] Pluggable SQL Parser Support"
This reverts commit 3ba5aaab82.
2015-04-30 20:33:36 -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
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
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
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
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
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
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 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