Commit graph

4719 commits

Author SHA1 Message Date
Shixiong Zhu c4979f6ea8 [SPARK-18655][SS] Ignore Structured Streaming 2.0.2 logs in history server
## What changes were proposed in this pull request?

As `queryStatus` in StreamingQueryListener events was removed in #15954, parsing 2.0.2 structured streaming logs will throw the following errror:

```
[info]   com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException: Unrecognized field "queryStatus" (class org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent), not marked as ignorable (2 known properties: "id", "exception"])
[info]  at [Source: {"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491532753,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getOffset.source":"1","triggerId":"1"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{}},"exception":null}; line: 1, column: 521] (through reference chain: org.apache.spark.sql.streaming.QueryTerminatedEvent["queryStatus"])
[info]   at com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException.from(UnrecognizedPropertyException.java:51)
[info]   at com.fasterxml.jackson.databind.DeserializationContext.reportUnknownProperty(DeserializationContext.java:839)
[info]   at com.fasterxml.jackson.databind.deser.std.StdDeserializer.handleUnknownProperty(StdDeserializer.java:1045)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperty(BeanDeserializerBase.java:1352)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperties(BeanDeserializerBase.java:1306)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:453)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1099)
...
```

This PR just ignores such errors and adds a test to make sure we can read 2.0.2 logs.

## How was this patch tested?

`query-event-logs-version-2.0.2.txt` has all types of events generated by Structured Streaming in Spark 2.0.2. `testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.2")` verified we can load them without any error.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16085 from zsxwing/SPARK-18655.
2016-11-30 16:18:53 -08:00
Wenchen Fan f135b70fd5 [SPARK-18251][SQL] the type of Dataset can't be Option of non-flat type
## What changes were proposed in this pull request?

For input object of non-flat type, we can't encode it to row if it's null, as Spark SQL doesn't allow the entire row to be null, only its columns can be null. That's the reason we forbid users to use top level null objects in https://github.com/apache/spark/pull/13469

However, if users wrap non-flat type with `Option`, then we may still encoder top level null object to row, which is not allowed.

This PR fixes this case, and suggests users to wrap their type with `Tuple1` if they do wanna top level null objects.

## How was this patch tested?

new test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15979 from cloud-fan/option.
2016-11-30 13:36:17 -08:00
Wenchen Fan 3f03c90a80 [SPARK-18220][SQL] read Hive orc table with varchar column should not fail
## What changes were proposed in this pull request?

Spark SQL only has `StringType`, when reading hive table with varchar column, we will read that column as `StringType`. However, we still need to use varchar `ObjectInspector` to read varchar column in hive table, which means we need to know the actual column type at hive side.

In Spark 2.1, after https://github.com/apache/spark/pull/14363 , we parse hive type string to catalyst type, which means the actual column type at hive side is erased. Then we may use string `ObjectInspector` to read varchar column and fail.

This PR keeps the original hive column type string in the metadata of `StructField`, and use it when we convert it to a hive column.

## How was this patch tested?

newly added regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16060 from cloud-fan/varchar.
2016-11-30 09:47:30 -08:00
jiangxingbo c24076dcf8 [SPARK-17932][SQL] Support SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards' statement
## What changes were proposed in this pull request?

Currently we haven't implemented `SHOW TABLE EXTENDED` in Spark 2.0. This PR is to implement the statement.
Goals:
1. Support `SHOW TABLES EXTENDED LIKE 'identifier_with_wildcards'`;
2. Explicitly output an unsupported error message for `SHOW TABLES [EXTENDED] ... PARTITION` statement;
3. Improve test cases for `SHOW TABLES` statement.

## How was this patch tested?
1. Add new test cases in file `show-tables.sql`.
2. Modify tests for `SHOW TABLES` in `DDLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15958 from jiangxb1987/show-table-extended.
2016-11-30 03:59:25 -08:00
gatorsmile 2eb093decb [SPARK-17897][SQL] Fixed IsNotNull Constraint Inference Rule
### What changes were proposed in this pull request?
The `constraints` of an operator is the expressions that evaluate to `true` for all the rows produced. That means, the expression result should be neither `false` nor `unknown` (NULL). Thus, we can conclude that `IsNotNull` on all the constraints, which are generated by its own predicates or propagated from the children. The constraint can be a complex expression. For better usage of these constraints, we try to push down `IsNotNull` to the lowest-level expressions (i.e., `Attribute`). `IsNotNull` can be pushed through an expression when it is null intolerant. (When the input is NULL, the null-intolerant expression always evaluates to NULL.)

Below is the existing code we have for `IsNotNull` pushdown.
```Scala
  private def scanNullIntolerantExpr(expr: Expression): Seq[Attribute] = expr match {
    case a: Attribute => Seq(a)
    case _: NullIntolerant | IsNotNull(_: NullIntolerant) =>
      expr.children.flatMap(scanNullIntolerantExpr)
    case _ => Seq.empty[Attribute]
  }
```

**`IsNotNull` itself is not null-intolerant.** It converts `null` to `false`. If the expression does not include any `Not`-like expression, it works; otherwise, it could generate a wrong result. This PR is to fix the above function by removing the `IsNotNull` from the inference. After the fix, when a constraint has a `IsNotNull` expression, we infer new attribute-specific `IsNotNull` constraints if and only if `IsNotNull` appears in the root.

Without the fix, the following test case will return empty.
```Scala
val data = Seq[java.lang.Integer](1, null).toDF("key")
data.filter("not key is not null").show()
```
Before the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter (isnotnull(value#1) && NOT isnotnull(value#1))
   +- LocalRelation [value#1]
```

After the fix, the optimized plan is like
```
== Optimized Logical Plan ==
Project [value#1 AS key#3]
+- Filter NOT isnotnull(value#1)
   +- LocalRelation [value#1]
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16067 from gatorsmile/isNotNull2.
2016-11-30 19:40:58 +08:00
Herman van Hovell 879ba71110 [SPARK-18622][SQL] Fix the datatype of the Sum aggregate function
## What changes were proposed in this pull request?
The result of a `sum` aggregate function is typically a Decimal, Double or a Long. Currently the output dataType is based on input's dataType.

The `FunctionArgumentConversion` rule will make sure that the input is promoted to the largest type, and that also ensures that the output uses a (hopefully) sufficiently large output dataType. The issue is that sum is in a resolved state when we cast the input type, this means that rules assuming that the dataType of the expression does not change anymore could have been applied in the mean time. This is what happens if we apply `WidenSetOperationTypes` before applying the casts, and this breaks analysis.

The most straight forward and future proof solution is to make `sum` always output the widest dataType in its class (Long for IntegralTypes, Decimal for DecimalTypes & Double for FloatType and DoubleType). This PR implements that solution.

We should move expression specific type casting rules into the given Expression at some point.

## How was this patch tested?
Added (regression) tests to SQLQueryTestSuite's `union.sql`.

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

Closes #16063 from hvanhovell/SPARK-18622.
2016-11-30 15:25:33 +08:00
gatorsmile a1d9138ab2 [SPARK-17680][SQL][TEST] Added a Testcase for Verifying Unicode Character Support for Column Names and Comments
### What changes were proposed in this pull request?

Spark SQL supports Unicode characters for column names when specified within backticks(`). When the Hive support is enabled, the version of the Hive metastore must be higher than 0.12,  See the JIRA: https://issues.apache.org/jira/browse/HIVE-6013 Hive metastore supports Unicode characters for column names since 0.13.

In Spark SQL, table comments, and view comments always allow Unicode characters without backticks.

BTW, a separate PR has been submitted for database and table name validation because we do not support Unicode characters in these two cases.
### How was this patch tested?

N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15255 from gatorsmile/unicodeSupport.
2016-11-30 15:17:29 +08:00
Tathagata Das bc09a2b8c3 [SPARK-18516][STRUCTURED STREAMING] Follow up PR to add StreamingQuery.status to Python
## What changes were proposed in this pull request?
- Add StreamingQueryStatus.json
- Make it not case class (to avoid unnecessarily exposing implicit object StreamingQueryStatus, consistent with StreamingQueryProgress)
- Add StreamingQuery.status to Python
- Fix post-termination status

## How was this patch tested?
New unit tests

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

Closes #16075 from tdas/SPARK-18516-1.
2016-11-29 23:08:56 -08:00
Herman van Hovell af9789a4f5 [SPARK-18632][SQL] AggregateFunction should not implement ImplicitCastInputTypes
## What changes were proposed in this pull request?
`AggregateFunction` currently implements `ImplicitCastInputTypes` (which enables implicit input type casting). There are actually quite a few situations in which we don't need this, or require more control over our input. A recent example is the aggregate for `CountMinSketch` which should only take string, binary or integral types inputs.

This PR removes `ImplicitCastInputTypes` from the `AggregateFunction` and makes a case-by-case decision on what kind of input validation we should use.

## How was this patch tested?
Refactoring only. Existing tests.

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

Closes #16066 from hvanhovell/SPARK-18632.
2016-11-29 20:05:15 -08:00
Tathagata Das c3d08e2f29 [SPARK-18516][SQL] Split state and progress in streaming
This PR separates the status of a `StreamingQuery` into two separate APIs:
 - `status` - describes the status of a `StreamingQuery` at this moment, including what phase of processing is currently happening and if data is available.
 - `recentProgress` - an array of statistics about the most recent microbatches that have executed.

A recent progress contains the following information:
```
{
  "id" : "2be8670a-fce1-4859-a530-748f29553bb6",
  "name" : "query-29",
  "timestamp" : 1479705392724,
  "inputRowsPerSecond" : 230.76923076923077,
  "processedRowsPerSecond" : 10.869565217391303,
  "durationMs" : {
    "triggerExecution" : 276,
    "queryPlanning" : 3,
    "getBatch" : 5,
    "getOffset" : 3,
    "addBatch" : 234,
    "walCommit" : 30
  },
  "currentWatermark" : 0,
  "stateOperators" : [ ],
  "sources" : [ {
    "description" : "KafkaSource[Subscribe[topic-14]]",
    "startOffset" : {
      "topic-14" : {
        "2" : 0,
        "4" : 1,
        "1" : 0,
        "3" : 0,
        "0" : 0
      }
    },
    "endOffset" : {
      "topic-14" : {
        "2" : 1,
        "4" : 2,
        "1" : 0,
        "3" : 0,
        "0" : 1
      }
    },
    "numRecords" : 3,
    "inputRowsPerSecond" : 230.76923076923077,
    "processedRowsPerSecond" : 10.869565217391303
  } ]
}
```

Additionally, in order to make it possible to correlate progress updates across restarts, we change the `id` field from an integer that is unique with in the JVM to a `UUID` that is globally unique.

Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #15954 from marmbrus/queryProgress.
2016-11-29 17:24:17 -08:00
Nattavut Sutyanyong 3600635215 [SPARK-18614][SQL] Incorrect predicate pushdown from ExistenceJoin
## What changes were proposed in this pull request?

ExistenceJoin should be treated the same as LeftOuter and LeftAnti, not InnerLike and LeftSemi. This is not currently exposed because the rewrite of [NOT] EXISTS OR ... to ExistenceJoin happens in rule RewritePredicateSubquery, which is in a separate rule set and placed after the rule PushPredicateThroughJoin. During the transformation in the rule PushPredicateThroughJoin, an ExistenceJoin never exists.

The semantics of ExistenceJoin says we need to preserve all the rows from the left table through the join operation as if it is a regular LeftOuter join. The ExistenceJoin augments the LeftOuter operation with a new column called exists, set to true when the join condition in the ON clause is true and false otherwise. The filter of any rows will happen in the Filter operation above the ExistenceJoin.

Example:

A(c1, c2): { (1, 1), (1, 2) }
// B can be any value as it is irrelevant in this example
B(c1): { (NULL) }

select A.*
from   A
where  exists (select 1 from B where A.c1 = A.c2)
       or A.c2=2

In this example, the correct result is all the rows from A. If the pattern ExistenceJoin around line 935 in Optimizer.scala is indeed active, the code will push down the predicate A.c1 = A.c2 to be a Filter on relation A, which will incorrectly filter the row (1,2) from A.

## How was this patch tested?

Since this is not an exposed case, no new test cases is added. The scenario is discovered via a code review of another PR and confirmed to be valid with peer.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16044 from nsyca/spark-18614.
2016-11-29 15:27:43 -08:00
Mark Hamstra f8878a4c6f [SPARK-18631][SQL] Changed ExchangeCoordinator re-partitioning to avoid more data skew
## What changes were proposed in this pull request?

Re-partitioning logic in ExchangeCoordinator changed so that adding another pre-shuffle partition to the post-shuffle partition will not be done if doing so would cause the size of the post-shuffle partition to exceed the target partition size.

## How was this patch tested?

Existing tests updated to reflect new expectations.

Author: Mark Hamstra <markhamstra@gmail.com>

