Commit graph

4754 commits

Author SHA1 Message Date
jiangxingbo 9c419698fe [SPARK-18191][CORE] Port RDD API to use commit protocol
## What changes were proposed in this pull request?

This PR port RDD API to use commit protocol, the changes made here:
1. Add new internal helper class that saves an RDD using a Hadoop OutputFormat named `SparkNewHadoopWriter`, it's similar with `SparkHadoopWriter` but uses commit protocol. This class supports the newer `mapreduce` API, instead of the old `mapred` API which is supported by `SparkHadoopWriter`;
2. Rewrite `PairRDDFunctions.saveAsNewAPIHadoopDataset` function, so it uses commit protocol now.

## How was this patch tested?
Exsiting test cases.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15769 from jiangxb1987/rdd-commit.
2016-11-08 09:41:01 -08:00
Wenchen Fan 73feaa30eb [SPARK-18346][SQL] TRUNCATE TABLE should fail if no partition is matched for the given non-partial partition spec
## What changes were proposed in this pull request?

a follow up of https://github.com/apache/spark/pull/15688

## How was this patch tested?

updated test in `DDLSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15805 from cloud-fan/truncate.
2016-11-08 22:28:29 +08:00
jiangxingbo 344dcad701 [SPARK-17868][SQL] Do not use bitmasks during parsing and analysis of CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?

We generate bitmasks for grouping sets during the parsing process, and use these during analysis. These bitmasks are difficult to work with in practice and have lead to numerous bugs. This PR removes these and use actual sets instead, however we still need to generate these offsets for the grouping_id.

This PR does the following works:
1. Replace bitmasks by actual grouping sets durning Parsing/Analysis stage of CUBE/ROLLUP/GROUPING SETS;
2. Add new testsuite `ResolveGroupingAnalyticsSuite` to test the `Analyzer.ResolveGroupingAnalytics` rule directly;
3. Fix a minor bug in `ResolveGroupingAnalytics`.
## How was this patch tested?

By existing test cases, and add new testsuite `ResolveGroupingAnalyticsSuite` to test directly.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15484 from jiangxb1987/group-set.
2016-11-08 15:11:03 +01:00
root c291bd2745 [SPARK-18137][SQL] Fix RewriteDistinctAggregates UnresolvedException when a UDAF has a foldable TypeCheck
## What changes were proposed in this pull request?

In RewriteDistinctAggregates rewrite funtion,after the UDAF's childs are mapped to AttributeRefference, If the UDAF(such as ApproximatePercentile) has a foldable TypeCheck for the input, It will failed because the AttributeRefference is not foldable,then the UDAF is not resolved, and then nullify on the unresolved object will throw a Exception.

In this PR, only map Unfoldable child to AttributeRefference, this can avoid the UDAF's foldable TypeCheck. and then only Expand Unfoldable child, there is no need to Expand a static value(foldable value).

**Before sql result**

> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> org.apache.spark.sql.catalyst.analysis.UnresolvedException: Invalid call to dataType on unresolved object, tree: 'percentile_approx(CAST(src.`key` AS DOUBLE), CAST(0.99999BD AS DOUBLE), 10000)
> at org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute.dataType(unresolved.scala:92)
>     at org.apache.spark.sql.catalyst.optimizer.RewriteDistinctAggregates$.org$apache$spark$sql$catalyst$optimizer$RewriteDistinctAggregates$$nullify(RewriteDistinctAggregates.scala:261)

**After sql result**

> select percentile_approxy(key,0.99999),count(distinct key),sume(distinc key) from src limit 1
> [498.0,309,79136]
## How was this patch tested?

Add a test case in HiveUDFSuit.

Author: root <root@iZbp1gsnrlfzjxh82cz80vZ.(none)>

Closes #15668 from windpiger/RewriteDistinctUDAFUnresolveExcep.
2016-11-08 12:09:32 +01:00
Kazuaki Ishizaki 47731e1865 [SPARK-18207][SQL] Fix a compilation error due to HashExpression.doGenCode
## What changes were proposed in this pull request?

This PR avoids a compilation error due to more than 64KB Java byte code size. This error occur since  generate java code for computing a hash value for a row is too big. This PR fixes this compilation error by splitting a big code chunk into multiple methods by calling `CodegenContext.splitExpression` at `HashExpression.doGenCode`

The test case requires a calculation of hash code for a row that includes 1000 String fields. `HashExpression.doGenCode` generate a lot of Java code for this computation into one function. As a result, the size of the corresponding Java bytecode is more than 64 KB.

Generated code without this PR
````java
/* 027 */   public UnsafeRow apply(InternalRow i) {
/* 028 */     boolean isNull = false;
/* 029 */
/* 030 */     int value1 = 42;
/* 031 */
/* 032 */     boolean isNull2 = i.isNullAt(0);
/* 033 */     UTF8String value2 = isNull2 ? null : (i.getUTF8String(0));
/* 034 */     if (!isNull2) {
/* 035 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value2.getBaseObject(), value2.getBaseOffset(), value2.numBytes(), value1);
/* 036 */     }
/* 037 */
/* 038 */
/* 039 */     boolean isNull3 = i.isNullAt(1);
/* 040 */     UTF8String value3 = isNull3 ? null : (i.getUTF8String(1));
/* 041 */     if (!isNull3) {
/* 042 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value3.getBaseObject(), value3.getBaseOffset(), value3.numBytes(), value1);
/* 043 */     }
/* 044 */
/* 045 */
...
/* 7024 */
/* 7025 */     boolean isNull1001 = i.isNullAt(999);
/* 7026 */     UTF8String value1001 = isNull1001 ? null : (i.getUTF8String(999));
/* 7027 */     if (!isNull1001) {
/* 7028 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1001.getBaseObject(), value1001.getBaseOffset(), value1001.numBytes(), value1);
/* 7029 */     }
/* 7030 */
/* 7031 */
/* 7032 */     boolean isNull1002 = i.isNullAt(1000);
/* 7033 */     UTF8String value1002 = isNull1002 ? null : (i.getUTF8String(1000));
/* 7034 */     if (!isNull1002) {
/* 7035 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1002.getBaseObject(), value1002.getBaseOffset(), value1002.numBytes(), value1);
/* 7036 */     }
````

Generated code with this PR
````java
/* 3807 */   private void apply_249(InternalRow i) {
/* 3808 */
/* 3809 */     boolean isNull998 = i.isNullAt(996);
/* 3810 */     UTF8String value998 = isNull998 ? null : (i.getUTF8String(996));
/* 3811 */     if (!isNull998) {
/* 3812 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value998.getBaseObject(), value998.getBaseOffset(), value998.numBytes(), value1);
/* 3813 */     }
/* 3814 */
/* 3815 */     boolean isNull999 = i.isNullAt(997);
/* 3816 */     UTF8String value999 = isNull999 ? null : (i.getUTF8String(997));
/* 3817 */     if (!isNull999) {
/* 3818 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value999.getBaseObject(), value999.getBaseOffset(), value999.numBytes(), value1);
/* 3819 */     }
/* 3820 */
/* 3821 */     boolean isNull1000 = i.isNullAt(998);
/* 3822 */     UTF8String value1000 = isNull1000 ? null : (i.getUTF8String(998));
/* 3823 */     if (!isNull1000) {
/* 3824 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1000.getBaseObject(), value1000.getBaseOffset(), value1000.numBytes(), value1);
/* 3825 */     }
/* 3826 */
/* 3827 */     boolean isNull1001 = i.isNullAt(999);
/* 3828 */     UTF8String value1001 = isNull1001 ? null : (i.getUTF8String(999));
/* 3829 */     if (!isNull1001) {
/* 3830 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value1001.getBaseObject(), value1001.getBaseOffset(), value1001.numBytes(), value1);
/* 3831 */     }
/* 3832 */
/* 3833 */   }
/* 3834 */
...
/* 4532 */   private void apply_0(InternalRow i) {
/* 4533 */
/* 4534 */     boolean isNull2 = i.isNullAt(0);
/* 4535 */     UTF8String value2 = isNull2 ? null : (i.getUTF8String(0));
/* 4536 */     if (!isNull2) {
/* 4537 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value2.getBaseObject(), value2.getBaseOffset(), value2.numBytes(), value1);
/* 4538 */     }
/* 4539 */
/* 4540 */     boolean isNull3 = i.isNullAt(1);
/* 4541 */     UTF8String value3 = isNull3 ? null : (i.getUTF8String(1));
/* 4542 */     if (!isNull3) {
/* 4543 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value3.getBaseObject(), value3.getBaseOffset(), value3.numBytes(), value1);
/* 4544 */     }
/* 4545 */
/* 4546 */     boolean isNull4 = i.isNullAt(2);
/* 4547 */     UTF8String value4 = isNull4 ? null : (i.getUTF8String(2));
/* 4548 */     if (!isNull4) {
/* 4549 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value4.getBaseObject(), value4.getBaseOffset(), value4.numBytes(), value1);
/* 4550 */     }
/* 4551 */
/* 4552 */     boolean isNull5 = i.isNullAt(3);
/* 4553 */     UTF8String value5 = isNull5 ? null : (i.getUTF8String(3));
/* 4554 */     if (!isNull5) {
/* 4555 */       value1 = org.apache.spark.unsafe.hash.Murmur3_x86_32.hashUnsafeBytes(value5.getBaseObject(), value5.getBaseOffset(), value5.numBytes(), value1);
/* 4556 */     }
/* 4557 */
/* 4558 */   }
...
/* 7344 */   public UnsafeRow apply(InternalRow i) {
/* 7345 */     boolean isNull = false;
/* 7346 */
/* 7347 */     value1 = 42;
/* 7348 */     apply_0(i);
/* 7349 */     apply_1(i);
...
/* 7596 */     apply_248(i);
/* 7597 */     apply_249(i);
/* 7598 */     apply_250(i);
/* 7599 */     apply_251(i);
...
````

## How was this patch tested?

Add a new test in `DataFrameSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #15745 from kiszk/SPARK-18207.
2016-11-08 12:01:54 +01:00
gatorsmile 1da64e1fa0 [SPARK-18217][SQL] Disallow creating permanent views based on temporary views or UDFs
### What changes were proposed in this pull request?
Based on the discussion in [SPARK-18209](https://issues.apache.org/jira/browse/SPARK-18209). It doesn't really make sense to create permanent views based on temporary views or temporary UDFs.

To disallow the supports and issue the exceptions, this PR needs to detect whether a temporary view/UDF is being used when defining a permanent view. Basically, this PR can be split to two sub-tasks:

**Task 1:** detecting a temporary view from the query plan of view definition.
When finding an unresolved temporary view, Analyzer replaces it by a `SubqueryAlias` with the corresponding logical plan, which is stored in an in-memory HashMap. After replacement, it is impossible to detect whether the `SubqueryAlias` is added/generated from a temporary view. Thus, to detect the usage of a temporary view in view definition, this PR traverses the unresolved logical plan and uses the name of an `UnresolvedRelation` to detect whether it is a (global) temporary view.

**Task 2:** detecting a temporary UDF from the query plan of view definition.
Detecting usage of a temporary UDF in view definition is not straightfoward.

First, in the analyzed plan, we are having different forms to represent the functions. More importantly, some classes (e.g., `HiveGenericUDF`) are not accessible from `CreateViewCommand`, which is part of  `sql/core`. Thus, we used the unanalyzed plan `child` of `CreateViewCommand` to detect the usage of a temporary UDF. Because the plan has already been successfully analyzed, we can assume the functions have been defined/registered.

Second, in Spark, the functions have four forms: Spark built-in functions, built-in hash functions, permanent UDFs and temporary UDFs. We do not have any direct way to determine whether a function is temporary or not. Thus, we introduced a function `isTemporaryFunction` in `SessionCatalog`. This function contains the detailed logics to determine whether a function is temporary or not.

### How was this patch tested?
Added test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15764 from gatorsmile/blockTempFromPermViewCreation.
2016-11-07 18:34:21 -08:00
Liwei Lin c1a0c66bd2 [SPARK-18261][STRUCTURED STREAMING] Add statistics to MemorySink for joining
## What changes were proposed in this pull request?

Right now, there is no way to join the output of a memory sink with any table:

> UnsupportedOperationException: LeafNode MemoryPlan must implement statistics

This patch adds statistics to MemorySink, making joining snapshots of memory streams with tables possible.

## How was this patch tested?

Added a test case.

Author: Liwei Lin <lwlin7@gmail.com>

Closes #15786 from lw-lin/memory-sink-stat.
2016-11-07 17:49:24 -08:00
Ryan Blue 9b0593d5e9 [SPARK-18086] Add support for Hive session vars.
## What changes were proposed in this pull request?

This adds support for Hive variables:

* Makes values set via `spark-sql --hivevar name=value` accessible
* Adds `getHiveVar` and `setHiveVar` to the `HiveClient` interface
* Adds a SessionVariables trait for sessions like Hive that support variables (including Hive vars)
* Adds SessionVariables support to variable substitution
* Adds SessionVariables support to the SET command

## How was this patch tested?

* Adds a test to all supported Hive versions for accessing Hive variables
* Adds HiveVariableSubstitutionSuite

Author: Ryan Blue <blue@apache.org>

Closes #15738 from rdblue/SPARK-18086-add-hivevar-support.
2016-11-07 17:36:15 -08:00
hyukjinkwon 3eda05703f [SPARK-18295][SQL] Make to_json function null safe (matching it to from_json)
## What changes were proposed in this pull request?

This PR proposes to match up the behaviour of `to_json` to `from_json` function for null-safety.

Currently, it throws `NullPointException` but this PR fixes this to produce `null` instead.

with the data below:

```scala
import spark.implicits._

val df = Seq(Some(Tuple1(Tuple1(1))), None).toDF("a")
df.show()
```

```
+----+
|   a|
+----+
| [1]|
|null|
+----+
```

the codes below

```scala
import org.apache.spark.sql.functions._

df.select(to_json($"a")).show()
```

produces..

**Before**

throws `NullPointException` as below:

```
java.lang.NullPointerException
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeFields(JacksonGenerator.scala:138)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator$$anonfun$write$1.apply$mcV$sp(JacksonGenerator.scala:194)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.org$apache$spark$sql$catalyst$json$JacksonGenerator$$writeObject(JacksonGenerator.scala:131)
  at org.apache.spark.sql.catalyst.json.JacksonGenerator.write(JacksonGenerator.scala:193)
  at org.apache.spark.sql.catalyst.expressions.StructToJson.eval(jsonExpressions.scala:544)
  at org.apache.spark.sql.catalyst.expressions.Alias.eval(namedExpressions.scala:142)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:48)
  at org.apache.spark.sql.catalyst.expressions.InterpretedProjection.apply(Projection.scala:30)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```

**After**

```
+---------------+
|structtojson(a)|
+---------------+
|       {"_1":1}|
|           null|
+---------------+
```

## How was this patch tested?

Unit test in `JsonExpressionsSuite.scala` and `JsonFunctionsSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15792 from HyukjinKwon/SPARK-18295.
2016-11-07 16:54:40 -08:00
Josh Rosen 3a710b94b0 [SPARK-18236] Reduce duplicate objects in Spark UI and HistoryServer
## What changes were proposed in this pull request?

When profiling heap dumps from the HistoryServer and live Spark web UIs, I found a large amount of memory being wasted on duplicated objects and strings. This patch's changes remove most of this duplication, resulting in over 40% memory savings for some benchmarks.

- **Task metrics** (6441f0624dfcda9c7193a64bfb416a145b5aabdf): previously, every `TaskUIData` object would have its own instances of `InputMetricsUIData`, `OutputMetricsUIData`, `ShuffleReadMetrics`, and `ShuffleWriteMetrics`, but for many tasks these metrics are irrelevant because they're all zero. This patch changes how we construct these metrics in order to re-use a single immutable "empty" value for the cases where these metrics are empty.
- **TaskInfo.accumulables** (ade86db901127bf13c0e0bdc3f09c933a093bb76): Previously, every `TaskInfo` object had its own empty `ListBuffer` for holding updates from named accumulators. Tasks which didn't use named accumulators still paid for the cost of allocating and storing this empty buffer. To avoid this overhead, I changed the `val` with a mutable buffer into a `var` which holds an immutable Scala list, allowing tasks which do not have named accumulator updates to share the same singleton `Nil` object.
- **String.intern() in JSONProtocol** (7e05630e9a78c455db8c8c499f0590c864624e05): in the HistoryServer, executor hostnames and ids are deserialized from JSON, leading to massive duplication of these string objects. By calling `String.intern()` on the deserialized values we can remove all of this duplication. Since Spark now requires Java 7+ we don't have to worry about string interning exhausting the permgen (see http://java-performance.info/string-intern-in-java-6-7-8/).

## How was this patch tested?

I ran

```
sc.parallelize(1 to 100000, 100000).count()
```

in `spark-shell` with event logging enabled, then loaded that event log in the HistoryServer, performed a full GC, and took a heap dump. According to YourKit, the changes in this patch reduced memory consumption by roughly 28 megabytes (or 770k Java objects):

![image](https://cloud.githubusercontent.com/assets/50748/19953276/4f3a28aa-a129-11e6-93df-d7fa91396f66.png)

Here's a table illustrating the drop in objects due to deduplication (the drop is <100k for some objects because some events were dropped from the listener bus; this is a separate, existing bug that I'll address separately after CPU-profiling):

![image](https://cloud.githubusercontent.com/assets/50748/19953290/6a271290-a129-11e6-93ad-b825f1448886.png)

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15743 from JoshRosen/spark-ui-memory-usage.
2016-11-07 16:14:19 -08:00
Kazuaki Ishizaki 19cf208063 [SPARK-17490][SQL] Optimize SerializeFromObject() for a primitive array
## What changes were proposed in this pull request?

Waiting for merging #13680

This PR optimizes `SerializeFromObject()` for an primitive array. This is derived from #13758 to address one of problems by using a simple way in #13758.

The current implementation always generates `GenericArrayData` from `SerializeFromObject()` for any type of an array in a logical plan. This involves a boxing at a constructor of `GenericArrayData` when `SerializedFromObject()` has an primitive array.

This PR enables to generate `UnsafeArrayData` from `SerializeFromObject()` for a primitive array. It can avoid boxing to create an instance of `ArrayData` in the generated code by Catalyst.

This PR also generate `UnsafeArrayData` in a case for `RowEncoder.serializeFor` or `CatalystTypeConverters.createToCatalystConverter`.

Performance improvement of `SerializeFromObject()` is up to 2.0x

```
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            556 /  608         15.1          66.3       1.0X
Double                                        1668 / 1746          5.0         198.8       0.3X

with this PR
Write an array in Dataset:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            352 /  401         23.8          42.0       1.0X
Double                                         821 /  885         10.2          97.9       0.4X
```

Here is an example program that will happen in mllib as described in [SPARK-16070](https://issues.apache.org/jira/browse/SPARK-16070).

```
sparkContext.parallelize(Seq(Array(1, 2)), 1).toDS.map(e => e).show
```

Generated code before applying this PR

``` java
/* 039 */   protected void processNext() throws java.io.IOException {
/* 040 */     while (inputadapter_input.hasNext()) {
/* 041 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 042 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 043 */
/* 044 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 045 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 046 */
/* 047 */       boolean mapelements_isNull = false || false;
/* 048 */       int[] mapelements_value = null;
/* 049 */       if (!mapelements_isNull) {
/* 050 */         Object mapelements_funcResult = null;
/* 051 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 052 */         if (mapelements_funcResult == null) {
/* 053 */           mapelements_isNull = true;
/* 054 */         } else {
/* 055 */           mapelements_value = (int[]) mapelements_funcResult;
/* 056 */         }
/* 057 */
/* 058 */       }
/* 059 */       mapelements_isNull = mapelements_value == null;
/* 060 */
/* 061 */       serializefromobject_argIsNulls[0] = mapelements_isNull;
/* 062 */       serializefromobject_argValue = mapelements_value;
/* 063 */
/* 064 */       boolean serializefromobject_isNull = false;
/* 065 */       for (int idx = 0; idx < 1; idx++) {
/* 066 */         if (serializefromobject_argIsNulls[idx]) { serializefromobject_isNull = true; break; }
/* 067 */       }
/* 068 */
/* 069 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_argValue);
/* 070 */       serializefromobject_holder.reset();
/* 071 */
/* 072 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 073 */
/* 074 */       if (serializefromobject_isNull) {
/* 075 */         serializefromobject_rowWriter.setNullAt(0);
/* 076 */       } else {
/* 077 */         // Remember the current cursor so that we can calculate how many bytes are
/* 078 */         // written later.
/* 079 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 080 */
/* 081 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 082 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 083 */           // grow the global buffer before writing data.
/* 084 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 085 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 086 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 087 */
/* 088 */         } else {
/* 089 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 090 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 091 */
/* 092 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 093 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 094 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 095 */             } else {
/* 096 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 097 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 098 */             }
/* 099 */           }
/* 100 */         }
/* 101 */
/* 102 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 103 */       }
/* 104 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 105 */       append(serializefromobject_result);
/* 106 */       if (shouldStop()) return;
/* 107 */     }
/* 108 */   }
/* 109 */ }
```

Generated code after applying this PR

``` java
/* 035 */   protected void processNext() throws java.io.IOException {
/* 036 */     while (inputadapter_input.hasNext()) {
/* 037 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 038 */       int[] inputadapter_value = (int[])inputadapter_row.get(0, null);
/* 039 */
/* 040 */       Object mapelements_obj = ((Expression) references[0]).eval(null);
/* 041 */       scala.Function1 mapelements_value1 = (scala.Function1) mapelements_obj;
/* 042 */
/* 043 */       boolean mapelements_isNull = false || false;
/* 044 */       int[] mapelements_value = null;
/* 045 */       if (!mapelements_isNull) {
/* 046 */         Object mapelements_funcResult = null;
/* 047 */         mapelements_funcResult = mapelements_value1.apply(inputadapter_value);
/* 048 */         if (mapelements_funcResult == null) {
/* 049 */           mapelements_isNull = true;
/* 050 */         } else {
/* 051 */           mapelements_value = (int[]) mapelements_funcResult;
/* 052 */         }
/* 053 */
/* 054 */       }
/* 055 */       mapelements_isNull = mapelements_value == null;
/* 056 */
/* 057 */       boolean serializefromobject_isNull = mapelements_isNull;
/* 058 */       final ArrayData serializefromobject_value = serializefromobject_isNull ? null : org.apache.spark.sql.catalyst.expressions.UnsafeArrayData.fromPrimitiveArray(mapelements_value);
/* 059 */       serializefromobject_isNull = serializefromobject_value == null;
/* 060 */       serializefromobject_holder.reset();
/* 061 */
/* 062 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 063 */
/* 064 */       if (serializefromobject_isNull) {
/* 065 */         serializefromobject_rowWriter.setNullAt(0);
/* 066 */       } else {
/* 067 */         // Remember the current cursor so that we can calculate how many bytes are
/* 068 */         // written later.
/* 069 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 070 */
/* 071 */         if (serializefromobject_value instanceof UnsafeArrayData) {
/* 072 */           final int serializefromobject_sizeInBytes = ((UnsafeArrayData) serializefromobject_value).getSizeInBytes();
/* 073 */           // grow the global buffer before writing data.
/* 074 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 075 */           ((UnsafeArrayData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 076 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 077 */
/* 078 */         } else {
/* 079 */           final int serializefromobject_numElements = serializefromobject_value.numElements();
/* 080 */           serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 081 */
/* 082 */           for (int serializefromobject_index = 0; serializefromobject_index < serializefromobject_numElements; serializefromobject_index++) {
/* 083 */             if (serializefromobject_value.isNullAt(serializefromobject_index)) {
/* 084 */               serializefromobject_arrayWriter.setNullInt(serializefromobject_index);
/* 085 */             } else {
/* 086 */               final int serializefromobject_element = serializefromobject_value.getInt(serializefromobject_index);
/* 087 */               serializefromobject_arrayWriter.write(serializefromobject_index, serializefromobject_element);
/* 088 */             }
/* 089 */           }
/* 090 */         }
/* 091 */
/* 092 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 093 */       }
/* 094 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 095 */       append(serializefromobject_result);
/* 096 */       if (shouldStop()) return;
/* 097 */     }
/* 098 */   }
/* 099 */ }
```
## How was this patch tested?

Added a test in `DatasetSuite`, `RowEncoderSuite`, and `CatalystTypeConvertersSuite`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #15044 from kiszk/SPARK-17490.
2016-11-08 00:14:57 +01:00
Weiqing Yang 0d95662e7f [SPARK-17108][SQL] Fix BIGINT and INT comparison failure in spark sql
## What changes were proposed in this pull request?

Add a function to check if two integers are compatible when invoking `acceptsType()` in `DataType`.
## How was this patch tested?

Manually.
E.g.

```
    spark.sql("create table t3(a map<bigint, array<string>>)")
    spark.sql("select * from t3 where a[1] is not null")
```

Before:

```
cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
org.apache.spark.sql.AnalysisException: cannot resolve 't.`a`[1]' due to data type mismatch: argument 2 requires bigint type, however, '1' is of int type.; line 1 pos 22
    at org.apache.spark.sql.catalyst.analysis.package$AnalysisErrorAt.failAnalysis(package.scala:42)
    at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:82)
    at org.apache.spark.sql.catalyst.analysis.CheckAnalysis$$anonfun$checkAnalysis$1$$anonfun$apply$2.applyOrElse(CheckAnalysis.scala:74)
at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformUp$1.apply(TreeNode.scala:307)
```

After:
 Run the sql queries above. No errors.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15448 from weiqingy/SPARK_17108.
2016-11-07 21:33:01 +01:00
Liang-Chi Hsieh a814eeac6b [SPARK-18125][SQL] Fix a compilation error in codegen due to splitExpression
## What changes were proposed in this pull request?

As reported in the jira, sometimes the generated java code in codegen will cause compilation error.

Code snippet to test it:

    case class Route(src: String, dest: String, cost: Int)
    case class GroupedRoutes(src: String, dest: String, routes: Seq[Route])

    val ds = sc.parallelize(Array(
      Route("a", "b", 1),
      Route("a", "b", 2),
      Route("a", "c", 2),
      Route("a", "d", 10),
      Route("b", "a", 1),
      Route("b", "a", 5),
      Route("b", "c", 6))
    ).toDF.as[Route]

    val grped = ds.map(r => GroupedRoutes(r.src, r.dest, Seq(r)))
      .groupByKey(r => (r.src, r.dest))
      .reduceGroups { (g1: GroupedRoutes, g2: GroupedRoutes) =>
        GroupedRoutes(g1.src, g1.dest, g1.routes ++ g2.routes)
      }.map(_._2)

The problem here is, in `ReferenceToExpressions` we evaluate the children vars to local variables. Then the result expression is evaluated to use those children variables. In the above case, the result expression code is too long and will be split by `CodegenContext.splitExpression`. So those local variables cannot be accessed and cause compilation error.

## How was this patch tested?

Jenkins tests.

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

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

Closes #15693 from viirya/fix-codege-compilation-error.
2016-11-07 12:18:19 +01:00
gatorsmile 57626a5570 [SPARK-16904][SQL] Removal of Hive Built-in Hash Functions and TestHiveFunctionRegistry
### What changes were proposed in this pull request?

Currently, the Hive built-in `hash` function is not being used in Spark since Spark 2.0. The public interface does not allow users to unregister the Spark built-in functions. Thus, users will never use Hive's built-in `hash` function.

The only exception here is `TestHiveFunctionRegistry`, which allows users to unregister the built-in functions. Thus, we can load Hive's hash function in the test cases. If we disable it, 10+ test cases will fail because the results are different from the Hive golden answer files.

This PR is to remove `hash` from the list of `hiveFunctions` in `HiveSessionCatalog`. It will also remove `TestHiveFunctionRegistry`. This removal makes us easier to remove `TestHiveSessionState` in the future.
### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #14498 from gatorsmile/removeHash.
2016-11-07 01:16:37 -08:00
Reynold Xin 9db06c442c [SPARK-18296][SQL] Use consistent naming for expression test suites
## What changes were proposed in this pull request?
We have an undocumented naming convention to call expression unit tests ExpressionsSuite, and the end-to-end tests FunctionsSuite. It'd be great to make all test suites consistent with this naming convention.

## How was this patch tested?
This is a test-only naming change.

Author: Reynold Xin <rxin@databricks.com>

Closes #15793 from rxin/SPARK-18296.
2016-11-06 22:44:55 -08:00
Reynold Xin 07ac3f09da [SPARK-18167][SQL] Disable flaky hive partition pruning test. 2016-11-06 22:42:05 -08:00
Wenchen Fan 46b2e49993 [SPARK-18173][SQL] data source tables should support truncating partition
## What changes were proposed in this pull request?

Previously `TRUNCATE TABLE ... PARTITION` will always truncate the whole table for data source tables, this PR fixes it and improve `InMemoryCatalog` to make this command work with it.
## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15688 from cloud-fan/truncate.
2016-11-06 18:57:13 -08:00
hyukjinkwon 556a3b7d07 [SPARK-18269][SQL] CSV datasource should read null properly when schema is lager than parsed tokens
## What changes were proposed in this pull request?

Currently, there are the three cases when reading CSV by datasource when it is `PERMISSIVE` parse mode.

- schema == parsed tokens (from each line)
  No problem to cast the value in the tokens to the field in the schema as they are equal.

- schema < parsed tokens (from each line)
  It slices the tokens into the number of fields in schema.

- schema > parsed tokens (from each line)
  It appends `null` into parsed tokens so that safely values can be casted with the schema.

However, when `null` is appended in the third case, we should take `null` into account when casting the values.

In case of `StringType`, it is fine as `UTF8String.fromString(datum)` produces `null` when the input is `null`. Therefore, this case will happen only when schema is explicitly given and schema includes data types that are not `StringType`.

The codes below:

```scala
val path = "/tmp/a"
Seq("1").toDF().write.text(path.getAbsolutePath)
val schema = StructType(
  StructField("a", IntegerType, true) ::
  StructField("b", IntegerType, true) :: Nil)