Closes #16065 from markhamstra/SPARK-17064.
2016-11-29 15:01:12 -08:00
wangzhenhua d57a594b8b [SPARK-18429][SQL] implement a new Aggregate for CountMinSketch
## What changes were proposed in this pull request?

This PR implements a new Aggregate to generate count min sketch, which is a wrapper of CountMinSketch.

## How was this patch tested?

add test cases

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15877 from wzhfy/cms.
2016-11-29 13:16:46 -08:00
Tyson Condie f643fe47f4 [SPARK-18498][SQL] Revise HDFSMetadataLog API for better testing
Revise HDFSMetadataLog API such that metadata object serialization and final batch file write are separated. This will allow serialization checks without worrying about batch file name formats. marmbrus zsxwing

Existing tests already ensure this API faithfully support core functionality i.e., creation of batch files.

Author: Tyson Condie <tcondie@gmail.com>

Closes #15924 from tcondie/SPARK-18498.

Signed-off-by: Michael Armbrust <michael@databricks.com>
2016-11-29 12:37:36 -08:00
hyukjinkwon 1a870090e4
[SPARK-18615][DOCS] Switch to multi-line doc to avoid a genjavadoc bug for backticks
## What changes were proposed in this pull request?

Currently, single line comment does not mark down backticks to `<code>..</code>` but prints as they are (`` `..` ``). For example, the line below:

```scala
/** Return an RDD with the pairs from `this` whose keys are not in `other`. */
```

So, we could work around this as below:

```scala
/**
 * Return an RDD with the pairs from `this` whose keys are not in `other`.
 */
```