spark.read.schema(schema).option("header", "false").csv(path).show()
```

prints

**Before**

```
java.lang.NumberFormatException: null
at java.lang.Integer.parseInt(Integer.java:542)
at java.lang.Integer.parseInt(Integer.java:615)
at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272)
at scala.collection.immutable.StringOps.toInt(StringOps.scala:29)
at org.apache.spark.sql.execution.datasources.csv.CSVTypeCast$.castTo(CSVInferSchema.scala:24)
```

**After**

```
+---+----+
|  a|   b|
+---+----+
|  1|null|
+---+----+
```

## How was this patch tested?

Unit test in `CSVSuite.scala` and `CSVTypeCastSuite.scala`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15767 from HyukjinKwon/SPARK-18269.
2016-11-06 18:52:05 -08:00
hyukjinkwon 340f09d100
[SPARK-17854][SQL] rand/randn allows null/long as input seed
## What changes were proposed in this pull request?

This PR proposes `rand`/`randn` accept `null` as input in Scala/SQL and `LongType` as input in SQL. In this case, it treats the values as `0`.

So, this PR includes both changes below:
- `null` support

  It seems MySQL also accepts this.

  ``` sql
  mysql> select rand(0);
  +---------------------+
  | rand(0)             |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)

  mysql> select rand(NULL);
  +---------------------+
  | rand(NULL)          |
  +---------------------+
  | 0.15522042769493574 |
  +---------------------+
  1 row in set (0.00 sec)
  ```

  and also Hive does according to [HIVE-14694](https://issues.apache.org/jira/browse/HIVE-14694)

  So the codes below:

  ``` scala
  spark.range(1).selectExpr("rand(null)").show()
  ```

  prints..

  **Before**

  ```
    Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:444)
  ```

  **After**

  ```
    +-----------------------+
    |rand(CAST(NULL AS INT))|
    +-----------------------+
    |    0.13385709732307427|
    +-----------------------+
  ```
- `LongType` support in SQL.

  In addition, it make the function allows to take `LongType` consistently within Scala/SQL.

  In more details, the codes below:

  ``` scala
  spark.range(1).select(rand(1), rand(1L)).show()
  spark.range(1).selectExpr("rand(1)", "rand(1L)").show()
  ```

  prints..

  **Before**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  Input argument to rand must be an integer literal.;; line 1 pos 0
  org.apache.spark.sql.AnalysisException: Input argument to rand must be an integer literal.;; line 1 pos 0
  at org.apache.spark.sql.catalyst.analysis.FunctionRegistry$$anonfun$5.apply(FunctionRegistry.scala:465)
  at
  ```

  **After**

  ```
  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+

  +------------------+------------------+
  |           rand(1)|           rand(1)|
  +------------------+------------------+
  |0.2630967864682161|0.2630967864682161|
  +------------------+------------------+
  ```
## How was this patch tested?

Unit tests in `DataFrameSuite.scala` and `RandomSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15432 from HyukjinKwon/SPARK-17854.
2016-11-06 14:11:37 +00:00
hyukjinkwon 15d3926884 [MINOR][DOCUMENTATION] Fix some minor descriptions in functions consistently with expressions
## What changes were proposed in this pull request?

This PR proposes to improve documentation and fix some descriptions equivalent to several minor fixes identified in https://github.com/apache/spark/pull/15677

Also, this suggests to change `Note:` and `NOTE:` to `.. note::` consistently with the others which marks up pretty.

## How was this patch tested?

Jenkins tests and manually.

For PySpark, `Note:` and `NOTE:` to `.. note::` make the document as below:

**From**

![2016-11-04 6 53 35](https://cloud.githubusercontent.com/assets/6477701/20002648/42989922-a2c5-11e6-8a32-b73eda49e8c3.png)
![2016-11-04 6 53 45](https://cloud.githubusercontent.com/assets/6477701/20002650/429fb310-a2c5-11e6-926b-e030d7eb0185.png)
![2016-11-04 6 54 11](https://cloud.githubusercontent.com/assets/6477701/20002649/429d570a-a2c5-11e6-9e7e-44090f337e32.png)
![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002647/4297fc74-a2c5-11e6-801a-b89fbcbfca44.png)
![2016-11-04 6 53 51](https://cloud.githubusercontent.com/assets/6477701/20002697/749f5780-a2c5-11e6-835f-022e1f2f82e3.png)

**To**

![2016-11-04 7 03 48](https://cloud.githubusercontent.com/assets/6477701/20002659/4961b504-a2c5-11e6-9ee0-ef0751482f47.png)
![2016-11-04 7 04 03](https://cloud.githubusercontent.com/assets/6477701/20002660/49871d3a-a2c5-11e6-85ea-d9a5d11efeff.png)
![2016-11-04 7 04 28](https://cloud.githubusercontent.com/assets/6477701/20002662/498e0f14-a2c5-11e6-803d-c0c5aeda4153.png)
![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png)
![2016-11-04 7 33 39](https://cloud.githubusercontent.com/assets/6477701/20002731/a76e30d2-a2c5-11e6-993b-0481b8342d6b.png)

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15765 from HyukjinKwon/minor-function-doc.
2016-11-05 21:47:33 -07:00
wangyang fb0d60814a [SPARK-17849][SQL] Fix NPE problem when using grouping sets
## What changes were proposed in this pull request?

Prior this pr, the following code would cause an NPE:
`case class point(a:String, b:String, c:String, d: Int)`

`val data = Seq(
point("1","2","3", 1),
point("4","5","6", 1),
point("7","8","9", 1)
)`
`sc.parallelize(data).toDF().registerTempTable("table")`
`spark.sql("select a, b, c, count(d) from table group by a, b, c GROUPING SETS ((a)) ").show()`

The reason is that when the grouping_id() behavior was changed in #10677, some code (which should be changed) was left out.

Take the above code for example, prior #10677, the bit mask for set "(a)" was `001`, while after #10677 the bit mask was changed to `011`. However, the `nonNullBitmask` was not changed accordingly.

This pr will fix this problem.
## How was this patch tested?

add integration tests

Author: wangyang <wangyang@haizhi.com>

Closes #15416 from yangw1234/groupingid.
2016-11-05 14:32:28 +01:00
hyukjinkwon a87471c830 [SPARK-18192][MINOR][FOLLOWUP] Missed json test in FileStreamSinkSuite
## What changes were proposed in this pull request?

This PR proposes to fix

```diff
 test("FileStreamSink - json") {
-  testFormat(Some("text"))
+  testFormat(Some("json"))
 }
```

`text` is being tested above

```
test("FileStreamSink - text") {
  testFormat(Some("text"))
}
```

## How was this patch tested?

Fixed test in `FileStreamSinkSuite.scala`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15785 from HyukjinKwon/SPARK-18192.
2016-11-05 13:41:35 +01:00
Reynold Xin e2648d3557 [SPARK-18287][SQL] Move hash expressions from misc.scala into hash.scala
## What changes were proposed in this pull request?
As the title suggests, this patch moves hash expressions from misc.scala into hash.scala, to make it easier to find the hash functions. I wanted to do this a while ago but decided to wait for the branch-2.1 cut so the chance of conflicts will be smaller.

## How was this patch tested?
Test cases were also moved out of MiscFunctionsSuite into HashExpressionsSuite.

Author: Reynold Xin <rxin@databricks.com>

Closes #15784 from rxin/SPARK-18287.
2016-11-05 11:29:17 +01:00
Wenchen Fan 95ec4e25bb [SPARK-17183][SPARK-17983][SPARK-18101][SQL] put hive serde table schema to table properties like data source table
## What changes were proposed in this pull request?

For data source tables, we will put its table schema, partition columns, etc. to table properties, to work around some hive metastore issues, e.g. not case-preserving, bad decimal type support, etc.

We should also do this for hive serde tables, to reduce the difference between hive serde tables and data source tables, e.g. column names should be case preserving.
## How was this patch tested?

existing tests, and a new test in `HiveExternalCatalog`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14750 from cloud-fan/minor1.
2016-11-05 00:58:50 -07:00
Burak Yavuz 6e27018157 [SPARK-18260] Make from_json null safe
## What changes were proposed in this pull request?

`from_json` is currently not safe against `null` rows. This PR adds a fix and a regression test for it.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15771 from brkyvz/json_fix.
2016-11-05 00:07:51 -07:00
Reynold Xin 0f7c9e84e0 [SPARK-18189] [SQL] [Followup] Move test from ReplSuite to prevent java.lang.ClassCircularityError
closes #15774
2016-11-04 23:34:29 -07:00
Eric Liang 4cee2ce251 [SPARK-18167] Re-enable the non-flaky parts of SQLQuerySuite
## What changes were proposed in this pull request?

It seems the proximate cause of the test failures is that `cast(str as decimal)` in derby will raise an exception instead of returning NULL. This is a problem since Hive sometimes inserts `__HIVE_DEFAULT_PARTITION__` entries into the partition table as documented here: https://github.com/apache/hive/blob/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreDirectSql.java#L1034

Basically, when these special default partitions are present, partition pruning pushdown using the SQL-direct mode will fail due this cast exception. As commented on in `MetaStoreDirectSql.java` above, this is normally fine since Hive falls back to JDO pruning, however when the pruning predicate contains an unsupported operator such as `>`, that will fail as well.

The only remaining question is why this behavior is nondeterministic. We know that when the test flakes, retries do not help, therefore the cause must be environmental. The current best hypothesis is that some config is different between different jenkins runs, which is why this PR prints out the Spark SQL and Hive confs for the test. The hope is that by comparing the config state for failure vs success we can isolate the root cause of the flakiness.

**Update:** we could not isolate the issue. It does not seem to be due to configuration differences. As such, I'm going to enable the non-flaky parts of the test since we are fairly confident these issues only occur with Derby (which is not used in production).

## How was this patch tested?

N/A

Author: Eric Liang <ekl@databricks.com>

Closes #15725 from ericl/print-confs-out.
2016-11-04 15:54:28 -07:00
Herman van Hovell 550cd56e8b [SPARK-17337][SQL] Do not pushdown predicates through filters with predicate subqueries
## What changes were proposed in this pull request?
The `PushDownPredicate` rule can create a wrong result if we try to push a filter containing a predicate subquery through a project when the subquery and the project share attributes (have the same source).

The current PR fixes this by making sure that we do not push down when there is a predicate subquery that outputs the same attributes as the filters new child plan.

## How was this patch tested?
Added a test to `SubquerySuite`. nsyca has done previous work this. I have taken test from his initial PR.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15761 from hvanhovell/SPARK-17337.
2016-11-04 21:18:13 +01:00
Herman van Hovell aa412c55e3 [SPARK-18259][SQL] Do not capture Throwable in QueryExecution
## What changes were proposed in this pull request?
`QueryExecution.toString` currently captures `java.lang.Throwable`s; this is far from a best practice and can lead to confusing situation or invalid application states. This PR fixes this by only capturing `AnalysisException`s.

## How was this patch tested?
Added a `QueryExecutionSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15760 from hvanhovell/SPARK-18259.
2016-11-03 21:59:59 -07:00
Reynold Xin f22954ad49 [SPARK-18257][SS] Improve error reporting for FileStressSuite
## What changes were proposed in this pull request?
This patch improves error reporting for FileStressSuite, when there is an error in Spark itself (not user code). This works by simply tightening the exception verification, and gets rid of the unnecessary thread for starting the stream.

Also renamed the class FileStreamStressSuite to make it more obvious it is a streaming suite.

## How was this patch tested?
This is a test only change and I manually verified error reporting by injecting some bug in the addBatch code for FileStreamSink.

Author: Reynold Xin <rxin@databricks.com>

Closes #15757 from rxin/SPARK-18257.
2016-11-03 15:30:45 -07:00
福星 16293311cd [SPARK-18237][HIVE] hive.exec.stagingdir have no effect
hive.exec.stagingdir have no effect in spark2.0.1,
Hive confs in hive-site.xml will be loaded in `hadoopConf`, so we should use `hadoopConf` in `InsertIntoHiveTable` instead of `SessionState.conf`

Author: 福星 <fuxing@wacai.com>

Closes #15744 from ClassNotFoundExp/master.
2016-11-03 12:02:01 -07:00
Reynold Xin b17057c0a6 [SPARK-18244][SQL] Rename partitionProviderIsHive -> tracksPartitionsInCatalog
## What changes were proposed in this pull request?
This patch renames partitionProviderIsHive to tracksPartitionsInCatalog, as the old name was too Hive specific.

## How was this patch tested?
Should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15750 from rxin/SPARK-18244.
2016-11-03 11:48:05 -07:00
Cheng Lian 27daf6bcde [SPARK-17949][SQL] A JVM object based aggregate operator
## What changes were proposed in this pull request?