- javadoc

  - **Before**
    ![2016-11-29 10 39 14](https://cloud.githubusercontent.com/assets/6477701/20693606/e64c8f90-b622-11e6-8dfc-4a029216e23d.png)

  - **After**
    ![2016-11-29 10 39 08](https://cloud.githubusercontent.com/assets/6477701/20693607/e7280d36-b622-11e6-8502-d2e21cd5556b.png)

- scaladoc (this one looks fine either way)

  - **Before**
    ![2016-11-29 10 38 22](https://cloud.githubusercontent.com/assets/6477701/20693640/12c18aa8-b623-11e6-901a-693e2f6f8066.png)

  - **After**
    ![2016-11-29 10 40 05](https://cloud.githubusercontent.com/assets/6477701/20693642/14eb043a-b623-11e6-82ac-7cd0000106d1.png)

I suspect this is related with SPARK-16153 and genjavadoc issue in ` typesafehub/genjavadoc#85`.

## How was this patch tested?

I found them via

```
grep -r "\/\*\*.*\`" . | grep .scala
````

and then checked if each is in the public API documentation with manually built docs (`jekyll build`) with Java 7.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16050 from HyukjinKwon/javadoc-markdown.
2016-11-29 13:50:24 +00:00
hyukjinkwon f830bb9170
[SPARK-3359][DOCS] Make javadoc8 working for unidoc/genjavadoc compatibility in Java API documentation
## What changes were proposed in this pull request?

This PR make `sbt unidoc` complete with Java 8.

This PR roughly includes several fixes as below:

- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``

  ```diff
  - * A column that will be computed based on the data in a [[DataFrame]].
  + * A column that will be computed based on the data in a `DataFrame`.
  ```

- Fix throws annotations so that they are recognisable in javadoc

- Fix URL links to `<a href="http..."></a>`.

  ```diff
  - * [[http://en.wikipedia.org/wiki/Decision_tree_learning Decision tree]] model for regression.
  + * <a href="http://en.wikipedia.org/wiki/Decision_tree_learning">
  + * Decision tree (Wikipedia)</a> model for regression.
  ```

  ```diff
  -   * see http://en.wikipedia.org/wiki/Receiver_operating_characteristic
  +   * see <a href="http://en.wikipedia.org/wiki/Receiver_operating_characteristic">
  +   * Receiver operating characteristic (Wikipedia)</a>
  ```

- Fix < to > to

  - `greater than`/`greater than or equal to` or `less than`/`less than or equal to` where applicable.

  - Wrap it with `{{{...}}}` to print them in javadoc or use `{code ...}` or `{literal ..}`. Please refer https://github.com/apache/spark/pull/16013#discussion_r89665558

- Fix `</p>` complaint

## How was this patch tested?

Manually tested by `jekyll build` with Java 7 and 8

```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```

```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16013 from HyukjinKwon/SPARK-3359-errors-more.
2016-11-29 09:41:32 +00:00
Tyson Condie 3c0beea475 [SPARK-18339][SPARK-18513][SQL] Don't push down current_timestamp for filters in StructuredStreaming and persist batch and watermark timestamps to offset log.
## What changes were proposed in this pull request?

For the following workflow:
1. I have a column called time which is at minute level precision in a Streaming DataFrame
2. I want to perform groupBy time, count
3. Then I want my MemorySink to only have the last 30 minutes of counts and I perform this by
.where('time >= current_timestamp().cast("long") - 30 * 60)
what happens is that the `filter` gets pushed down before the aggregation, and the filter happens on the source data for the aggregation instead of the result of the aggregation (where I actually want to filter).
I guess the main issue here is that `current_timestamp` is non-deterministic in the streaming context and shouldn't be pushed down the filter.
Does this require us to store the `current_timestamp` for each trigger of the streaming job, that is something to discuss.

Furthermore, we want to persist current batch timestamp and watermark timestamp to the offset log so that these values are consistent across multiple executions of the same batch.

brkyvz zsxwing tdas

## How was this patch tested?

A test was added to StreamingAggregationSuite ensuring the above use case is handled. The test injects a stream of time values (in seconds) to a query that runs in complete mode and only outputs the (count) aggregation results for the past 10 seconds.

Author: Tyson Condie <tcondie@gmail.com>

Closes #15949 from tcondie/SPARK-18339.
2016-11-28 23:07:17 -08:00
Eric Liang e2318ede04 [SPARK-18544][SQL] Append with df.saveAsTable writes data to wrong location
## What changes were proposed in this pull request?

We failed to properly propagate table metadata for existing tables for the saveAsTable command. This caused a downstream component to think the table was MANAGED, writing data to the wrong location.

## How was this patch tested?

Unit test that fails before the patch.

Author: Eric Liang <ekl@databricks.com>

Closes #15983 from ericl/spark-18544.
2016-11-28 21:58:01 -08:00
Herman van Hovell d449988b88 [SPARK-18058][SQL][TRIVIAL] Use dataType.sameResult(...) instead equality on asNullable datatypes
## What changes were proposed in this pull request?
This is absolutely minor. PR https://github.com/apache/spark/pull/15595 uses `dt1.asNullable == dt2.asNullable` expressions in a few places. It is however more efficient to call `dt1.sameType(dt2)`. I have replaced every instance of the first pattern with the second pattern (3/5 were introduced by #15595).

## How was this patch tested?
Existing tests.

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

Closes #16041 from hvanhovell/SPARK-18058.
2016-11-28 21:43:33 -08:00
Shuai Lin e64a2047ea [SPARK-16282][SQL] Follow-up: remove "percentile" from temp function detection after implementing it natively
## What changes were proposed in this pull request?

In #15764 we added a mechanism to detect if a function is temporary or not. Hive functions are treated as non-temporary. Of the three hive functions, now "percentile" has been implemented natively, and "hash" has been removed. So we should update the list.

## How was this patch tested?

Unit tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #16049 from lins05/update-temp-function-detect-hive-list.
2016-11-28 20:23:48 -08:00
Cheng Lian 2e809903d4 [SPARK-18403][SQL] Fix unsafe data false sharing issue in ObjectHashAggregateExec
## What changes were proposed in this pull request?

This PR fixes a random OOM issue occurred while running `ObjectHashAggregateSuite`.

This issue can be steadily reproduced under the following conditions:

1. The aggregation must be evaluated using `ObjectHashAggregateExec`;
2. There must be an input column whose data type involves `ArrayType` (an input column of `MapType` may even cause SIGSEGV);
3. Sort-based aggregation fallback must be triggered during evaluation.

The root cause is that while falling back to sort-based aggregation, we must sort and feed already evaluated partial aggregation buffers living in the hash map to the sort-based aggregator using an external sorter. However, the underlying mutable byte buffer of `UnsafeRow`s produced by the iterator of the external sorter is reused and may get overwritten when the iterator steps forward. After the last entry is consumed, the byte buffer points to a block of uninitialized memory filled by `5a`. Therefore, while reading an `UnsafeArrayData` out of the `UnsafeRow`, `5a5a5a5a` is treated as array size and triggers a memory allocation for a ridiculously large array and immediately blows up the JVM with an OOM.

To fix this issue, we only need to add `.copy()` accordingly.

## How was this patch tested?

New regression test case added in `ObjectHashAggregateSuite`.

Author: Cheng Lian <lian@databricks.com>

Closes #15976 from liancheng/investigate-oom.
2016-11-29 09:01:03 +08:00
Kazuaki Ishizaki ad67993b73 [SPARK-17680][SQL][TEST] Added test cases for InMemoryRelation
## What changes were proposed in this pull request?

This pull request adds test cases for the following cases:
- keep all data types with null or without null
- access `CachedBatch` disabling whole stage codegen
- access only some columns in `CachedBatch`

This PR is a part of https://github.com/apache/spark/pull/15219. Here are motivations to add these tests. When https://github.com/apache/spark/pull/15219 is enabled, the first two cases are handled by specialized (generated) code. The third one is a pitfall.

In general, even for now, it would be helpful to increase test coverage.
## How was this patch tested?

added test suites itself

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

Closes #15462 from kiszk/columnartestsuites.
2016-11-28 14:06:37 -05:00
jiangxingbo 0f5f52a3d1 [SPARK-16282][SQL] Implement percentile SQL function.
## What changes were proposed in this pull request?

Implement percentile SQL function. It computes the exact percentile(s) of expr at pc with range in [0, 1].

## How was this patch tested?

Add a new testsuite `PercentileSuite` to test percentile directly.
Updated related testcases in `ExpressionToSQLSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>
Author: 蒋星博 <jiangxingbo@meituan.com>
Author: jiangxingbo <jiangxingbo@meituan.com>

Closes #14136 from jiangxb1987/percentile.
2016-11-28 11:05:58 -08:00
Wenchen Fan 185642846e [SQL][MINOR] DESC should use 'Catalog' as partition provider
## What changes were proposed in this pull request?

`CatalogTable` has a parameter named `tracksPartitionsInCatalog`, and in `CatalogTable.toString` we use `"Partition Provider: Catalog"` to represent it. This PR fixes `DESC TABLE` to make it consistent with `CatalogTable.toString`.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16035 from cloud-fan/minor.
2016-11-28 10:57:17 -08:00
Yin Huai eba727757e [SPARK-18602] Set the version of org.codehaus.janino:commons-compiler to 3.0.0 to match the version of org.codehaus.janino:janino
## What changes were proposed in this pull request?
org.codehaus.janino:janino depends on org.codehaus.janino:commons-compiler and we have been upgraded to org.codehaus.janino:janino 3.0.0.

However, seems we are still pulling in org.codehaus.janino:commons-compiler 2.7.6 because of calcite. It looks like an accident because we exclude janino from calcite (see here https://github.com/apache/spark/blob/branch-2.1/pom.xml#L1759). So, this PR upgrades org.codehaus.janino:commons-compiler to 3.0.0.

## How was this patch tested?
jenkins

Author: Yin Huai <yhuai@databricks.com>

Closes #16025 from yhuai/janino-commons-compile.
2016-11-28 10:09:30 -08:00
Wenchen Fan d31ff9b7ca [SPARK-17732][SQL] Revert ALTER TABLE DROP PARTITION should support comparators
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/15704 will fail if we use int literal in `DROP PARTITION`, and we have reverted it in branch-2.1.

This PR reverts it in master branch, and add a regression test for it, to make sure the master branch is healthy.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16036 from cloud-fan/revert.
2016-11-28 08:46:00 -08:00
Herman van Hovell 38e29824d9 [SPARK-18597][SQL] Do not push-down join conditions to the right side of a LEFT ANTI join
## What changes were proposed in this pull request?
We currently push down join conditions of a Left Anti join to both sides of the join. This is similar to Inner, Left Semi and Existence (a specialized left semi) join. The problem is that this changes the semantics of the join; a left anti join filters out rows that matches the join condition.

This PR fixes this by only pushing down conditions to the left hand side of the join. This is similar to the behavior of left outer join.

## How was this patch tested?
Added tests to `FilterPushdownSuite.scala` and created a SQLQueryTestSuite file for left anti joins with a regression test.

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

Closes #16026 from hvanhovell/SPARK-18597.
2016-11-28 07:10:52 -08:00
gatorsmile 9f273c5173 [SPARK-17783][SQL] Hide Credentials in CREATE and DESC FORMATTED/EXTENDED a PERSISTENT/TEMP Table for JDBC
### What changes were proposed in this pull request?

We should never expose the Credentials in the EXPLAIN and DESC FORMATTED/EXTENDED command. However, below commands exposed the credentials.

In the related PR: https://github.com/apache/spark/pull/10452

> URL patterns to specify credential seems to be vary between different databases.

Thus, we hide the whole `url` value if it contains the keyword `password`. We also hide the `password` property.

Before the fix, the command outputs look like:

``` SQL
CREATE TABLE tab1
USING org.apache.spark.sql.jdbc
OPTIONS (
 url 'jdbc:h2:mem:testdb0;user=testUser;password=testPass',
 dbtable 'TEST.PEOPLE',
 user 'testUser',
 password '$password')

DESC FORMATTED tab1
DESC EXTENDED tab1
```

Before the fix,
- The output of SQL statement EXPLAIN
```
== Physical Plan ==
ExecutedCommand
   +- CreateDataSourceTableCommand CatalogTable(
	Table: `tab1`
	Created: Wed Nov 16 23:00:10 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: org.apache.spark.sql.jdbc
	Storage(Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass])), false
```

- The output of `DESC FORMATTED`
```
...
|Storage Desc Parameters:    |                                                                  |       |
|  url                       |jdbc:h2:mem:testdb0;user=testUser;password=testPass               |       |
|  dbtable                   |TEST.PEOPLE                                                       |       |
|  user                      |testUser                                                          |       |
|  password                  |testPass                                                          |       |
+----------------------------+------------------------------------------------------------------+-------+
```

- The output of `DESC EXTENDED`
```
|# Detailed Table Information|CatalogTable(
	Table: `default`.`tab1`
	Created: Wed Nov 16 23:00:10 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)]
	Provider: org.apache.spark.sql.jdbc
	Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=jdbc:h2:mem:testdb0;user=testUser;password=testPass, dbtable=TEST.PEOPLE, user=testUser, password=testPass]))|       |
```

After the fix,
- The output of SQL statement EXPLAIN
```
== Physical Plan ==
ExecutedCommand
   +- CreateDataSourceTableCommand CatalogTable(
	Table: `tab1`
	Created: Wed Nov 16 22:43:49 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Provider: org.apache.spark.sql.jdbc
	Storage(Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###])), false
```
- The output of `DESC FORMATTED`
```
...
|Storage Desc Parameters:    |                                                                  |       |
|  url                       |###                                                               |       |
|  dbtable                   |TEST.PEOPLE                                                       |       |
|  user                      |testUser                                                          |       |
|  password                  |###                                                               |       |
+----------------------------+------------------------------------------------------------------+-------+
```

- The output of `DESC EXTENDED`
```
|# Detailed Table Information|CatalogTable(
	Table: `default`.`tab1`
	Created: Wed Nov 16 22:43:49 PST 2016
	Last Access: Wed Dec 31 15:59:59 PST 1969
	Type: MANAGED
	Schema: [StructField(NAME,StringType,false), StructField(THEID,IntegerType,false)]
	Provider: org.apache.spark.sql.jdbc
	Storage(Location: file:/Users/xiaoli/IdeaProjects/sparkDelivery/spark-warehouse/tab1, Properties: [url=###, dbtable=TEST.PEOPLE, user=testUser, password=###]))|       |
```

### How was this patch tested?

Added test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15358 from gatorsmile/maskCredentials.
2016-11-28 07:04:38 -08:00
Herman van Hovell 70dfdcbbf1 [SPARK-18118][SQL] fix a compilation error due to nested JavaBeans\nRemove this reference. 2016-11-28 04:41:43 -08:00
Kazuaki Ishizaki f075cd9cb7 [SPARK-18118][SQL] fix a compilation error due to nested JavaBeans
## 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 generated java code `SpecificSafeProjection.apply()` for nested JavaBeans is too big. This PR avoids this compilation error by splitting a big code chunk into multiple methods by calling `CodegenContext.splitExpression` at `InitializeJavaBean.doGenCode`
An object reference for JavaBean is stored to an instance variable `javaBean...`. Then, the instance variable will be referenced in the split methods.

Generated code with this PR
````
/* 22098 */   private void apply130_0(InternalRow i) {
...
/* 22125 */     boolean isNull238 = i.isNullAt(2);
/* 22126 */     InternalRow value238 = isNull238 ? null : (i.getStruct(2, 3));
/* 22127 */     boolean isNull236 = false;
/* 22128 */     test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value236 = null;
/* 22129 */     if (!false && isNull238) {
/* 22130 */
/* 22131 */       final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value239 = null;
/* 22132 */       isNull236 = true;
/* 22133 */       value236 = value239;
/* 22134 */     } else {
/* 22135 */
/* 22136 */       final test.org.apache.spark.sql.JavaDatasetSuite$Nesting1 value241 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$Nesting1();
/* 22137 */       this.javaBean14 = value241;
/* 22138 */       if (!false) {
/* 22139 */         apply25_0(i);
/* 22140 */         apply25_1(i);
/* 22141 */         apply25_2(i);
/* 22142 */       }
/* 22143 */       isNull236 = false;
/* 22144 */       value236 = value241;
/* 22145 */     }
/* 22146 */     this.javaBean.setField2(value236);
/* 22147 */
/* 22148 */   }
...
/* 22928 */   public java.lang.Object apply(java.lang.Object _i) {
/* 22929 */     InternalRow i = (InternalRow) _i;
/* 22930 */
/* 22931 */     final test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean value1 = false ? null : new test.org.apache.spark.sql.JavaDatasetSuite$NestedComplicatedJavaBean();
/* 22932 */     this.javaBean = value1;
/* 22933 */     if (!false) {
/* 22934 */       apply130_0(i);
/* 22935 */       apply130_1(i);
/* 22936 */       apply130_2(i);
/* 22937 */       apply130_3(i);
/* 22938 */       apply130_4(i);
/* 22939 */     }
/* 22940 */     if (false) {
/* 22941 */       mutableRow.setNullAt(0);
/* 22942 */     } else {
/* 22943 */
/* 22944 */       mutableRow.update(0, value1);
/* 22945 */     }
/* 22946 */
/* 22947 */     return mutableRow;
/* 22948 */   }
````

## How was this patch tested?

added a test suite into `JavaDatasetSuite.java`

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

Closes #16032 from kiszk/SPARK-18118.
2016-11-28 04:18:35 -08:00
Herman van Hovell 454b804991 [SPARK-18604][SQL] Make sure CollapseWindow returns the attributes in the same order.
## What changes were proposed in this pull request?
The `CollapseWindow` optimizer rule changes the order of output attributes. This modifies the output of the plan, which the optimizer cannot do. This also breaks things like `collect()` for which we use a `RowEncoder` that assumes that the output attributes of the executed plan are equal to those outputted by the logical plan.

## How was this patch tested?
I have updated an incorrect test in `CollapseWindowSuite`.

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

Closes #16027 from hvanhovell/SPARK-18604.
2016-11-28 02:56:26 -08:00
Takuya UESHIN 87141622ee [SPARK-18585][SQL] Use ev.isNull = "false" if possible for Janino to have a chance to optimize.
## What changes were proposed in this pull request?

Janino can optimize `true ? a : b` into `a` or `false ? a : b` into `b`, or if/else with literal condition, so we should use literal as `ev.isNull` if possible.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16008 from ueshin/issues/SPARK-18585.
2016-11-27 23:30:18 -08:00
Wenchen Fan fc2c13bdf0 [SPARK-18482][SQL] make sure Spark can access the table metadata created by older version of spark
## What changes were proposed in this pull request?

In Spark 2.1, we did a lot of refactor for `HiveExternalCatalog` and related code path. These refactor may introduce external behavior changes and break backward compatibility. e.g. http://issues.apache.org/jira/browse/SPARK-18464

To avoid future compatibility problems of `HiveExternalCatalog`, this PR dumps some typical table metadata from tables created by 2.0, and test if they can recognized by current version of Spark.

## How was this patch tested?

test only change

Author: Wenchen Fan <wenchen@databricks.com>

Closes #16003 from cloud-fan/test.
2016-11-27 21:45:50 -08:00
gatorsmile 07f32c2283 [SPARK-18594][SQL] Name Validation of Databases/Tables
### What changes were proposed in this pull request?
Currently, the name validation checks are limited to table creation. It is enfored by Analyzer rule: `PreWriteCheck`.

However, table renaming and database creation have the same issues. It makes more sense to do the checks in `SessionCatalog`. This PR is to add it into `SessionCatalog`.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #16018 from gatorsmile/nameValidate.
2016-11-27 19:43:24 -08:00
Dongjoon Hyun 9c03c56460 [SPARK-17251][SQL] Improve OuterReference to be NamedExpression
## What changes were proposed in this pull request?

Currently, `OuterReference` is not `NamedExpression`. So, it raises 'ClassCastException` when it used in projection lists of IN correlated subqueries. This PR aims to support that by making `OuterReference` as `NamedExpression` to show correct error messages.

```scala
scala> sql("CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES 1, 2 AS t1(a)")
scala> sql("CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES 1 AS t2(b)")
scala> sql("SELECT a FROM t1 WHERE a IN (SELECT a FROM t2)").show
java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.OuterReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression
```

## How was this patch tested?

Pass the Jenkins test with new test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16015 from dongjoon-hyun/SPARK-17251-2.
2016-11-26 14:57:48 -08:00
Weiqing Yang f4a98e421e
[WIP][SQL][DOC] Fix incorrect code tag
## What changes were proposed in this pull request?
This PR is to fix incorrect `code` tag in `sql-programming-guide.md`

## How was this patch tested?
Manually.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15941 from weiqingy/fixtag.
2016-11-26 15:41:37 +00:00
Takuya UESHIN a88329d455 [SPARK-18583][SQL] Fix nullability of InputFileName.
## What changes were proposed in this pull request?

The nullability of `InputFileName` should be `false`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #16007 from ueshin/issues/SPARK-18583.
2016-11-25 20:25:29 -08:00
jiangxingbo e2fb9fd365 [SPARK-18436][SQL] isin causing SQL syntax error with JDBC
## What changes were proposed in this pull request?

The expression `in(empty seq)` is invalid in some data source. Since `in(empty seq)` is always false, we should generate `in(empty seq)` to false literal in optimizer.
The sql `SELECT * FROM t WHERE a IN ()` throws a `ParseException` which is consistent with Hive, don't need to change that behavior.

## How was this patch tested?
Add new test case in `OptimizeInSuite`.

Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15977 from jiangxb1987/isin-empty.
2016-11-25 12:44:34 -08:00
Dongjoon Hyun fb07bbe575 [SPARK-18413][SQL][FOLLOW-UP] Use numPartitions instead of maxConnections
## What changes were proposed in this pull request?

This is a follow-up PR of #15868 to merge `maxConnections` option into `numPartitions` options.

## How was this patch tested?

Pass the existing tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15966 from dongjoon-hyun/SPARK-18413-2.
2016-11-25 10:35:07 -08:00
Zhenhua Wang 5ecdc7c5c0 [SPARK-18559][SQL] Fix HLL++ with small relative error
## What changes were proposed in this pull request?

In `HyperLogLogPlusPlus`, if the relative error is so small that p >= 19, it will cause ArrayIndexOutOfBoundsException in `THRESHOLDS(p-4)` . We should check `p` and when p >= 19, regress to the original HLL result and use the small range correction they use.

The pr also fixes the upper bound in the log info in `require()`.
The upper bound is computed by:
```
val relativeSD = 1.106d / Math.pow(Math.E, p * Math.log(2.0d) / 2.0d)
```
which is derived from the equation for computing `p`:
```
val p = 2.0d * Math.log(1.106d / relativeSD) / Math.log(2.0d)
```

## How was this patch tested?

add test cases for:
1. checking validity of parameter relatvieSD
2. estimation with smaller relative error so that p >= 19

Author: Zhenhua Wang <wzh_zju@163.com>
Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15990 from wzhfy/hllppRsd.
2016-11-25 05:02:48 -08:00
hyukjinkwon 51b1c1551d
[SPARK-3359][BUILD][DOCS] More changes to resolve javadoc 8 errors that will help unidoc/genjavadoc compatibility
## What changes were proposed in this pull request?

This PR only tries to fix things that looks pretty straightforward and were fixed in other previous PRs before.

This PR roughly fixes several things as below:

- Fix unrecognisable class and method links in javadoc by changing it from `[[..]]` to `` `...` ``

  ```
  [error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/DataStreamReader.java:226: error: reference not found
  [error]    * Loads text files and returns a {link DataFrame} whose schema starts with a string column named
  ```

- Fix an exception annotation and remove code backticks in `throws` annotation

  Currently, sbt unidoc with Java 8 complains as below:

  ```
  [error] .../java/org/apache/spark/sql/streaming/StreamingQuery.java:72: error: unexpected text
  [error]    * throws StreamingQueryException, if <code>this</code> query has terminated with an exception.
  ```

  `throws` should specify the correct class name from `StreamingQueryException,` to `StreamingQueryException` without backticks. (see [JDK-8007644](https://bugs.openjdk.java.net/browse/JDK-8007644)).

- Fix `[[http..]]` to `<a href="http..."></a>`.

  ```diff
  -   * [[https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https Oracle
  -   * blog page]].
  +   * <a href="https://blogs.oracle.com/java-platform-group/entry/diagnosing_tls_ssl_and_https">
  +   * Oracle blog page</a>.
  ```

   `[[http...]]` link markdown in scaladoc is unrecognisable in javadoc.

- It seems class can't have `return` annotation. So, two cases of this were removed.

  ```
  [error] .../java/org/apache/spark/mllib/regression/IsotonicRegression.java:27: error: invalid use of return
  [error]    * return New instance of IsotonicRegression.
  ```

- Fix < to `&lt;` and > to `&gt;` according to HTML rules.

- Fix `</p>` complaint

- Exclude unrecognisable in javadoc, `constructor`, `todo` and `groupname`.

## How was this patch tested?

Manually tested by `jekyll build` with Java 7 and 8

```
java version "1.7.0_80"
Java(TM) SE Runtime Environment (build 1.7.0_80-b15)
Java HotSpot(TM) 64-Bit Server VM (build 24.80-b11, mixed mode)
```

```
java version "1.8.0_45"
Java(TM) SE Runtime Environment (build 1.8.0_45-b14)
Java HotSpot(TM) 64-Bit Server VM (build 25.45-b02, mixed mode)
```

Note: this does not yet make sbt unidoc suceed with Java 8 yet but it reduces the number of errors with Java 8.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15999 from HyukjinKwon/SPARK-3359-errors.
2016-11-25 11:27:07 +00:00
Nattavut Sutyanyong a367d5ff00 [SPARK-18578][SQL] Full outer join in correlated subquery returns incorrect results
## What changes were proposed in this pull request?

- Raise Analysis exception when correlated predicates exist in the descendant operators of either operand of a Full outer join in a subquery as well as in a FOJ operator itself
- Raise Analysis exception when correlated predicates exists in a Window operator (a side effect inadvertently introduced by SPARK-17348)

## How was this patch tested?

Run sql/test catalyst/test and new test cases, added to SubquerySuite, showing the reported incorrect results.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #16005 from nsyca/FOJ-incorrect.1.
2016-11-24 12:07:55 -08:00
Shixiong Zhu 223fa218e1 [SPARK-18510][SQL] Follow up to address comments in #15951
## What changes were proposed in this pull request?

This PR addressed the rest comments in #15951.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15997 from zsxwing/SPARK-18510-follow-up.
2016-11-23 16:15:35 -08:00
Burak Yavuz 0d1bf2b6c8 [SPARK-18510] Fix data corruption from inferred partition column dataTypes
## What changes were proposed in this pull request?

### The Issue

If I specify my schema when doing
```scala
spark.read
  .schema(someSchemaWherePartitionColumnsAreStrings)
```
but if the partition inference can infer it as IntegerType or I assume LongType or DoubleType (basically fixed size types), then once UnsafeRows are generated, your data will be corrupted.

### Proposed solution

The partition handling code path is kind of a mess. In my fix I'm probably adding to the mess, but at least trying to standardize the code path.

The real issue is that a user that uses the `spark.read` code path can never clearly specify what the partition columns are. If you try to specify the fields in `schema`, we practically ignore what the user provides, and fall back to our inferred data types. What happens in the end is data corruption.

My solution tries to fix this by always trying to infer partition columns the first time you specify the table. Once we find what the partition columns are, we try to find them in the user specified schema and use the dataType provided there, or fall back to the smallest common data type.

We will ALWAYS append partition columns to the user's schema, even if they didn't ask for it. We will only use the data type they provided if they specified it. While this is confusing, this has been the behavior since Spark 1.6, and I didn't want to change this behavior in the QA period of Spark 2.1. We may revisit this decision later.

A side effect of this PR is that we won't need https://github.com/apache/spark/pull/15942 if this PR goes in.

## How was this patch tested?

Regression tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15951 from brkyvz/partition-corruption.
2016-11-23 11:48:59 -08:00
Wenchen Fan f129ebcd30 [SPARK-18050][SQL] do not create default database if it already exists
## What changes were proposed in this pull request?

When we try to create the default database, we ask hive to do nothing if it already exists. However, Hive will log an error message instead of doing nothing, and the error message is quite annoying and confusing.

In this PR, we only create default database if it doesn't exist.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15993 from cloud-fan/default-db.
2016-11-23 12:54:18 -05:00
Reynold Xin 70ad07a9d2 [SPARK-18522][SQL] Explicit contract for column stats serialization
## What changes were proposed in this pull request?
The current implementation of column stats uses the base64 encoding of the internal UnsafeRow format to persist statistics (in table properties in Hive metastore). This is an internal format that is not stable across different versions of Spark and should NOT be used for persistence. In addition, it would be better if statistics stored in the catalog is human readable.

This pull request introduces the following changes:

1. Created a single ColumnStat class to for all data types. All data types track the same set of statistics.
2. Updated the implementation for stats collection to get rid of the dependency on internal data structures (e.g. InternalRow, or storing DateType as an int32). For example, previously dates were stored as a single integer, but are now stored as java.sql.Date. When we implement the next steps of CBO, we can add code to convert those back into internal types again.
3. Documented clearly what JVM data types are being used to store what data.
4. Defined a simple Map[String, String] interface for serializing and deserializing column stats into/from the catalog.
5. Rearranged the method/function structure so it is more clear what the supported data types are, and also moved how stats are generated into ColumnStat class so they are easy to find.

## How was this patch tested?
Removed most of the original test cases created for column statistics, and added three very simple ones to cover all the cases. The three test cases validate:
1. Roundtrip serialization works.
2. Behavior when analyzing non-existent column or unsupported data type column.
3. Result for stats collection for all valid data types.

Also moved parser related tests into a parser test suite and added an explicit serialization test for the Hive external catalog.

Author: Reynold Xin <rxin@databricks.com>

Closes #15959 from rxin/SPARK-18522.
2016-11-23 20:48:41 +08:00
Wenchen Fan 84284e8c82 [SPARK-18053][SQL] compare unsafe and safe complex-type values correctly
## What changes were proposed in this pull request?

In Spark SQL, some expression may output safe format values, e.g. `CreateArray`, `CreateStruct`, `Cast`, etc. When we compare 2 values, we should be able to compare safe and unsafe formats.

The `GreaterThan`, `LessThan`, etc. in Spark SQL already handles it, but the `EqualTo` doesn't. This PR fixes it.

## How was this patch tested?

new unit test and regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15929 from cloud-fan/type-aware.
2016-11-23 04:15:19 -08:00
Eric Liang 85235ed6c6 [SPARK-18545][SQL] Verify number of hive client RPCs in PartitionedTablePerfStatsSuite
## What changes were proposed in this pull request?

This would help catch accidental O(n) calls to the hive client as in https://issues.apache.org/jira/browse/SPARK-18507

## How was this patch tested?

Checked that the test fails before https://issues.apache.org/jira/browse/SPARK-18507 was patched. cc cloud-fan

Author: Eric Liang <ekl@databricks.com>

Closes #15985 from ericl/spark-18545.
2016-11-23 20:14:08 +08:00
Sean Owen 7e0cd1d9b1
[SPARK-18073][DOCS][WIP] Migrate wiki to spark.apache.org web site
## What changes were proposed in this pull request?

Updates links to the wiki to links to the new location of content on spark.apache.org.

## How was this patch tested?

Doc builds

Author: Sean Owen <sowen@cloudera.com>

Closes #15967 from srowen/SPARK-18073.1.
2016-11-23 11:25:47 +00:00
hyukjinkwon 2559fb4b40 [SPARK-18179][SQL] Throws analysis exception with a proper message for unsupported argument types in reflect/java_method function
## What changes were proposed in this pull request?

This PR proposes throwing an `AnalysisException` with a proper message rather than `NoSuchElementException` with the message ` key not found: TimestampType` when unsupported types are given to `reflect` and `java_method` functions.

```scala
spark.range(1).selectExpr("reflect('java.lang.String', 'valueOf', cast('1990-01-01' as timestamp))")
```

produces

**Before**

```
java.util.NoSuchElementException: key not found: TimestampType
  at scala.collection.MapLike$class.default(MapLike.scala:228)
  at scala.collection.AbstractMap.default(Map.scala:59)
  at scala.collection.MapLike$class.apply(MapLike.scala:141)
  at scala.collection.AbstractMap.apply(Map.scala:59)
  at org.apache.spark.sql.catalyst.expressions.CallMethodViaReflection$$anonfun$findMethod$1$$anonfun$apply$1.apply(CallMethodViaReflection.scala:159)
...
```

**After**

```
cannot resolve 'reflect('java.lang.String', 'valueOf', CAST('1990-01-01' AS TIMESTAMP))' due to data type mismatch: arguments from the third require boolean, byte, short, integer, long, float, double or string expressions; line 1 pos 0;
'Project [unresolvedalias(reflect(java.lang.String, valueOf, cast(1990-01-01 as timestamp)), Some(<function1>))]
+- Range (0, 1, step=1, splits=Some(2))
...
```

Added message is,

```
arguments from the third require boolean, byte, short, integer, long, float, double or string expressions
```

## How was this patch tested?

Tests added in `CallMethodViaReflection`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15694 from HyukjinKwon/SPARK-18179.
2016-11-22 22:25:27 -08:00
Dilip Biswal 39a1d30636 [SPARK-18533] Raise correct error upon specification of schema for datasource tables created using CTAS
## What changes were proposed in this pull request?
Fixes the inconsistency of error raised between data source and hive serde
tables when schema is specified in CTAS scenario. In the process the grammar for
create table (datasource) is simplified.

**before:**
``` SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1;
Error in query:
mismatched input 'as' expecting {<EOF>, '.', 'OPTIONS', 'CLUSTERED', 'PARTITIONED'}(line 1, pos 64)

== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
----------------------------------------------------------------^^^
```

**After:**
```SQL
spark-sql> create table t2 (c1 int, c2 int) using parquet as select * from t1
         > ;
Error in query:
Operation not allowed: Schema may not be specified in a Create Table As Select (CTAS) statement(line 1, pos 0)

== SQL ==
create table t2 (c1 int, c2 int) using parquet as select * from t1
^^^
```
## How was this patch tested?
Added a new test in CreateTableAsSelectSuite

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

Closes #15968 from dilipbiswal/ctas.
2016-11-22 15:57:07 -08:00
gatorsmile 9c42d4a76c [SPARK-16803][SQL] SaveAsTable does not work when target table is a Hive serde table
### What changes were proposed in this pull request?

In Spark 2.0, `SaveAsTable` does not work when the target table is a Hive serde table, but Spark 1.6 works.

**Spark 1.6**

``` Scala
scala> sql("create table sample.sample stored as SEQUENCEFILE as select 1 as key, 'abc' as value")
res2: org.apache.spark.sql.DataFrame = []

scala> val df = sql("select key, value as value from sample.sample")
df: org.apache.spark.sql.DataFrame = [key: int, value: string]

scala> df.write.mode("append").saveAsTable("sample.sample")

scala> sql("select * from sample.sample").show()
+---+-----+
|key|value|
+---+-----+
|  1|  abc|
|  1|  abc|
+---+-----+
```

**Spark 2.0**

``` Scala
scala> df.write.mode("append").saveAsTable("sample.sample")
org.apache.spark.sql.AnalysisException: Saving data in MetastoreRelation sample, sample
 is not supported.;
```

So far, we do not plan to support it in Spark 2.1 due to the risk. Spark 1.6 works because it internally uses insertInto. But, if we change it back it will break the semantic of saveAsTable (this method uses by-name resolution instead of using by-position resolution used by insertInto). More extra changes are needed to support `hive` as a `format` in DataFrameWriter.

Instead, users should use insertInto API. This PR corrects the error messages. Users can understand how to bypass it before we support it in a separate PR.
### How was this patch tested?

Test cases are added

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15926 from gatorsmile/saveAsTableFix5.
2016-11-22 15:10:49 -08:00
Burak Yavuz bdc8153e86 [SPARK-18465] Add 'IF EXISTS' clause to 'UNCACHE' to not throw exceptions when table doesn't exist
## What changes were proposed in this pull request?

While this behavior is debatable, consider the following use case:
```sql
UNCACHE TABLE foo;
CACHE TABLE foo AS
SELECT * FROM bar
```
The command above fails the first time you run it. But I want to run the command above over and over again, and I don't want to change my code just for the first run of it.
The issue is that subsequent `CACHE TABLE` commands do not overwrite the existing table.

Now we can do:
```sql
UNCACHE TABLE IF EXISTS foo;
CACHE TABLE foo AS
SELECT * FROM bar
```

## How was this patch tested?

Unit tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15896 from brkyvz/uncache.
2016-11-22 13:03:50 -08:00
Wenchen Fan 702cd403fc [SPARK-18507][SQL] HiveExternalCatalog.listPartitions should only call getTable once
## What changes were proposed in this pull request?

HiveExternalCatalog.listPartitions should only call `getTable` once, instead of calling it for every partitions.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15978 from cloud-fan/perf.
2016-11-22 15:25:22 -05:00
Nattavut Sutyanyong 45ea46b7b3 [SPARK-18504][SQL] Scalar subquery with extra group by columns returning incorrect result
## What changes were proposed in this pull request?

This PR blocks an incorrect result scenario in scalar subquery where there are GROUP BY column(s)
that are not part of the correlated predicate(s).

Example:
// Incorrect result
Seq(1).toDF("c1").createOrReplaceTempView("t1")
Seq((1,1),(1,2)).toDF("c1","c2").createOrReplaceTempView("t2")
sql("select (select sum(-1) from t2 where t1.c1=t2.c1 group by t2.c2) from t1").show

// How can selecting a scalar subquery from a 1-row table return 2 rows?

## How was this patch tested?
sql/test, catalyst/test
new test case covering the reported problem is added to SubquerySuite.scala

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #15936 from nsyca/scalarSubqueryIncorrect-1.
2016-11-22 12:06:21 -08:00
Wenchen Fan bb152cdfbb [SPARK-18519][SQL] map type can not be used in EqualTo
## What changes were proposed in this pull request?

Technically map type is not orderable, but can be used in equality comparison. However, due to the limitation of the current implementation, map type can't be used in equality comparison so that it can't be join key or grouping key.

This PR makes this limitation explicit, to avoid wrong result.

## How was this patch tested?

updated tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15956 from cloud-fan/map-type.
2016-11-22 09:16:20 -08:00
Liwei Lin ebeb0830a3 [SPARK-18425][STRUCTURED STREAMING][TESTS] Test CompactibleFileStreamLog directly
## What changes were proposed in this pull request?

Right now we are testing the most of `CompactibleFileStreamLog` in `FileStreamSinkLogSuite` (because `FileStreamSinkLog` once was the only subclass of `CompactibleFileStreamLog`, but now it's not the case any more).

Let's refactor the tests so that `CompactibleFileStreamLog` is directly tested, making future changes (like https://github.com/apache/spark/pull/15828, https://github.com/apache/spark/pull/15827) to `CompactibleFileStreamLog` much easier to test and much easier to review.

## How was this patch tested?

the PR itself is about tests

Author: Liwei Lin <lwlin7@gmail.com>

Closes #15870 from lw-lin/test-compact-1113.
2016-11-21 21:14:13 -08:00
Burak Yavuz 97a8239a62 [SPARK-18493] Add missing python APIs: withWatermark and checkpoint to dataframe
## What changes were proposed in this pull request?

This PR adds two of the newly added methods of `Dataset`s to Python:
`withWatermark` and `checkpoint`

## How was this patch tested?

Doc tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15921 from brkyvz/py-watermark.
2016-11-21 17:24:02 -08:00
hyukjinkwon a2d464770c [SPARK-17765][SQL] Support for writing out user-defined type in ORC datasource
## What changes were proposed in this pull request?

This PR adds the support for `UserDefinedType` when writing out instead of throwing `ClassCastException` in ORC data source.

In more details, `OrcStruct` is being created based on string from`DataType.catalogString`. For user-defined type, it seems it returns `sqlType.simpleString` for `catalogString` by default[1]. However, during type-dispatching to match the output with the schema, it tries to cast to, for example, `StructType`[2].

So, running the codes below (`MyDenseVector` was borrowed[3]) :

``` scala
val data = Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25))))
val udtDF = data.toDF("id", "vectors")
udtDF.write.orc("/tmp/test.orc")
```

ends up throwing an exception as below:

```
java.lang.ClassCastException: org.apache.spark.sql.UDT$MyDenseVectorUDT cannot be cast to org.apache.spark.sql.types.ArrayType
    at org.apache.spark.sql.hive.HiveInspectors$class.wrapperFor(HiveInspectors.scala:381)
    at org.apache.spark.sql.hive.orc.OrcSerializer.wrapperFor(OrcFileFormat.scala:164)
...
```

So, this PR uses `UserDefinedType.sqlType` during finding the correct converter when writing out in ORC data source.

[1]dfdcab00c7/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UserDefinedType.scala (L95)
[2]d2dc8c4a16/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala (L326)
[3]2bfed1a0c5/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala (L38-L70)
## How was this patch tested?

Unit tests in `OrcQuerySuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15361 from HyukjinKwon/SPARK-17765.
2016-11-21 13:23:32 -08:00
Dongjoon Hyun ddd02f50bb [SPARK-18517][SQL] DROP TABLE IF EXISTS should not warn for non-existing tables
## What changes were proposed in this pull request?

Currently, `DROP TABLE IF EXISTS` shows warning for non-existing tables. However, it had better be quiet for this case by definition of the command.

**BEFORE**
```scala
scala> sql("DROP TABLE IF EXISTS nonexist")
16/11/20 20:48:26 WARN DropTableCommand: org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'nonexist' not found in database 'default';
```

**AFTER**
```scala
scala> sql("DROP TABLE IF EXISTS nonexist")
res0: org.apache.spark.sql.DataFrame = []
```

## How was this patch tested?

Manual because this is related to the warning messages instead of exceptions.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15953 from dongjoon-hyun/SPARK-18517.
2016-11-21 16:14:59 -05:00
Dongjoon Hyun 07beb5d21c
[SPARK-18413][SQL] Add maxConnections JDBCOption
## What changes were proposed in this pull request?

This PR adds a new JDBCOption `maxConnections` which means the maximum number of simultaneous JDBC connections allowed. This option applies only to writing with coalesce operation if needed. It defaults to the number of partitions of RDD. Previously, SQL users cannot cannot control this while Scala/Java/Python users can use `coalesce` (or `repartition`) API.

**Reported Scenario**

For the following cases, the number of connections becomes 200 and database cannot handle all of them.

```sql
CREATE OR REPLACE TEMPORARY VIEW resultview
USING org.apache.spark.sql.jdbc
OPTIONS (
  url "jdbc:oracle:thin:10.129.10.111:1521:BKDB",
  dbtable "result",
  user "HIVE",
  password "HIVE"
);
-- set spark.sql.shuffle.partitions=200
INSERT OVERWRITE TABLE resultview SELECT g, count(1) AS COUNT FROM tnet.DT_LIVE_INFO GROUP BY g
```

## How was this patch tested?

Manual. Do the followings and see Spark UI.

**Step 1 (MySQL)**
```
CREATE TABLE t1 (a INT);
CREATE TABLE data (a INT);
INSERT INTO data VALUES (1);
INSERT INTO data VALUES (2);
INSERT INTO data VALUES (3);
```

**Step 2 (Spark)**
```scala
SPARK_HOME=$PWD bin/spark-shell --driver-memory 4G --driver-class-path mysql-connector-java-5.1.40-bin.jar
scala> sql("SET spark.sql.shuffle.partitions=3")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW data USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 'data', user 'root', password '')")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '1')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '2')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '3')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
scala> sql("CREATE OR REPLACE TEMPORARY VIEW t1 USING org.apache.spark.sql.jdbc OPTIONS (url 'jdbc:mysql://localhost:3306/t', dbtable 't1', user 'root', password '', maxConnections '4')")
scala> sql("INSERT OVERWRITE TABLE t1 SELECT a FROM data GROUP BY a")
```

![maxconnections](https://cloud.githubusercontent.com/assets/9700541/20287987/ed8409c2-aa84-11e6-8aab-ae28e63fe54d.png)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15868 from dongjoon-hyun/SPARK-18413.
2016-11-21 13:57:36 +00:00
Takuya UESHIN 9f262ae163 [SPARK-18398][SQL] Fix nullabilities of MapObjects and ExternalMapToCatalyst.
## What changes were proposed in this pull request?

The nullabilities of `MapObject` can be made more strict by relying on `inputObject.nullable` and `lambdaFunction.nullable`.

Also `ExternalMapToCatalyst.dataType` can be made more strict by relying on `valueConverter.nullable`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15840 from ueshin/issues/SPARK-18398.
2016-11-21 05:50:35 -08:00
Takuya UESHIN 6585479749 [SPARK-18467][SQL] Extracts method for preparing arguments from StaticInvoke, Invoke and NewInstance and modify to short circuit if arguments have null when needNullCheck == true.
## What changes were proposed in this pull request?

This pr extracts method for preparing arguments from `StaticInvoke`, `Invoke` and `NewInstance` and modify to short circuit if arguments have `null` when `propageteNull == true`.

The steps are as follows:

1. Introduce `InvokeLike` to extract common logic from `StaticInvoke`, `Invoke` and `NewInstance` to prepare arguments.
`StaticInvoke` and `Invoke` had a risk to exceed 64kb JVM limit to prepare arguments but after this patch they can handle them because they share the preparing code of NewInstance, which handles the limit well.

2. Remove unneeded null checking and fix nullability of `NewInstance`.
Avoid some of nullabilty checking which are not needed because the expression is not nullable.

3. Modify to short circuit if arguments have `null` when `needNullCheck == true`.
If `needNullCheck == true`, preparing arguments can be skipped if we found one of them is `null`, so modified to short circuit in the case.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15901 from ueshin/issues/SPARK-18467.
2016-11-21 12:05:01 +08:00
Reynold Xin b625a36ebc [HOTFIX][SQL] Fix DDLSuite failure. 2016-11-20 20:00:59 -08:00
Herman van Hovell 7ca7a63524 [SPARK-15214][SQL] Code-generation for Generate
## What changes were proposed in this pull request?

This PR adds code generation to `Generate`. It supports two code paths:
- General `TraversableOnce` based iteration. This used for regular `Generator` (code generation supporting) expressions. This code path expects the expression to return a `TraversableOnce[InternalRow]` and it will iterate over the returned collection. This PR adds code generation for the `stack` generator.
- Specialized `ArrayData/MapData` based iteration. This is used for the `explode`, `posexplode` & `inline` functions and operates directly on the `ArrayData`/`MapData` result that the child of the generator returns.

### Benchmarks
I have added some benchmarks and it seems we can create a nice speedup for explode:
#### Environment
```
Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6
Intel(R) Core(TM) i7-4980HQ CPU  2.80GHz
```
#### Explode Array
##### Before
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7377 / 7607          2.3         439.7       1.0X
generate explode array wholestage on          6055 / 6086          2.8         360.9       1.2X
```
##### After
```
generate explode array:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode array wholestage off         7432 / 7696          2.3         443.0       1.0X
generate explode array wholestage on           631 /  646         26.6          37.6      11.8X
```
#### Explode Map
##### Before
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         12792 / 12848          1.3         762.5       1.0X
generate explode map wholestage on          11181 / 11237          1.5         666.5       1.1X
```
##### After
```
generate explode map:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate explode map wholestage off         10949 / 10972          1.5         652.6       1.0X
generate explode map wholestage on             870 /  913         19.3          51.9      12.6X
```
#### Posexplode
##### Before
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7547 / 7580          2.2         449.8       1.0X
generate posexplode array wholestage on       5786 / 5838          2.9         344.9       1.3X
```
##### After
```
generate posexplode array:               Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate posexplode array wholestage off      7535 / 7548          2.2         449.1       1.0X
generate posexplode array wholestage on        620 /  624         27.1          37.0      12.1X
```
#### Inline
##### Before
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6935 / 6978          2.4         413.3       1.0X
generate inline array wholestage on           6360 / 6400          2.6         379.1       1.1X
```
##### After
```
generate inline array:                   Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate inline array wholestage off          6940 / 6966          2.4         413.6       1.0X
generate inline array wholestage on           1002 / 1012         16.7          59.7       6.9X
```
#### Stack
##### Before
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12980 / 13104          1.3         773.7       1.0X
generate stack wholestage on                11566 / 11580          1.5         689.4       1.1X
```
##### After
```
generate stack:                          Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
generate stack wholestage off               12875 / 12949          1.3         767.4       1.0X
generate stack wholestage on                   840 /  845         20.0          50.0      15.3X
```
## How was this patch tested?

Existing tests.

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

Closes #13065 from hvanhovell/SPARK-15214.
2016-11-19 23:55:09 -08:00
Reynold Xin a64f25d8b4 [SQL] Fix documentation for Concat and ConcatWs 2016-11-19 21:57:49 -08:00
Reynold Xin bce9a03677 [SPARK-18508][SQL] Fix documentation error for DateDiff
## What changes were proposed in this pull request?
The previous documentation and example for DateDiff was wrong.

## How was this patch tested?
Doc only change.

Author: Reynold Xin <rxin@databricks.com>

Closes #15937 from rxin/datediff-doc.
2016-11-19 21:57:09 -08:00
Sean Owen ded5fefb6f
[SPARK-18448][CORE] Fix @since 2.1.0 on new SparkSession.close() method
## What changes were proposed in this pull request?

Fix since 2.1.0 on new SparkSession.close() method. I goofed in https://github.com/apache/spark/pull/15932 because it was back-ported to 2.1 instead of just master as originally planned.

Author: Sean Owen <sowen@cloudera.com>

Closes #15938 from srowen/SPARK-18448.2.
2016-11-19 13:48:56 +00:00
hyukjinkwon d5b1d5fc80
[SPARK-18445][BUILD][DOCS] Fix the markdown for Note:/NOTE:/Note that/'''Note:''' across Scala/Java API documentation
## What changes were proposed in this pull request?

It seems in Scala/Java,

- `Note:`
- `NOTE:`
- `Note that`
- `'''Note:'''`
- `note`

This PR proposes to fix those to `note` to be consistent.

**Before**

- Scala
  ![2016-11-17 6 16 39](https://cloud.githubusercontent.com/assets/6477701/20383180/1a7aed8c-acf2-11e6-9611-5eaf6d52c2e0.png)

- Java
  ![2016-11-17 6 14 41](https://cloud.githubusercontent.com/assets/6477701/20383096/c8ffc680-acf1-11e6-914a-33460bf1401d.png)

**After**

- Scala
  ![2016-11-17 6 16 44](https://cloud.githubusercontent.com/assets/6477701/20383167/09940490-acf2-11e6-937a-0d5e1dc2cadf.png)

- Java
  ![2016-11-17 6 13 39](https://cloud.githubusercontent.com/assets/6477701/20383132/e7c2a57e-acf1-11e6-9c47-b849674d4d88.png)

## How was this patch tested?

The notes were found via

```bash
grep -r "NOTE: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// NOTE: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \ # note that this is a regular expression. So actual matches were mostly `org/apache/spark/api/java/functions ...`
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note that " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note that " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "Note: " . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// Note: " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

```bash
grep -r "'''Note:'''" . | \ # Note:|NOTE:|Note that|'''Note:'''
grep -v "// '''Note:''' " | \  # starting with // does not appear in API documentation.
grep -E '.scala|.java' | \ # java/scala files
grep -v Suite | \ # exclude tests
grep -v Test | \ # exclude tests
grep -e 'org.apache.spark.api.java' \ # packages appear in API documenation
-e 'org.apache.spark.api.java.function' \
-e 'org.apache.spark.api.r' \
...
```

And then fixed one by one comparing with API documentation/access modifiers.

After that, manually tested via `jekyll build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15889 from HyukjinKwon/SPARK-18437.
2016-11-19 11:24:15 +00:00
Sean Owen db9fb9baac
[SPARK-18448][CORE] SparkSession should implement java.lang.AutoCloseable like JavaSparkContext
## What changes were proposed in this pull request?

Just adds `close()` + `Closeable` as a synonym for `stop()`. This makes it usable in Java in try-with-resources, as suggested by ash211  (`Closeable` extends `AutoCloseable` BTW)

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #15932 from srowen/SPARK-18448.
2016-11-19 09:00:11 +00:00
Shixiong Zhu 2a40de408b [SPARK-18497][SS] Make ForeachSink support watermark
## What changes were proposed in this pull request?

The issue in ForeachSink is the new created DataSet still uses the old QueryExecution. When `foreachPartition` is called, `QueryExecution.toString` will be called and then fail because it doesn't know how to plan EventTimeWatermark.

This PR just replaces the QueryExecution with IncrementalExecution to fix the issue.

## How was this patch tested?

`test("foreach with watermark")`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15934 from zsxwing/SPARK-18497.
2016-11-18 16:34:38 -08:00
Reynold Xin 6f7ff75091 [SPARK-18505][SQL] Simplify AnalyzeColumnCommand
## What changes were proposed in this pull request?
I'm spending more time at the design & code level for cost-based optimizer now, and have found a number of issues related to maintainability and compatibility that I will like to address.

This is a small pull request to clean up AnalyzeColumnCommand:

1. Removed warning on duplicated columns. Warnings in log messages are useless since most users that run SQL don't see them.
2. Removed the nested updateStats function, by just inlining the function.
3. Renamed a few functions to better reflect what they do.
4. Removed the factory apply method for ColumnStatStruct. It is a bad pattern to use a apply method that returns an instantiation of a class that is not of the same type (ColumnStatStruct.apply used to return CreateNamedStruct).
5. Renamed ColumnStatStruct to just AnalyzeColumnCommand.
6. Added more documentation explaining some of the non-obvious return types and code blocks.

In follow-up pull requests, I'd like to address the following:

1. Get rid of the Map[String, ColumnStat] map, since internally we should be using Attribute to reference columns, rather than strings.
2. Decouple the fields exposed by ColumnStat and internals of Spark SQL's execution path. Currently the two are coupled because ColumnStat takes in an InternalRow.
3. Correctness: Remove code path that stores statistics in the catalog using the base64 encoding of the UnsafeRow format, which is not stable across Spark versions.
4. Clearly document the data representation stored in the catalog for statistics.

## How was this patch tested?
Affected test cases have been updated.

Author: Reynold Xin <rxin@databricks.com>

Closes #15933 from rxin/SPARK-18505.
2016-11-18 16:34:11 -08:00
Shixiong Zhu e5f5c29e02 [SPARK-18477][SS] Enable interrupts for HDFS in HDFSMetadataLog
## What changes were proposed in this pull request?

HDFS `write` may just hang until timeout if some network error happens. It's better to enable interrupts to allow stopping the query fast on HDFS.

This PR just changes the logic to only disable interrupts for local file system, as HADOOP-10622 only happens for local file system.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15911 from zsxwing/interrupt-on-dfs.
2016-11-18 16:13:02 -08:00
Andrew Ray 795e9fc921 [SPARK-18457][SQL] ORC and other columnar formats using HiveShim read all columns when doing a simple count
## What changes were proposed in this pull request?

When reading zero columns (e.g., count(*)) from ORC or any other format that uses HiveShim, actually set the read column list to empty for Hive to use.

## How was this patch tested?

Query correctness is handled by existing unit tests. I'm happy to add more if anyone can point out some case that is not covered.

Reduction in data read can be verified in the UI when built with a recent version of Hadoop say:
```
build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.0 -Phive -DskipTests clean package
```
However the default Hadoop 2.2 that is used for unit tests does not report actual bytes read and instead just full file sizes (see FileScanRDD.scala line 80). Therefore I don't think there is a good way to add a unit test for this.

I tested with the following setup using above build options
```
case class OrcData(intField: Long, stringField: String)
spark.range(1,1000000).map(i => OrcData(i, s"part-$i")).toDF().write.format("orc").save("orc_test")

sql(
      s"""CREATE EXTERNAL TABLE orc_test(
         |  intField LONG,
         |  stringField STRING
         |)
         |STORED AS ORC
         |LOCATION '${System.getProperty("user.dir") + "/orc_test"}'
       """.stripMargin)
```

## Results

query | Spark 2.0.2 | this PR
---|---|---
`sql("select count(*) from orc_test").collect`|4.4 MB|199.4 KB
`sql("select intField from orc_test").collect`|743.4 KB|743.4 KB
`sql("select * from orc_test").collect`|4.4 MB|4.4 MB

Author: Andrew Ray <ray.andrew@gmail.com>

Closes #15898 from aray/sql-orc-no-col.
2016-11-18 11:19:49 -08:00
Tyson Condie 51baca2219 [SPARK-18187][SQL] CompactibleFileStreamLog should not use "compactInterval" direcly with user setting.
## What changes were proposed in this pull request?
CompactibleFileStreamLog relys on "compactInterval" to detect a compaction batch. If the "compactInterval" is reset by user, CompactibleFileStreamLog will return wrong answer, resulting data loss. This PR procides a way to check the validity of 'compactInterval', and calculate an appropriate value.

## How was this patch tested?
When restart a stream, we change the 'spark.sql.streaming.fileSource.log.compactInterval' different with the former one.

The primary solution to this issue was given by uncleGen
Added extensions include an additional metadata field in OffsetSeq and CompactibleFileStreamLog APIs. zsxwing

Author: Tyson Condie <tcondie@gmail.com>
Author: genmao.ygm <genmao.ygm@genmaoygmdeMacBook-Air.local>

Closes #15852 from tcondie/spark-18187.
2016-11-18 11:11:24 -08:00
Josh Rosen d9dd979d17 [SPARK-18462] Fix ClassCastException in SparkListenerDriverAccumUpdates event
## What changes were proposed in this pull request?

This patch fixes a `ClassCastException: java.lang.Integer cannot be cast to java.lang.Long` error which could occur in the HistoryServer while trying to process a deserialized `SparkListenerDriverAccumUpdates` event.

The problem stems from how `jackson-module-scala` handles primitive type parameters (see https://github.com/FasterXML/jackson-module-scala/wiki/FAQ#deserializing-optionint-and-other-primitive-challenges for more details). This was causing a problem where our code expected a field to be deserialized as a `(Long, Long)` tuple but we got an `(Int, Int)` tuple instead.

This patch hacks around this issue by registering a custom `Converter` with Jackson in order to deserialize the tuples as `(Object, Object)` and perform the appropriate casting.

## How was this patch tested?

New regression tests in `SQLListenerSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15922 from JoshRosen/SPARK-18462.
2016-11-17 18:45:15 -08:00
Wenchen Fan ce13c26723 [SPARK-18360][SQL] default table path of tables in default database should depend on the location of default database
## What changes were proposed in this pull request?

The current semantic of the warehouse config:

1. it's a static config, which means you can't change it once your spark application is launched.
2. Once a database is created, its location won't change even the warehouse path config is changed.
3. default database is a special case, although its location is fixed, but the locations of tables created in it are not. If a Spark app starts with warehouse path B(while the location of default database is A), then users create a table `tbl` in default database, its location will be `B/tbl` instead of `A/tbl`. If uses change the warehouse path config to C, and create another table `tbl2`, its location will still be `B/tbl2` instead of `C/tbl2`.

rule 3 doesn't make sense and I think we made it by mistake, not intentionally. Data source tables don't follow rule 3 and treat default database like normal ones.

This PR fixes hive serde tables to make it consistent with data source tables.

## How was this patch tested?

HiveSparkSubmitSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15812 from cloud-fan/default-db.
2016-11-17 17:31:12 -08:00
root b0aa1aa1af
[SPARK-18490][SQL] duplication nodename extrainfo for ShuffleExchange
## What changes were proposed in this pull request?

   In ShuffleExchange, the nodename's extraInfo are the same when exchangeCoordinator.isEstimated
 is true or false.

Merge the two situation in the PR.

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

Closes #15920 from windpiger/DupNodeNameShuffleExchange.
2016-11-17 17:04:19 +00:00
anabranch 49b6f456ac
[SPARK-18365][DOCS] Improve Sample Method Documentation
## What changes were proposed in this pull request?

I found the documentation for the sample method to be confusing, this adds more clarification across all languages.

- [x] Scala
- [x] Python
- [x] R
- [x] RDD Scala
- [ ] RDD Python with SEED
- [X] RDD Java
- [x] RDD Java with SEED
- [x] RDD Python

## How was this patch tested?

NA

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

Author: anabranch <wac.chambers@gmail.com>
Author: Bill Chambers <bill@databricks.com>

Closes #15815 from anabranch/SPARK-18365.
2016-11-17 11:34:55 +00:00
Wenchen Fan 07b3f045cd [SPARK-18464][SQL] support old table which doesn't store schema in metastore
## What changes were proposed in this pull request?

Before Spark 2.1, users can create an external data source table without schema, and we will infer the table schema at runtime. In Spark 2.1, we decided to infer the schema when the table was created, so that we don't need to infer it again and again at runtime.

This is a good improvement, but we should still respect and support old tables which doesn't store table schema in metastore.

## How was this patch tested?

regression test.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15900 from cloud-fan/hive-catalog.
2016-11-17 00:00:38 -08:00
Takuya UESHIN 170eeb345f [SPARK-18442][SQL] Fix nullability of WrapOption.
## What changes were proposed in this pull request?

The nullability of `WrapOption` should be `false`.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@happy-camper.st>

Closes #15887 from ueshin/issues/SPARK-18442.
2016-11-17 11:21:08 +08:00
Cheng Lian 2ca8ae9aa1 [SPARK-18186] Migrate HiveUDAFFunction to TypedImperativeAggregate for partial aggregation support
## What changes were proposed in this pull request?

While being evaluated in Spark SQL, Hive UDAFs don't support partial aggregation. This PR migrates `HiveUDAFFunction`s to `TypedImperativeAggregate`, which already provides partial aggregation support for aggregate functions that may use arbitrary Java objects as aggregation states.

The following snippet shows the effect of this PR:

```scala
import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax
sql(s"CREATE FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'")

spark.range(100).createOrReplaceTempView("t")

// A query using both Spark SQL native `max` and Hive `max`
sql(s"SELECT max(id), hive_max(id) FROM t").explain()
```

Before this PR:

```
== Physical Plan ==
SortAggregate(key=[], functions=[max(id#1L), default.hive_max(default.hive_max, HiveFunctionWrapper(org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax,org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax7475f57e), id#1L, false, 0, 0)])
+- Exchange SinglePartition
   +- *Range (0, 100, step=1, splits=Some(1))
```

After this PR:

```
== Physical Plan ==
SortAggregate(key=[], functions=[max(id#1L), default.hive_max(default.hive_max, HiveFunctionWrapper(org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax,org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax5e18a6a7), id#1L, false, 0, 0)])
+- Exchange SinglePartition
   +- SortAggregate(key=[], functions=[partial_max(id#1L), partial_default.hive_max(default.hive_max, HiveFunctionWrapper(org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax,org.apache.hadoop.hive.ql.udf.generic.GenericUDAFMax5e18a6a7), id#1L, false, 0, 0)])
      +- *Range (0, 100, step=1, splits=Some(1))
```

The tricky part of the PR is mostly about updating and passing around aggregation states of `HiveUDAFFunction`s since the aggregation state of a Hive UDAF may appear in three different forms. Let's take a look at the testing `MockUDAF` added in this PR as an example. This UDAF computes the count of non-null values together with the count of nulls of a given column. Its aggregation state may appear as the following forms at different time:

1. A `MockUDAFBuffer`, which is a concrete subclass of `GenericUDAFEvaluator.AggregationBuffer`

   The form used by Hive UDAF API. This form is required by the following scenarios:

   - Calling `GenericUDAFEvaluator.iterate()` to update an existing aggregation state with new input values.
   - Calling `GenericUDAFEvaluator.terminate()` to get the final aggregated value from an existing aggregation state.
   - Calling `GenericUDAFEvaluator.merge()` to merge other aggregation states into an existing aggregation state.

     The existing aggregation state to be updated must be in this form.

   Conversions:

   - To form 2:

     `GenericUDAFEvaluator.terminatePartial()`

   - To form 3:

     Convert to form 2 first, and then to 3.

2. An `Object[]` array containing two `java.lang.Long` values.

   The form used to interact with Hive's `ObjectInspector`s. This form is required by the following scenarios:

   - Calling `GenericUDAFEvaluator.terminatePartial()` to convert an existing aggregation state in form 1 to form 2.
   - Calling `GenericUDAFEvaluator.merge()` to merge other aggregation states into an existing aggregation state.

     The input aggregation state must be in this form.

   Conversions:

   - To form 1:

     No direct method. Have to create an empty `AggregationBuffer` and merge it into the empty buffer.

   - To form 3:

     `unwrapperFor()`/`unwrap()` method of `HiveInspectors`

3. The byte array that holds data of an `UnsafeRow` with two `LongType` fields.

   The form used by Spark SQL to shuffle partial aggregation results. This form is required because `TypedImperativeAggregate` always asks its subclasses to serialize their aggregation states into a byte array.

   Conversions:

   - To form 1:

     Convert to form 2 first, and then to 1.

   - To form 2:

     `wrapperFor()`/`wrap()` method of `HiveInspectors`

Here're some micro-benchmark results produced by the most recent master and this PR branch.

Master:

```
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
------------------------------------------------------------------------------------------------
w/o groupBy                                    339 /  372          3.1         323.2       1.0X
w/ groupBy                                     503 /  529          2.1         479.7       0.7X
```

This PR:

```
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
------------------------------------------------------------------------------------------------
w/o groupBy                                    116 /  126          9.0         110.8       1.0X
w/ groupBy                                     151 /  159          6.9         144.0       0.8X
```

Benchmark code snippet:

```scala
  test("Hive UDAF benchmark") {
    val N = 1 << 20

    sparkSession.sql(s"CREATE TEMPORARY FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'")

    val benchmark = new Benchmark(
      name = "hive udaf vs spark af",
      valuesPerIteration = N,
      minNumIters = 5,
      warmupTime = 5.seconds,
      minTime = 5.seconds,
      outputPerIteration = true
    )

    benchmark.addCase("w/o groupBy") { _ =>
      sparkSession.range(N).agg("id" -> "hive_max").collect()
    }

    benchmark.addCase("w/ groupBy") { _ =>
      sparkSession.range(N).groupBy($"id" % 10).agg("id" -> "hive_max").collect()
    }

    benchmark.run()

    sparkSession.sql(s"DROP TEMPORARY FUNCTION IF EXISTS hive_max")
  }
```

## How was this patch tested?

New test suite `HiveUDAFSuite` is added.

Author: Cheng Lian <lian@databricks.com>

Closes #15703 from liancheng/partial-agg-hive-udaf.
2016-11-16 14:32:36 -08:00
Tathagata Das 0048ce7ce6 [SPARK-18459][SPARK-18460][STRUCTUREDSTREAMING] Rename triggerId to batchId and add triggerDetails to json in StreamingQueryStatus
## What changes were proposed in this pull request?

SPARK-18459: triggerId seems like a number that should be increasing with each trigger, whether or not there is data in it. However, actually, triggerId increases only where there is a batch of data in a trigger. So its better to rename it to batchId.

SPARK-18460: triggerDetails was missing from json representation. Fixed it.

## How was this patch tested?
Updated existing unit tests.

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

Closes #15895 from tdas/SPARK-18459.
2016-11-16 10:00:59 -08:00
gatorsmile 608ecc512b [SPARK-18415][SQL] Weird Plan Output when CTE used in RunnableCommand
### What changes were proposed in this pull request?
Currently, when CTE is used in RunnableCommand, the Analyzer does not replace the logical node `With`. The child plan of RunnableCommand is not resolved. Thus, the output of the `With` plan node looks very confusing.
For example,
```
sql(
  """
    |CREATE VIEW cte_view AS
    |WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
    |SELECT n FROM w
  """.stripMargin).explain()
```
The output is like
```
ExecutedCommand
   +- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
SELECT n FROM w, false, false, PersistedView
         +- 'With [(w,SubqueryAlias w
+- Project [1 AS n#16]
   +- OneRowRelation$
), (cte1,'SubqueryAlias cte1
+- 'Project [unresolvedalias(2, None)]
   +- OneRowRelation$
), (cte2,'SubqueryAlias cte2
+- 'Project [unresolvedalias(3, None)]
   +- OneRowRelation$
)]
            +- 'Project ['n]
               +- 'UnresolvedRelation `w`
```
After the fix, the output is as shown below.
```
ExecutedCommand
   +- CreateViewCommand `cte_view`, WITH w AS (SELECT 1 AS n), cte1 (select 2), cte2 as (select 3)
SELECT n FROM w, false, false, PersistedView
         +- CTE [w, cte1, cte2]
            :  :- SubqueryAlias w
            :  :  +- Project [1 AS n#16]
            :  :     +- OneRowRelation$
            :  :- 'SubqueryAlias cte1
            :  :  +- 'Project [unresolvedalias(2, None)]
            :  :     +- OneRowRelation$
            :  +- 'SubqueryAlias cte2
            :     +- 'Project [unresolvedalias(3, None)]
            :        +- OneRowRelation$
            +- 'Project ['n]
               +- 'UnresolvedRelation `w`
```

BTW, this PR also fixes the output of the view type.

### How was this patch tested?
Manual

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15854 from gatorsmile/cteName.
2016-11-16 08:25:15 -08:00
Xianyang Liu 7569cf6cb8
[SPARK-18420][BUILD] Fix the errors caused by lint check in Java
## What changes were proposed in this pull request?

Small fix, fix the errors caused by lint check in Java

- Clear unused objects and `UnusedImports`.
- Add comments around the method `finalize` of `NioBufferedFileInputStream`to turn off checkstyle.
- Cut the line which is longer than 100 characters into two lines.

## How was this patch tested?
Travis CI.
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
```
Before:
```
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[21,8] (imports) UnusedImports: Unused import - org.apache.commons.crypto.cipher.CryptoCipherFactory.
[ERROR] src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java:[516,5] (modifier) RedundantModifier: Redundant 'public' modifier.
[ERROR] src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java:[133] (coding) NoFinalizer: Avoid using finalizer method.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java:[71] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java:[112] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[31,17] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR]src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.ml.linalg.Vectors.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
```

After:
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
Using `mvn` from path: /home/travis/build/ConeyLiu/spark/build/apache-maven-3.3.9/bin/mvn
Checkstyle checks passed.
```

Author: Xianyang Liu <xyliu0530@icloud.com>

Closes #15865 from ConeyLiu/master.
2016-11-16 11:59:00 +00:00
Dongjoon Hyun 74f5c2176d [SPARK-18433][SQL] Improve DataSource option keys to be more case-insensitive
## What changes were proposed in this pull request?

This PR aims to improve DataSource option keys to be more case-insensitive

DataSource partially use CaseInsensitiveMap in code-path. For example, the following fails to find url.

```scala
val df = spark.createDataFrame(sparkContext.parallelize(arr2x2), schema2)
df.write.format("jdbc")
    .option("UrL", url1)
    .option("dbtable", "TEST.SAVETEST")
    .options(properties.asScala)
    .save()
```

This PR makes DataSource options to use CaseInsensitiveMap internally and also makes DataSource to use CaseInsensitiveMap generally except `InMemoryFileIndex` and `InsertIntoHadoopFsRelationCommand`. We can not pass them CaseInsensitiveMap because they creates new case-sensitive HadoopConfs by calling newHadoopConfWithOptions(options) inside.

## How was this patch tested?

Pass the Jenkins test with newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15884 from dongjoon-hyun/SPARK-18433.
2016-11-16 17:12:18 +08:00
Wenchen Fan 4ac9759f80 [SPARK-18377][SQL] warehouse path should be a static conf
## What changes were proposed in this pull request?

it's weird that every session can set its own warehouse path at runtime, we should forbid it and make it a static conf.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15825 from cloud-fan/warehouse.
2016-11-15 20:24:36 -08:00
Herman van Hovell 4b35d13bac [SPARK-18300][SQL] Fix scala 2.10 build for FoldablePropagation
## What changes were proposed in this pull request?
Commit f14ae4900a broke the scala 2.10 build. This PR fixes this by simplifying the used pattern match.

## How was this patch tested?
Tested building manually. Ran `build/sbt -Dscala-2.10 -Pscala-2.10 package`.

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

Closes #15891 from hvanhovell/SPARK-18300-scala-2.10.
2016-11-15 16:55:02 -08:00
Dongjoon Hyun 3ce057d001 [SPARK-17732][SQL] ALTER TABLE DROP PARTITION should support comparators
## What changes were proposed in this pull request?

This PR aims to support `comparators`, e.g. '<', '<=', '>', '>=', again in Apache Spark 2.0 for backward compatibility.

**Spark 1.6**

``` scala
scala> sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)")
res0: org.apache.spark.sql.DataFrame = [result: string]

scala> sql("ALTER TABLE sales DROP PARTITION (country < 'KR')")
res1: org.apache.spark.sql.DataFrame = [result: string]
```

**Spark 2.0**

``` scala
scala> sql("CREATE TABLE sales(id INT) PARTITIONED BY (country STRING, quarter STRING)")
res0: org.apache.spark.sql.DataFrame = []

scala> sql("ALTER TABLE sales DROP PARTITION (country < 'KR')")
org.apache.spark.sql.catalyst.parser.ParseException:
mismatched input '<' expecting {')', ','}(line 1, pos 42)
```

After this PR, it's supported.

## How was this patch tested?

Pass the Jenkins test with a newly added testcase.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15704 from dongjoon-hyun/SPARK-17732-2.
2016-11-15 15:59:04 -08:00
Tathagata Das 1ae4652b7e [SPARK-18440][STRUCTURED STREAMING] Pass correct query execution to FileFormatWriter
## What changes were proposed in this pull request?

SPARK-18012 refactored the file write path in FileStreamSink using FileFormatWriter which always uses the default non-streaming QueryExecution to perform the writes. This is wrong for FileStreamSink, because the streaming QueryExecution (i.e. IncrementalExecution) should be used for correctly incrementalizing aggregation. The addition of watermarks in SPARK-18124, file stream sink should logically supports aggregation + watermark + append mode. But actually it fails with
```
16:23:07.389 ERROR org.apache.spark.sql.execution.streaming.StreamExecution: Query query-0 terminated with error
java.lang.AssertionError: assertion failed: No plan for EventTimeWatermark timestamp#7: timestamp, interval 10 seconds
+- LocalRelation [timestamp#7]

	at scala.Predef$.assert(Predef.scala:170)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:92)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:77)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:74)
	at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
	at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
	at scala.collection.Iterator$class.foreach(Iterator.scala:893)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
	at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157)
	at scala.collection.AbstractIterator.foldLeft(Iterator.scala:1336)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2.apply(QueryPlanner.scala:74)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2.apply(QueryPlanner.scala:66)
	at scala.collection.Iterator$$anon$12.nextCur(Iterator.scala:434)
	at scala.collection.Iterator$$anon$12.hasNext(Iterator.scala:440)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner.plan(QueryPlanner.scala:92)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:77)
	at org.apache.spark.sql.catalyst.planning.QueryPlanner$$anonfun$2$$anonfun$apply$2.apply(QueryPlanner.scala:74)
```

This PR fixes it by passing the correct query execution.

## How was this patch tested?
New unit test

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

Closes #15885 from tdas/SPARK-18440.
2016-11-15 15:12:30 -08:00
Burak Yavuz 2afdaa9805 [SPARK-18337] Complete mode memory sinks should be able to recover from checkpoints
## What changes were proposed in this pull request?

It would be nice if memory sinks can also recover from checkpoints. For correctness reasons, the only time we should support it is in `Complete` OutputMode. We can support this in CompleteMode, because the output of the StateStore is already persisted in the checkpoint directory.

## How was this patch tested?

Unit test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15801 from brkyvz/mem-stream.
2016-11-15 13:09:29 -08:00
genmao.ygm 745ab8bc50 [SPARK-18379][SQL] Make the parallelism of parallelPartitionDiscovery configurable.
## What changes were proposed in this pull request?

The largest parallelism in PartitioningAwareFileIndex #listLeafFilesInParallel() is 10000 in hard code. We may need to make this number configurable. And in PR, I reduce it to 100.

## How was this patch tested?

Existing ut.

Author: genmao.ygm <genmao.ygm@genmaoygmdeMacBook-Air.local>
Author: dylon <hustyugm@gmail.com>

Closes #15829 from uncleGen/SPARK-18379.
2016-11-15 10:32:43 -08:00
Herman van Hovell f14ae4900a [SPARK-18300][SQL] Do not apply foldable propagation with expand as a child.
## What changes were proposed in this pull request?
The `FoldablePropagation` optimizer rule, pulls foldable values out from under an `Expand`. This breaks the `Expand` in two ways:

- It rewrites the output attributes of the `Expand`. We explicitly define output attributes for `Expand`, these are (unfortunately) considered as part of the expressions of the `Expand` and can be rewritten.
- Expand can actually change the column (it will typically re-use the attributes or the underlying plan). This means that we cannot safely propagate the expressions from under an `Expand`.

This PR fixes this and (hopefully) other issues by explicitly whitelisting allowed operators.

## How was this patch tested?
Added tests to `FoldablePropagationSuite` and to `SQLQueryTestSuite`.

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

Closes #15857 from hvanhovell/SPARK-18300.
2016-11-15 06:59:25 -08:00
gatorsmile 86430cc4e8 [SPARK-18430][SQL] Fixed Exception Messages when Hitting an Invocation Exception of Function Lookup
### What changes were proposed in this pull request?
When the exception is an invocation exception during function lookup, we return a useless/confusing error message:

For example,
```Scala
df.selectExpr("concat_ws()")
```
Below is the error message we got:
```
null; line 1 pos 0
org.apache.spark.sql.AnalysisException: null; line 1 pos 0
```

To get the meaningful error message, we need to get the cause. The fix is exactly the same as what we did in https://github.com/apache/spark/pull/12136. After the fix, the message we got is the exception issued in the constuctor of function implementation:
```
requirement failed: concat_ws requires at least one argument.; line 1 pos 0
org.apache.spark.sql.AnalysisException: requirement failed: concat_ws requires at least one argument.; line 1 pos 0
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #15878 from gatorsmile/functionNotFound.
2016-11-14 21:21:34 -08:00
Michael Armbrust c07187823a [SPARK-18124] Observed delay based Event Time Watermarks
This PR adds a new method `withWatermark` to the `Dataset` API, which can be used specify an _event time watermark_.  An event time watermark allows the streaming engine to reason about the point in time after which we no longer expect to see late data.  This PR also has augmented `StreamExecution` to use this watermark for several purposes:
  - To know when a given time window aggregation is finalized and thus results can be emitted when using output modes that do not allow updates (e.g. `Append` mode).
  - To minimize the amount of state that we need to keep for on-going aggregations, by evicting state for groups that are no longer expected to change.  Although, we do still maintain all state if the query requires (i.e. if the event time is not present in the `groupBy` or when running in `Complete` mode).

An example that emits windowed counts of records, waiting up to 5 minutes for late data to arrive.
```scala
df.withWatermark("eventTime", "5 minutes")
  .groupBy(window($"eventTime", "1 minute") as 'window)
  .count()
  .writeStream
  .format("console")
  .mode("append") // In append mode, we only output finalized aggregations.
  .start()
```

### Calculating the watermark.
The current event time is computed by looking at the `MAX(eventTime)` seen this epoch across all of the partitions in the query minus some user defined _delayThreshold_.  An additional constraint is that the watermark must increase monotonically.

Note that since we must coordinate this value across partitions occasionally, the actual watermark used is only guaranteed to be at least `delay` behind the actual event time.  In some cases we may still process records that arrive more than delay late.

This mechanism was chosen for the initial implementation over processing time for two reasons:
  - it is robust to downtime that could affect processing delay
  - it does not require syncing of time or timezones between the producer and the processing engine.

### Other notable implementation details
 - A new trigger metric `eventTimeWatermark` outputs the current value of the watermark.
 - We mark the event time column in the `Attribute` metadata using the key `spark.watermarkDelay`.  This allows downstream operations to know which column holds the event time.  Operations like `window` propagate this metadata.
 - `explain()` marks the watermark with a suffix of `-T${delayMs}` to ease debugging of how this information is propagated.
 - Currently, we don't filter out late records, but instead rely on the state store to avoid emitting records that are both added and filtered in the same epoch.

### Remaining in this PR
 - [ ] The test for recovery is currently failing as we don't record the watermark used in the offset log.  We will need to do so to ensure determinism, but this is deferred until #15626 is merged.

### Other follow-ups
There are some natural additional features that we should consider for future work:
 - Ability to write records that arrive too late to some external store in case any out-of-band remediation is required.
 - `Update` mode so you can get partial results before a group is evicted.
 - Other mechanisms for calculating the watermark.  In particular a watermark based on quantiles would be more robust to outliers.

Author: Michael Armbrust <michael@databricks.com>

Closes #15702 from marmbrus/watermarks.
2016-11-14 16:46:26 -08:00
Nattavut Sutyanyong bd85603ba5 [SPARK-17348][SQL] Incorrect results from subquery transformation
## What changes were proposed in this pull request?

Return an Analysis exception when there is a correlated non-equality predicate in a subquery and the correlated column from the outer reference is not from the immediate parent operator of the subquery. This PR prevents incorrect results from subquery transformation in such case.

Test cases, both positive and negative tests, are added.

## How was this patch tested?

sql/test, catalyst/test, hive/test, and scenarios that will produce incorrect results without this PR and product correct results when subquery transformation does happen.

Author: Nattavut Sutyanyong <nsy.can@gmail.com>

Closes #15763 from nsyca/spark-17348.
2016-11-14 20:59:15 +01:00
Tathagata Das bdfe60ac92 [SPARK-18416][STRUCTURED STREAMING] Fixed temp file leak in state store
## What changes were proposed in this pull request?

StateStore.get() causes temporary files to be created immediately, even if the store is not used to make updates for new version. The temp file is not closed as store.commit() is not called in those cases, thus keeping the output stream to temp file open forever.

This PR fixes it by opening the temp file only when there are updates being made.

## How was this patch tested?

New unit test

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

Closes #15859 from tdas/SPARK-18416.
2016-11-14 10:03:01 -08:00
Ryan Blue 6e95325fc3 [SPARK-18387][SQL] Add serialization to checkEvaluation.
## What changes were proposed in this pull request?

This removes the serialization test from RegexpExpressionsSuite and
replaces it by serializing all expressions in checkEvaluation.

This also fixes math constant expressions by making LeafMathExpression
Serializable and fixes NumberFormat values that are null or invalid
after serialization.

## How was this patch tested?

This patch is to tests.

Author: Ryan Blue <blue@apache.org>

Closes #15847 from rdblue/SPARK-18387-fix-serializable-expressions.
2016-11-11 13:52:10 -08:00
Dongjoon Hyun d42bb7cc4e [SPARK-17982][SQL] SQLBuilder should wrap the generated SQL with parenthesis for LIMIT
## What changes were proposed in this pull request?

Currently, `SQLBuilder` handles `LIMIT` by always adding `LIMIT` at the end of the generated subSQL. It makes `RuntimeException`s like the following. This PR adds a parenthesis always except `SubqueryAlias` is used together with `LIMIT`.

**Before**

``` scala
scala> sql("CREATE TABLE tbl(id INT)")
scala> sql("CREATE VIEW v1(id2) AS SELECT id FROM tbl LIMIT 2")
java.lang.RuntimeException: Failed to analyze the canonicalized SQL: ...
```

**After**

``` scala
scala> sql("CREATE TABLE tbl(id INT)")
scala> sql("CREATE VIEW v1(id2) AS SELECT id FROM tbl LIMIT 2")
scala> sql("SELECT id2 FROM v1")
res4: org.apache.spark.sql.DataFrame = [id2: int]
```

**Fixed cases in this PR**

The following two cases are the detail query plans having problematic SQL generations.

1. `SELECT * FROM (SELECT id FROM tbl LIMIT 2)`

    Please note that **FROM SELECT** part of the generated SQL in the below. When we don't use '()' for limit, this fails.

```scala
# Original logical plan:
Project [id#1]
+- GlobalLimit 2
   +- LocalLimit 2
      +- Project [id#1]
         +- MetastoreRelation default, tbl

# Canonicalized logical plan:
Project [gen_attr_0#1 AS id#4]
+- SubqueryAlias tbl
   +- Project [gen_attr_0#1]
      +- GlobalLimit 2
         +- LocalLimit 2
            +- Project [gen_attr_0#1]
               +- SubqueryAlias gen_subquery_0
                  +- Project [id#1 AS gen_attr_0#1]
                     +- SQLTable default, tbl, [id#1]

# Generated SQL:
SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM SELECT `gen_attr_0` FROM (SELECT `id` AS `gen_attr_0` FROM `default`.`tbl`) AS gen_subquery_0 LIMIT 2) AS tbl
```

2. `SELECT * FROM (SELECT id FROM tbl TABLESAMPLE (2 ROWS))`

    Please note that **((~~~) AS gen_subquery_0 LIMIT 2)** in the below. When we use '()' for limit on `SubqueryAlias`, this fails.

```scala
# Original logical plan:
Project [id#1]
+- Project [id#1]
   +- GlobalLimit 2
      +- LocalLimit 2
         +- MetastoreRelation default, tbl

# Canonicalized logical plan:
Project [gen_attr_0#1 AS id#4]
+- SubqueryAlias tbl
   +- Project [gen_attr_0#1]
      +- GlobalLimit 2
         +- LocalLimit 2
            +- SubqueryAlias gen_subquery_0
               +- Project [id#1 AS gen_attr_0#1]
                  +- SQLTable default, tbl, [id#1]

# Generated SQL:
SELECT `gen_attr_0` AS `id` FROM (SELECT `gen_attr_0` FROM ((SELECT `id` AS `gen_attr_0` FROM `default`.`tbl`) AS gen_subquery_0 LIMIT 2)) AS tbl
```

## How was this patch tested?

Pass the Jenkins test with a newly added test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #15546 from dongjoon-hyun/SPARK-17982.
2016-11-11 13:28:18 -08:00
Eric Liang a3356343cb [SPARK-18185] Fix all forms of INSERT / OVERWRITE TABLE for Datasource tables
## What changes were proposed in this pull request?

As of current 2.1, INSERT OVERWRITE with dynamic partitions against a Datasource table will overwrite the entire table instead of only the partitions matching the static keys, as in Hive. It also doesn't respect custom partition locations.

This PR adds support for all these operations to Datasource tables managed by the Hive metastore. It is implemented as follows
- During planning time, the full set of partitions affected by an INSERT or OVERWRITE command is read from the Hive metastore.
- The planner identifies any partitions with custom locations and includes this in the write task metadata.
- FileFormatWriter tasks refer to this custom locations map when determining where to write for dynamic partition output.
- When the write job finishes, the set of written partitions is compared against the initial set of matched partitions, and the Hive metastore is updated to reflect the newly added / removed partitions.

It was necessary to introduce a method for staging files with absolute output paths to `FileCommitProtocol`. These files are not handled by the Hadoop output committer but are moved to their final locations when the job commits.

The overwrite behavior of legacy Datasource tables is also changed: no longer will the entire table be overwritten if a partial partition spec is present.

cc cloud-fan yhuai

## How was this patch tested?

Unit tests, existing tests.

Author: Eric Liang <ekl@databricks.com>
Author: Wenchen Fan <wenchen@databricks.com>

Closes #15814 from ericl/sc-5027.
2016-11-10 17:00:43 -08:00