This PR adds a new hash-based aggregate operator named `ObjectHashAggregateExec` that supports `TypedImperativeAggregate`, which may use arbitrary Java objects as aggregation states. Please refer to the [design doc](https://issues.apache.org/jira/secure/attachment/12834260/%5BDesign%20Doc%5D%20Support%20for%20Arbitrary%20Aggregation%20States.pdf) attached in [SPARK-17949](https://issues.apache.org/jira/browse/SPARK-17949) for more details about it.

The major benefit of this operator is better performance when evaluating `TypedImperativeAggregate` functions, especially when there are relatively few distinct groups. Functions like Hive UDAFs, `collect_list`, and `collect_set` may also benefit from this after being migrated to `TypedImperativeAggregate`.

The following feature flag is introduced to enable or disable the new aggregate operator:
- Name: `spark.sql.execution.useObjectHashAggregateExec`
- Default value: `true`

We can also configure the fallback threshold using the following SQL operation:
- Name: `spark.sql.objectHashAggregate.sortBased.fallbackThreshold`
- Default value: 128

  Fallback to sort-based aggregation when more than 128 distinct groups are accumulated in the aggregation hash map. This number is intentionally made small to avoid GC problems since aggregation buffers of this operator may contain arbitrary Java objects.

  This may be improved by implementing size tracking for this operator, but that can be done in a separate PR.

Code generation and size tracking are planned to be implemented in follow-up PRs.
## Benchmark results
### `ObjectHashAggregateExec` vs `SortAggregateExec`

The first benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating `typed_count`, a testing `TypedImperativeAggregate` version of the SQL `count` function.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

object agg v.s. sort agg:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by                        31251 / 31908          3.4         298.0       1.0X
object agg w/ group by w/o fallback           6903 / 7141         15.2          65.8       4.5X
object agg w/ group by w/ fallback          20945 / 21613          5.0         199.7       1.5X
sort agg w/o group by                         4734 / 5463         22.1          45.2       6.6X
object agg w/o group by w/o fallback          4310 / 4529         24.3          41.1       7.3X
```

The next benchmark compares `ObjectHashAggregateExec` and `SortAggregateExec` by evaluating the Spark native version of `percentile_approx`.

Note that `percentile_approx` is so heavy an aggregate function that the bottleneck of the benchmark is evaluating the aggregate function itself rather than the aggregate operator since I couldn't run a large scale benchmark on my laptop. That's why the results are so close and looks counter-intuitive (aggregation with grouping is even faster than that aggregation without grouping).

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

object agg v.s. sort agg:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
sort agg w/ group by                          3418 / 3530          0.6        1630.0       1.0X
object agg w/ group by w/o fallback           3210 / 3314          0.7        1530.7       1.1X
object agg w/ group by w/ fallback            3419 / 3511          0.6        1630.1       1.0X
sort agg w/o group by                         4336 / 4499          0.5        2067.3       0.8X
object agg w/o group by w/o fallback          4271 / 4372          0.5        2036.7       0.8X
```
### Hive UDAF vs Spark AF

This benchmark compares the following two kinds of aggregate functions:
- "hive udaf": Hive implementation of `percentile_approx`, without partial aggregation supports, evaluated using `SortAggregateExec`.
- "spark af": Spark native implementation of `percentile_approx`, with partial aggregation support, evaluated using `ObjectHashAggregateExec`

The performance differences are mostly due to faster implementation and partial aggregation support in the Spark native version of `percentile_approx`.

This benchmark basically shows the performance differences between the worst case, where an aggregate function without partial aggregation support is evaluated using `SortAggregateExec`, and the best case, where a `TypedImperativeAggregate` with partial aggregation support is evaluated using `ObjectHashAggregateExec`.

```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5
Intel(R) Core(TM) i7-4960HQ CPU  2.60GHz

hive udaf vs spark af:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hive udaf w/o group by                        5326 / 5408          0.0       81264.2       1.0X
spark af w/o group by                           93 /  111          0.7        1415.6      57.4X
hive udaf w/ group by                         3804 / 3946          0.0       58050.1       1.4X
spark af w/ group by w/o fallback               71 /   90          0.9        1085.7      74.8X
spark af w/ group by w/ fallback                98 /  111          0.7        1501.6      54.1X
```
### Real world benchmark

We also did a relatively large benchmark using a real world query involving `percentile_approx`:
- Hive UDAF implementation, sort-based aggregation, w/o partial aggregation support

  24.77 minutes
- Native implementation, sort-based aggregation, w/ partial aggregation support

  4.64 minutes
- Native implementation, object hash aggregator, w/ partial aggregation support

  1.80 minutes
## How was this patch tested?

New unit tests and randomized test cases are added in `ObjectAggregateFunctionSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #15590 from liancheng/obj-hash-agg.
2016-11-03 09:34:51 -07:00
gatorsmile 66a99f4a41 [SPARK-17981][SPARK-17957][SQL] Fix Incorrect Nullability Setting to False in FilterExec
### What changes were proposed in this pull request?

When `FilterExec` contains `isNotNull`, which could be inferred and pushed down or users specified, we convert the nullability of the involved columns if the top-layer expression is null-intolerant. However, this is not correct, if the top-layer expression is not a leaf expression, it could still tolerate the null when it has null-tolerant child expressions.

For example, `cast(coalesce(a#5, a#15) as double)`. Although `cast` is a null-intolerant expression, but obviously`coalesce` is null-tolerant. Thus, it could eat null.

When the nullability is wrong, we could generate incorrect results in different cases. For example,

``` Scala
    val df1 = Seq((1, 2), (2, 3)).toDF("a", "b")
    val df2 = Seq((2, 5), (3, 4)).toDF("a", "c")
    val joinedDf = df1.join(df2, Seq("a"), "outer").na.fill(0)
    val df3 = Seq((3, 1)).toDF("a", "d")
    joinedDf.join(df3, "a").show
```

The optimized plan is like

```
Project [a#29, b#30, c#31, d#42]
+- Join Inner, (a#29 = a#41)
   :- Project [cast(coalesce(cast(coalesce(a#5, a#15) as double), 0.0) as int) AS a#29, cast(coalesce(cast(b#6 as double), 0.0) as int) AS b#30, cast(coalesce(cast(c#16 as double), 0.0) as int) AS c#31]
   :  +- Filter isnotnull(cast(coalesce(cast(coalesce(a#5, a#15) as double), 0.0) as int))
   :     +- Join FullOuter, (a#5 = a#15)
   :        :- LocalRelation [a#5, b#6]
   :        +- LocalRelation [a#15, c#16]
   +- LocalRelation [a#41, d#42]
```

Without the fix, it returns an empty result. With the fix, it can return a correct answer:

```
+---+---+---+---+
|  a|  b|  c|  d|
+---+---+---+---+
|  3|  0|  4|  1|
+---+---+---+---+
```
### How was this patch tested?

Added test cases to verify the nullability changes in FilterExec. Also added a test case for verifying the reported incorrect result.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15523 from gatorsmile/nullabilityFilterExec.
2016-11-03 16:35:36 +01:00
Reynold Xin 0ea5d5b24c [SQL] minor - internal doc improvement for InsertIntoTable.
## What changes were proposed in this pull request?
I was reading this part of the code and was really confused by the "partition" parameter. This patch adds some documentation for it to reduce confusion in the future.

I also looked around other logical plans but most of them are either already documented, or pretty self-evident to people that know Spark SQL.

## How was this patch tested?
N/A - doc change only.

Author: Reynold Xin <rxin@databricks.com>

Closes #15749 from rxin/doc-improvement.
2016-11-03 02:45:54 -07:00
Reynold Xin 937af592e6 [SPARK-18219] Move commit protocol API (internal) from sql/core to core module
## What changes were proposed in this pull request?
This patch moves the new commit protocol API from sql/core to core module, so we can use it in the future in the RDD API.

As part of this patch, I also moved the speficiation of the random uuid for the write path out of the commit protocol, and instead pass in a job id.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #15731 from rxin/SPARK-18219.
2016-11-03 02:42:48 -07:00
Daoyuan Wang 96cc1b5675 [SPARK-17122][SQL] support drop current database
## What changes were proposed in this pull request?

In Spark 1.6 and earlier, we can drop the database we are using. In Spark 2.0, native implementation prevent us from dropping current database, which may break some old queries. This PR would re-enable the feature.
## How was this patch tested?

one new unit test in `SessionCatalogSuite`.

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

Closes #15011 from adrian-wang/dropcurrent.
2016-11-03 00:18:03 -07:00
gatorsmile 9ddec8636c [SPARK-18175][SQL] Improve the test case coverage of implicit type casting
### What changes were proposed in this pull request?

So far, we have limited test case coverage about implicit type casting. We need to draw a matrix to find all the possible casting pairs.
- Reorged the existing test cases
- Added all the possible type casting pairs
- Drawed a matrix to show the implicit type casting. The table is very wide. Maybe hard to review. Thus, you also can access the same table via the link to [a google sheet](https://docs.google.com/spreadsheets/d/19PS4ikrs-Yye_mfu-rmIKYGnNe-NmOTt5DDT1fOD3pI/edit?usp=sharing).

SourceType\CastToType | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType | NumericType | IntegralType
------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ | ------------ |  -----------
**ByteType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(3, 0) | ByteType | ByteType
**ShortType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(5, 0) | ShortType | ShortType
**IntegerType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(10, 0) | IntegerType | IntegerType
**LongType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(20, 0) | LongType | LongType
**DoubleType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(30, 15) | DoubleType | IntegerType
**FloatType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(14, 7) | FloatType | IntegerType
**Dec(10, 2)** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | X    | X    | StringType | X    | X    | X    | X    | X    | X    | X    | DecimalType(10, 2) | Dec(10, 2) | IntegerType
**BinaryType** | X    | X    | X    | X    | X    | X    | X    | BinaryType | X    | StringType | X    | X    | X    | X    | X    | X    | X    | X    | X    | X
**BooleanType** | X    | X    | X    | X    | X    | X    | X    | X    | BooleanType | StringType | X    | X    | X    | X    | X    | X    | X    | X    | X    | X
**StringType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | DecimalType(38, 18) | DoubleType | X
**DateType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | X    | X    | X
**TimestampType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | StringType | DateType | TimestampType | X    | X    | X    | X    | X    | X    | X    | X
**ArrayType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | ArrayType* | X    | X    | X    | X    | X    | X    | X
**MapType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | MapType* | X    | X    | X    | X    | X    | X
**StructType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | StructType* | X    | X    | X    | X    | X
**NullType** | ByteType | ShortType | IntegerType | LongType | DoubleType | FloatType | Dec(10, 2) | BinaryType | BooleanType | StringType | DateType | TimestampType | ArrayType | MapType | StructType | NullType | CalendarIntervalType | DecimalType(38, 18) | DoubleType | IntegerType
**CalendarIntervalType** | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | X    | CalendarIntervalType | X    | X    | X
Note: ArrayType\*, MapType\*, StructType\* are castable only when the internal child types also match; otherwise, not castable
### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15691 from gatorsmile/implicitTypeCasting.
2016-11-02 21:01:03 -07:00
hyukjinkwon 7eb2ca8e33 [SPARK-17963][SQL][DOCUMENTATION] Add examples (extend) in each expression and improve documentation
## What changes were proposed in this pull request?

This PR proposes to change the documentation for functions. Please refer the discussion from https://github.com/apache/spark/pull/15513

The changes include
- Re-indent the documentation
- Add examples/arguments in `extended` where the arguments are multiple or specific format (e.g. xml/ json).

For examples, the documentation was updated as below:
### Functions with single line usage

**Before**
- `pow`

  ``` sql
  Usage: pow(x1, x2) - Raise x1 to the power of x2.
  Extended Usage:
  > SELECT pow(2, 3);
   8.0
  ```
- `current_timestamp`

  ``` sql
  Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
  Extended Usage:
  No example for current_timestamp.
  ```

**After**
- `pow`

  ``` sql
  Usage: pow(expr1, expr2) - Raises `expr1` to the power of `expr2`.
  Extended Usage:
      Examples:
        > SELECT pow(2, 3);
         8.0
  ```

- `current_timestamp`

  ``` sql
  Usage: current_timestamp() - Returns the current timestamp at the start of query evaluation.
  Extended Usage:
      No example/argument for current_timestamp.
  ```
### Functions with (already) multiple line usage

**Before**
- `approx_count_distinct`

  ``` sql
  Usage: approx_count_distinct(expr) - Returns the estimated cardinality by HyperLogLog++.
      approx_count_distinct(expr, relativeSD=0.05) - Returns the estimated cardinality by HyperLogLog++
        with relativeSD, the maximum estimation error allowed.

  Extended Usage:
  No example for approx_count_distinct.
  ```
- `percentile_approx`

  ``` sql
  Usage:
        percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
        column `col` at the given percentage. The value of percentage must be between 0.0
        and 1.0. The `accuracy` parameter (default: 10000) is a positive integer literal which
        controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
        better accuracy, `1.0/accuracy` is the relative error of the approximation.

        percentile_approx(col, array(percentage1 [, percentage2]...) [, accuracy]) - Returns the approximate
        percentile array of column `col` at the given percentage array. Each value of the
        percentage array must be between 0.0 and 1.0. The `accuracy` parameter (default: 10000) is
        a positive integer literal which controls approximation accuracy at the cost of memory.
        Higher value of `accuracy` yields better accuracy, `1.0/accuracy` is the relative error of
        the approximation.

  Extended Usage:
  No example for percentile_approx.
  ```

**After**
- `approx_count_distinct`

  ``` sql
  Usage:
      approx_count_distinct(expr[, relativeSD]) - Returns the estimated cardinality by HyperLogLog++.
        `relativeSD` defines the maximum estimation error allowed.

  Extended Usage:
      No example/argument for approx_count_distinct.
  ```

- `percentile_approx`

  ``` sql
  Usage:
      percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
        column `col` at the given percentage. The value of percentage must be between 0.0
        and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
        controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
        better accuracy, `1.0/accuracy` is the relative error of the approximation.
        When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
        In this case, returns the approximate percentile array of column `col` at the given
        percentage array.

  Extended Usage:
      Examples:
        > SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
         [10.0,10.0,10.0]
        > SELECT percentile_approx(10.0, 0.5, 100);
         10.0
  ```
## How was this patch tested?

Manually tested

**When examples are multiple**

``` sql
spark-sql> describe function extended reflect;
Function: reflect
Class: org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection
Usage: reflect(class, method[, arg1[, arg2 ..]]) - Calls a method with reflection.
Extended Usage:
    Examples:
      > SELECT reflect('java.util.UUID', 'randomUUID');
       c33fb387-8500-4bfa-81d2-6e0e3e930df2
      > SELECT reflect('java.util.UUID', 'fromString', 'a5cf6c42-0c85-418f-af6c-3e4e5b1328f2');
       a5cf6c42-0c85-418f-af6c-3e4e5b1328f2
```

**When `Usage` is in single line**

``` sql
spark-sql> describe function extended min;
Function: min
Class: org.apache.spark.sql.catalyst.expressions.aggregate.Min
Usage: min(expr) - Returns the minimum value of `expr`.
Extended Usage:
    No example/argument for min.
```

**When `Usage` is already in multiple lines**

``` sql
spark-sql> describe function extended percentile_approx;
Function: percentile_approx
Class: org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile
Usage:
    percentile_approx(col, percentage [, accuracy]) - Returns the approximate percentile value of numeric
      column `col` at the given percentage. The value of percentage must be between 0.0
      and 1.0. The `accuracy` parameter (default: 10000) is a positive numeric literal which
      controls approximation accuracy at the cost of memory. Higher value of `accuracy` yields
      better accuracy, `1.0/accuracy` is the relative error of the approximation.
      When `percentage` is an array, each value of the percentage array must be between 0.0 and 1.0.
      In this case, returns the approximate percentile array of column `col` at the given
      percentage array.

Extended Usage:
    Examples:
      > SELECT percentile_approx(10.0, array(0.5, 0.4, 0.1), 100);
       [10.0,10.0,10.0]
      > SELECT percentile_approx(10.0, 0.5, 100);
       10.0
```

**When example/argument is missing**

``` sql
spark-sql> describe function extended rank;
Function: rank
Class: org.apache.spark.sql.catalyst.expressions.Rank
Usage:
    rank() - Computes the rank of a value in a group of values. The result is one plus the number
      of rows preceding or equal to the current row in the ordering of the partition. The values
      will produce gaps in the sequence.

Extended Usage:
    No example/argument for rank.
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15677 from HyukjinKwon/SPARK-17963-1.
2016-11-02 20:56:30 -07:00
Wenchen Fan 3a1bc6f478 [SPARK-17470][SQL] unify path for data source table and locationUri for hive serde table
## What changes were proposed in this pull request?

Due to a limitation of hive metastore(table location must be directory path, not file path), we always store `path` for data source table in storage properties, instead of the `locationUri` field. However, we should not expose this difference to `CatalogTable` level, but just treat it as a hack in `HiveExternalCatalog`, like we store table schema of data source table in table properties.

This PR unifies `path` and `locationUri` outside of `HiveExternalCatalog`, both data source table and hive serde table should use the `locationUri` field.

This PR also unifies the way we handle default table location for managed table. Previously, the default table location of hive serde managed table is set by external catalog, but the one of data source table is set by command. After this PR, we follow the hive way and the default table location is always set by external catalog.

For managed non-file-based tables, we will assign a default table location and create an empty directory for it, the table location will be removed when the table is dropped. This is reasonable as metastore doesn't care about whether a table is file-based or not, and an empty table directory has no harm.
For external non-file-based tables, ideally we can omit the table location, but due to a hive metastore issue, we will assign a random location to it, and remove it right after the table is created. See SPARK-15269 for more details. This is fine as it's well isolated in `HiveExternalCatalog`.

To keep the existing behaviour of the `path` option, in this PR we always add the `locationUri` to storage properties using key `path`, before passing storage properties to `DataSource` as data source options.
## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15024 from cloud-fan/path.
2016-11-02 18:05:14 -07:00
Reynold Xin fd90541c35 [SPARK-18214][SQL] Simplify RuntimeReplaceable type coercion
## What changes were proposed in this pull request?
RuntimeReplaceable is used to create aliases for expressions, but the way it deals with type coercion is pretty weird (each expression is responsible for how to handle type coercion, which does not obey the normal implicit type cast rules).

This patch simplifies its handling by allowing the analyzer to traverse into the actual expression of a RuntimeReplaceable.

## How was this patch tested?
- Correctness should be guaranteed by existing unit tests already
- Removed SQLCompatibilityFunctionSuite and moved it sql-compatibility-functions.sql
- Added a new test case in sql-compatibility-functions.sql for verifying explain behavior.

Author: Reynold Xin <rxin@databricks.com>

Closes #15723 from rxin/SPARK-18214.
2016-11-02 15:53:02 -07:00
Xiangrui Meng 02f203107b [SPARK-14393][SQL] values generated by non-deterministic functions shouldn't change after coalesce or union
## What changes were proposed in this pull request?

When a user appended a column using a "nondeterministic" function to a DataFrame, e.g., `rand`, `randn`, and `monotonically_increasing_id`, the expected semantic is the following:
- The value in each row should remain unchanged, as if we materialize the column immediately, regardless of later DataFrame operations.

However, since we use `TaskContext.getPartitionId` to get the partition index from the current thread, the values from nondeterministic columns might change if we call `union` or `coalesce` after. `TaskContext.getPartitionId` returns the partition index of the current Spark task, which might not be the corresponding partition index of the DataFrame where we defined the column.

See the unit tests below or JIRA for examples.

This PR uses the partition index from `RDD.mapPartitionWithIndex` instead of `TaskContext` and fixes the partition initialization logic in whole-stage codegen, normal codegen, and codegen fallback. `initializeStatesForPartition(partitionIndex: Int)` was added to `Projection`, `Nondeterministic`, and `Predicate` (codegen) and initialized right after object creation in `mapPartitionWithIndex`. `newPredicate` now returns a `Predicate` instance rather than a function for proper initialization.
## How was this patch tested?

Unit tests. (Actually I'm not very confident that this PR fixed all issues without introducing new ones ...)

cc: rxin davies

Author: Xiangrui Meng <meng@databricks.com>

Closes #15567 from mengxr/SPARK-14393.
2016-11-02 11:41:49 -07:00
buzhihuojie 742e0fea53 [SPARK-17895] Improve doc for rangeBetween and rowsBetween
## What changes were proposed in this pull request?

Copied description for row and range based frame boundary from https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala#L56

Added examples to show different behavior of rangeBetween and rowsBetween when involving duplicate values.

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

Author: buzhihuojie <ren.weiluo@gmail.com>

Closes #15727 from david-weiluo-ren/improveDocForRangeAndRowsBetween.
2016-11-02 11:36:20 -07:00
Takeshi YAMAMURO 4af0ce2d96 [SPARK-17683][SQL] Support ArrayType in Literal.apply
## What changes were proposed in this pull request?

This pr is to add pattern-matching entries for array data in `Literal.apply`.
## How was this patch tested?

Added tests in `LiteralExpressionSuite`.

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

Closes #15257 from maropu/SPARK-17683.
2016-11-02 11:29:26 -07:00
eyal farago f151bd1af8 [SPARK-16839][SQL] Simplify Struct creation code path
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?
Running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

Modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Author: eyal farago <eyal farago>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: eyal farago <eyal.farago@gmail.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #15718 from hvanhovell/SPARK-16839-2.
2016-11-02 11:12:20 +01:00
Sean Owen 9c8deef64e
[SPARK-18076][CORE][SQL] Fix default Locale used in DateFormat, NumberFormat to Locale.US
## What changes were proposed in this pull request?

Fix `Locale.US` for all usages of `DateFormat`, `NumberFormat`
## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15610 from srowen/SPARK-18076.
2016-11-02 09:39:15 +00:00
CodingCat 85c5424d46 [SPARK-18144][SQL] logging StreamingQueryListener$QueryStartedEvent
## What changes were proposed in this pull request?

The PR fixes the bug that the QueryStartedEvent is not logged

the postToAll() in the original code is actually calling StreamingQueryListenerBus.postToAll() which has no listener at all....we shall post by sparkListenerBus.postToAll(s) and this.postToAll() to trigger local listeners as well as the listeners registered in LiveListenerBus

zsxwing
## How was this patch tested?

The following snapshot shows that QueryStartedEvent has been logged correctly

![image](https://cloud.githubusercontent.com/assets/678008/19821553/007a7d28-9d2d-11e6-9f13-49851559cdaa.png)

Author: CodingCat <zhunansjtu@gmail.com>

Closes #15675 from CodingCat/SPARK-18144.
2016-11-01 23:39:53 -07:00
Reynold Xin a36653c5b7 [SPARK-18192] Support all file formats in structured streaming
## What changes were proposed in this pull request?
This patch adds support for all file formats in structured streaming sinks. This is actually a very small change thanks to all the previous refactoring done using the new internal commit protocol API.

## How was this patch tested?
Updated FileStreamSinkSuite to add test cases for json, text, and parquet.

Author: Reynold Xin <rxin@databricks.com>

Closes #15711 from rxin/SPARK-18192.
2016-11-01 23:37:03 -07:00
Eric Liang abefe2ec42 [SPARK-18183][SPARK-18184] Fix INSERT [INTO|OVERWRITE] TABLE ... PARTITION for Datasource tables
## What changes were proposed in this pull request?

There are a couple issues with the current 2.1 behavior when inserting into Datasource tables with partitions managed by Hive.

(1) OVERWRITE TABLE ... PARTITION will actually overwrite the entire table instead of just the specified partition.
(2) INSERT|OVERWRITE does not work with partitions that have custom locations.

This PR fixes both of these issues for Datasource tables managed by Hive. The behavior for legacy tables or when `manageFilesourcePartitions = false` is unchanged.

There is one other issue in that INSERT OVERWRITE with dynamic partitions will overwrite the entire table instead of just the updated partitions, but this behavior is pretty complicated to implement for Datasource tables. We should address that in a future release.

## How was this patch tested?

Unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15705 from ericl/sc-4942.
2016-11-02 14:15:10 +08:00
frreiss 620da3b482 [SPARK-17475][STREAMING] Delete CRC files if the filesystem doesn't use checksum files
## What changes were proposed in this pull request?

When the metadata logs for various parts of Structured Streaming are stored on non-HDFS filesystems such as NFS or ext4, the HDFSMetadataLog class leaves hidden HDFS-style checksum (CRC) files in the log directory, one file per batch. This PR modifies HDFSMetadataLog so that it detects the use of a filesystem that doesn't use CRC files and removes the CRC files.
## How was this patch tested?

Modified an existing test case in HDFSMetadataLogSuite to check whether HDFSMetadataLog correctly removes CRC files on the local POSIX filesystem.  Ran the entire regression suite.

Author: frreiss <frreiss@us.ibm.com>

Closes #15027 from frreiss/fred-17475.
2016-11-01 23:00:17 -07:00
Michael Allman 1bbf9ff634 [SPARK-17992][SQL] Return all partitions from HiveShim when Hive throws a metastore exception when attempting to fetch partitions by filter
(Link to Jira issue: https://issues.apache.org/jira/browse/SPARK-17992)
## What changes were proposed in this pull request?

We recently added table partition pruning for partitioned Hive tables converted to using `TableFileCatalog`. When the Hive configuration option `hive.metastore.try.direct.sql` is set to `false`, Hive will throw an exception for unsupported filter expressions. For example, attempting to filter on an integer partition column will throw a `org.apache.hadoop.hive.metastore.api.MetaException`.

I discovered this behavior because VideoAmp uses the CDH version of Hive with a Postgresql metastore DB. In this configuration, CDH sets `hive.metastore.try.direct.sql` to `false` by default, and queries that filter on a non-string partition column will fail.

Rather than throw an exception in query planning, this patch catches this exception, logs a warning and returns all table partitions instead. Clients of this method are already expected to handle the possibility that the filters will not be honored.
## How was this patch tested?

A unit test was added.

Author: Michael Allman <michael@videoamp.com>

Closes #15673 from mallman/spark-17992-catch_hive_partition_filter_exception.
2016-11-01 22:20:19 -07:00
Reynold Xin ad4832a9fa [SPARK-18216][SQL] Make Column.expr public
## What changes were proposed in this pull request?
Column.expr is private[sql], but it's an actually really useful field to have for debugging. We should open it up, similar to how we use QueryExecution.

## How was this patch tested?
N/A - this is a simple visibility change.

Author: Reynold Xin <rxin@databricks.com>

Closes #15724 from rxin/SPARK-18216.
2016-11-01 21:20:53 -07:00
Reynold Xin 77a98162d1 [SPARK-18025] Use commit protocol API in structured streaming
## What changes were proposed in this pull request?
This patch adds a new commit protocol implementation ManifestFileCommitProtocol that follows the existing streaming flow, and uses it in FileStreamSink to consolidate the write path in structured streaming with the batch mode write path.

This deletes a lot of code, and would make it trivial to support other functionalities that are currently available in batch but not in streaming, including all file formats and bucketing.

## How was this patch tested?
Should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15710 from rxin/SPARK-18025.
2016-11-01 18:06:57 -07:00
Josh Rosen 6e6298154a [SPARK-17350][SQL] Disable default use of KryoSerializer in Thrift Server
In SPARK-4761 / #3621 (December 2014) we enabled Kryo serialization by default in the Spark Thrift Server. However, I don't think that the original rationale for doing this still holds now that most Spark SQL serialization is now performed via encoders and our UnsafeRow format.

In addition, the use of Kryo as the default serializer can introduce performance problems because the creation of new KryoSerializer instances is expensive and we haven't performed instance-reuse optimizations in several code paths (including DirectTaskResult deserialization).

Given all of this, I propose to revert back to using JavaSerializer as the default serializer in the Thrift Server.

/cc liancheng

Author: Josh Rosen <joshrosen@databricks.com>

Closes #14906 from JoshRosen/disable-kryo-in-thriftserver.
2016-11-01 16:23:47 -07:00
hyukjinkwon 01dd008301 [SPARK-17764][SQL] Add to_json supporting to convert nested struct column to JSON string
## What changes were proposed in this pull request?

This PR proposes to add `to_json` function in contrast with `from_json` in Scala, Java and Python.

It'd be useful if we can convert a same column from/to json. Also, some datasources do not support nested types. If we are forced to save a dataframe into those data sources, we might be able to work around by this function.

The usage is as below:

``` scala
val df = Seq(Tuple1(Tuple1(1))).toDF("a")
df.select(to_json($"a").as("json")).show()
```

``` bash
+--------+
|    json|
+--------+
|{"_1":1}|
+--------+
```
## How was this patch tested?

Unit tests in `JsonFunctionsSuite` and `JsonExpressionsSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15354 from HyukjinKwon/SPARK-17764.
2016-11-01 12:46:41 -07:00
Eric Liang cfac17ee1c [SPARK-18167] Disable flaky SQLQuerySuite test
We now know it's a persistent environmental issue that is causing this test to sometimes fail. One hypothesis is that some configuration is leaked from another suite, and depending on suite ordering this can cause this test to fail.

I am planning on mining the jenkins logs to try to narrow down which suite could be causing this. For now, disable the test.

Author: Eric Liang <ekl@databricks.com>

Closes #15720 from ericl/disable-flaky-test.
2016-11-01 12:35:34 -07:00
jiangxingbo d0272b4365 [SPARK-18148][SQL] Misleading Error Message for Aggregation Without Window/GroupBy
## What changes were proposed in this pull request?

Aggregation Without Window/GroupBy expressions will fail in `checkAnalysis`, the error message is a bit misleading, we should generate a more specific error message for this case.

For example,

```
spark.read.load("/some-data")
  .withColumn("date_dt", to_date($"date"))
  .withColumn("year", year($"date_dt"))
  .withColumn("week", weekofyear($"date_dt"))
  .withColumn("user_count", count($"userId"))
  .withColumn("daily_max_in_week", max($"user_count").over(weeklyWindow))
)
```

creates the following output:

```
org.apache.spark.sql.AnalysisException: expression '`randomColumn`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;
```

In the error message above, `randomColumn` doesn't appear in the query(acturally it's added by function `withColumn`), so the message is not enough for the user to address the problem.
## How was this patch tested?

Manually test

Before:

```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: expression 'tbl.`col`' is neither present in the group by, nor is it an aggregate function. Add to group by or wrap in first() (or first_value) if you don't care which value you get.;;
```

After:

```
scala> spark.sql("select col, count(col) from tbl")
org.apache.spark.sql.AnalysisException: grouping expressions sequence is empty, and 'tbl.`col`' is not an aggregate function. Wrap '(count(col#231L) AS count(col)#239L)' in windowing function(s) or wrap 'tbl.`col`' in first() (or first_value) if you don't care which value you get.;;
```

Also add new test sqls in `group-by.sql`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15672 from jiangxb1987/groupBy-empty.
2016-11-01 11:25:11 -07:00
Ergin Seyfe 8a538c97b5 [SPARK-18189][SQL] Fix serialization issue in KeyValueGroupedDataset
## What changes were proposed in this pull request?
Likewise [DataSet.scala](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala#L156) KeyValueGroupedDataset should mark the queryExecution as transient.

As mentioned in the Jira ticket, without transient we saw serialization issues like

```
Caused by: java.io.NotSerializableException: org.apache.spark.sql.execution.QueryExecution
Serialization stack:
        - object not serializable (class: org.apache.spark.sql.execution.QueryExecution, value: ==
```

## How was this patch tested?

Run the query which is specified in the Jira ticket before and after:
```
val a = spark.createDataFrame(sc.parallelize(Seq((1,2),(3,4)))).as[(Int,Int)]
val grouped = a.groupByKey(
{x:(Int,Int)=>x._1}
)
val mappedGroups = grouped.mapGroups((k,x)=>
{(k,1)}
)
val yyy = sc.broadcast(1)
val last = mappedGroups.rdd.map(xx=>
{ val simpley = yyy.value 1 }
)
```

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #15706 from seyfe/keyvaluegrouped_serialization.
2016-11-01 11:18:42 -07:00
Liwei Lin 8cdf143f4b [SPARK-18103][FOLLOW-UP][SQL][MINOR] Rename MetadataLogFileCatalog to MetadataLogFileIndex
## What changes were proposed in this pull request?

This is a follow-up to https://github.com/apache/spark/pull/15634.

## How was this patch tested?

N/A

Author: Liwei Lin <lwlin7@gmail.com>

Closes #15712 from lw-lin/18103.
2016-11-01 11:17:35 -07:00
Herman van Hovell 0cba535af3 Revert "[SPARK-16839][SQL] redundant aliases after cleanupAliases"
This reverts commit 5441a6269e.
2016-11-01 17:30:37 +01:00
eyal farago 5441a6269e [SPARK-16839][SQL] redundant aliases after cleanupAliases
## What changes were proposed in this pull request?

Simplify struct creation, especially the aspect of `CleanupAliases` which missed some aliases when handling trees created by `CreateStruct`.

This PR includes:

1. A failing test (create struct with nested aliases, some of the aliases survive `CleanupAliases`).
2. A fix that transforms `CreateStruct` into a `CreateNamedStruct` constructor, effectively eliminating `CreateStruct` from all expression trees.
3. A `NamePlaceHolder` used by `CreateStruct` when column names cannot be extracted from unresolved `NamedExpression`.
4. A new Analyzer rule that resolves `NamePlaceHolder` into a string literal once the `NamedExpression` is resolved.
5. `CleanupAliases` code was simplified as it no longer has to deal with `CreateStruct`'s top level columns.

## How was this patch tested?

running all tests-suits in package org.apache.spark.sql, especially including the analysis suite, making sure added test initially fails, after applying suggested fix rerun the entire analysis package successfully.

modified few tests that expected `CreateStruct` which is now transformed into `CreateNamedStruct`.

Credit goes to hvanhovell for assisting with this PR.

Author: eyal farago <eyal farago>
Author: eyal farago <eyal.farago@gmail.com>
Author: Herman van Hovell <hvanhovell@databricks.com>
Author: Eyal Farago <eyal.farago@actimize.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>
Author: eyalfa <eyal.farago@gmail.com>

Closes #14444 from eyalfa/SPARK-16839_redundant_aliases_after_cleanupAliases.
2016-11-01 17:12:20 +01:00
Herman van Hovell f7c145d8ce [SPARK-17996][SQL] Fix unqualified catalog.getFunction(...)
## What changes were proposed in this pull request?

Currently an unqualified `getFunction(..)`call returns a wrong result; the returned function is shown as temporary function without a database. For example:

```
scala> sql("create function fn1 as 'org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs'")
res0: org.apache.spark.sql.DataFrame = []

scala> spark.catalog.getFunction("fn1")
res1: org.apache.spark.sql.catalog.Function = Function[name='fn1', className='org.apache.hadoop.hive.ql.udf.generic.GenericUDFAbs', isTemporary='true']
```

This PR fixes this by adding database information to ExpressionInfo (which is used to store the function information).
## How was this patch tested?

Added more thorough tests to `CatalogSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15542 from hvanhovell/SPARK-17996.
2016-11-01 15:41:45 +01:00
wangzhenhua cb80edc263
[SPARK-18111][SQL] Wrong ApproximatePercentile answer when multiple records have the minimum value
## What changes were proposed in this pull request?

When multiple records have the minimum value, the answer of ApproximatePercentile is wrong.
## How was this patch tested?

add a test case

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15641 from wzhfy/percentile.
2016-11-01 13:11:24 +00:00
Liang-Chi Hsieh dd85eb5448 [SPARK-18107][SQL] Insert overwrite statement runs much slower in spark-sql than it does in hive-client
## What changes were proposed in this pull request?

As reported on the jira, insert overwrite statement runs much slower in Spark, compared with hive-client.

It seems there is a patch [HIVE-11940](ba21806b77) which largely improves insert overwrite performance on Hive. HIVE-11940 is patched after Hive 2.0.0.

Because Spark SQL uses older Hive library, we can not benefit from such improvement.

The reporter verified that there is also a big performance gap between Hive 1.2.1 (520.037 secs) and Hive 2.0.1 (35.975 secs) on insert overwrite execution.

Instead of upgrading to Hive 2.0 in Spark SQL, which might not be a trivial task, this patch provides an approach to delete the partition before asking Hive to load data files into the partition.

Note: The case reported on the jira is insert overwrite to partition. Since `Hive.loadTable` also uses the function to replace files, insert overwrite to table should has the same issue. We can take the same approach to delete the table first. I will upgrade this to include this.
## How was this patch tested?

Jenkins tests.

There are existing tests using insert overwrite statement. Those tests should be passed. I added a new test to specially test insert overwrite into partition.

For performance issue, as I don't have Hive 2.0 environment, this needs the reporter to verify it. Please refer to the jira.

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

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

Closes #15667 from viirya/improve-hive-insertoverwrite.
2016-11-01 00:24:08 -07:00
Reynold Xin d9d1465009 [SPARK-18024][SQL] Introduce an internal commit protocol API
## What changes were proposed in this pull request?
This patch introduces an internal commit protocol API that is used by the batch data source to do write commits. It currently has only one implementation that uses Hadoop MapReduce's OutputCommitter API. In the future, this commit API can be used to unify streaming and batch commits.

## How was this patch tested?
Should be covered by existing write tests.

Author: Reynold Xin <rxin@databricks.com>
Author: Eric Liang <ekl@databricks.com>

Closes #15707 from rxin/SPARK-18024-2.
2016-10-31 22:23:38 -07:00
Eric Liang 7d6c87155c [SPARK-18167][SQL] Retry when the SQLQuerySuite test flakes
## What changes were proposed in this pull request?

This will re-run the flaky test a few times after it fails. This will help determine if it's due to nondeterministic test setup, or because of some environment issue (e.g. leaked config from another test).

cc yhuai

Author: Eric Liang <ekl@databricks.com>

Closes #15708 from ericl/spark-18167-3.
2016-10-31 20:23:22 -07:00
Eric Liang efc254a82b [SPARK-18087][SQL] Optimize insert to not require REPAIR TABLE
## What changes were proposed in this pull request?

When inserting into datasource tables with partitions managed by the hive metastore, we need to notify the metastore of newly added partitions. Previously this was implemented via `msck repair table`, but this is more expensive than needed.

This optimizes the insertion path to add only the updated partitions.
## How was this patch tested?

Existing tests (I verified manually that tests fail if the repair operation is omitted).

Author: Eric Liang <ekl@databricks.com>

Closes #15633 from ericl/spark-18087.
2016-10-31 19:46:55 -07:00
Eric Liang 6633b97b57 [SPARK-18167][SQL] Also log all partitions when the SQLQuerySuite test flakes
## What changes were proposed in this pull request?

One possibility for this test flaking is that we have corrupted the partition schema somehow in the tests, which causes the cast to decimal to fail in the call. This should at least show us the actual partition values.

## How was this patch tested?

Run it locally, it prints out something like `ArrayBuffer(test(partcol=0), test(partcol=1), test(partcol=2), test(partcol=3), test(partcol=4))`.

Author: Eric Liang <ekl@databricks.com>

Closes #15701 from ericl/print-more-info.
2016-10-31 16:26:52 -07:00
Shixiong Zhu de3f87fa71 [SPARK-18030][TESTS] Fix flaky FileStreamSourceSuite by not deleting the files
## What changes were proposed in this pull request?

The test `when schema inference is turned on, should read partition data` should not delete files because the source maybe is listing files. This PR just removes the delete actions since they are not necessary.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15699 from zsxwing/SPARK-18030.
2016-10-31 16:05:17 -07:00
Cheng Lian 8bfc3b7aac [SPARK-17972][SQL] Add Dataset.checkpoint() to truncate large query plans
## What changes were proposed in this pull request?
### Problem

Iterative ML code may easily create query plans that grow exponentially. We found that query planning time also increases exponentially even when all the sub-plan trees are cached.

The following snippet illustrates the problem:

``` scala
(0 until 6).foldLeft(Seq(1, 2, 3).toDS) { (plan, iteration) =>
  println(s"== Iteration $iteration ==")
  val time0 = System.currentTimeMillis()
  val joined = plan.join(plan, "value").join(plan, "value").join(plan, "value").join(plan, "value")
  joined.cache()
  println(s"Query planning takes ${System.currentTimeMillis() - time0} ms")
  joined.as[Int]
}

// == Iteration 0 ==
// Query planning takes 9 ms
// == Iteration 1 ==
// Query planning takes 26 ms
// == Iteration 2 ==
// Query planning takes 53 ms
// == Iteration 3 ==
// Query planning takes 163 ms
// == Iteration 4 ==
// Query planning takes 700 ms
// == Iteration 5 ==
// Query planning takes 3418 ms
```

This is because when building a new Dataset, the new plan is always built upon `QueryExecution.analyzed`, which doesn't leverage existing cached plans.

On the other hand, usually, doing caching every a few iterations may not be the right direction for this problem since caching is too memory consuming (imaging computing connected components over a graph with 50 billion nodes). What we really need here is to truncate both the query plan (to minimize query planning time) and the lineage of the underlying RDD (to avoid stack overflow).
### Changes introduced in this PR

This PR tries to fix this issue by introducing a `checkpoint()` method into `Dataset[T]`, which does exactly the things described above. The following snippet, which is essentially the same as the one above but invokes `checkpoint()` instead of `cache()`, shows the micro benchmark result of this PR:

One key point is that the checkpointed Dataset should preserve the origianl partitioning and ordering information of the original Dataset, so that we can avoid unnecessary shuffling (similar to reading from a pre-bucketed table). This is done by adding `outputPartitioning` and `outputOrdering` to `LogicalRDD` and `RDDScanExec`.
### Micro benchmark

``` scala
spark.sparkContext.setCheckpointDir("/tmp/cp")

(0 until 100).foldLeft(Seq(1, 2, 3).toDS) { (plan, iteration) =>
  println(s"== Iteration $iteration ==")
  val time0 = System.currentTimeMillis()
  val cp = plan.checkpoint()
  cp.count()
  System.out.println(s"Checkpointing takes ${System.currentTimeMillis() - time0} ms")

  val time1 = System.currentTimeMillis()
  val joined = cp.join(cp, "value").join(cp, "value").join(cp, "value").join(cp, "value")
  val result = joined.as[Int]

  println(s"Query planning takes ${System.currentTimeMillis() - time1} ms")
  result
}

// == Iteration 0 ==
// Checkpointing takes 591 ms
// Query planning takes 13 ms
// == Iteration 1 ==
// Checkpointing takes 1605 ms
// Query planning takes 16 ms
// == Iteration 2 ==
// Checkpointing takes 782 ms
// Query planning takes 8 ms
// == Iteration 3 ==
// Checkpointing takes 729 ms
// Query planning takes 10 ms
// == Iteration 4 ==
// Checkpointing takes 734 ms
// Query planning takes 9 ms
// == Iteration 5 ==
// ...
// == Iteration 50 ==
// Checkpointing takes 571 ms
// Query planning takes 7 ms
// == Iteration 51 ==
// Checkpointing takes 548 ms
// Query planning takes 7 ms
// == Iteration 52 ==
// Checkpointing takes 596 ms
// Query planning takes 8 ms
// == Iteration 53 ==
// Checkpointing takes 568 ms
// Query planning takes 7 ms
// ...
```

You may see that although checkpointing is more heavy weight an operation, it always takes roughly the same amount of time to perform both checkpointing and query planning.
### Open question

mengxr mentioned that it would be more convenient if we can make `Dataset.checkpoint()` eager, i.e., always performs a `RDD.count()` after calling `RDD.checkpoint()`. Not quite sure whether this is a universal requirement. Maybe we can add a `eager: Boolean` argument for `Dataset.checkpoint()` to support that.
## How was this patch tested?

Unit test added in `DatasetSuite`.

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

Closes #15651 from liancheng/ds-checkpoint.
2016-10-31 13:39:59 -07:00
Shixiong Zhu d2923f1732 [SPARK-18143][SQL] Ignore Structured Streaming event logs to avoid breaking history server
## What changes were proposed in this pull request?

Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed.

This PR just ignores these logs in ReplayListenerBus because no places use them.
## How was this patch tested?
- Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as `structured-streaming-query-event-logs-2.0.0.txt` and `structured-streaming-query-event-logs-2.0.1.txt`
- The new added test makes sure ReplayListenerBus will skip these bad jsons.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15663 from zsxwing/fix-event-log.
2016-10-31 00:11:33 -07:00
Dongjoon Hyun 8ae2da0b25 [SPARK-18106][SQL] ANALYZE TABLE should raise a ParseException for invalid option
## What changes were proposed in this pull request?

Currently, `ANALYZE TABLE` command accepts `identifier` for option `NOSCAN`. This PR raises a ParseException for unknown option.

**Before**
```scala
scala> sql("create table test(a int)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("analyze table test compute statistics blah")
res1: org.apache.spark.sql.DataFrame = []
```

**After**
```scala
scala> sql("create table test(a int)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("analyze table test compute statistics blah")
org.apache.spark.sql.catalyst.parser.ParseException:
Expected `NOSCAN` instead of `blah`(line 1, pos 0)
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15640 from dongjoon-hyun/SPARK-18106.
2016-10-30 23:24:30 +01:00
Eric Liang 90d3b91f4c [SPARK-18103][SQL] Rename *FileCatalog to *FileIndex
## What changes were proposed in this pull request?

To reduce the number of components in SQL named *Catalog, rename *FileCatalog to *FileIndex. A FileIndex is responsible for returning the list of partitions / files to scan given a filtering expression.

```
TableFileCatalog => CatalogFileIndex
FileCatalog => FileIndex
ListingFileCatalog => InMemoryFileIndex
MetadataLogFileCatalog => MetadataLogFileIndex
PrunedTableFileCatalog => PrunedInMemoryFileIndex
```

cc yhuai marmbrus

## How was this patch tested?

N/A

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15634 from ericl/rename-file-provider.
2016-10-30 13:14:45 -07:00
Eric Liang 3ad99f1664 [SPARK-18146][SQL] Avoid using Union to chain together create table and repair partition commands
## What changes were proposed in this pull request?

The behavior of union is not well defined here. It is safer to explicitly execute these commands in order. The other use of `Union` in this way will be removed by https://github.com/apache/spark/pull/15633

## How was this patch tested?

Existing tests.

cc yhuai cloud-fan

Author: Eric Liang <ekhliang@gmail.com>
Author: Eric Liang <ekl@databricks.com>

Closes #15665 from ericl/spark-18146.
2016-10-30 20:27:38 +08:00
Eric Liang d2d438d1d5 [SPARK-18167][SQL] Add debug code for SQLQuerySuite flakiness when metastore partition pruning is enabled
## What changes were proposed in this pull request?

org.apache.spark.sql.hive.execution.SQLQuerySuite is flaking when hive partition pruning is enabled.
Based on the stack traces, it seems to be an old issue where Hive fails to cast a numeric partition column ("Invalid character string format for type DECIMAL"). There are two possibilities here: either we are somehow corrupting the partition table to have non-decimal values in that column, or there is a transient issue with Derby.

This PR logs the result of the retry when this exception is encountered, so we can confirm what is going on.

## How was this patch tested?

n/a

cc yhuai

Author: Eric Liang <ekl@databricks.com>

Closes #15676 from ericl/spark-18167.
2016-10-29 06:49:57 +02:00
Shixiong Zhu 59cccbda48 [SPARK-18164][SQL] ForeachSink should fail the Spark job if process throws exception
## What changes were proposed in this pull request?

Fixed the issue that ForeachSink didn't rethrow the exception.

## How was this patch tested?

The fixed unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15674 from zsxwing/foreach-sink-error.
2016-10-28 20:14:38 -07:00
Sunitha Kambhampati ab5f938bc7 [SPARK-18121][SQL] Unable to query global temp views when hive support is enabled
## What changes were proposed in this pull request?

Issue:
Querying on a global temp view throws Table or view not found exception.

Fix:
Update the lookupRelation in HiveSessionCatalog to check for global temp views similar to the SessionCatalog.lookupRelation.

Before fix:
Querying on a global temp view ( for. e.g.:  select * from global_temp.v1)  throws Table or view not found exception

After fix:
Query succeeds and returns the right result.

## How was this patch tested?
- Two unit tests are added to check for global temp view for the code path when hive support is enabled.
- Regression unit tests were run successfully. ( build/sbt -Phive hive/test, build/sbt sql/test, build/sbt catalyst/test)

Author: Sunitha Kambhampati <skambha@us.ibm.com>

Closes #15649 from skambha/lookuprelationChanges.
2016-10-28 08:39:02 +08:00
Eric Liang ccb1154304 [SPARK-17970][SQL] store partition spec in metastore for data source table
## What changes were proposed in this pull request?

We should follow hive table and also store partition spec in metastore for data source table.
This brings 2 benefits:

1. It's more flexible to manage the table data files, as users can use `ADD PARTITION`, `DROP PARTITION` and `RENAME PARTITION`
2. We don't need to cache all file status for data source table anymore.

## How was this patch tested?

existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15515 from cloud-fan/partition.
2016-10-27 14:22:30 -07:00
Shixiong Zhu 79fd0cc058 [SPARK-16963][SQL] Fix test "StreamExecution metadata garbage collection"
## What changes were proposed in this pull request?

A follow up PR for #14553 to fix the flaky test. It's flaky because the file list API doesn't guarantee any order of the return list.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15661 from zsxwing/fix-StreamingQuerySuite.
2016-10-27 12:32:58 -07:00
VinceShieh 0b076d4cb6 [SPARK-17219][ML] enhanced NaN value handling in Bucketizer
## What changes were proposed in this pull request?

This PR is an enhancement of PR with commit ID:57dc326bd00cf0a49da971e9c573c48ae28acaa2.
NaN is a special type of value which is commonly seen as invalid. But We find that there are certain cases where NaN are also valuable, thus need special handling. We provided user when dealing NaN values with 3 options, to either reserve an extra bucket for NaN values, or remove the NaN values, or report an error, by setting handleNaN "keep", "skip", or "error"(default) respectively.

'''Before:
val bucketizer: Bucketizer = new Bucketizer()
          .setInputCol("feature")
          .setOutputCol("result")
          .setSplits(splits)
'''After:
val bucketizer: Bucketizer = new Bucketizer()
          .setInputCol("feature")
          .setOutputCol("result")
          .setSplits(splits)
          .setHandleNaN("keep")

## How was this patch tested?
Tests added in QuantileDiscretizerSuite, BucketizerSuite and DataFrameStatSuite

Signed-off-by: VinceShieh <vincent.xieintel.com>

Author: VinceShieh <vincent.xie@intel.com>
Author: Vincent Xie <vincent.xie@intel.com>
Author: Joseph K. Bradley <joseph@databricks.com>

Closes #15428 from VinceShieh/spark-17219_followup.
2016-10-27 11:52:15 -07:00
Felix Cheung 44c8bfda79 [SQL][DOC] updating doc for JSON source to link to jsonlines.org
## What changes were proposed in this pull request?

API and programming guide doc changes for Scala, Python and R.

## How was this patch tested?

manual test

Author: Felix Cheung <felixcheung_m@hotmail.com>

Closes #15629 from felixcheung/jsondoc.
2016-10-26 23:06:11 -07:00
Dilip Biswal dd4f088c1d [SPARK-18009][SQL] Fix ClassCastException while calling toLocalIterator() on dataframe produced by RunnableCommand
## What changes were proposed in this pull request?
A short code snippet that uses toLocalIterator() on a dataframe produced by a RunnableCommand
reproduces the problem. toLocalIterator() is called by thriftserver when
`spark.sql.thriftServer.incrementalCollect`is set to handle queries producing large result
set.

**Before**
```SQL
scala> spark.sql("show databases")
res0: org.apache.spark.sql.DataFrame = [databaseName: string]

scala> res0.toLocalIterator()
16/10/26 03:00:24 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.GenericInternalRow cannot be cast to org.apache.spark.sql.catalyst.expressions.UnsafeRow
```

**After**
```SQL
scala> spark.sql("drop database databases")
res30: org.apache.spark.sql.DataFrame = []

scala> spark.sql("show databases")
res31: org.apache.spark.sql.DataFrame = [databaseName: string]

scala> res31.toLocalIterator().asScala foreach println
[default]
[parquet]
```
## How was this patch tested?
Added a test in DDLSuite

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

Closes #15642 from dilipbiswal/SPARK-18009.
2016-10-27 13:12:14 +08:00
ALeksander Eskilson f1aeed8b02 [SPARK-17770][CATALYST] making ObjectType public
## What changes were proposed in this pull request?

In order to facilitate the writing of additional Encoders, I proposed opening up the ObjectType SQL DataType. This DataType is used extensively in the JavaBean Encoder, but would also be useful in writing other custom encoders.

As mentioned by marmbrus, it is understood that the Expressions API is subject to potential change.

## How was this patch tested?

The change only affects the visibility of the ObjectType class, and the existing SQL test suite still runs without error.

Author: ALeksander Eskilson <alek.eskilson@cerner.com>

Closes #15453 from bdrillard/master.
2016-10-26 18:03:31 -07:00
frreiss 5b27598ff5 [SPARK-16963][STREAMING][SQL] Changes to Source trait and related implementation classes
## What changes were proposed in this pull request?

This PR contains changes to the Source trait such that the scheduler can notify data sources when it is safe to discard buffered data. Summary of changes:
* Added a method `commit(end: Offset)` that tells the Source that is OK to discard all offsets up `end`, inclusive.
* Changed the semantics of a `None` value for the `getBatch` method to mean "from the very beginning of the stream"; as opposed to "all data present in the Source's buffer".
* Added notes that the upper layers of the system will never call `getBatch` with a start value less than the last value passed to `commit`.
* Added a `lastCommittedOffset` method to allow the scheduler to query the status of each Source on restart. This addition is not strictly necessary, but it seemed like a good idea -- Sources will be maintaining their own persistent state, and there may be bugs in the checkpointing code.
* The scheduler in `StreamExecution.scala` now calls `commit` on its stream sources after marking each batch as complete in its checkpoint.
* `MemoryStream` now cleans committed batches out of its internal buffer.
* `TextSocketSource` now cleans committed batches from its internal buffer.

## How was this patch tested?
Existing regression tests already exercise the new code.

Author: frreiss <frreiss@us.ibm.com>

Closes #14553 from frreiss/fred-16963.
2016-10-26 17:33:08 -07:00
jiangxingbo 5b7d403c18 [SPARK-18094][SQL][TESTS] Move group analytics test cases from SQLQuerySuite into a query file test.
## What changes were proposed in this pull request?

Currently we have several test cases for group analytics(ROLLUP/CUBE/GROUPING SETS) in `SQLQuerySuite`, should better move them into a query file test.
The following test cases are moved to `group-analytics.sql`:
```
test("rollup")
test("grouping sets when aggregate functions containing groupBy columns")
test("cube")
test("grouping sets")
test("grouping and grouping_id")
test("grouping and grouping_id in having")
test("grouping and grouping_id in sort")
```

This is followup work of #15582

## How was this patch tested?

Modified query file `group-analytics.sql`, which will be tested by `SQLQueryTestSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15624 from jiangxb1987/group-analytics-test.
2016-10-26 23:51:16 +02:00
jiangxingbo fa7d9d7082 [SPARK-18063][SQL] Failed to infer constraints over multiple aliases
## What changes were proposed in this pull request?

The `UnaryNode.getAliasedConstraints` function fails to replace all expressions by their alias where constraints contains more than one expression to be replaced.
For example:
```
val tr = LocalRelation('a.int, 'b.string, 'c.int)
val multiAlias = tr.where('a === 'c + 10).select('a.as('x), 'c.as('y))
multiAlias.analyze.constraints
```
currently outputs:
```
ExpressionSet(Seq(
    IsNotNull(resolveColumn(multiAlias.analyze, "x")),
    IsNotNull(resolveColumn(multiAlias.analyze, "y"))
)
```
The constraint `resolveColumn(multiAlias.analyze, "x") === resolveColumn(multiAlias.analyze, "y") + 10)` is missing.

## How was this patch tested?

Add new test cases in `ConstraintPropagationSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15597 from jiangxb1987/alias-constraints.
2016-10-26 20:12:20 +02:00
Shixiong Zhu 7ac70e7ba8 [SPARK-13747][SQL] Fix concurrent executions in ForkJoinPool for SQL
## What changes were proposed in this pull request?

Calling `Await.result` will allow other tasks to be run on the same thread when using ForkJoinPool. However, SQL uses a `ThreadLocal` execution id to trace Spark jobs launched by a query, which doesn't work perfectly in ForkJoinPool.

This PR just uses `Awaitable.result` instead to  prevent ForkJoinPool from running other tasks in the current waiting thread.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15520 from zsxwing/SPARK-13747.
2016-10-26 10:36:36 -07:00
Mark Grover 4bee954079 [SPARK-18093][SQL] Fix default value test in SQLConfSuite to work rega…
…rdless of warehouse dir's existence

## What changes were proposed in this pull request?
Appending a trailing slash, if there already isn't one for the
sake comparison of the two paths. It doesn't take away from
the essence of the check, but removes any potential mismatch
due to lack of trailing slash.

## How was this patch tested?
Ran unit tests and they passed.

Author: Mark Grover <mark@apache.org>

Closes #15623 from markgrover/spark-18093.
2016-10-26 09:07:30 -07:00
jiangxingbo 3c023570b2 [SPARK-17733][SQL] InferFiltersFromConstraints rule never terminates for query
## What changes were proposed in this pull request?

The function `QueryPlan.inferAdditionalConstraints` and `UnaryNode.getAliasedConstraints` can produce a non-converging set of constraints for recursive functions. For instance, if we have two constraints of the form(where a is an alias):
`a = b, a = f(b, c)`
Applying both these rules in the next iteration would infer:
`f(b, c) = f(f(b, c), c)`
This process repeated, the iteration won't converge and the set of constraints will grow larger and larger until OOM.

~~To fix this problem, we collect alias from expressions and skip infer constraints if we are to transform an `Expression` to another which contains it.~~
To fix this problem, we apply additional check in `inferAdditionalConstraints`, when it's possible to generate recursive constraints, we skip generate that.

## How was this patch tested?

Add new testcase in `SQLQuerySuite`/`InferFiltersFromConstraintsSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15319 from jiangxb1987/constraints.
2016-10-26 17:09:48 +02:00
Sean Owen 6c7d094ec4
[SPARK-18022][SQL] java.lang.NullPointerException instead of real exception when saving DF to MySQL
## What changes were proposed in this pull request?

On null next exception in JDBC, don't init it as cause or suppressed

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #15599 from srowen/SPARK-18022.
2016-10-26 14:19:40 +02:00
gatorsmile 93b8ad184a [SPARK-17693][SQL] Fixed Insert Failure To Data Source Tables when the Schema has the Comment Field
### What changes were proposed in this pull request?
```SQL
CREATE TABLE tab1(col1 int COMMENT 'a', col2 int) USING parquet
INSERT INTO TABLE tab1 SELECT 1, 2
```
The insert attempt will fail if the target table has a column with comments. The error is strange to the external users:
```
assertion failed: No plan for InsertIntoTable Relation[col1#15,col2#16] parquet, false, false
+- Project [1 AS col1#19, 2 AS col2#20]
   +- OneRowRelation$
```

This PR is to fix the above bug by checking the metadata when comparing the schema between the table and the query. If not matched, we also copy the metadata. This is an alternative to https://github.com/apache/spark/pull/15266

### How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15615 from gatorsmile/insertDataSourceTableWithCommentSolution2.
2016-10-26 00:38:34 -07:00
Wenchen Fan a21791e316 [SPARK-18070][SQL] binary operator should not consider nullability when comparing input types
## What changes were proposed in this pull request?

Binary operator requires its inputs to be of same type, but it should not consider nullability, e.g. `EqualTo` should be able to compare an element-nullable array and an element-non-nullable array.

## How was this patch tested?

a regression test in `DataFrameSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15606 from cloud-fan/type-bug.
2016-10-25 12:08:17 -07:00
Wenchen Fan 6f31833dbe [SPARK-18026][SQL] should not always lowercase partition columns of partition spec in parser
## What changes were proposed in this pull request?

Currently we always lowercase the partition columns of partition spec in parser, with the assumption that table partition columns are always lowercased.

However, this is not true for data source tables, which are case preserving. It's safe for now because data source tables don't store partition spec in metastore and don't support `ADD PARTITION`, `DROP PARTITION`, `RENAME PARTITION`, but we should make our code future-proof.

This PR makes partition spec case preserving at parser, and improve the `PreprocessTableInsertion` analyzer rule to normalize the partition columns in partition spec, w.r.t. the table partition columns.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15566 from cloud-fan/partition-spec.
2016-10-25 15:00:33 +08:00
gatorsmile d479c52622 [SPARK-17409][SQL][FOLLOW-UP] Do Not Optimize Query in CTAS More Than Once
### What changes were proposed in this pull request?
This follow-up PR is for addressing the [comment](https://github.com/apache/spark/pull/15048).

We added two test cases based on the suggestion from yhuai . One is a new test case using the `saveAsTable` API to create a data source table. Another is for CTAS on Hive serde table.

Note: No need to backport this PR to 2.0. Will submit a new PR to backport the whole fix with new test cases to Spark 2.0

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15459 from gatorsmile/ctasOptimizedTestCases.
2016-10-25 10:47:11 +08:00
Wenchen Fan 84a3399908 [SPARK-18028][SQL] simplify TableFileCatalog
## What changes were proposed in this pull request?

Simplify/cleanup TableFileCatalog:

1. pass a `CatalogTable` instead of `databaseName` and `tableName` into `TableFileCatalog`, so that we don't need to fetch table metadata from metastore again
2. In `TableFileCatalog.filterPartitions0`, DO NOT set `PartitioningAwareFileCatalog.BASE_PATH_PARAM`. According to the [classdoc](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala#L189-L209), the default value of `basePath` already satisfies our need. What's more, if we set this parameter, we may break the case 2 which is metioned in the classdoc.
3. add `equals` and `hashCode` to `TableFileCatalog`
4. add `SessionCatalog.listPartitionsByFilter` which handles case sensitivity.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15568 from cloud-fan/table-file-catalog.
2016-10-25 08:42:21 +08:00
Tathagata Das 407c3cedf2 [SPARK-17624][SQL][STREAMING][TEST] Fixed flaky StateStoreSuite.maintenance
## What changes were proposed in this pull request?

The reason for the flakiness was follows. The test starts the maintenance background thread, and then writes 20 versions of the state store. The maintenance thread is expected to create snapshots in the middle, and clean up old files that are not needed any more. The earliest delta file (1.delta) is expected to be deleted as snapshots will ensure that the earliest delta would not be needed.

However, the default configuration for the maintenance thread is to retain files such that last 2 versions can be recovered, and delete the rest. Now while generating the versions, the maintenance thread can kick in and create snapshots anywhere between version 10 and 20 (at least 10 deltas needed for snapshot). Then later it will choose to retain only version 20 and 19 (last 2). There are two cases.

- Common case: One of the version between 10 and 19 gets snapshotted. Then recovering versions 19 and 20 just needs 19.snapshot and 20.delta, so 1.delta gets deleted.

- Uncommon case (reason for flakiness): Only version 20 gets snapshotted. Then recovering versoin 20 requires 20.snapshot, and recovering version 19 all the previous 19...1.delta. So 1.delta does not get deleted.

This PR rearranges the checks such that it create 20 versions, and then waits that there is at least one snapshot, then creates another 20. This will ensure that the latest 2 versions cannot require anything older than the first snapshot generated, and therefore will 1.delta will be deleted.

In addition, I have added more logs, and comments that I felt would help future debugging and understanding what is going on.

## How was this patch tested?

Ran the StateStoreSuite > 6K times in a heavily loaded machine (10 instances of tests running in parallel). No failures.

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

Closes #15592 from tdas/SPARK-17624.
2016-10-24 17:21:16 -07:00
Sean Owen 4ecbe1b92f
[SPARK-17810][SQL] Default spark.sql.warehouse.dir is relative to local FS but can resolve as HDFS path
## What changes were proposed in this pull request?

Always resolve spark.sql.warehouse.dir as a local path, and as relative to working dir not home dir

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15382 from srowen/SPARK-17810.
2016-10-24 10:44:45 +01:00
CodingCat a81fba048f [SPARK-18058][SQL] Comparing column types ignoring Nullability in Union and SetOperation
## What changes were proposed in this pull request?

The PR tries to fix [SPARK-18058](https://issues.apache.org/jira/browse/SPARK-18058) which refers to a bug that the column types are compared with the extra care about Nullability in Union and SetOperation.

This PR converts the columns types by setting all fields as nullable before comparison

## How was this patch tested?

regular unit test cases

Author: CodingCat <zhunansjtu@gmail.com>

Closes #15595 from CodingCat/SPARK-18058.
2016-10-23 19:42:11 +02:00
jiangxingbo b158256c2e [SPARK-18045][SQL][TESTS] Move HiveDataFrameAnalyticsSuite to package sql
## What changes were proposed in this pull request?

The testsuite `HiveDataFrameAnalyticsSuite` has nothing to do with HIVE, we should move it to package `sql`.
The original test cases in that suite are splited into two existing testsuites: `DataFrameAggregateSuite` tests for the functions and ~~`SQLQuerySuite`~~`SQLQueryTestSuite` tests for the SQL statements.

## How was this patch tested?
~~Modified `SQLQuerySuite` in package `sql`.~~
Add query file for `SQLQueryTestSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15582 from jiangxb1987/group-analytics-test.
2016-10-23 13:28:35 +02:00
Tejas Patil 21c7539a52 [SPARK-18038][SQL] Move output partitioning definition from UnaryNodeExec to its children
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-18038

This was a suggestion by rxin over one of the dev list discussion : http://apache-spark-developers-list.1001551.n3.nabble.com/Project-not-preserving-child-partitioning-td19417.html

His words:

>> It would be better (safer) to move the output partitioning definition into each of the operator and remove it from UnaryExecNode.

With this PR, following is the output partitioning and ordering for all the impls of `UnaryExecNode`.

UnaryExecNode's impl | outputPartitioning | outputOrdering | comment
------------ | ------------- | ------------ | ------------
AppendColumnsExec | child's | Nil | child's ordering can be used
AppendColumnsWithObjectExec | child's | Nil | child's ordering can be used
BroadcastExchangeExec | BroadcastPartitioning | Nil | -
CoalesceExec | UnknownPartitioning | Nil | -
CollectLimitExec | SinglePartition | Nil | -
DebugExec | child's | Nil | child's ordering can be used
DeserializeToObjectExec | child's | Nil | child's ordering can be used
ExpandExec | UnknownPartitioning | Nil | -
FilterExec | child's | child's | -
FlatMapGroupsInRExec | child's | Nil | child's ordering can be used
GenerateExec | child's | Nil | need to dig more
GlobalLimitExec | child's | child's | -
HashAggregateExec | child's | Nil | -
InputAdapter | child's | child's | -
InsertIntoHiveTable | child's | Nil | terminal node, doesn't need partitioning
LocalLimitExec | child's | child's | -
MapElementsExec | child's | child's | -
MapGroupsExec | child's | Nil | child's ordering can be used
MapPartitionsExec | child's | Nil | child's ordering can be used
ProjectExec | child's | child's | -
SampleExec | child's | Nil | child's ordering can be used
ScriptTransformation | child's | Nil | child's ordering can be used
SerializeFromObjectExec | child's | Nil | child's ordering can be used
ShuffleExchange | custom | Nil | -
SortAggregateExec | child's | sort over grouped exprs | -
SortExec | child's | custom | -
StateStoreRestoreExec  | child's | Nil | child's ordering can be used
StateStoreSaveExec | child's | Nil | child's ordering can be used
SubqueryExec | child's | child's | -
TakeOrderedAndProjectExec | SinglePartition | custom | -
WholeStageCodegenExec | child's | child's | -
WindowExec | child's | child's | -

## How was this patch tested?

This does NOT change any existing functionality so relying on existing tests

Author: Tejas Patil <tejasp@fb.com>

Closes #15575 from tejasapatil/SPARK-18038_UnaryNodeExec_output_partitioning.
2016-10-23 13:25:47 +02:00
Tejas Patil eff4aed1ac [SPARK-18035][SQL] Introduce performant and memory efficient APIs to create ArrayBasedMapData
## What changes were proposed in this pull request?

Jira: https://issues.apache.org/jira/browse/SPARK-18035

In HiveInspectors, I saw that converting Java map to Spark's `ArrayBasedMapData` spent quite sometime in buffer copying : https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala#L658

The reason being `map.toSeq` allocates a new buffer and copies the map entries to it: https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/MapLike.scala#L323

This copy is not needed as we get rid of it once we extract the key and value arrays.

Here is the call trace:

```
org.apache.spark.sql.hive.HiveInspectors$$anonfun$unwrapperFor$41.apply(HiveInspectors.scala:664)
scala.collection.AbstractMap.toSeq(Map.scala:59)
scala.collection.MapLike$class.toSeq(MapLike.scala:323)
scala.collection.AbstractMap.toBuffer(Map.scala:59)
scala.collection.MapLike$class.toBuffer(MapLike.scala:326)
scala.collection.AbstractTraversable.copyToBuffer(Traversable.scala:104)
scala.collection.TraversableOnce$class.copyToBuffer(TraversableOnce.scala:275)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:48)
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:104)
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
scala.collection.AbstractIterable.foreach(Iterable.scala:54)
scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
scala.collection.Iterator$class.foreach(Iterator.scala:893)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
scala.collection.generic.Growable$$anonfun$$plus$plus$eq$1.apply(Growable.scala:59)
```

Also, earlier code was populating keys and values arrays separately by iterating twice. The PR avoids double iteration of the map and does it in one iteration.

EDIT: During code review, there were several more places in the code which were found to do similar thing. The PR dedupes those instances and introduces convenient APIs which are performant and memory efficient

## Performance gains

The number is subjective and depends on how many map columns are accessed in the query and average entries per map. For one the queries that I tried out, I saw 3% CPU savings (end-to-end) for the query.

## How was this patch tested?

This does not change the end result produced so relying on existing tests.

Author: Tejas Patil <tejasp@fb.com>

Closes #15573 from tejasapatil/SPARK-18035_avoid_toSeq.
2016-10-22 20:43:43 -07:00
hyukjinkwon 5fa9f8795a [SPARK-17123][SQL] Use type-widened encoder for DataFrame rather than existing encoder to allow type-widening from set operations
# What changes were proposed in this pull request?

This PR fixes set operations in `DataFrame` to be performed fine without exceptions when the types are non-scala native types. (e.g, `TimestampType`, `DateType` and `DecimalType`).

The problem is, it seems set operations such as `union`, `intersect` and `except` uses the encoder belonging to the `Dataset` in caller.

So, `Dataset` of the caller holds `ExpressionEncoder[Row]` as it is when the set operations are performed. However, the return types can be actually widen. So, we should use `ExpressionEncoder[Row]` constructed from executed plan rather than using existing one. Otherwise, this will generate some codes wrongly via `StaticInvoke`.

Running the codes below:

```scala
val dates = Seq(
  (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)),
  (new Date(3), BigDecimal.valueOf(4), new Timestamp(5))
).toDF("date", "timestamp", "decimal")

val widenTypedRows = Seq(
  (new Timestamp(2), 10.5D, "string")
).toDF("date", "timestamp", "decimal")

val results = dates.union(widenTypedRows).collect()
results.foreach(println)
```

prints below:

**Before**

```java
23:08:54.490 ERROR org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator: failed to compile: org.codehaus.commons.compiler.CompileException: File 'generated.java', Line 28, Column 107: No applicable constructor/method found for actual parameters "long"; candidates are: "public static java.sql.Date org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(int)"
/* 001 */ public java.lang.Object generate(Object[] references) {
/* 002 */   return new SpecificSafeProjection(references);
/* 003 */ }
/* 004 */
/* 005 */ class SpecificSafeProjection extends org.apache.spark.sql.catalyst.expressions.codegen.BaseProjection {
/* 006 */
/* 007 */   private Object[] references;
/* 008 */   private MutableRow mutableRow;
/* 009 */   private Object[] values;
/* 010 */   private org.apache.spark.sql.types.StructType schema;
/* 011 */
/* 012 */
/* 013 */   public SpecificSafeProjection(Object[] references) {
/* 014 */     this.references = references;
/* 015 */     mutableRow = (MutableRow) references[references.length - 1];
/* 016 */
/* 017 */     this.schema = (org.apache.spark.sql.types.StructType) references[0];
/* 018 */   }
/* 019 */
/* 020 */   public java.lang.Object apply(java.lang.Object _i) {
/* 021 */     InternalRow i = (InternalRow) _i;
/* 022 */
/* 023 */     values = new Object[3];
/* 024 */
/* 025 */     boolean isNull2 = i.isNullAt(0);
/* 026 */     long value2 = isNull2 ? -1L : (i.getLong(0));
/* 027 */     boolean isNull1 = isNull2;
/* 028 */     final java.sql.Date value1 = isNull1 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaDate(value2);
/* 029 */     isNull1 = value1 == null;
/* 030 */     if (isNull1) {
/* 031 */       values[0] = null;
/* 032 */     } else {
/* 033 */       values[0] = value1;
/* 034 */     }
/* 035 */
/* 036 */     boolean isNull4 = i.isNullAt(1);
/* 037 */     double value4 = isNull4 ? -1.0 : (i.getDouble(1));
/* 038 */
/* 039 */     boolean isNull3 = isNull4;
/* 040 */     java.math.BigDecimal value3 = null;
/* 041 */     if (!isNull3) {
/* 042 */
/* 043 */       Object funcResult = null;
/* 044 */       funcResult = value4.toJavaBigDecimal();
/* 045 */       if (funcResult == null) {
/* 046 */         isNull3 = true;
/* 047 */       } else {
/* 048 */         value3 = (java.math.BigDecimal) funcResult;
/* 049 */       }
/* 050 */
/* 051 */     }
/* 052 */     isNull3 = value3 == null;
/* 053 */     if (isNull3) {
/* 054 */       values[1] = null;
/* 055 */     } else {
/* 056 */       values[1] = value3;
/* 057 */     }
/* 058 */
/* 059 */     boolean isNull6 = i.isNullAt(2);
/* 060 */     UTF8String value6 = isNull6 ? null : (i.getUTF8String(2));
/* 061 */     boolean isNull5 = isNull6;
/* 062 */     final java.sql.Timestamp value5 = isNull5 ? null : org.apache.spark.sql.catalyst.util.DateTimeUtils.toJavaTimestamp(value6);
/* 063 */     isNull5 = value5 == null;
/* 064 */     if (isNull5) {
/* 065 */       values[2] = null;
/* 066 */     } else {
/* 067 */       values[2] = value5;
/* 068 */     }
/* 069 */
/* 070 */     final org.apache.spark.sql.Row value = new org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema(values, schema);
/* 071 */     if (false) {
/* 072 */       mutableRow.setNullAt(0);
/* 073 */     } else {
/* 074 */
/* 075 */       mutableRow.update(0, value);
/* 076 */     }
/* 077 */
/* 078 */     return mutableRow;
/* 079 */   }
/* 080 */ }
```

**After**

```bash
[1969-12-31 00:00:00.0,1.0,1969-12-31 16:00:00.002]
[1969-12-31 00:00:00.0,4.0,1969-12-31 16:00:00.005]
[1969-12-31 16:00:00.002,10.5,string]
```

## How was this patch tested?

Unit tests in `DataFrameSuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15072 from HyukjinKwon/SPARK-17123.
2016-10-22 20:09:04 +02:00
Eric Liang 3eca283aca [SPARK-17994][SQL] Add back a file status cache for catalog tables
## What changes were proposed in this pull request?

In SPARK-16980, we removed the full in-memory cache of table partitions in favor of loading only needed partitions from the metastore. This greatly improves the initial latency of queries that only read a small fraction of table partitions.

However, since the metastore does not store file statistics, we need to discover those from remote storage. With the loss of the in-memory file status cache this has to happen on each query, increasing the latency of repeated queries over the same partitions.

The proposal is to add back a per-table cache of partition contents, i.e. Map[Path, Array[FileStatus]]. This cache would be retained per-table, and can be invalidated through refreshTable() and refreshByPath(). Unlike the prior cache, it can be incrementally updated as new partitions are read.

## How was this patch tested?

Existing tests and new tests in `HiveTablePerfStatsSuite`.

cc mallman

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15539 from ericl/meta-cache.
2016-10-22 22:08:28 +08:00
Sean Owen 7178c56433 [SPARK-16606][MINOR] Tiny follow-up to , to correct more instances of the same log message typo
## What changes were proposed in this pull request?

Tiny follow-up to SPARK-16606 / https://github.com/apache/spark/pull/14533 , to correct more instances of the same log message typo

## How was this patch tested?

Existing tests (no functional change anyway)

Author: Sean Owen <sowen@cloudera.com>

Closes #15586 from srowen/SPARK-16606.2.
2016-10-21 22:20:52 -07:00
Reynold Xin 3fbf5a58c2 [SPARK-18042][SQL] OutputWriter should expose file path written
## What changes were proposed in this pull request?
This patch adds a new "path" method on OutputWriter that returns the path of the file written by the OutputWriter. This is part of the necessary work to consolidate structured streaming and batch write paths.

The batch write path has a nice feature that each data source can define the extension of the files, and allow Spark to specify the staging directory and the prefix for the files. However, in the streaming path we need to collect the list of files written, and there is no interface right now to do that.

## How was this patch tested?
N/A - there is no behavior change and this should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15580 from rxin/SPARK-18042.
2016-10-21 17:27:18 -07:00
Wenchen Fan 140570252f [SPARK-18044][STREAMING] FileStreamSource should not infer partitions in every batch
## What changes were proposed in this pull request?

In `FileStreamSource.getBatch`, we will create a `DataSource` with specified schema, to avoid inferring the schema again and again. However, we don't pass the partition columns, and will infer the partition again and again.

This PR fixes it by keeping the partition columns in `FileStreamSource`, like schema.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15581 from cloud-fan/stream.
2016-10-21 15:28:16 -07:00
Tathagata Das 7a531e3054 [SPARK-17926][SQL][STREAMING] Added json for statuses
## What changes were proposed in this pull request?

StreamingQueryStatus exposed through StreamingQueryListener often needs to be recorded (similar to SparkListener events). This PR adds `.json` and `.prettyJson` to `StreamingQueryStatus`, `SourceStatus` and `SinkStatus`.

## How was this patch tested?
New unit tests

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

Closes #15476 from tdas/SPARK-17926.
2016-10-21 13:07:29 -07:00
Zheng RuiFeng a8ea4da8d0
[SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.

cc srowen

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15564 from zhengruifeng/avoid_0_length_array.
2016-10-21 09:49:37 +01:00
Wenchen Fan 57e97fcbd6 [SPARK-18029][SQL] PruneFileSourcePartitions should not change the output of LogicalRelation
## What changes were proposed in this pull request?

In `PruneFileSourcePartitions`, we will replace the `LogicalRelation` with a pruned one. However, this replacement may change the output of the `LogicalRelation` if it doesn't have `expectedOutputAttributes`. This PR fixes it.

## How was this patch tested?

the new `PruneFileSourcePartitionsSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15569 from cloud-fan/partition-bug.
2016-10-21 12:27:53 +08:00
Shixiong Zhu 1bb99c4887 [SPARK-18030][TESTS] Adds more checks to collect more info about FileStreamSourceSuite failure
## What changes were proposed in this pull request?

My hunch is `mkdirs` fails. Just add more checks to collect more info.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15577 from zsxwing/SPARK-18030-debug.
2016-10-20 20:44:32 -07:00
Reynold Xin 7f9ec19eae [SPARK-18021][SQL] Refactor file name specification for data sources
## What changes were proposed in this pull request?
Currently each data source OutputWriter is responsible for specifying the entire file name for each file output. This, however, does not make any sense because we rely on file naming schemes for certain behaviors in Spark SQL, e.g. bucket id. The current approach allows individual data sources to break the implementation of bucketing.

On the flip side, we also don't want to move file naming entirely out of data sources, because different data sources do want to specify different extensions.

This patch divides file name specification into two parts: the first part is a prefix specified by the caller of OutputWriter (in WriteOutput), and the second part is the suffix that can be specified by the OutputWriter itself. Note that a side effect of this change is that now all file based data sources also support bucketing automatically.

There are also some other minor cleanups:

- Removed the UUID passed through generic Configuration string
- Some minor rewrites for better clarity
- Renamed "path" in multiple places to "stagingDir", to more accurately reflect its meaning

## How was this patch tested?
This should be covered by existing data source tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15562 from rxin/SPARK-18021.
2016-10-20 12:18:56 -07:00
Koert Kuipers 84b245f2dd [SPARK-15780][SQL] Support mapValues on KeyValueGroupedDataset
## What changes were proposed in this pull request?

Add mapValues to KeyValueGroupedDataset

## How was this patch tested?

New test in DatasetSuite for groupBy function, mapValues, flatMap

Author: Koert Kuipers <koert@tresata.com>

Closes #13526 from koertkuipers/feat-keyvaluegroupeddataset-mapvalues.
2016-10-20 10:08:12 -07:00
Tejas Patil fb0894b3a8 [SPARK-17698][SQL] Join predicates should not contain filter clauses
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-17698

`ExtractEquiJoinKeys` is incorrectly using filter predicates as the join condition for joins. `canEvaluate` [0] tries to see if the an `Expression` can be evaluated using output of a given `Plan`. In case of filter predicates (eg. `a.id='1'`), the `Expression` passed for the right hand side (ie. '1' ) is a `Literal` which does not have any attribute references. Thus `expr.references` is an empty set which theoretically is a subset of any set. This leads to `canEvaluate` returning `true` and `a.id='1'` is treated as a join predicate. While this does not lead to incorrect results but in case of bucketed + sorted tables, we might miss out on avoiding un-necessary shuffle + sort. See example below:

[0] : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala#L91

eg.

```
val df = (1 until 10).toDF("id").coalesce(1)
hc.sql("DROP TABLE IF EXISTS table1").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table1")
hc.sql("DROP TABLE IF EXISTS table2").collect
df.write.bucketBy(8, "id").sortBy("id").saveAsTable("table2")

sqlContext.sql("""
  SELECT a.id, b.id
  FROM table1 a
  FULL OUTER JOIN table2 b
  ON a.id = b.id AND a.id='1' AND b.id='1'
""").explain(true)
```

BEFORE: This is doing shuffle + sort over table scan outputs which is not needed as both tables are bucketed and sorted on the same columns and have same number of buckets. This should be a single stage job.

```
SortMergeJoin [id#38, cast(id#38 as double), 1.0], [id#39, 1.0, cast(id#39 as double)], FullOuter
:- *Sort [id#38 ASC NULLS FIRST, cast(id#38 as double) ASC NULLS FIRST, 1.0 ASC NULLS FIRST], false, 0
:  +- Exchange hashpartitioning(id#38, cast(id#38 as double), 1.0, 200)
:     +- *FileScan parquet default.table1[id#38] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *Sort [id#39 ASC NULLS FIRST, 1.0 ASC NULLS FIRST, cast(id#39 as double) ASC NULLS FIRST], false, 0
   +- Exchange hashpartitioning(id#39, 1.0, cast(id#39 as double), 200)
      +- *FileScan parquet default.table2[id#39] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

AFTER :

```
SortMergeJoin [id#32], [id#33], FullOuter, ((cast(id#32 as double) = 1.0) && (cast(id#33 as double) = 1.0))
:- *FileScan parquet default.table1[id#32] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table1, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
+- *FileScan parquet default.table2[id#33] Batched: true, Format: ParquetFormat, InputPaths: file:spark-warehouse/table2, PartitionFilters: [], PushedFilters: [], ReadSchema: struct<id:int>
```

## How was this patch tested?

- Added a new test case for this scenario : `SPARK-17698 Join predicates should not contain filter clauses`
- Ran all the tests in `BucketedReadSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15272 from tejasapatil/SPARK-17698_join_predicate_filter_clause.
2016-10-20 09:50:55 -07:00
Dilip Biswal e895bc2548 [SPARK-17860][SQL] SHOW COLUMN's database conflict check should respect case sensitivity configuration
## What changes were proposed in this pull request?
SHOW COLUMNS command validates the user supplied database
name with database name from qualified table name name to make
sure both of them are consistent. This comparison should respect
case sensitivity.

## How was this patch tested?
Added tests in DDLSuite and existing tests were moved to use new sql based test infrastructure.

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

Closes #15423 from dilipbiswal/dkb_show_column_fix.
2016-10-20 19:39:25 +08:00
Dongjoon Hyun 986a3b8b5b
[SPARK-17796][SQL] Support wildcard character in filename for LOAD DATA LOCAL INPATH
## What changes were proposed in this pull request?

Currently, Spark 2.0 raises an `input path does not exist` AnalysisException if the file name contains '*'. It is misleading since it occurs when there exist some matched files. Also, it was a supported feature in Spark 1.6.2. This PR aims to support wildcard characters in filename for `LOAD DATA LOCAL INPATH` SQL command like Spark 1.6.2.

**Reported Error Scenario**
```scala
scala> sql("CREATE TABLE t(a string)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("LOAD DATA LOCAL INPATH '/tmp/x*' INTO TABLE t")
org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: /tmp/x*;
```

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15376 from dongjoon-hyun/SPARK-17796.
2016-10-20 09:53:12 +01:00
Eric Liang 4bd17c4606 [SPARK-17991][SQL] Enable metastore partition pruning by default.
## What changes were proposed in this pull request?

This should apply to non-converted metastore relations. WIP to see if this causes any test failures.

## How was this patch tested?

Existing tests.

Author: Eric Liang <ekl@databricks.com>

Closes #15475 from ericl/try-enabling-pruning.
2016-10-19 23:55:05 -07:00
Reynold Xin f313117bc9 [SPARK-18012][SQL] Simplify WriterContainer
## What changes were proposed in this pull request?
This patch refactors WriterContainer to simplify the logic and make control flow more obvious.The previous code setup made it pretty difficult to track the actual dependencies on variables and setups because the driver side and the executor side were using the same set of variables.

## How was this patch tested?
N/A - this should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15551 from rxin/writercontainer-refactor.
2016-10-19 22:22:35 -07:00
hyukjinkwon 4b2011ec9d [SPARK-17989][SQL] Check ascendingOrder type in sort_array function rather than throwing ClassCastException
## What changes were proposed in this pull request?

This PR proposes to check the second argument, `ascendingOrder`  rather than throwing `ClassCastException` exception message.

```sql
select sort_array(array('b', 'd'), '1');
```

**Before**

```
16/10/19 13:16:08 ERROR SparkSQLDriver: Failed in [select sort_array(array('b', 'd'), '1')]
java.lang.ClassCastException: org.apache.spark.unsafe.types.UTF8String cannot be cast to java.lang.Boolean
	at scala.runtime.BoxesRunTime.unboxToBoolean(BoxesRunTime.java:85)
	at org.apache.spark.sql.catalyst.expressions.SortArray.nullSafeEval(collectionOperations.scala:185)
	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:416)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:50)
	at org.apache.spark.sql.catalyst.optimizer.ConstantFolding$$anonfun$apply$1$$anonfun$applyOrElse$1.applyOrElse(expressions.scala:43)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$3.apply(TreeNode.scala:292)
	at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:74)
	at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:291)
	at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:297)
```

**After**

```
Error in query: cannot resolve 'sort_array(array('b', 'd'), '1')' due to data type mismatch: Sort order in second argument requires a boolean literal.; line 1 pos 7;
```

## How was this patch tested?

Unit test in `DataFrameFunctionsSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15532 from HyukjinKwon/SPARK-17989.
2016-10-19 19:36:21 -07:00
Wenchen Fan 4329c5cea4 [SPARK-17873][SQL] ALTER TABLE RENAME TO should allow users to specify database in destination table name(but have to be same as source table)
## What changes were proposed in this pull request?

Unlike Hive, in Spark SQL, ALTER TABLE RENAME TO cannot move a table from one database to another(e.g. `ALTER TABLE db1.tbl RENAME TO db2.tbl2`), and will report error if the database in source table and destination table is different. So in #14955 , we forbid users to specify database of destination table in ALTER TABLE RENAME TO, to be consistent with other database systems and also make it easier to rename tables in non-current database, e.g. users can write `ALTER TABLE db1.tbl RENAME TO tbl2`, instead of `ALTER TABLE db1.tbl RENAME TO db1.tbl2`.

However, this is a breaking change. Users may already have queries that specify database of destination table in ALTER TABLE RENAME TO.

This PR reverts most of #14955 , and simplify the usage of ALTER TABLE RENAME TO by making database of source table the default database of destination table, instead of current database, so that users can still write `ALTER TABLE db1.tbl RENAME TO tbl2`, which is consistent with other databases like MySQL, Postgres, etc.

## How was this patch tested?

The added back tests and some new tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15434 from cloud-fan/revert.
2016-10-18 20:23:13 -07:00
Eric Liang 5f20ae0394 [SPARK-17980][SQL] Fix refreshByPath for converted Hive tables
## What changes were proposed in this pull request?

There was a bug introduced in https://github.com/apache/spark/pull/14690 which broke refreshByPath with converted hive tables (though, it turns out it was very difficult to refresh converted hive tables anyways, since you had to specify the exact path of one of the partitions).

This changes refreshByPath to invalidate by prefix instead of exact match, and fixes the issue.

cc sameeragarwal for refreshByPath changes
mallman

## How was this patch tested?

Extended unit test.

Author: Eric Liang <ekl@databricks.com>

Closes #15521 from ericl/fix-caching.
2016-10-19 10:20:12 +08:00
Tathagata Das 941b3f9aca [SPARK-17731][SQL][STREAMING][FOLLOWUP] Refactored StreamingQueryListener APIs
## What changes were proposed in this pull request?

As per rxin request, here are further API changes
- Changed `Stream(Started/Progress/Terminated)` events to `Stream*Event`
- Changed the fields in `StreamingQueryListener.on***` from `query*` to `event`

## How was this patch tested?
Existing unit tests.

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

Closes #15530 from tdas/SPARK-17731-1.
2016-10-18 17:32:16 -07:00
hyukjinkwon b3130c7b6a [SPARK-17955][SQL] Make DataFrameReader.jdbc call DataFrameReader.format("jdbc").load
## What changes were proposed in this pull request?

This PR proposes to make `DataFrameReader.jdbc` call `DataFrameReader.format("jdbc").load` consistently with other APIs in `DataFrameReader`/`DataFrameWriter` and avoid calling `sparkSession.baseRelationToDataFrame(..)` here and there.

The changes were mostly copied from `DataFrameWriter.jdbc()` which was recently updated.

```diff
-    val params = extraOptions.toMap ++ connectionProperties.asScala.toMap
-    val options = new JDBCOptions(url, table, params)
-    val relation = JDBCRelation(parts, options)(sparkSession)
-    sparkSession.baseRelationToDataFrame(relation)
+    this.extraOptions = this.extraOptions ++ connectionProperties.asScala
+    // explicit url and dbtable should override all
+    this.extraOptions += ("url" -> url, "dbtable" -> table)
+    format("jdbc").load()
```

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15499 from HyukjinKwon/SPARK-17955.
2016-10-18 13:49:02 -07:00
Eric Liang 4ef39c2f44 [SPARK-17974] try 2) Refactor FileCatalog classes to simplify the inheritance tree
## What changes were proposed in this pull request?

This renames `BasicFileCatalog => FileCatalog`, combines  `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.

In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```

(note that this is a re-submission of https://github.com/apache/spark/pull/15518 which got reverted)

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #15533 from ericl/fix-scalastyle-revert.
2016-10-18 13:33:46 -07:00
hyukjinkwon 37686539f5 [SPARK-17388] [SQL] Support for inferring type date/timestamp/decimal for partition column
## What changes were proposed in this pull request?

Currently, Spark only supports to infer `IntegerType`, `LongType`, `DoubleType` and `StringType`.

`DecimalType` is being tried but it seems it never infers type as `DecimalType` as `DoubleType` is being tried first. Also, it seems `DateType` and `TimestampType` could be inferred.

As far as I know, it is pretty common to use both for a partition column.

This PR fixes the incorrect `DecimalType` try and also adds the support for both `DateType` and `TimestampType` for inferring partition column type.

## How was this patch tested?

Unit tests in `ParquetPartitionDiscoverySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14947 from HyukjinKwon/SPARK-17388.
2016-10-18 13:20:42 -07:00
Wenchen Fan e59df62e62 [SPARK-17899][SQL][FOLLOW-UP] debug mode should work for corrupted table
## What changes were proposed in this pull request?

Debug mode should work for corrupted table, so that we can really debug

## How was this patch tested?

new test in `MetastoreDataSourcesSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15528 from cloud-fan/debug.
2016-10-18 11:03:10 -07:00
Tathagata Das a9e79a41ee [SQL][STREAMING][TEST] Follow up to remove Option.contains for Scala 2.10 compatibility
## What changes were proposed in this pull request?

Scala 2.10 does not have Option.contains, which broke Scala 2.10 build.

## How was this patch tested?
Locally compiled and ran sql/core unit tests in 2.10

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

Closes #15531 from tdas/metrics-flaky-test-fix-1.
2016-10-18 02:29:55 -07:00
Liwei Lin 7d878cf2da [SQL][STREAMING][TEST] Fix flaky tests in StreamingQueryListenerSuite
This work has largely been done by lw-lin in his PR #15497. This is a slight refactoring of it.

## What changes were proposed in this pull request?
There were two sources of flakiness in StreamingQueryListener test.

- When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock.
```
+-----------------------------------+--------------------------------+
|      StreamExecution thread       |         testing thread         |
+-----------------------------------+--------------------------------+
|  ManualClock.waitTillTime(100) {  |                                |
|        _isWaiting = true          |                                |
|            wait(10)               |                                |
|        still in wait(10)          |  if (_isWaiting) advance(100)  |
|        still in wait(10)          |  if (_isWaiting) advance(200)  | <- this should be disallowed !
|        still in wait(10)          |  if (_isWaiting) advance(300)  | <- this should be disallowed !
|      wake up from wait(10)        |                                |
|       current time is 600         |                                |
|       _isWaiting = false          |                                |
|  }                                |                                |
+-----------------------------------+--------------------------------+
```

- Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger.

My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(200)` (see above) will wait for stream execution thread to complete the wait that started at time 0, and start a new wait at time 200 (i.e. time stamp after the previous `advance(100)`).

In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest.

## How was this patch tested?
Ran existing unit test MANY TIME in Jenkins

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

Closes #15519 from tdas/metrics-flaky-test-fix.
2016-10-18 00:49:57 -07:00
Reynold Xin 1c5a7d7f64 Revert "[SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree"
This reverts commit 8daa1a29b6.
2016-10-17 21:26:28 -07:00
Eric Liang 8daa1a29b6 [SPARK-17974] Refactor FileCatalog classes to simplify the inheritance tree
## What changes were proposed in this pull request?

This renames `BasicFileCatalog => FileCatalog`, combines  `SessionFileCatalog` with `PartitioningAwareFileCatalog`, and removes the old `FileCatalog` trait.

In summary,
```
MetadataLogFileCatalog extends PartitioningAwareFileCatalog
ListingFileCatalog extends PartitioningAwareFileCatalog
PartitioningAwareFileCatalog extends FileCatalog
TableFileCatalog extends FileCatalog
```

cc cloud-fan mallman

## How was this patch tested?

Existing tests

Author: Eric Liang <ekl@databricks.com>

Closes #15518 from ericl/refactor-session-file-catalog.
2016-10-17 21:01:22 -07:00
Dilip Biswal 813ab5e025 [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables
## What changes were proposed in this pull request?
Reopens the closed PR https://github.com/apache/spark/pull/15190
(Please refer to the above link for review comments on the PR)

Make sure the hive.default.fileformat is used to when creating the storage format metadata.

Output
``` SQL
scala> spark.sql("SET hive.default.fileformat=orc")
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]

scala> spark.sql("CREATE TABLE tmp_default(id INT)")
res2: org.apache.spark.sql.DataFrame = []
```
Before
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]
```
After
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]

```
## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Added new tests to HiveDDLCommandSuite, SQLQuerySuite

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

Closes #15495 from dilipbiswal/orc2.
2016-10-17 20:46:30 -07:00
gatorsmile d88a1bae6a [SPARK-17751][SQL] Remove spark.sql.eagerAnalysis and Output the Plan if Existed in AnalysisException
### What changes were proposed in this pull request?
Dataset always does eager analysis now. Thus, `spark.sql.eagerAnalysis` is not used any more. Thus, we need to remove it.

This PR also outputs the plan. Without the fix, the analysis error is like
```
cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12
```

After the fix, the analysis error becomes:
```
org.apache.spark.sql.AnalysisException: cannot resolve '`k1`' given input columns: [k, v]; line 1 pos 12;
'Project [unresolvedalias(CASE WHEN ('k1 = 2) THEN 22 WHEN ('k1 = 4) THEN 44 ELSE 0 END, None), v#6]
+- SubqueryAlias t
   +- Project [_1#2 AS k#5, _2#3 AS v#6]
      +- LocalRelation [_1#2, _2#3]
```

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15316 from gatorsmile/eagerAnalysis.
2016-10-17 11:33:06 -07:00
Sital Kedia c7ac027d5f [SPARK-17839][CORE] Use Nio's directbuffer instead of BufferedInputStream in order to avoid additional copy from os buffer cache to user buffer
## What changes were proposed in this pull request?

Currently we use BufferedInputStream to read the shuffle file which copies the file content from os buffer cache to the user buffer. This adds additional latency in reading the spill files. We made a change to use java nio's direct buffer to read the spill files and for certain pipelines spilling significant amount of data, we see up to 7% speedup for the entire pipeline.

## How was this patch tested?
Tested by running the job in the cluster and observed up to 7% speedup.

Author: Sital Kedia <skedia@fb.com>

Closes #15408 from sitalkedia/skedia/nio_spill_read.
2016-10-17 11:03:04 -07:00
Weiqing Yang 56b0f5f4d1 [MINOR][SQL] Add prettyName for current_database function
## What changes were proposed in this pull request?
Added a `prettyname` for current_database function.

## How was this patch tested?
Manually.

Before:
```
scala> sql("select current_database()").show
+-----------------+
|currentdatabase()|
+-----------------+
|          default|
+-----------------+
```

After:
```
scala> sql("select current_database()").show
+------------------+
|current_database()|
+------------------+
|           default|
+------------------+
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15506 from weiqingy/prettyName.
2016-10-16 22:38:30 -07:00
gatorsmile e18d02c5a8 [SPARK-17947][SQL] Add Doc and Comment about spark.sql.debug
### What changes were proposed in this pull request?
Just document the impact of `spark.sql.debug`:

When enabling the debug, Spark SQL internal table properties are not filtered out; however, some related DDL commands (e.g., Analyze Table and CREATE TABLE LIKE) might not work properly.

### How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15494 from gatorsmile/addDocForSQLDebug.
2016-10-17 12:08:25 +08:00
Dongjoon Hyun 59e3eb5af8 [SPARK-17819][SQL] Support default database in connection URIs for Spark Thrift Server
## What changes were proposed in this pull request?

Currently, Spark Thrift Server ignores the default database in URI. This PR supports that like the following.

```sql
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "create database testdb"
$ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "create table t(a int)"
$ bin/beeline -u jdbc:hive2://localhost:10000/testdb -e "show tables"
...
+------------+--------------+--+
| tableName  | isTemporary  |
+------------+--------------+--+
| t          | false        |
+------------+--------------+--+
1 row selected (0.347 seconds)
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "show tables"
...
+------------+--------------+--+
| tableName  | isTemporary  |
+------------+--------------+--+
+------------+--------------+--+
No rows selected (0.098 seconds)
```

## How was this patch tested?

Manual.

Note: I tried to add a test case for this, but I cannot found a suitable testsuite for this. I'll add the testcase if some advice is given.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15399 from dongjoon-hyun/SPARK-17819.
2016-10-16 20:15:32 -07:00
Jun Kim 36d81c2c68 [SPARK-17953][DOCUMENTATION] Fix typo in SparkSession scaladoc
## What changes were proposed in this pull request?

### Before:
```scala
SparkSession.builder()
     .master("local")
     .appName("Word Count")
     .config("spark.some.config.option", "some-value").
     .getOrCreate()
```

### After:
```scala
SparkSession.builder()
     .master("local")
     .appName("Word Count")
     .config("spark.some.config.option", "some-value")
     .getOrCreate()
```

There was one unexpected dot!

Author: Jun Kim <i2r.jun@gmail.com>

Closes #15498 from tae-jun/SPARK-17953.
2016-10-15 00:36:55 -07:00
Michael Allman 6ce1b675ee [SPARK-16980][SQL] Load only catalog table partition metadata required to answer a query
(This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.)

## What changes were proposed in this pull request?

In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference.

If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild.

In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space.

This PR proposes an alternative approach. Basically, it makes four changes:

1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates.
1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates.
1. It removes partition loading and caching from `HiveMetastoreCatalog`.
1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog.

The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters.

As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted.

## Open Issues

1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR.
1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue.
1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`.
1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly.

## How was this patch tested?

The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded.

Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #14690 from mallman/spark-16980-lazy_partition_fetching.
2016-10-14 18:26:18 -07:00
Srinath Shankar 2d96d35dc0 [SPARK-17946][PYSPARK] Python crossJoin API similar to Scala
## What changes were proposed in this pull request?

Add a crossJoin function to the DataFrame API similar to that in Scala. Joins with no condition (cartesian products) must be specified with the crossJoin API

## How was this patch tested?
Added python tests to ensure that an AnalysisException if a cartesian product is specified without crossJoin(), and that cartesian products can execute if specified via crossJoin()

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

Author: Srinath Shankar <srinath@databricks.com>

Closes #15493 from srinathshankar/crosspython.
2016-10-14 18:24:47 -07:00
Reynold Xin 72adfbf94a [SPARK-17900][SQL] Graduate a list of Spark SQL APIs to stable
## What changes were proposed in this pull request?
This patch graduates a list of Spark SQL APIs and mark them stable.

The following are marked stable:

Dataset/DataFrame
- functions, since 1.3
- ColumnName, since 1.3
- DataFrameNaFunctions, since 1.3.1
- DataFrameStatFunctions, since 1.4
- UserDefinedFunction, since 1.3
- UserDefinedAggregateFunction, since 1.5
- Window and WindowSpec, since 1.4

Data sources:
- DataSourceRegister, since 1.5
- RelationProvider, since 1.3
- SchemaRelationProvider, since 1.3
- CreatableRelationProvider, since 1.3
- BaseRelation, since 1.3
- TableScan, since 1.3
- PrunedScan, since 1.3
- PrunedFilteredScan, since 1.3
- InsertableRelation, since 1.3

The following are kept experimental / evolving:

Data sources:
- CatalystScan (tied to internal logical plans so it is not stable by definition)

Structured streaming:
- all classes (introduced new in 2.0 and will likely change)

Dataset typed operations (introduced in 1.6 and 2.0 and might change, although probability is low)
- all typed methods on Dataset
- KeyValueGroupedDataset
- o.a.s.sql.expressions.javalang.typed
- o.a.s.sql.expressions.scalalang.typed
- methods that return typed Dataset in SparkSession

We should discuss more whether we want to mark Dataset typed operations stable in 2.1.

## How was this patch tested?
N/A - just annotation changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #15469 from rxin/SPARK-17900.
2016-10-14 16:13:42 -07:00
Jeff Zhang f00df40cfe [SPARK-11775][PYSPARK][SQL] Allow PySpark to register Java UDF
Currently pyspark can only call the builtin java UDF, but can not call custom java UDF. It would be better to allow that. 2 benefits:
* Leverage the power of rich third party java library
* Improve the performance. Because if we use python UDF, python daemons will be started on worker which will affect the performance.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #9766 from zjffdu/SPARK-11775.
2016-10-14 15:50:35 -07:00
Nick Pentreath 5aeb7384c7 [SPARK-16063][SQL] Add storageLevel to Dataset
[SPARK-11905](https://issues.apache.org/jira/browse/SPARK-11905) added support for `persist`/`cache` for `Dataset`. However, there is no user-facing API to check if a `Dataset` is cached and if so what the storage level is. This PR adds `getStorageLevel` to `Dataset`, analogous to `RDD.getStorageLevel`.

Updated `DatasetCacheSuite`.

Author: Nick Pentreath <nickp@za.ibm.com>

Closes #13780 from MLnick/ds-storagelevel.

Signed-off-by: Michael Armbrust <michael@databricks.com>
2016-10-14 15:09:49 -07:00
Davies Liu da9aeb0fde [SPARK-17863][SQL] should not add column into Distinct
## What changes were proposed in this pull request?

We are trying to resolve the attribute in sort by pulling up some column for grandchild into child, but that's wrong when the child is Distinct, because the added column will change the behavior of Distinct, we should not do that.

## How was this patch tested?

Added regression test.

Author: Davies Liu <davies@databricks.com>

Closes #15489 from davies/order_distinct.
2016-10-14 14:45:20 -07:00
Yin Huai 522dd0d0e5 Revert "[SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables"
This reverts commit 7ab86244e3.
2016-10-14 14:09:35 -07:00
Dilip Biswal 7ab86244e3 [SPARK-17620][SQL] Determine Serde by hive.default.fileformat when Creating Hive Serde Tables
## What changes were proposed in this pull request?
Make sure the hive.default.fileformat is used to when creating the storage format metadata.

Output
``` SQL
scala> spark.sql("SET hive.default.fileformat=orc")
res1: org.apache.spark.sql.DataFrame = [key: string, value: string]

scala> spark.sql("CREATE TABLE tmp_default(id INT)")
res2: org.apache.spark.sql.DataFrame = []
```
Before
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]
```
After
```SQL
scala> spark.sql("DESC FORMATTED tmp_default").collect.foreach(println)
..
[# Storage Information,,]
[SerDe Library:,org.apache.hadoop.hive.ql.io.orc.OrcSerde,]
[InputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcInputFormat,]
[OutputFormat:,org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat,]
[Compressed:,No,]
[Storage Desc Parameters:,,]
[  serialization.format,1,]

```

## How was this patch tested?
Added new tests to HiveDDLCommandSuite

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

Closes #15190 from dilipbiswal/orc.
2016-10-14 13:22:59 -07:00
Tathagata Das 05800b4b4e [TEST] Ignore flaky test in StreamingQueryListenerSuite
## What changes were proposed in this pull request?

Ignoring the flaky test introduced in #15307

https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/1736/testReport/junit/org.apache.spark.sql.streaming/StreamingQueryListenerSuite/single_listener__check_trigger_statuses/

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

Closes #15491 from tdas/metrics-flaky-test.
2016-10-14 12:39:25 -07:00
Andrew Ash fa37877af0
Typo: form -> from
## What changes were proposed in this pull request?

Minor typo fix

## How was this patch tested?

Existing unit tests on Jenkins

Author: Andrew Ash <andrew@andrewash.com>

Closes #15486 from ash211/patch-8.
2016-10-14 18:13:19 +01:00
wangzhenhua 7486442fe0 [SPARK-17073][SQL][FOLLOWUP] generate column-level statistics
## What changes were proposed in this pull request?
This pr adds some test cases for statistics: case sensitive column names, non ascii column names, refresh table, and also improves some documentation.

## How was this patch tested?
add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15360 from wzhfy/colStats2.
2016-10-14 21:18:49 +08:00
Wenchen Fan 2fb12b0a33 [SPARK-17903][SQL] MetastoreRelation should talk to external catalog instead of hive client
## What changes were proposed in this pull request?

`HiveExternalCatalog` should be the only interface to talk to the hive metastore. In `MetastoreRelation` we can just use `ExternalCatalog` instead of `HiveClient` to interact with hive metastore,  and add missing API in `ExternalCatalog`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15460 from cloud-fan/relation.
2016-10-14 15:53:50 +08:00
Reynold Xin 6c29b3de76 [SPARK-17925][SQL] Break fileSourceInterfaces.scala into multiple pieces
## What changes were proposed in this pull request?
This patch does a few changes to the file structure of data sources:

- Break fileSourceInterfaces.scala into multiple pieces (HadoopFsRelation, FileFormat, OutputWriter)
- Move ParquetOutputWriter into its own file

I created this as a separate patch so it'd be easier to review my future PRs that focus on refactoring this internal logic. This patch only moves code around, and has no logic changes.

## How was this patch tested?
N/A - should be covered by existing tests.

Author: Reynold Xin <rxin@databricks.com>

Closes #15473 from rxin/SPARK-17925.
2016-10-14 14:14:52 +08:00
Reynold Xin 8543996c3f [SPARK-17927][SQL] Remove dead code in WriterContainer.
## What changes were proposed in this pull request?
speculationEnabled and DATASOURCE_OUTPUTPATH seem like just dead code.

## How was this patch tested?
Tests should fail if they are not dead code.

Author: Reynold Xin <rxin@databricks.com>

Closes #15477 from rxin/SPARK-17927.
2016-10-14 12:35:59 +08:00
Jakob Odersky 9dc0ca060d [SPARK-17368][SQL] Add support for value class serialization and deserialization
## What changes were proposed in this pull request?
Value classes were unsupported because catalyst data types were
obtained through reflection on erased types, which would resolve to a
value class' wrapped type and hence lead to unavailable methods during
code generation.

E.g. the following class
```scala
case class Foo(x: Int) extends AnyVal
```
would be seen as an `int` in catalyst and will cause instance cast failures when generated java code tries to treat it as a `Foo`.

This patch simply removes the erasure step when getting data types for
catalyst.

## How was this patch tested?
Additional tests in `ExpressionEncoderSuite`.

Author: Jakob Odersky <jakob@odersky.com>

Closes #15284 from jodersky/value-classes.
2016-10-13 17:48:09 -07:00
petermaxlee adc112429d [SPARK-17661][SQL] Consolidate various listLeafFiles implementations
## What changes were proposed in this pull request?
There are 4 listLeafFiles-related functions in Spark:

- ListingFileCatalog.listLeafFiles (which calls HadoopFsRelation.listLeafFilesInParallel if the number of paths passed in is greater than a threshold; if it is lower, then it has its own serial version implemented)
- HadoopFsRelation.listLeafFiles (called only by HadoopFsRelation.listLeafFilesInParallel)
- HadoopFsRelation.listLeafFilesInParallel (called only by ListingFileCatalog.listLeafFiles)

It is actually very confusing and error prone because there are effectively two distinct implementations for the serial version of listing leaf files. As an example, SPARK-17599 updated only one of the code path and ignored the other one.

This code can be improved by:

- Move all file listing code into ListingFileCatalog, since it is the only class that needs this.
- Keep only one function for listing files in serial.

## How was this patch tested?
This change should be covered by existing unit and integration tests. I also moved a test case for HadoopFsRelation.shouldFilterOut from HadoopFsRelationSuite to ListingFileCatalogSuite.

Author: petermaxlee <petermaxlee@gmail.com>

Closes #15235 from petermaxlee/SPARK-17661.
2016-10-13 14:16:39 -07:00
Tathagata Das 7106866c22 [SPARK-17731][SQL][STREAMING] Metrics for structured streaming
## What changes were proposed in this pull request?

Metrics are needed for monitoring structured streaming apps. Here is the design doc for implementing the necessary metrics.
https://docs.google.com/document/d/1NIdcGuR1B3WIe8t7VxLrt58TJB4DtipWEbj5I_mzJys/edit?usp=sharing

Specifically, this PR adds the following public APIs changes.

### New APIs
- `StreamingQuery.status` returns a `StreamingQueryStatus` object (renamed from `StreamingQueryInfo`, see later)

- `StreamingQueryStatus` has the following important fields
  - inputRate - Current rate (rows/sec) at which data is being generated by all the sources
  - processingRate - Current rate (rows/sec) at which the query is processing data from
                                  all the sources
  - ~~outputRate~~ - *Does not work with wholestage codegen*
  - latency - Current average latency between the data being available in source and the sink writing the corresponding output
  - sourceStatuses: Array[SourceStatus] - Current statuses of the sources
  - sinkStatus: SinkStatus - Current status of the sink
  - triggerStatus - Low-level detailed status of the last completed/currently active trigger
    - latencies - getOffset, getBatch, full trigger, wal writes
    - timestamps - trigger start, finish, after getOffset, after getBatch
    - numRows - input, output, state total/updated rows for aggregations

- `SourceStatus` has the following important fields
  - inputRate - Current rate (rows/sec) at which data is being generated by the source
  - processingRate - Current rate (rows/sec) at which the query is processing data from the source
  - triggerStatus - Low-level detailed status of the last completed/currently active trigger

- Python API for `StreamingQuery.status()`

### Breaking changes to existing APIs
**Existing direct public facing APIs**
- Deprecated direct public-facing APIs `StreamingQuery.sourceStatuses` and `StreamingQuery.sinkStatus` in favour of `StreamingQuery.status.sourceStatuses/sinkStatus`.
  - Branch 2.0 should have it deprecated, master should have it removed.

**Existing advanced listener APIs**
- `StreamingQueryInfo` renamed to `StreamingQueryStatus` for consistency with `SourceStatus`, `SinkStatus`
   - Earlier StreamingQueryInfo was used only in the advanced listener API, but now it is used in direct public-facing API (StreamingQuery.status)

- Field `queryInfo` in listener events `QueryStarted`, `QueryProgress`, `QueryTerminated` changed have name `queryStatus` and return type `StreamingQueryStatus`.

- Field `offsetDesc` in `SourceStatus` was Option[String], converted it to `String`.

- For `SourceStatus` and `SinkStatus` made constructor private instead of private[sql] to make them more java-safe. Instead added `private[sql] object SourceStatus/SinkStatus.apply()` which are harder to accidentally use in Java.

## How was this patch tested?

Old and new unit tests.
- Rate calculation and other internal logic of StreamMetrics tested by StreamMetricsSuite.
- New info in statuses returned through StreamingQueryListener is tested in StreamingQueryListenerSuite.
- New and old info returned through StreamingQuery.status is tested in StreamingQuerySuite.
- Source-specific tests for making sure input rows are counted are is source-specific test suites.
- Additional tests to test minor additions in LocalTableScanExec, StateStore, etc.

Metrics also manually tested using Ganglia sink

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

Closes #15307 from tdas/SPARK-17731.
2016-10-13 13:36:26 -07:00
Pete Robbins 84f149e414 [SPARK-17827][SQL] maxColLength type should be Int for String and Binary
## What changes were proposed in this pull request?
correct the expected type from Length function to be Int

## How was this patch tested?
Test runs on little endian and big endian platforms

Author: Pete Robbins <robbinspg@gmail.com>

Closes #15464 from robbinspg/SPARK-17827.
2016-10-13 11:26:30 -07:00
Reynold Xin 04d417a7ca [SPARK-17830][SQL] Annotate remaining SQL APIs with InterfaceStability
## What changes were proposed in this pull request?
This patch annotates all the remaining APIs in SQL (excluding streaming) with InterfaceStability.

## How was this patch tested?
N/A - just annotation change.

Author: Reynold Xin <rxin@databricks.com>

Closes #15457 from rxin/SPARK-17830-2.
2016-10-13 11:12:30 -07:00
gatorsmile 0a8e51a5e4 [SPARK-17657][SQL] Disallow Users to Change Table Type
### What changes were proposed in this pull request?
Hive allows users to change the table type from `Managed` to `External` or from `External` to `Managed` by altering table's property `EXTERNAL`. See the JIRA: https://issues.apache.org/jira/browse/HIVE-1329

So far, Spark SQL does not correctly support it, although users can do it. Many assumptions are broken in the implementation. Thus, this PR is to disallow users to change it.

In addition, we also do not allow users to set the property `EXTERNAL` when creating a table.

### How was this patch tested?
Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15230 from gatorsmile/alterTableSetExternal.
2016-10-13 21:36:39 +08:00
Wenchen Fan db8784feaa [SPARK-17899][SQL] add a debug mode to keep raw table properties in HiveExternalCatalog
## What changes were proposed in this pull request?

Currently `HiveExternalCatalog` will filter out the Spark SQL internal table properties, e.g. `spark.sql.sources.provider`, `spark.sql.sources.schema`, etc. This is reasonable for external users as they don't want to see these internal properties in `DESC TABLE`.

However, as a Spark developer, sometimes we do wanna see the raw table properties. This PR adds a new internal SQL conf, `spark.sql.debug`, to enable debug mode and keep these raw table properties.

This config can also be used in similar places where we wanna retain debug information in the future.

## How was this patch tested?

new test in MetastoreDataSourcesSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15458 from cloud-fan/debug.
2016-10-13 03:26:29 -04:00
buzhihuojie 7222a25a11 minor doc fix for Row.scala
## What changes were proposed in this pull request?

minor doc fix for "getAnyValAs" in class Row

## How was this patch tested?

None.

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

Author: buzhihuojie <ren.weiluo@gmail.com>

Closes #15452 from david-weiluo-ren/minorDocFixForRow.
2016-10-12 22:51:54 -07:00
Liang-Chi Hsieh 064d6650e9 [SPARK-17866][SPARK-17867][SQL] Fix Dataset.dropduplicates
## What changes were proposed in this pull request?

Two issues regarding Dataset.dropduplicates:

1. Dataset.dropDuplicates should consider the columns with same column name

    We find and get the first resolved attribute from output with the given column name in `Dataset.dropDuplicates`. When we have the more than one columns with the same name. Other columns are put into aggregation columns, instead of grouping columns.

2. Dataset.dropDuplicates should not change the output of child plan

    We create new `Alias` with new exprId in `Dataset.dropDuplicates` now. However it causes problem when we want to select the columns as follows:

        val ds = Seq(("a", 1), ("a", 2), ("b", 1), ("a", 1)).toDS()
        // ds("_2") will cause analysis exception
        ds.dropDuplicates("_1").select(ds("_1").as[String], ds("_2").as[Int])

Because the two issues are both related to `Dataset.dropduplicates` and the code changes are not big, so submitting them together as one PR.

## How was this patch tested?

Jenkins tests.

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

Closes #15427 from viirya/fix-dropduplicates.
2016-10-13 13:27:57 +08:00
Burak Yavuz edeb51a39d [SPARK-17876] Write StructuredStreaming WAL to a stream instead of materializing all at once
## What changes were proposed in this pull request?

The CompactibleFileStreamLog materializes the whole metadata log in memory as a String. This can cause issues when there are lots of files that are being committed, especially during a compaction batch.
You may come across stacktraces that look like:
```
java.lang.OutOfMemoryError: Requested array size exceeds VM limit
at java.lang.StringCoding.encode(StringCoding.java:350)
at java.lang.String.getBytes(String.java:941)
at org.apache.spark.sql.execution.streaming.FileStreamSinkLog.serialize(FileStreamSinkLog.scala:127)

```
The safer way is to write to an output stream so that we don't have to materialize a huge string.

## How was this patch tested?

Existing unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15437 from brkyvz/ser-to-stream.
2016-10-12 21:40:45 -07:00
Reynold Xin 6f20a92ca3 [SPARK-17845] [SQL] More self-evident window function frame boundary API
## What changes were proposed in this pull request?
This patch improves the window function frame boundary API to make it more obvious to read and to use. The two high level changes are:

1. Create Window.currentRow, Window.unboundedPreceding, Window.unboundedFollowing to indicate the special values in frame boundaries. These methods map to the special integral values so we are not breaking backward compatibility here. This change makes the frame boundaries more self-evident (instead of Long.MinValue, it becomes Window.unboundedPreceding).

2. In Python, for any value less than or equal to JVM's Long.MinValue, treat it as Window.unboundedPreceding. For any value larger than or equal to JVM's Long.MaxValue, treat it as Window.unboundedFollowing. Before this change, if the user specifies any value that is less than Long.MinValue but not -sys.maxsize (e.g. -sys.maxsize + 1), the number we pass over to the JVM would overflow, resulting in a frame that does not make sense.

Code example required to specify a frame before this patch:
```
Window.rowsBetween(-Long.MinValue, 0)
```

While the above code should still work, the new way is more obvious to read:
```
Window.rowsBetween(Window.unboundedPreceding, Window.currentRow)
```

## How was this patch tested?
- Updated DataFrameWindowSuite (for Scala/Java)
- Updated test_window_functions_cumulative_sum (for Python)
- Renamed DataFrameWindowSuite DataFrameWindowFunctionsSuite to better reflect its purpose

Author: Reynold Xin <rxin@databricks.com>

Closes #15438 from rxin/SPARK-17845.
2016-10-12 16:45:10 -07:00
Imran Rashid 9ce7d3e542 [SPARK-17675][CORE] Expand Blacklist for TaskSets
## What changes were proposed in this pull request?

This is a step along the way to SPARK-8425.

To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)

Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.

Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).

## How was this patch tested?

Added unit tests, run tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>

Closes #15249 from squito/taskset_blacklist_only.
2016-10-12 16:43:03 -05:00
Shixiong Zhu 47776e7c0c [SPARK-17850][CORE] Add a flag to ignore corrupt files
## What changes were proposed in this pull request?

Add a flag to ignore corrupt files. For Spark core, the configuration is `spark.files.ignoreCorruptFiles`. For Spark SQL, it's `spark.sql.files.ignoreCorruptFiles`.

## How was this patch tested?

The added unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15422 from zsxwing/SPARK-17850.
2016-10-12 13:51:53 -07:00
prigarg d5580ebaa0 [SPARK-17884][SQL] To resolve Null pointer exception when casting from empty string to interval type.
## What changes were proposed in this pull request?
This change adds a check in castToInterval method of Cast expression , such that if converted value is null , then isNull variable should be set to true.

Earlier, the expression Cast(Literal(), CalendarIntervalType) was throwing NullPointerException because of the above mentioned reason.

## How was this patch tested?
Added test case in CastSuite.scala

jira entry for detail: https://issues.apache.org/jira/browse/SPARK-17884

Author: prigarg <prigarg@adobe.com>

Closes #15449 from priyankagargnitk/SPARK-17884.
2016-10-12 10:14:45 -07:00
Wenchen Fan b9a147181d [SPARK-17720][SQL] introduce static SQL conf
## What changes were proposed in this pull request?

SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897.

Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.

## How was this patch tested?

new tests in SQLConfSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15295 from cloud-fan/global-conf.
2016-10-11 20:27:08 -07:00
Liang-Chi Hsieh c8c090640a [SPARK-17821][SQL] Support And and Or in Expression Canonicalize
## What changes were proposed in this pull request?

Currently `Canonicalize` object doesn't support `And` and `Or`. So we can compare canonicalized form of predicates consistently. We should add the support.

## How was this patch tested?

Jenkins tests.

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

Closes #15388 from viirya/canonicalize-and-or.
2016-10-11 16:06:40 +08:00
Reynold Xin 3694ba48f0 [SPARK-17864][SQL] Mark data type APIs as stable (not DeveloperApi)
## What changes were proposed in this pull request?
The data type API has not been changed since Spark 1.3.0, and is ready for graduation. This patch marks them as stable APIs using the new InterfaceStability annotation.

This patch also looks at the various files in the catalyst module (not the "package") and marks the remaining few classes appropriately as well.

## How was this patch tested?
This is an annotation change. No functional changes.

Author: Reynold Xin <rxin@databricks.com>

Closes #15426 from rxin/SPARK-17864.
2016-10-11 15:35:52 +08:00
Wenchen Fan 7388ad94d7 [SPARK-17338][SQL][FOLLOW-UP] add global temp view
## What changes were proposed in this pull request?

address post hoc review comments for https://github.com/apache/spark/pull/14897

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15424 from cloud-fan/global-temp-view.
2016-10-11 15:21:28 +08:00
Reynold Xin b515768f26 [SPARK-17844] Simplify DataFrame API for defining frame boundaries in window functions
## What changes were proposed in this pull request?
When I was creating the example code for SPARK-10496, I realized it was pretty convoluted to define the frame boundaries for window functions when there is no partition column or ordering column. The reason is that we don't provide a way to create a WindowSpec directly with the frame boundaries. We can trivially improve this by adding rowsBetween and rangeBetween to Window object.

As an example, to compute cumulative sum using the natural ordering, before this pr:
```
df.select('key, sum("value").over(Window.partitionBy(lit(1)).rowsBetween(Long.MinValue, 0)))
```

After this pr:
```
df.select('key, sum("value").over(Window.rowsBetween(Long.MinValue, 0)))
```

Note that you could argue there is no point specifying a window frame without partitionBy/orderBy -- but it is strange that only rowsBetween and rangeBetween are not the only two APIs not available.

This also fixes https://issues.apache.org/jira/browse/SPARK-17656 (removing _root_.scala).

## How was this patch tested?
Added test cases to compute cumulative sum in DataFrameWindowSuite for Scala/Java and tests.py for Python.

Author: Reynold Xin <rxin@databricks.com>

Closes #15412 from rxin/SPARK-17844.
2016-10-10 22:33:20 -07:00
hyukjinkwon 0c0ad436ad [SPARK-17719][SPARK-17776][SQL] Unify and tie up options in a single place in JDBC datasource package
## What changes were proposed in this pull request?

This PR proposes to fix arbitrary usages among `Map[String, String]`, `Properties` and `JDBCOptions` instances for options in `execution/jdbc` package and make the connection properties exclude Spark-only options.

This PR includes some changes as below:

  - Unify `Map[String, String]`, `Properties` and `JDBCOptions` in `execution/jdbc` package to `JDBCOptions`.

- Move `batchsize`, `fetchszie`, `driver` and `isolationlevel` options into `JDBCOptions` instance.

- Document `batchSize` and `isolationlevel` with marking both read-only options and write-only options. Also, this includes minor types and detailed explanation for some statements such as url.

- Throw exceptions fast by checking arguments first rather than in execution time (e.g. for `fetchsize`).

- Exclude Spark-only options in connection properties.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15292 from HyukjinKwon/SPARK-17719.
2016-10-10 22:22:41 -07:00
hyukjinkwon 90217f9dee [SPARK-16896][SQL] Handle duplicated field names in header consistently with null or empty strings in CSV
## What changes were proposed in this pull request?

Currently, CSV datasource allows to load duplicated empty string fields or fields having `nullValue` in the header. It'd be great if this can deal with normal fields as well.

This PR proposes handling the duplicates consistently with the existing behaviour with considering case-sensitivity (`spark.sql.caseSensitive`) as below:

data below:

```
fieldA,fieldB,,FIELDA,fielda,,
1,2,3,4,5,6,7
```

is parsed as below:

```scala
spark.read.format("csv").option("header", "true").load("test.csv").show()
```

- when `spark.sql.caseSensitive` is `false` (by default).

  ```
  +-------+------+---+-------+-------+---+---+
  |fieldA0|fieldB|_c2|FIELDA3|fieldA4|_c5|_c6|
  +-------+------+---+-------+-------+---+---+
  |      1|     2|  3|      4|      5|  6|  7|
  +-------+------+---+-------+-------+---+---+
  ```

- when `spark.sql.caseSensitive` is `true`.

  ```
  +-------+------+---+-------+-------+---+---+
  |fieldA0|fieldB|_c2| FIELDA|fieldA4|_c5|_c6|
  +-------+------+---+-------+-------+---+---+
  |      1|     2|  3|      4|      5|  6|  7|
  +-------+------+---+-------+-------+---+---+
  ```

**In more details**,

There is a good reference about this problem, `read.csv()` in R. So, I initially wanted to propose the similar behaviour.

In case of R,  the CSV data below:

```
fieldA,fieldB,,fieldA,fieldA,,
1,2,3,4,5,6,7
```

is parsed as below:

```r
test <- read.csv(file="test.csv",header=TRUE,sep=",")
> test
  fieldA fieldB X fieldA.1 fieldA.2 X.1 X.2
1      1      2 3        4        5   6   7
```

However, Spark CSV datasource already is handling duplicated empty strings and `nullValue` as field names. So the data below:

```
,,,fieldA,,fieldB,
1,2,3,4,5,6,7
```

is parsed as below:

```scala
spark.read.format("csv").option("header", "true").load("test.csv").show()
```
```
+---+---+---+------+---+------+---+
|_c0|_c1|_c2|fieldA|_c4|fieldB|_c6|
+---+---+---+------+---+------+---+
|  1|  2|  3|     4|  5|     6|  7|
+---+---+---+------+---+------+---+
```

R starts the number for each duplicate but Spark adds the number for its position for all fields for `nullValue` and empty strings.

In terms of case-sensitivity, it seems R is case-sensitive as below: (it seems it is not configurable).

```
a,a,a,A,A
1,2,3,4,5
```

is parsed as below:

```r
test <- read.csv(file="test.csv",header=TRUE,sep=",")
> test
  a a.1 a.2 A A.1
1 1   2   3 4   5
```

## How was this patch tested?

Unit test in `CSVSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14745 from HyukjinKwon/SPARK-16896.
2016-10-11 10:21:22 +08:00
Davies Liu d5ec4a3e01 [SPARK-17738][TEST] Fix flaky test in ColumnTypeSuite
## What changes were proposed in this pull request?

The default buffer size is not big enough for randomly generated MapType.

## How was this patch tested?

Ran the tests in 100 times, it never fail (it fail 8 times before the patch).

Author: Davies Liu <davies@databricks.com>

Closes #15395 from davies/flaky_map.
2016-10-10 19:14:01 -07:00
Reynold Xin 689de92005 [SPARK-17830] Annotate spark.sql package with InterfaceStability
## What changes were proposed in this pull request?
This patch annotates the InterfaceStability level for top level classes in o.a.spark.sql and o.a.spark.sql.util packages, to experiment with this new annotation.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #15392 from rxin/SPARK-17830.
2016-10-10 11:29:09 -07:00
jiangxingbo 7e16c94f18
[HOT-FIX][SQL][TESTS] Remove unused function in SparkSqlParserSuite
## What changes were proposed in this pull request?

The function `SparkSqlParserSuite.createTempViewUsing` is not used for now and causes build failure, this PR simply removes it.

## How was this patch tested?
N/A

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15418 from jiangxb1987/parserSuite.
2016-10-10 13:49:25 +01:00
Wenchen Fan 23ddff4b2b [SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?

Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

changes for `SessionCatalog`:

1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.

changes for SQL commands:

1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.

changes for other public API

1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`

## How was this patch tested?

new tests in `SQLViewSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 15:48:57 +08:00
jiangxingbo 16590030c1 [SPARK-17741][SQL] Grammar to parse top level and nested data fields separately
## What changes were proposed in this pull request?

Currently we use the same rule to parse top level and nested data fields. For example:
```
create table tbl_x(
  id bigint,
  nested struct<col1:string,col2:string>
)
```
Shows both syntaxes. In this PR we split this rule in a top-level and nested rule.

Before this PR,
```
sql("CREATE TABLE my_tab(column1: INT)")
```
works fine.
After this PR, it will throw a `ParseException`:
```
scala> sql("CREATE TABLE my_tab(column1: INT)")
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'CREATE TABLE my_tab(column1:'(line 1, pos 27)
```

## How was this patch tested?
Add new testcases in `SparkSqlParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15346 from jiangxb1987/cdt.
2016-10-09 22:00:54 -07:00
jiangxingbo 26fbca4806 [SPARK-17832][SQL] TableIdentifier.quotedString creates un-parseable names when name contains a backtick
## What changes were proposed in this pull request?

The `quotedString` method in `TableIdentifier` and `FunctionIdentifier` produce an illegal (un-parseable) name when the name contains a backtick. For example:
```
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser._
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
val complexName = TableIdentifier("`weird`table`name", Some("`d`b`1"))
parseTableIdentifier(complexName.unquotedString) // Does not work
parseTableIdentifier(complexName.quotedString) // Does not work
parseExpression(complexName.unquotedString) // Does not work
parseExpression(complexName.quotedString) // Does not work
```
We should handle the backtick properly to make `quotedString` parseable.

## How was this patch tested?
Add new testcases in `TableIdentifierParserSuite` and `ExpressionParserSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15403 from jiangxb1987/backtick.
2016-10-09 21:52:46 -07:00
Weiqing Yang 8a6bbe095b
[MINOR][SQL] Use resource path for test_script.sh
## What changes were proposed in this pull request?
This PR modified the test case `test("script")` to use resource path for `test_script.sh`. Make the test case portable (even in IntelliJ).

## How was this patch tested?
Passed the test case.
Before:
Run `test("script")` in IntelliJ:
```
Caused by: org.apache.spark.SparkException: Subprocess exited with status 127. Error: bash: src/test/resources/test_script.sh: No such file or directory
```
After:
Test passed.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15246 from weiqingy/hivetest.
2016-10-08 12:12:35 +01:00
hyukjinkwon 24850c9415 [HOTFIX][BUILD] Do not use contains in Option in JdbcRelationProvider
## What changes were proposed in this pull request?

This PR proposes the fix the use of `contains` API which only exists from Scala 2.11.

## How was this patch tested?

Manually checked:

```scala
scala> val o: Option[Boolean] = None
o: Option[Boolean] = None

scala> o == Some(false)
res17: Boolean = false

scala> val o: Option[Boolean] = Some(true)
o: Option[Boolean] = Some(true)

scala> o == Some(false)
res18: Boolean = false

scala> val o: Option[Boolean] = Some(false)
o: Option[Boolean] = Some(false)

scala> o == Some(false)
res19: Boolean = true
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15393 from HyukjinKwon/hotfix.
2016-10-07 17:59:24 -07:00
Davies Liu 94b24b84a6 [SPARK-17806] [SQL] fix bug in join key rewritten in HashJoin
## What changes were proposed in this pull request?

In HashJoin, we try to rewrite the join key as Long to improve the performance of finding a match. The rewriting part is not well tested, has a bug that could cause wrong result when there are at least three integral columns in the joining key also the total length of the key exceed 8 bytes.

## How was this patch tested?

Added unit test to covering the rewriting with different number of columns and different data types. Manually test the reported case and confirmed that this PR fix the bug.

Author: Davies Liu <davies@databricks.com>

Closes #15390 from davies/rewrite_key.
2016-10-07 15:03:47 -07:00
Herman van Hovell 97594c29b7 [SPARK-17761][SQL] Remove MutableRow
## What changes were proposed in this pull request?
In practice we cannot guarantee that an `InternalRow` is immutable. This makes the `MutableRow` almost redundant. This PR folds `MutableRow` into `InternalRow`.

The code below illustrates the immutability issue with InternalRow:
```scala
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
val struct = new GenericMutableRow(1)
val row = InternalRow(struct, 1)
println(row)
scala> [[null], 1]
struct.setInt(0, 42)
println(row)
scala> [[42], 1]
```

This might be somewhat controversial, so feedback is appreciated.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15333 from hvanhovell/SPARK-17761.
2016-10-07 14:03:45 -07:00
Davies Liu 2badb58cdd [SPARK-15621][SQL] Support spilling for Python UDF
## What changes were proposed in this pull request?

When execute a Python UDF, we buffer the input row into as queue, then pull them out to join with the result from Python UDF. In the case that Python UDF is slow or the input row is too wide, we could ran out of memory because of the queue. Since we can't flush all the buffers (sockets) between JVM and Python process from JVM side, we can't limit the rows in the queue, otherwise it could deadlock.

This PR will manage the memory used by the queue, spill that into disk when there is no enough memory (also release the memory and disk space as soon as possible).

## How was this patch tested?

Added unit tests. Also manually ran a workload with large input row and slow python UDF (with  large broadcast) like this:

```
b = range(1<<24)
add = udf(lambda x: x + len(b), IntegerType())
df = sqlContext.range(1, 1<<26, 1, 4)
print df.select(df.id, lit("adf"*10000).alias("s"), add(df.id).alias("add")).groupBy(length("s")).sum().collect()
```

It ran out of memory (hang because of full GC) before the patch, ran smoothly after the patch.

Author: Davies Liu <davies@databricks.com>

Closes #15089 from davies/spill_udf.
2016-10-07 13:45:00 -07:00
Prashant Sharma bb1aaf28ec [SPARK-16411][SQL][STREAMING] Add textFile to Structured Streaming.
## What changes were proposed in this pull request?

Adds the textFile API which exists in DataFrameReader and serves same purpose.

## How was this patch tested?

Added corresponding testcase.

Author: Prashant Sharma <prashsh1@in.ibm.com>

Closes #14087 from ScrapCodes/textFile.
2016-10-07 11:16:24 -07:00
hyukjinkwon aa3a6841eb [SPARK-14525][SQL][FOLLOWUP] Clean up JdbcRelationProvider
## What changes were proposed in this pull request?

This PR proposes cleaning up the confusing part in `createRelation` as discussed in https://github.com/apache/spark/pull/12601/files#r80627940

Also, this PR proposes the changes below:

 - Add documentation for `batchsize` and `isolationLevel`.
 - Move property names into `JDBCOptions` so that they can be managed in a single place. which were, `fetchsize`, `batchsize`, `isolationLevel` and `driver`.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15263 from HyukjinKwon/SPARK-14525.
2016-10-07 10:52:32 -07:00
Sean Owen cff5607552 [SPARK-17707][WEBUI] Web UI prevents spark-submit application to be finished
## What changes were proposed in this pull request?

This expands calls to Jetty's simple `ServerConnector` constructor to explicitly specify a `ScheduledExecutorScheduler` that makes daemon threads. It should otherwise result in exactly the same configuration, because the other args are copied from the constructor that is currently called.

(I'm not sure we should change the Hive Thriftserver impl, but I did anyway.)

This also adds `sc.stop()` to the quick start guide example.

## How was this patch tested?

Existing tests; _pending_ at least manual verification of the fix.

Author: Sean Owen <sowen@cloudera.com>

Closes #15381 from srowen/SPARK-17707.
2016-10-07 10:31:41 -07:00
hyukjinkwon 2b01d3c701
[SPARK-16960][SQL] Deprecate approxCountDistinct, toDegrees and toRadians according to FunctionRegistry
## What changes were proposed in this pull request?

It seems `approxCountDistinct`, `toDegrees` and `toRadians` are also missed while matching the names to the ones in `FunctionRegistry`. (please see [approx_count_distinct](5c2ae79bfc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L244)), [degrees](5c2ae79bfc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L203)) and [radians](5c2ae79bfc/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala (L222)) in `FunctionRegistry`).

I took a scan between `functions.scala` and `FunctionRegistry` and it seems these are all left. For `countDistinct` and `sumDistinct`, they are not registered in `FunctionRegistry`.

This PR deprecates `approxCountDistinct`, `toDegrees` and `toRadians` and introduces `approx_count_distinct`, `degrees` and `radians`.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Hyukjin Kwon <gurwls223@gmail.com>

Closes #14538 from HyukjinKwon/SPARK-16588-followup.
2016-10-07 11:49:34 +01:00
Shixiong Zhu 9a48e60e63 [SPARK-17780][SQL] Report Throwable to user in StreamExecution
## What changes were proposed in this pull request?

When using an incompatible source for structured streaming, it may throw NoClassDefFoundError. It's better to just catch Throwable and report it to the user since the streaming thread is dying.

## How was this patch tested?

`test("NoClassDefFoundError from an incompatible source")`

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15352 from zsxwing/SPARK-17780.
2016-10-06 12:51:12 -07:00
Reynold Xin 79accf45ac [SPARK-17798][SQL] Remove redundant Experimental annotations in sql.streaming
## What changes were proposed in this pull request?
I was looking through API annotations to catch mislabeled APIs, and realized DataStreamReader and DataStreamWriter classes are already annotated as Experimental, and as a result there is no need to annotate each method within them.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #15373 from rxin/SPARK-17798.
2016-10-06 10:33:45 -07:00
Dongjoon Hyun 92b7e57280 [SPARK-17750][SQL] Fix CREATE VIEW with INTERVAL arithmetic.
## What changes were proposed in this pull request?

Currently, Spark raises `RuntimeException` when creating a view with timestamp with INTERVAL arithmetic like the following. The root cause is the arithmetic expression, `TimeAdd`, was transformed into `timeadd` function as a VIEW definition. This PR fixes the SQL definition of `TimeAdd` and `TimeSub` expressions.

```scala
scala> sql("CREATE TABLE dates (ts TIMESTAMP)")

scala> sql("CREATE VIEW view1 AS SELECT ts + INTERVAL 1 DAY FROM dates")
java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ...
```

## How was this patch tested?

Pass Jenkins with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15318 from dongjoon-hyun/SPARK-17750.
2016-10-06 09:42:30 -07:00
Shixiong Zhu b678e465af [SPARK-17346][SQL][TEST-MAVEN] Generate the sql test jar to fix the maven build
## What changes were proposed in this pull request?

Generate the sql test jar to fix the maven build

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15368 from zsxwing/sql-test-jar.
2016-10-05 18:11:31 -07:00
Shixiong Zhu 9293734d35 [SPARK-17346][SQL] Add Kafka source for Structured Streaming
## What changes were proposed in this pull request?

This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source.

It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing

tdas did most of work and part of them was inspired by koeninger's work.

### Introduction

The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows:

Column | Type
---- | ----
key | binary
value | binary
topic | string
partition | int
offset | long
timestamp | long
timestampType | int

The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic.

### Configuration

The user can use `DataStreamReader.option` to set the following configurations.

Kafka Source's options | value | default | meaning
------ | ------- | ------ | -----
startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off.
failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected.
subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors
fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets.
fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets

Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")`

### Usage

* Subscribe to 1 topic
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1")
  .load()
```

* Subscribe to multiple topics
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1,topic2")
  .load()
```

* Subscribe to a pattern
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribePattern", "topic.*")
  .load()
```

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Author: cody koeninger <cody@koeninger.org>

Closes #15102 from zsxwing/kafka-source.
2016-10-05 16:45:45 -07:00
Herman van Hovell 5fd54b994e [SPARK-17758][SQL] Last returns wrong result in case of empty partition
## What changes were proposed in this pull request?
The result of the `Last` function can be wrong when the last partition processed is empty. It can return `null` instead of the expected value. For example, this can happen when we process partitions in the following order:
```
- Partition 1 [Row1, Row2]
- Partition 2 [Row3]
- Partition 3 []
```
In this case the `Last` function will currently return a null, instead of the value of `Row3`.

This PR fixes this by adding a `valueSet` flag to the `Last` function.

## How was this patch tested?
We only used end to end tests for `DeclarativeAggregateFunction`s. I have added an evaluator for these functions so we can tests them in catalyst. I have added a `LastTestSuite` to test the `Last` aggregate function.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15348 from hvanhovell/SPARK-17758.
2016-10-05 16:05:30 -07:00
Dongjoon Hyun 6a05eb24d0 [SPARK-17328][SQL] Fix NPE with EXPLAIN DESCRIBE TABLE
## What changes were proposed in this pull request?

This PR fixes the following NPE scenario in two ways.

**Reported Error Scenario**
```scala
scala> sql("EXPLAIN DESCRIBE TABLE x").show(truncate = false)
INFO SparkSqlParser: Parsing command: EXPLAIN DESCRIBE TABLE x
java.lang.NullPointerException
```

- **DESCRIBE**: Extend `DESCRIBE` syntax to accept `TABLE`.
- **EXPLAIN**: Prevent NPE in case of the parsing failure of target statement, e.g., `EXPLAIN DESCRIBE TABLES x`.

## How was this patch tested?

Pass the Jenkins test with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15357 from dongjoon-hyun/SPARK-17328.
2016-10-05 10:52:43 -07:00
Herman van Hovell 89516c1c4a [SPARK-17258][SQL] Parse scientific decimal literals as decimals
## What changes were proposed in this pull request?
Currently Spark SQL parses regular decimal literals (e.g. `10.00`) as decimals and scientific decimal literals (e.g. `10.0e10`) as doubles. The difference between the two confuses most users. This PR unifies the parsing behavior and also parses scientific decimal literals as decimals.

This implications in tests are limited to a single Hive compatibility test.

## How was this patch tested?
Updated tests in `ExpressionParserSuite` and `SQLQueryTestSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #14828 from hvanhovell/SPARK-17258.
2016-10-04 23:48:26 -07:00
Tejas Patil a99743d053 [SPARK-17495][SQL] Add Hash capability semantically equivalent to Hive's
## What changes were proposed in this pull request?

Jira : https://issues.apache.org/jira/browse/SPARK-17495

Spark internally uses Murmur3Hash for partitioning. This is different from the one used by Hive. For queries which use bucketing this leads to different results if one tries the same query on both engines. For us, we want users to have backward compatibility to that one can switch parts of applications across the engines without observing regressions.

This PR includes `HiveHash`, `HiveHashFunction`, `HiveHasher` which mimics Hive's hashing at https://github.com/apache/hive/blob/master/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/ObjectInspectorUtils.java#L638

I am intentionally not introducing any usages of this hash function in rest of the code to keep this PR small. My eventual goal is to have Hive bucketing support in Spark. Once this PR gets in, I will make hash function pluggable in relevant areas (eg. `HashPartitioning`'s `partitionIdExpression` has Murmur3 hardcoded : https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala#L265)

## How was this patch tested?

Added `HiveHashSuite`

Author: Tejas Patil <tejasp@fb.com>

Closes #15047 from tejasapatil/SPARK-17495_hive_hash.
2016-10-04 18:59:31 -07:00
Marcelo Vanzin 8d969a2125 [SPARK-17549][SQL] Only collect table size stat in driver for cached relation.
This reverts commit 9ac68dbc57. Turns out
the original fix was correct.

Original change description:
The existing code caches all stats for all columns for each partition
in the driver; for a large relation, this causes extreme memory usage,
which leads to gc hell and application failures.

It seems that only the size in bytes of the data is actually used in the
driver, so instead just colllect that. In executors, the full stats are
still kept, but that's not a big problem; we expect the data to be distributed
and thus not really incur in too much memory pressure in each individual
executor.

There are also potential improvements on the executor side, since the data
being stored currently is very wasteful (e.g. storing boxed types vs.
primitive types for stats). But that's a separate issue.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15304 from vanzin/SPARK-17549.2.
2016-10-04 09:38:44 -07:00
sumansomasundar 7d51608835
[SPARK-16962][CORE][SQL] Fix misaligned record accesses for SPARC architectures
## What changes were proposed in this pull request?

Made changes to record length offsets to make them uniform throughout various areas of Spark core and unsafe

## How was this patch tested?

This change affects only SPARC architectures and was tested on X86 architectures as well for regression.

Author: sumansomasundar <suman.somasundar@oracle.com>

Closes #14762 from sumansomasundar/master.
2016-10-04 10:31:56 +01:00
Ergin Seyfe d2dc8c4a16 [SPARK-17773] Input/Output] Add VoidObjectInspector
## What changes were proposed in this pull request?
Added VoidObjectInspector to the list of PrimitiveObjectInspectors

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Executing following query was failing.
select SOME_UDAF*(a.arr)
from (
select Array(null) as arr from dim_one_row
) a

After the fix, I am getting the correct output:
res0: Array[org.apache.spark.sql.Row] = Array([null])

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #15337 from seyfe/add_void_object_inspector.
2016-10-03 23:28:39 -07:00
Takuya UESHIN b1b47274bf [SPARK-17702][SQL] Code generation including too many mutable states exceeds JVM size limit.
## What changes were proposed in this pull request?

Code generation including too many mutable states exceeds JVM size limit to extract values from `references` into fields in the constructor.
We should split the generated extractions in the constructor into smaller functions.

## How was this patch tested?

I added some tests to check if the generated codes for the expressions exceed or not.

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

Closes #15275 from ueshin/issues/SPARK-17702.
2016-10-03 21:48:58 -07:00
Dongjoon Hyun c571cfb2d0 [SPARK-17112][SQL] "select null" via JDBC triggers IllegalArgumentException in Thriftserver
## What changes were proposed in this pull request?

Currently, Spark Thrift Server raises `IllegalArgumentException` for queries whose column types are `NullType`, e.g., `SELECT null` or `SELECT if(true,null,null)`. This PR fixes that by returning `void` like Hive 1.2.

**Before**
```sql
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "select null"
Connecting to jdbc:hive2://localhost:10000
Connected to: Spark SQL (version 2.1.0-SNAPSHOT)
Driver: Hive JDBC (version 1.2.1.spark2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Error: java.lang.IllegalArgumentException: Unrecognized type name: null (state=,code=0)
Closing: 0: jdbc:hive2://localhost:10000

$ bin/beeline -u jdbc:hive2://localhost:10000 -e "select if(true,null,null)"
Connecting to jdbc:hive2://localhost:10000
Connected to: Spark SQL (version 2.1.0-SNAPSHOT)
Driver: Hive JDBC (version 1.2.1.spark2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Error: java.lang.IllegalArgumentException: Unrecognized type name: null (state=,code=0)
Closing: 0: jdbc:hive2://localhost:10000
```

**After**
```sql
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "select null"
Connecting to jdbc:hive2://localhost:10000
Connected to: Spark SQL (version 2.1.0-SNAPSHOT)
Driver: Hive JDBC (version 1.2.1.spark2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
+-------+--+
| NULL  |
+-------+--+
| NULL  |
+-------+--+
1 row selected (3.242 seconds)
Beeline version 1.2.1.spark2 by Apache Hive
Closing: 0: jdbc:hive2://localhost:10000

$ bin/beeline -u jdbc:hive2://localhost:10000 -e "select if(true,null,null)"
Connecting to jdbc:hive2://localhost:10000
Connected to: Spark SQL (version 2.1.0-SNAPSHOT)
Driver: Hive JDBC (version 1.2.1.spark2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
+-------------------------+--+
| (IF(true, NULL, NULL))  |
+-------------------------+--+
| NULL                    |
+-------------------------+--+
1 row selected (0.201 seconds)
Beeline version 1.2.1.spark2 by Apache Hive
Closing: 0: jdbc:hive2://localhost:10000
```

## How was this patch tested?

* Pass the Jenkins test with a new testsuite.
* Also, Manually, after starting Spark Thrift Server, run the following command.
```sql
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "select null"
$ bin/beeline -u jdbc:hive2://localhost:10000 -e "select if(true,null,null)"
```

**Hive 1.2**
```sql
hive> create table null_table as select null;
hive> desc null_table;
OK
_c0                     void
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15325 from dongjoon-hyun/SPARK-17112.
2016-10-03 21:28:16 -07:00
Herman van Hovell 2bbecdec20 [SPARK-17753][SQL] Allow a complex expression as the input a value based case statement
## What changes were proposed in this pull request?
We currently only allow relatively simple expressions as the input for a value based case statement. Expressions like `case (a > 1) or (b = 2) when true then 1 when false then 0 end` currently fail. This PR adds support for such expressions.

## How was this patch tested?
Added a test to the ExpressionParserSuite.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15322 from hvanhovell/SPARK-17753.
2016-10-03 19:32:59 -07:00
Zhenhua Wang 7bf9212764 [SPARK-17073][SQL] generate column-level statistics
## What changes were proposed in this pull request?

Generate basic column statistics for all the atomic types:
- numeric types: max, min, num of nulls, ndv (number of distinct values)
- date/timestamp types: they are also represented as numbers internally, so they have the same stats as above.
- string: avg length, max length, num of nulls, ndv
- binary: avg length, max length, num of nulls
- boolean: num of nulls, num of trues, num of falsies

Also support storing and loading these statistics.

One thing to notice:
We support analyzing columns independently, e.g.:
sql1: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key;`
sql2: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS value;`
when running sql2 to collect column stats for `value`, we don’t remove stats of columns `key` which are analyzed in sql1 and not in sql2. As a result, **users need to guarantee consistency** between sql1 and sql2. If the table has been changed before sql2, users should re-analyze column `key` when they want to analyze column `value`:
`ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key, value;`

## How was this patch tested?

add unit tests

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #15090 from wzhfy/colStats.
2016-10-03 10:12:02 -07:00
Tao LI 76dc2d9073 [SPARK-14914][CORE][SQL] Skip/fix some test cases on Windows due to limitation of Windows
## What changes were proposed in this pull request?

This PR proposes to fix/skip some tests failed on Windows. This PR takes over https://github.com/apache/spark/pull/12696.

**Before**

- **SparkSubmitSuite**

  ```
[info] - launch simple application with spark-submit *** FAILED *** (202 milliseconds)
[info]   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specifie

[info] - includes jars passed in through --jars *** FAILED *** (1 second, 625 milliseconds)
[info]   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```

- **DiskStoreSuite**

  ```
[info] - reads of memory-mapped and non memory-mapped files are equivalent *** FAILED *** (1 second, 78 milliseconds)
[info]   diskStoreMapped.remove(blockId) was false (DiskStoreSuite.scala:41)
```

**After**

- **SparkSubmitSuite**

  ```
[info] - launch simple application with spark-submit (578 milliseconds)
[info] - includes jars passed in through --jars (1 second, 875 milliseconds)
```

- **DiskStoreSuite**

  ```
[info] DiskStoreSuite:
[info] - reads of memory-mapped and non memory-mapped files are equivalent !!! CANCELED !!! (766 milliseconds
```

For `CreateTableAsSelectSuite` and `FsHistoryProviderSuite`, I could not reproduce as the Java version seems higher than the one that has the bugs about `setReadable(..)` and `setWritable(...)` but as they are bugs reported clearly, it'd be sensible to skip those. We should revert the changes for both back as soon as we drop the support of Java 7.

## How was this patch tested?

Manually tested via AppVeyor.

Closes #12696

Author: Tao LI <tl@microsoft.com>
Author: U-FAREAST\tl <tl@microsoft.com>
Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15320 from HyukjinKwon/SPARK-14914.
2016-10-02 16:01:02 -07:00
Sital Kedia f8d7fade4b [SPARK-17509][SQL] When wrapping catalyst datatype to Hive data type avoid…
## What changes were proposed in this pull request?

When wrapping catalyst datatypes to Hive data type, wrap function was doing an expensive pattern matching which was consuming around 11% of cpu time. Avoid the pattern matching by returning the wrapper only once and reuse it.

## How was this patch tested?

Tested by running the job on cluster and saw around 8% cpu improvements.

Author: Sital Kedia <skedia@fb.com>

Closes #15064 from sitalkedia/skedia/hive_wrapper.
2016-10-02 15:47:36 -07:00
Herman van Hovell af6ece33d3 [SPARK-17717][SQL] Add Exist/find methods to Catalog [FOLLOW-UP]
## What changes were proposed in this pull request?
We added find and exists methods for Databases, Tables and Functions to the user facing Catalog in PR https://github.com/apache/spark/pull/15301. However, it was brought up that the semantics of the  `find` methods are more in line a `get` method (get an object or else fail). So we rename these in this PR.

## How was this patch tested?
Existing tests.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15308 from hvanhovell/SPARK-17717-2.
2016-10-01 00:50:16 -07:00
Eric Liang 4bcd9b728b [SPARK-17740] Spark tests should mock / interpose HDFS to ensure that streams are closed
## What changes were proposed in this pull request?

As a followup to SPARK-17666, ensure filesystem connections are not leaked at least in unit tests. This is done here by intercepting filesystem calls as suggested by JoshRosen . At the end of each test, we assert no filesystem streams are left open.

This applies to all tests using SharedSQLContext or SharedSparkContext.

## How was this patch tested?

I verified that tests in sql and core are indeed using the filesystem backend, and fixed the detected leaks. I also checked that reverting https://github.com/apache/spark/pull/15245 causes many actual test failures due to connection leaks.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15306 from ericl/sc-4672.
2016-09-30 23:51:36 -07:00
Dongjoon Hyun aef506e39a [SPARK-17739][SQL] Collapse adjacent similar Window operators
## What changes were proposed in this pull request?

Currently, Spark does not collapse adjacent windows with the same partitioning and sorting. This PR implements `CollapseWindow` optimizer to do the followings.

1. If the partition specs and order specs are the same, collapse into the parent.
2. If the partition specs are the same and one order spec is a prefix of the other, collapse to the more specific one.

For example:
```scala
val df = spark.range(1000).select($"id" % 100 as "grp", $"id", rand() as "col1", rand() as "col2")

// Add summary statistics for all columns
import org.apache.spark.sql.expressions.Window
val cols = Seq("id", "col1", "col2")
val window = Window.partitionBy($"grp").orderBy($"id")
val result = cols.foldLeft(df) { (base, name) =>
  base.withColumn(s"${name}_avg", avg(col(name)).over(window))
      .withColumn(s"${name}_stddev", stddev(col(name)).over(window))
      .withColumn(s"${name}_min", min(col(name)).over(window))
      .withColumn(s"${name}_max", max(col(name)).over(window))
}
```

**Before**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#234], [grp#17L], [id#14L ASC NULLS FIRST]
+- Window [min(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#216], [grp#17L], [id#14L ASC NULLS FIRST]
   +- Window [stddev_samp(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#191], [grp#17L], [id#14L ASC NULLS FIRST]
      +- Window [avg(col2#19) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#167], [grp#17L], [id#14L ASC NULLS FIRST]
         +- Window [max(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#152], [grp#17L], [id#14L ASC NULLS FIRST]
            +- Window [min(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#138], [grp#17L], [id#14L ASC NULLS FIRST]
               +- Window [stddev_samp(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#117], [grp#17L], [id#14L ASC NULLS FIRST]
                  +- Window [avg(col1#18) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#97], [grp#17L], [id#14L ASC NULLS FIRST]
                     +- Window [max(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#86L], [grp#17L], [id#14L ASC NULLS FIRST]
                        +- Window [min(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#76L], [grp#17L], [id#14L ASC NULLS FIRST]
                           +- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, id_stddev#42]
                              +- Window [stddev_samp(_w0#59) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#42], [grp#17L], [id#14L ASC NULLS FIRST]
                                 +- *Project [grp#17L, id#14L, col1#18, col2#19, id_avg#26, cast(id#14L as double) AS _w0#59]
                                    +- Window [avg(id#14L) windowspecdefinition(grp#17L, id#14L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#26], [grp#17L], [id#14L ASC NULLS FIRST]
                                       +- *Sort [grp#17L ASC NULLS FIRST, id#14L ASC NULLS FIRST], false, 0
                                          +- Exchange hashpartitioning(grp#17L, 200)
                                             +- *Project [(id#14L % 100) AS grp#17L, id#14L, rand(-6329949029880411066) AS col1#18, rand(-7251358484380073081) AS col2#19]
                                                +- *Range (0, 1000, step=1, splits=Some(8))
```

**After**
```scala
scala> result.explain
== Physical Plan ==
Window [max(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_max#220, min(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_min#202, stddev_samp(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_stddev#177, avg(col2#5) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col2_avg#153, max(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_max#138, min(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_min#124, stddev_samp(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_stddev#103, avg(col1#4) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS col1_avg#83, max(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_max#72L, min(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_min#62L], [grp#3L], [id#0L ASC NULLS FIRST]
+- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, id_stddev#28]
   +- Window [stddev_samp(_w0#45) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_stddev#28], [grp#3L], [id#0L ASC NULLS FIRST]
      +- *Project [grp#3L, id#0L, col1#4, col2#5, id_avg#12, cast(id#0L as double) AS _w0#45]
         +- Window [avg(id#0L) windowspecdefinition(grp#3L, id#0L ASC NULLS FIRST, RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS id_avg#12], [grp#3L], [id#0L ASC NULLS FIRST]
            +- *Sort [grp#3L ASC NULLS FIRST, id#0L ASC NULLS FIRST], false, 0
               +- Exchange hashpartitioning(grp#3L, 200)
                  +- *Project [(id#0L % 100) AS grp#3L, id#0L, rand(6537478539664068821) AS col1#4, rand(-8961093871295252795) AS col2#5]
                     +- *Range (0, 1000, step=1, splits=Some(8))
```

## How was this patch tested?

Pass the Jenkins tests with a newly added testsuite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15317 from dongjoon-hyun/SPARK-17739.
2016-09-30 21:05:06 -07:00
Takuya UESHIN 81455a9cd9 [SPARK-17703][SQL] Add unnamed version of addReferenceObj for minor objects.
## What changes were proposed in this pull request?

There are many minor objects in references, which are extracted to the generated class field, e.g. `errMsg` in `GetExternalRowField` or `ValidateExternalType`, but number of fields in class is limited so we should reduce the number.
This pr adds unnamed version of `addReferenceObj` for these minor objects not to store the object into field but refer it from the `references` field at the time of use.

## How was this patch tested?

Existing tests.

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

Closes #15276 from ueshin/issues/SPARK-17703.
2016-09-30 17:31:59 -07:00
Davies Liu f327e16863 [SPARK-17738] [SQL] fix ARRAY/MAP in columnar cache
## What changes were proposed in this pull request?

The actualSize() of array and map is different from the actual size, the header is Int, rather than Long.

## How was this patch tested?

The flaky test should be fixed.

Author: Davies Liu <davies@databricks.com>

Closes #15305 from davies/fix_MAP.
2016-09-30 09:59:12 -07:00
Herman van Hovell 74ac1c4381 [SPARK-17717][SQL] Add exist/find methods to Catalog.
## What changes were proposed in this pull request?
The current user facing catalog does not implement methods for checking object existence or finding objects. You could theoretically do this using the `list*` commands, but this is rather cumbersome and can actually be costly when there are many objects. This PR adds `exists*` and `find*` methods for Databases, Table and Functions.

## How was this patch tested?
Added tests to `org.apache.spark.sql.internal.CatalogSuite`

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15301 from hvanhovell/SPARK-17717.
2016-09-29 17:56:32 -07:00
Dongjoon Hyun 4ecc648ad7 [SPARK-17612][SQL] Support DESCRIBE table PARTITION SQL syntax
## What changes were proposed in this pull request?

This PR implements `DESCRIBE table PARTITION` SQL Syntax again. It was supported until Spark 1.6.2, but was dropped since 2.0.0.

**Spark 1.6.2**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res1: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res2: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
+----------------------------------------------------------------+
|result                                                          |
+----------------------------------------------------------------+
|a                      string                                   |
|b                      int                                      |
|c                      string                                   |
|d                      string                                   |
|                                                                |
|# Partition Information                                         |
|# col_name             data_type               comment          |
|                                                                |
|c                      string                                   |
|d                      string                                   |
+----------------------------------------------------------------+
```

**Spark 2.0**
- **Before**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
org.apache.spark.sql.catalyst.parser.ParseException:
Unsupported SQL statement
```

- **After**
```scala
scala> sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
res1: org.apache.spark.sql.DataFrame = []

scala> sql("DESC partitioned_table PARTITION (c='Us', d=1)").show(false)
+-----------------------+---------+-------+
|col_name               |data_type|comment|
+-----------------------+---------+-------+
|a                      |string   |null   |
|b                      |int      |null   |
|c                      |string   |null   |
|d                      |string   |null   |
|# Partition Information|         |       |
|# col_name             |data_type|comment|
|c                      |string   |null   |
|d                      |string   |null   |
+-----------------------+---------+-------+

scala> sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)").show(100,false)
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+
|col_name                                                                                                                                                                                                                                                                                                                                                                                                                                                                           |data_type|comment|
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+
|a                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|b                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |int      |null   |
|c                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|d                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|# Partition Information                                                                                                                                                                                                                                                                                                                                                                                                                                                            |         |       |
|# col_name                                                                                                                                                                                                                                                                                                                                                                                                                                                                         |data_type|comment|
|c                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|d                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |string   |null   |
|                                                                                                                                                                                                                                                                                                                                                                                                                                                                                   |         |       |
|Detailed Partition Information CatalogPartition(
        Partition Values: [Us, 1]
        Storage(Location: file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1])
        Partition Parameters:{transient_lastDdlTime=1475001066})|         |       |
+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+---------+-------+

scala> sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)").show(100,false)
+--------------------------------+---------------------------------------------------------------------------------------+-------+
|col_name                        |data_type                                                                              |comment|
+--------------------------------+---------------------------------------------------------------------------------------+-------+
|a                               |string                                                                                 |null   |
|b                               |int                                                                                    |null   |
|c                               |string                                                                                 |null   |
|d                               |string                                                                                 |null   |
|# Partition Information         |                                                                                       |       |
|# col_name                      |data_type                                                                              |comment|
|c                               |string                                                                                 |null   |
|d                               |string                                                                                 |null   |
|                                |                                                                                       |       |
|# Detailed Partition Information|                                                                                       |       |
|Partition Value:                |[Us, 1]                                                                                |       |
|Database:                       |default                                                                                |       |
|Table:                          |partitioned_table                                                                      |       |
|Location:                       |file:/Users/dhyun/SPARK-17612-DESC-PARTITION/spark-warehouse/partitioned_table/c=Us/d=1|       |
|Partition Parameters:           |                                                                                       |       |
|  transient_lastDdlTime         |1475001066                                                                             |       |
|                                |                                                                                       |       |
|# Storage Information           |                                                                                       |       |
|SerDe Library:                  |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe                                     |       |
|InputFormat:                    |org.apache.hadoop.mapred.TextInputFormat                                               |       |
|OutputFormat:                   |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat                             |       |
|Compressed:                     |No                                                                                     |       |
|Storage Desc Parameters:        |                                                                                       |       |
|  serialization.format          |1                                                                                      |       |
+--------------------------------+---------------------------------------------------------------------------------------+-------+
```

## How was this patch tested?

Pass the Jenkins tests with a new testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15168 from dongjoon-hyun/SPARK-17612.
2016-09-29 15:30:18 -07:00
Liang-Chi Hsieh 566d7f2827 [SPARK-17653][SQL] Remove unnecessary distincts in multiple unions
## What changes were proposed in this pull request?

Currently for `Union [Distinct]`, a `Distinct` operator is necessary to be on the top of `Union`. Once there are adjacent `Union [Distinct]`,  there will be multiple `Distinct` in the query plan.

E.g.,

For a query like: select 1 a union select 2 b union select 3 c

Before this patch, its physical plan looks like:

    *HashAggregate(keys=[a#13], functions=[])
    +- Exchange hashpartitioning(a#13, 200)
       +- *HashAggregate(keys=[a#13], functions=[])
          +- Union
             :- *HashAggregate(keys=[a#13], functions=[])
             :  +- Exchange hashpartitioning(a#13, 200)
             :     +- *HashAggregate(keys=[a#13], functions=[])
             :        +- Union
             :           :- *Project [1 AS a#13]
             :           :  +- Scan OneRowRelation[]
             :           +- *Project [2 AS b#14]
             :              +- Scan OneRowRelation[]
             +- *Project [3 AS c#15]
                +- Scan OneRowRelation[]

Only the top distinct should be necessary.

After this patch, the physical plan looks like:

    *HashAggregate(keys=[a#221], functions=[], output=[a#221])
    +- Exchange hashpartitioning(a#221, 5)
       +- *HashAggregate(keys=[a#221], functions=[], output=[a#221])
          +- Union
             :- *Project [1 AS a#221]
             :  +- Scan OneRowRelation[]
             :- *Project [2 AS b#222]
             :  +- Scan OneRowRelation[]
             +- *Project [3 AS c#223]
                +- Scan OneRowRelation[]

## How was this patch tested?

Jenkins tests.

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

Closes #15238 from viirya/remove-extra-distinct-union.
2016-09-29 14:30:23 -07:00
Michael Armbrust fe33121a53 [SPARK-17699] Support for parsing JSON string columns
Spark SQL has great support for reading text files that contain JSON data.  However, in many cases the JSON data is just one column amongst others.  This is particularly true when reading from sources such as Kafka.  This PR adds a new functions `from_json` that converts a string column into a nested `StructType` with a user specified schema.

Example usage:
```scala
val df = Seq("""{"a": 1}""").toDS()
val schema = new StructType().add("a", IntegerType)

df.select(from_json($"value", schema) as 'json) // => [json: <a: int>]
```

This PR adds support for java, scala and python.  I leveraged our existing JSON parsing support by moving it into catalyst (so that we could define expressions using it).  I left SQL out for now, because I'm not sure how users would specify a schema.

Author: Michael Armbrust <michael@databricks.com>

Closes #15274 from marmbrus/jsonParser.
2016-09-29 13:01:10 -07:00
Sean Owen b35b0dbbfa
[SPARK-17614][SQL] sparkSession.read() .jdbc(***) use the sql syntax "where 1=0" that Cassandra does not support
## What changes were proposed in this pull request?

Use dialect's table-exists query rather than hard-coded WHERE 1=0 query

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15196 from srowen/SPARK-17614.
2016-09-29 08:24:34 -04:00
Josh Rosen 37eb9184f1 [SPARK-17712][SQL] Fix invalid pushdown of data-independent filters beneath aggregates
## What changes were proposed in this pull request?

This patch fixes a minor correctness issue impacting the pushdown of filters beneath aggregates. Specifically, if a filter condition references no grouping or aggregate columns (e.g. `WHERE false`) then it would be incorrectly pushed beneath an aggregate.

Intuitively, the only case where you can push a filter beneath an aggregate is when that filter is deterministic and is defined over the grouping columns / expressions, since in that case the filter is acting to exclude entire groups from the query (like a `HAVING` clause). The existing code would only push deterministic filters beneath aggregates when all of the filter's references were grouping columns, but this logic missed the case where a filter has no references. For example, `WHERE false` is deterministic but is independent of the actual data.

This patch fixes this minor bug by adding a new check to ensure that we don't push filters beneath aggregates when those filters don't reference any columns.

## How was this patch tested?

New regression test in FilterPushdownSuite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15289 from JoshRosen/SPARK-17712.
2016-09-28 19:03:05 -07:00
Herman van Hovell 7d09232028 [SPARK-17641][SQL] Collect_list/Collect_set should not collect null values.
## What changes were proposed in this pull request?
We added native versions of `collect_set` and `collect_list` in Spark 2.0. These currently also (try to) collect null values, this is different from the original Hive implementation. This PR fixes this by adding a null check to the `Collect.update` method.

## How was this patch tested?
Added a regression test to `DataFrameAggregateSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15208 from hvanhovell/SPARK-17641.
2016-09-28 16:25:10 -07:00
Eric Liang 557d6e3227 [SPARK-17713][SQL] Move row-datasource related tests out of JDBCSuite
## What changes were proposed in this pull request?

As a followup for https://github.com/apache/spark/pull/15273 we should move non-JDBC specific tests out of that suite.

## How was this patch tested?

Ran the test.

Author: Eric Liang <ekl@databricks.com>

Closes #15287 from ericl/spark-17713.
2016-09-28 16:20:49 -07:00
Eric Liang a6cfa3f38b [SPARK-17673][SQL] Incorrect exchange reuse with RowDataSourceScan
## What changes were proposed in this pull request?

It seems the equality check for reuse of `RowDataSourceScanExec` nodes doesn't respect the output schema. This can cause self-joins or unions over the same underlying data source to return incorrect results if they select different fields.

## How was this patch tested?

New unit test passes after the fix.

Author: Eric Liang <ekl@databricks.com>

Closes #15273 from ericl/spark-17673.
2016-09-28 13:22:45 -07:00
Josh Rosen b03b4adf6d [SPARK-17666] Ensure that RecordReaders are closed by data source file scans
## What changes were proposed in this pull request?

This patch addresses a potential cause of resource leaks in data source file scans. As reported in [SPARK-17666](https://issues.apache.org/jira/browse/SPARK-17666), tasks which do not fully-consume their input may cause file handles / network connections (e.g. S3 connections) to be leaked. Spark's `NewHadoopRDD` uses a TaskContext callback to [close its record readers](https://github.com/apache/spark/blame/master/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L208), but the new data source file scans will only close record readers once their iterators are fully-consumed.

This patch modifies `RecordReaderIterator` and `HadoopFileLinesReader` to add `close()` methods and modifies all six implementations of `FileFormat.buildReader()` to register TaskContext task completion callbacks to guarantee that cleanup is eventually performed.

## How was this patch tested?

Tested manually for now.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15245 from JoshRosen/SPARK-17666-close-recordreader.
2016-09-27 17:52:57 -07:00
Josh Rosen 2f84a68660 [SPARK-17618] Guard against invalid comparisons between UnsafeRow and other formats
This patch ports changes from #15185 to Spark 2.x. In that patch, a  correctness bug in Spark 1.6.x which was caused by an invalid `equals()` comparison between an `UnsafeRow` and another row of a different format. Spark 2.x is not affected by that specific correctness bug but it can still reap the error-prevention benefits of that patch's changes, which modify  ``UnsafeRow.equals()` to throw an IllegalArgumentException if it is called with an object that is not an `UnsafeRow`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15265 from JoshRosen/SPARK-17618-master.
2016-09-27 14:14:27 -07:00
Reynold Xin 67c73052b8 [SPARK-17677][SQL] Break WindowExec.scala into multiple files
## What changes were proposed in this pull request?
As of Spark 2.0, all the window function execution code are in WindowExec.scala. This file is pretty large (over 1k loc) and has a lot of different abstractions in them. This patch creates a new package sql.execution.window, moves WindowExec.scala in it, and breaks WindowExec.scala into multiple, more maintainable pieces:

- AggregateProcessor.scala
- BoundOrdering.scala
- RowBuffer.scala
- WindowExec
- WindowFunctionFrame.scala

## How was this patch tested?
This patch mostly moves code around, and should not change any existing test coverage.

Author: Reynold Xin <rxin@databricks.com>

Closes #15252 from rxin/SPARK-17677.
2016-09-27 12:37:19 -07:00
gatorsmile 2ab24a7bf6 [SPARK-17660][SQL] DESC FORMATTED for VIEW Lacks View Definition
### What changes were proposed in this pull request?
Before this PR, `DESC FORMATTED` does not have a section for the view definition. We should add it for permanent views, like what Hive does.

```
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------+-------+
|col_name                    |data_type                                                                                                                            |comment|
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------+-------+
|a                           |int                                                                                                                                  |null   |
|                            |                                                                                                                                     |       |
|# Detailed Table Information|                                                                                                                                     |       |
|Database:                   |default                                                                                                                              |       |
|Owner:                      |xiaoli                                                                                                                               |       |
|Create Time:                |Sat Sep 24 21:46:19 PDT 2016                                                                                                         |       |
|Last Access Time:           |Wed Dec 31 16:00:00 PST 1969                                                                                                         |       |
|Location:                   |                                                                                                                                     |       |
|Table Type:                 |VIEW                                                                                                                                 |       |
|Table Parameters:           |                                                                                                                                     |       |
|  transient_lastDdlTime     |1474778779                                                                                                                           |       |
|                            |                                                                                                                                     |       |
|# Storage Information       |                                                                                                                                     |       |
|SerDe Library:              |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe                                                                                   |       |
|InputFormat:                |org.apache.hadoop.mapred.SequenceFileInputFormat                                                                                     |       |
|OutputFormat:               |org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat                                                                            |       |
|Compressed:                 |No                                                                                                                                   |       |
|Storage Desc Parameters:    |                                                                                                                                     |       |
|  serialization.format      |1                                                                                                                                    |       |
|                            |                                                                                                                                     |       |
|# View Information          |                                                                                                                                     |       |
|View Original Text:         |SELECT * FROM tbl                                                                                                                    |       |
|View Expanded Text:         |SELECT `gen_attr_0` AS `a` FROM (SELECT `gen_attr_0` FROM (SELECT `a` AS `gen_attr_0` FROM `default`.`tbl`) AS gen_subquery_0) AS tbl|       |
+----------------------------+-------------------------------------------------------------------------------------------------------------------------------------+-------+
```

### How was this patch tested?
Added a test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15234 from gatorsmile/descFormattedView.
2016-09-27 10:52:26 -07:00
Reynold Xin 120723f934 [SPARK-17682][SQL] Mark children as final for unary, binary, leaf expressions and plan nodes
## What changes were proposed in this pull request?
This patch marks the children method as final in unary, binary, and leaf expressions and plan nodes (both logical plan and physical plan), as brought up in http://apache-spark-developers-list.1001551.n3.nabble.com/Should-LeafExpression-have-children-final-override-like-Nondeterministic-td19104.html

## How was this patch tested?
This is a simple modifier change and has no impact on test coverage.

Author: Reynold Xin <rxin@databricks.com>

Closes #15256 from rxin/SPARK-17682.
2016-09-27 10:20:30 -07:00
hyukjinkwon 2cac3b2d4a [SPARK-16516][SQL] Support for pushing down filters for decimal and timestamp types in ORC
## What changes were proposed in this pull request?

It seems ORC supports all the types in  ([`PredicateLeaf.Type`](e085b7e9bd/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/PredicateLeaf.java (L50-L56))) which includes timestamp type and decimal type.

In more details, the types listed in [`SearchArgumentImpl.boxLiteral()`](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L1068-L1093) can be used as a filter value.

FYI, inital `case` caluse for supported types was introduced in 65d71bd9fb and this was not changed overtime. At that time, Hive version was, 0.13 which supports only some types for filter-push down (See [SearchArgumentImpl.java#L945-L965](https://github.com/apache/hive/blob/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L945-L965) at 0.13).

However, the version was upgraded into 1.2.x and now it supports more types (See [SearchArgumentImpl.java#L1068-L1093](https://github.com/apache/hive/blob/branch-1.2/ql/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java#L1068-L1093) at 1.2.0)

## How was this patch tested?

Unit tests in `OrcFilterSuite` and `OrcQuerySuite`

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14172 from HyukjinKwon/SPARK-16516.
2016-09-28 00:50:12 +08:00
hyukjinkwon 5de1737b02 [SPARK-16777][SQL] Do not use deprecated listType API in ParquetSchemaConverter
## What changes were proposed in this pull request?

This PR removes build waning as below.

```scala
[WARNING] .../spark/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala:448: method listType in object ConversionPatterns is deprecated: see corresponding Javadoc for more information.
[WARNING]         ConversionPatterns.listType(
[WARNING]                            ^
[WARNING] .../spark/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala:464: method listType in object ConversionPatterns is deprecated: see corresponding Javadoc for more information.
[WARNING]         ConversionPatterns.listType(
[WARNING]                            ^
```

This should not use `listOfElements` (recommended to be replaced from `listType`) instead because the new method checks if the name of elements in Parquet's `LIST` is `element` in Parquet schema and throws an exception if not. However, It seems Spark prior to 1.4.x writes `ArrayType` with Parquet's `LIST` but with `array` as its element name.

Therefore, this PR avoids to use both `listOfElements` and `listType` but just use the existing schema builder to construct the same `GroupType`.

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #14399 from HyukjinKwon/SPARK-16777.
2016-09-28 00:39:47 +08:00
Kazuaki Ishizaki 85b0a15754 [SPARK-15962][SQL] Introduce implementation with a dense format for UnsafeArrayData
## What changes were proposed in this pull request?

This PR introduces more compact representation for ```UnsafeArrayData```.

```UnsafeArrayData``` needs to accept ```null``` value in each entry of an array. In the current version, it has three parts
```
[numElements] [offsets] [values]
```
`Offsets` has the number of `numElements`, and represents `null` if its value is negative. It may increase memory footprint, and introduces an indirection for accessing each of `values`.

This PR uses bitvectors to represent nullability for each element like `UnsafeRow`, and eliminates an indirection for accessing each element. The new ```UnsafeArrayData``` has four parts.
```
[numElements][null bits][values or offset&length][variable length portion]
```
In the `null bits` region, we store 1 bit per element, represents whether an element is null. Its total size is ceil(numElements / 8) bytes, and it is aligned to 8-byte boundaries.
In the `values or offset&length` region, we store the content of elements. For fields that hold fixed-length primitive types, such as long, double, or int, we store the value directly in the field. For fields with non-primitive or variable-length values, we store a relative offset (w.r.t. the base address of the array) that points to the beginning of the variable-length field and length (they are combined into a long). Each is word-aligned. For `variable length portion`, each is aligned to 8-byte boundaries.

The new format can reduce memory footprint and improve performance of accessing each element. An example of memory foot comparison:
1024x1024 elements integer array
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024 + 1024x1024 = 2M bytes
Size of ```baseObject``` for ```UnsafeArrayData```: 8 + 1024x1024/8 + 1024x1024 = 1.25M bytes

In summary, we got 1.0-2.6x performance improvements over the code before applying this PR.
Here are performance results of [benchmark programs](04d2e4b6db/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala):

**Read UnsafeArrayData**: 1.7x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Read UnsafeArrayData:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            430 /  436        390.0           2.6       1.0X
Double                                         456 /  485        367.8           2.7       0.9X

With SPARK-15962
Read UnsafeArrayData:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            252 /  260        666.1           1.5       1.0X
Double                                         281 /  292        597.7           1.7       0.9X
````
**Write UnsafeArrayData**: 1.0x and 1.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Write UnsafeArrayData:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            203 /  273        103.4           9.7       1.0X
Double                                         239 /  356         87.9          11.4       0.8X

With SPARK-15962
Write UnsafeArrayData:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            196 /  249        107.0           9.3       1.0X
Double                                         227 /  367         92.3          10.8       0.9X
````

**Get primitive array from UnsafeArrayData**: 2.6x and 1.6x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            207 /  217        304.2           3.3       1.0X
Double                                         257 /  363        245.2           4.1       0.8X

With SPARK-15962
Get primitive array from UnsafeArrayData: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            151 /  198        415.8           2.4       1.0X
Double                                         214 /  394        293.6           3.4       0.7X
````

**Create UnsafeArrayData from primitive array**: 1.7x and 2.1x performance improvements over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.0.4-301.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            340 /  385        185.1           5.4       1.0X
Double                                         479 /  705        131.3           7.6       0.7X

With SPARK-15962
Create UnsafeArrayData from primitive array: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Int                                            206 /  211        306.0           3.3       1.0X
Double                                         232 /  406        271.6           3.7       0.9X
````

1.7x and 1.4x performance improvements in [```UDTSerializationBenchmark```](https://github.com/apache/spark/blob/master/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala)  over the code before applying this PR
````
OpenJDK 64-Bit Server VM 1.8.0_91-b14 on Linux 4.4.11-200.fc22.x86_64
Intel Xeon E3-12xx v2 (Ivy Bridge)

Without SPARK-15962
VectorUDT de/serialization:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
serialize                                      442 /  533          0.0      441927.1       1.0X
deserialize                                    217 /  274          0.0      217087.6       2.0X

With SPARK-15962
VectorUDT de/serialization:              Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
serialize                                      265 /  318          0.0      265138.5       1.0X
deserialize                                    155 /  197          0.0      154611.4       1.7X
````

## How was this patch tested?

Added unit tests into ```UnsafeArraySuite```

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #13680 from kiszk/SPARK-15962.
2016-09-27 14:18:32 +08:00
Sameer Agarwal 7c7586aef9 [SPARK-17652] Fix confusing exception message while reserving capacity
## What changes were proposed in this pull request?

This minor patch fixes a confusing exception message while reserving additional capacity in the vectorized parquet reader.

## How was this patch tested?

Exisiting Unit Tests

Author: Sameer Agarwal <sameerag@cs.berkeley.edu>

Closes #15225 from sameeragarwal/error-msg.
2016-09-26 13:21:08 -07:00
Liang-Chi Hsieh 8135e0e5eb [SPARK-17153][SQL] Should read partition data when reading new files in filestream without globbing
## What changes were proposed in this pull request?

When reading file stream with non-globbing path, the results return data with all `null`s for the
partitioned columns. E.g.,

    case class A(id: Int, value: Int)
    val data = spark.createDataset(Seq(
      A(1, 1),
      A(2, 2),
      A(2, 3))
    )
    val url = "/tmp/test"
    data.write.partitionBy("id").parquet(url)
    spark.read.parquet(url).show

    +-----+---+
    |value| id|
    +-----+---+
    |    2|  2|
    |    3|  2|
    |    1|  1|
    +-----+---+

    val s = spark.readStream.schema(spark.read.load(url).schema).parquet(url)
    s.writeStream.queryName("test").format("memory").start()

    sql("SELECT * FROM test").show

    +-----+----+
    |value|  id|
    +-----+----+
    |    2|null|
    |    3|null|
    |    1|null|
    +-----+----+

## How was this patch tested?

Jenkins tests.

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

Closes #14803 from viirya/filestreamsource-option.
2016-09-26 13:07:11 -07:00
Justin Pihony 50b89d05b7
[SPARK-14525][SQL] Make DataFrameWrite.save work for jdbc
## What changes were proposed in this pull request?

This change modifies the implementation of DataFrameWriter.save such that it works with jdbc, and the call to jdbc merely delegates to save.

## How was this patch tested?

This was tested via unit tests in the JDBCWriteSuite, of which I added one new test to cover this scenario.

## Additional details

rxin This seems to have been most recently touched by you and was also commented on in the JIRA.

This contribution is my original work and I license the work to the project under the project's open source license.

Author: Justin Pihony <justin.pihony@gmail.com>
Author: Justin Pihony <justin.pihony@typesafe.com>

Closes #12601 from JustinPihony/jdbc_reconciliation.
2016-09-26 09:54:22 +01:00
xin wu de333d121d [SPARK-17551][SQL] Add DataFrame API for null ordering
## What changes were proposed in this pull request?
This pull request adds Scala/Java DataFrame API for null ordering (NULLS FIRST | LAST).

Also did some minor clean up for related code (e.g. incorrect indentation), and renamed "orderby-nulls-ordering.sql" to be consistent with existing test files.

## How was this patch tested?
Added a new test case in DataFrameSuite.

Author: petermaxlee <petermaxlee@gmail.com>
Author: Xin Wu <xinwu@us.ibm.com>

Closes #15123 from petermaxlee/SPARK-17551.
2016-09-25 16:46:12 -07:00
Michael Armbrust 988c714573 [SPARK-17643] Remove comparable requirement from Offset
For some sources, it is difficult to provide a global ordering based only on the data in the offset.  Since we don't use comparison for correctness, lets remove it.

Author: Michael Armbrust <michael@databricks.com>

Closes #15207 from marmbrus/removeComparable.
2016-09-23 12:17:59 -07:00
Shixiong Zhu 62ccf27ab4 [SPARK-17640][SQL] Avoid using -1 as the default batchId for FileStreamSource.FileEntry
## What changes were proposed in this pull request?

Avoid using -1 as the default batchId for FileStreamSource.FileEntry so that we can make sure not writing any FileEntry(..., batchId = -1) into the log. This also avoids people misusing it in future (#15203 is an example).

## How was this patch tested?

Jenkins.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15206 from zsxwing/cleanup.
2016-09-22 23:35:08 -07:00
Yucai Yu 79159a1e87 [SPARK-17635][SQL] Remove hardcode "agg_plan" in HashAggregateExec
## What changes were proposed in this pull request?

"agg_plan" are hardcoded in HashAggregateExec, which have potential issue, so removing them.

## How was this patch tested?

existing tests.

Author: Yucai Yu <yucai.yu@intel.com>

Closes #15199 from yucai/agg_plan.
2016-09-22 17:22:56 -07:00
Burak Yavuz a166196831 [SPARK-17569][SPARK-17569][TEST] Make the unit test added for work again
## What changes were proposed in this pull request?

A [PR](a6aade0042) was merged concurrently that made the unit test for PR #15122 not test anything anymore. This PR fixes the test.

## How was this patch tested?

Changed line 0d63487502/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala (L137)
from `false` to `true` and made sure the unit test failed.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15203 from brkyvz/fix-test.
2016-09-22 16:50:22 -07:00
Herman van Hovell 0d63487502 [SPARK-17616][SQL] Support a single distinct aggregate combined with a non-partial aggregate
## What changes were proposed in this pull request?
We currently cannot execute an aggregate that contains a single distinct aggregate function and an one or more non-partially plannable aggregate functions, for example:
```sql
select   grp,
         collect_list(col1),
         count(distinct col2)
from     tbl_a
group by 1
```
This is a regression from Spark 1.6. This is caused by the fact that the single distinct aggregation code path assumes that all aggregates can be planned in two phases (is partially aggregatable). This PR works around this issue by triggering the `RewriteDistinctAggregates` in such cases (this is similar to the approach taken in 1.6).

## How was this patch tested?
Created `RewriteDistinctAggregatesSuite` which checks if the aggregates with distinct aggregate functions get rewritten into two `Aggregates` and an `Expand`. Added a regression test to `DataFrameAggregateSuite`.

Author: Herman van Hovell <hvanhovell@databricks.com>

Closes #15187 from hvanhovell/SPARK-17616.
2016-09-22 14:29:27 -07:00
Burak Yavuz 85d609cf25 [SPARK-17613] S3A base paths with no '/' at the end return empty DataFrames
## What changes were proposed in this pull request?

Consider you have a bucket as `s3a://some-bucket`
and under it you have files:
```
s3a://some-bucket/file1.parquet
s3a://some-bucket/file2.parquet
```
Getting the parent path of `s3a://some-bucket/file1.parquet` yields
`s3a://some-bucket/` and the ListingFileCatalog uses this as the key in the hash map.

When catalog.allFiles is called, we use `s3a://some-bucket` (no slash at the end) to get the list of files, and we're left with an empty list!

This PR fixes this by adding a `/` at the end of the `URI` iff the given `Path` doesn't have a parent, i.e. is the root. This is a no-op if the path already had a `/` at the end, and is handled through the Hadoop Path, path merging semantics.

## How was this patch tested?

Unit test in `FileCatalogSuite`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15169 from brkyvz/SPARK-17613.
2016-09-22 13:05:41 -07:00
Wenchen Fan 8a02410a92 [SQL][MINOR] correct the comment of SortBasedAggregationIterator.safeProj
## What changes were proposed in this pull request?

This comment went stale long time ago, this PR fixes it according to my understanding.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15095 from cloud-fan/update-comment.
2016-09-22 23:25:32 +08:00
Zhenhua Wang de7df7defc [SPARK-17625][SQL] set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation
## What changes were proposed in this pull request?

We should set expectedOutputAttributes when converting SimpleCatalogRelation to LogicalRelation, otherwise the outputs of LogicalRelation are different from outputs of SimpleCatalogRelation - they have different exprId's.

## How was this patch tested?

add a test case

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #15182 from wzhfy/expectedAttributes.
2016-09-22 14:48:49 +08:00
gatorsmile 3a80f92f8f [SPARK-17492][SQL] Fix Reading Cataloged Data Sources without Extending SchemaRelationProvider
### What changes were proposed in this pull request?
For data sources without extending `SchemaRelationProvider`, we expect users to not specify schemas when they creating tables. If the schema is input from users, an exception is issued.

Since Spark 2.1, for any data source, to avoid infer the schema every time, we store the schema in the metastore catalog. Thus, when reading a cataloged data source table, the schema could be read from metastore catalog. In this case, we also got an exception. For example,

```Scala
sql(
  s"""
     |CREATE TABLE relationProvierWithSchema
     |USING org.apache.spark.sql.sources.SimpleScanSource
     |OPTIONS (
     |  From '1',
     |  To '10'
     |)
   """.stripMargin)
spark.table(tableName).show()
```
```
org.apache.spark.sql.sources.SimpleScanSource does not allow user-specified schemas.;
```

This PR is to fix the above issue. When building a data source, we introduce a flag `isSchemaFromUsers` to indicate whether the schema is really input from users. If true, we issue an exception. Otherwise, we will call the `createRelation` of `RelationProvider` to generate the `BaseRelation`, in which it contains the actual schema.

### How was this patch tested?
Added a few cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15046 from gatorsmile/tempViewCases.
2016-09-22 13:19:06 +08:00
Yadong Qi cb324f6115 [SPARK-17425][SQL] Override sameResult in HiveTableScanExec to make ReuseExchange work in text format table
## What changes were proposed in this pull request?
The PR will override the `sameResult` in `HiveTableScanExec` to make `ReuseExchange` work in text format table.

## How was this patch tested?
# SQL
```sql
SELECT * FROM src t1
JOIN src t2 ON t1.key = t2.key
JOIN src t3 ON t1.key = t3.key;
```

# Before
```
== Physical Plan ==
*BroadcastHashJoin [key#30], [key#34], Inner, BuildRight
:- *BroadcastHashJoin [key#30], [key#32], Inner, BuildRight
:  :- *Filter isnotnull(key#30)
:  :  +- HiveTableScan [key#30, value#31], MetastoreRelation default, src
:  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
:     +- *Filter isnotnull(key#32)
:        +- HiveTableScan [key#32, value#33], MetastoreRelation default, src
+- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
   +- *Filter isnotnull(key#34)
      +- HiveTableScan [key#34, value#35], MetastoreRelation default, src
```

# After
```
== Physical Plan ==
*BroadcastHashJoin [key#2], [key#6], Inner, BuildRight
:- *BroadcastHashJoin [key#2], [key#4], Inner, BuildRight
:  :- *Filter isnotnull(key#2)
:  :  +- HiveTableScan [key#2, value#3], MetastoreRelation default, src
:  +- BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
:     +- *Filter isnotnull(key#4)
:        +- HiveTableScan [key#4, value#5], MetastoreRelation default, src
+- ReusedExchange [key#6, value#7], BroadcastExchange HashedRelationBroadcastMode(List(cast(input[0, int, false] as bigint)))
```

cc: davies cloud-fan

Author: Yadong Qi <qiyadong2010@gmail.com>

Closes #14988 from watermen/SPARK-17425.
2016-09-22 13:04:42 +08:00
Wenchen Fan b50b34f561 [SPARK-17609][SQL] SessionCatalog.tableExists should not check temp view
## What changes were proposed in this pull request?

After #15054 , there is no place in Spark SQL that need `SessionCatalog.tableExists` to check temp views, so this PR makes `SessionCatalog.tableExists` only check permanent table/view and removes some hacks.

This PR also improves the `getTempViewOrPermanentTableMetadata` that is introduced in  #15054 , to make the code simpler.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15160 from cloud-fan/exists.
2016-09-22 12:52:09 +08:00
Davies Liu 8bde03bf9a [SPARK-17494][SQL] changePrecision() on compact decimal should respect rounding mode
## What changes were proposed in this pull request?

Floor()/Ceil() of decimal is implemented using changePrecision() by passing a rounding mode, but the rounding mode is not respected when the decimal is in compact mode (could fit within a Long).

This Update the changePrecision() to respect rounding mode, which could be ROUND_FLOOR, ROUND_CEIL, ROUND_HALF_UP, ROUND_HALF_EVEN.

## How was this patch tested?

Added regression tests.

Author: Davies Liu <davies@databricks.com>

Closes #15154 from davies/decimal_round.
2016-09-21 21:02:30 -07:00
Michael Armbrust 3497ebe511 [SPARK-17627] Mark Streaming Providers Experimental
All of structured streaming is experimental in its first release.  We missed the annotation on two of the APIs.

Author: Michael Armbrust <michael@databricks.com>

Closes #15188 from marmbrus/experimentalApi.
2016-09-21 20:59:46 -07:00
Burak Yavuz 7cbe216449 [SPARK-17569] Make StructuredStreaming FileStreamSource batch generation faster
## What changes were proposed in this pull request?

While getting the batch for a `FileStreamSource` in StructuredStreaming, we know which files we must take specifically. We already have verified that they exist, and have committed them to a metadata log. When creating the FileSourceRelation however for an incremental execution, the code checks the existence of every single file once again!

When you have 100,000s of files in a folder, creating the first batch takes 2 hours+ when working with S3! This PR disables that check

## How was this patch tested?

Added a unit test to `FileStreamSource`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15122 from brkyvz/SPARK-17569.
2016-09-21 17:12:52 -07:00
Liang-Chi Hsieh 248922fd4f [SPARK-17590][SQL] Analyze CTE definitions at once and allow CTE subquery to define CTE
## What changes were proposed in this pull request?

We substitute logical plan with CTE definitions in the analyzer rule CTESubstitution. A CTE definition can be used in the logical plan for multiple times, and its analyzed logical plan should be the same. We should not analyze CTE definitions multiple times when they are reused in the query.

By analyzing CTE definitions before substitution, we can support defining CTE in subquery.

## How was this patch tested?

Jenkins tests.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>

Closes #15146 from viirya/cte-analysis-once.
2016-09-21 06:53:42 -07:00
hyukjinkwon 25a020be99
[SPARK-17583][SQL] Remove uesless rowSeparator variable and set auto-expanding buffer as default for maxCharsPerColumn option in CSV
## What changes were proposed in this pull request?

This PR includes the changes below:

1. Upgrade Univocity library from 2.1.1 to 2.2.1

  This includes some performance improvement and also enabling auto-extending buffer in `maxCharsPerColumn` option in CSV. Please refer the [release notes](https://github.com/uniVocity/univocity-parsers/releases).

2. Remove useless `rowSeparator` variable existing in `CSVOptions`

  We have this unused variable in [CSVOptions.scala#L127](29952ed096/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala (L127)) but it seems possibly causing confusion that it actually does not care of `\r\n`. For example, we have an issue open about this, [SPARK-17227](https://issues.apache.org/jira/browse/SPARK-17227), describing this variable.

  This variable is virtually not being used because we rely on `LineRecordReader` in Hadoop which deals with only both `\n` and `\r\n`.

3. Set the default value of `maxCharsPerColumn` to auto-expending.

  We are setting 1000000 for the length of each column. It'd be more sensible we allow auto-expending rather than fixed length by default.

  To make sure, using `-1` is being described in the release note, [2.2.0](https://github.com/uniVocity/univocity-parsers/releases/tag/v2.2.0).

## How was this patch tested?

N/A

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15138 from HyukjinKwon/SPARK-17583.
2016-09-21 10:35:29 +01:00
VinceShieh 57dc326bd0
[SPARK-17219][ML] Add NaN value handling in Bucketizer
## What changes were proposed in this pull request?
This PR fixes an issue when Bucketizer is called to handle a dataset containing NaN value.
Sometimes, null value might also be useful to users, so in these cases, Bucketizer should
reserve one extra bucket for NaN values, instead of throwing an illegal exception.
Before:
```
Bucketizer.transform on NaN value threw an illegal exception.
```
After:
```
NaN values will be grouped in an extra bucket.
```
## How was this patch tested?
New test cases added in `BucketizerSuite`.
Signed-off-by: VinceShieh <vincent.xieintel.com>

Author: VinceShieh <vincent.xie@intel.com>

Closes #14858 from VinceShieh/spark-17219.
2016-09-21 10:20:57 +01:00
Burak Yavuz 28fafa3ee8 [SPARK-17599] Prevent ListingFileCatalog from failing if path doesn't exist
## What changes were proposed in this pull request?

The `ListingFileCatalog` lists files given a set of resolved paths. If a folder is deleted at any time between the paths were resolved and the file catalog can check for the folder, the Spark job fails. This may abruptly stop long running StructuredStreaming jobs for example.

Folders may be deleted by users or automatically by retention policies. These cases should not prevent jobs from successfully completing.

## How was this patch tested?

Unit test in `FileCatalogSuite`

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15153 from brkyvz/SPARK-17599.
2016-09-21 17:07:16 +08:00
Sean Zhong 3977223a32 [SPARK-17617][SQL] Remainder(%) expression.eval returns incorrect result on double value
## What changes were proposed in this pull request?

Remainder(%) expression's `eval()` returns incorrect result when the dividend is a big double. The reason is that Remainder converts the double dividend to decimal to do "%", and that lose precision.

This bug only affects the `eval()` that is used by constant folding, the codegen path is not impacted.

### Before change
```
scala> -5083676433652386516D % 10
res2: Double = -6.0

scala> spark.sql("select -5083676433652386516D % 10 as a").show
+---+
|  a|
+---+
|0.0|
+---+
```

### After change
```
scala> spark.sql("select -5083676433652386516D % 10 as a").show
+----+
|   a|
+----+
|-6.0|
+----+
```

## How was this patch tested?

Unit test.

Author: Sean Zhong <seanzhong@databricks.com>

Closes #15171 from clockfly/SPARK-17617.
2016-09-21 16:53:34 +08:00
wm624@hotmail.com 61876a4279
[CORE][DOC] Fix errors in comments
## What changes were proposed in this pull request?
While reading source code of CORE and SQL core, I found some minor errors in comments such as extra space, missing blank line and grammar error.

I fixed these minor errors and might find more during my source code study.

## How was this patch tested?
Manually build

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #15151 from wangmiao1981/mem.
2016-09-21 09:33:29 +01:00