Commit graph

4183 commits

Author SHA1 Message Date
Marcos P. Sanchez 312bebfb6d [SPARK-21640][FOLLOW-UP][SQL] added errorifexists on IllegalArgumentException message
## What changes were proposed in this pull request?

This commit adds a new argument for IllegalArgumentException message. This recent commit added the argument:

[dcac1d57f0)

## How was this patch tested?

Unit test have been passed

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Marcos P. Sanchez <mpenate@stratio.com>

Closes #18862 from mpenate/feature/exception-errorifexists.
2017-08-07 22:41:57 -07:00
gatorsmile baf5cac0f8 [SPARK-21648][SQL] Fix confusing assert failure in JDBC source when parallel fetching parameters are not properly provided.
### What changes were proposed in this pull request?
```SQL
CREATE TABLE mytesttable1
USING org.apache.spark.sql.jdbc
  OPTIONS (
  url 'jdbc:mysql://${jdbcHostname}:${jdbcPort}/${jdbcDatabase}?user=${jdbcUsername}&password=${jdbcPassword}',
  dbtable 'mytesttable1',
  paritionColumn 'state_id',
  lowerBound '0',
  upperBound '52',
  numPartitions '53',
  fetchSize '10000'
)
```

The above option name `paritionColumn` is wrong. That mean, users did not provide the value for `partitionColumn`. In such case, users hit a confusing error.

```
AssertionError: assertion failed
java.lang.AssertionError: assertion failed
	at scala.Predef$.assert(Predef.scala:156)
	at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:39)
	at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:312)
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18864 from gatorsmile/jdbcPartCol.
2017-08-07 13:04:04 -07:00
Jose Torres cce25b360e [SPARK-21565][SS] Propagate metadata in attribute replacement.
## What changes were proposed in this pull request?

Propagate metadata in attribute replacement during streaming execution. This is necessary for EventTimeWatermarks consuming replaced attributes.

## How was this patch tested?
new unit test, which was verified to fail before the fix

Author: Jose Torres <joseph-torres@databricks.com>

Closes #18840 from joseph-torres/SPARK-21565.
2017-08-07 12:27:16 -07:00
Mac 4f7ec3a316 [SPARK][DOCS] Added note on meaning of position to substring function
## What changes were proposed in this pull request?

Enhanced some existing documentation

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Mac <maclockard@gmail.com>

Closes #18710 from maclockard/maclockard-patch-1.
2017-08-07 17:16:03 +01:00
Xiao Li bbfd6b5d24 [SPARK-21647][SQL] Fix SortMergeJoin when using CROSS
### What changes were proposed in this pull request?
author: BoleynSu
closes https://github.com/apache/spark/pull/18836

```Scala
val df = Seq((1, 1)).toDF("i", "j")
df.createOrReplaceTempView("T")
withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") {
  sql("select * from (select a.i from T a cross join T t where t.i = a.i) as t1 " +
    "cross join T t2 where t2.i = t1.i").explain(true)
}
```
The above code could cause the following exception:
```
SortMergeJoinExec should not take Cross as the JoinType
java.lang.IllegalArgumentException: SortMergeJoinExec should not take Cross as the JoinType
	at org.apache.spark.sql.execution.joins.SortMergeJoinExec.outputOrdering(SortMergeJoinExec.scala:100)
```

Our SortMergeJoinExec supports CROSS. We should not hit such an exception. This PR is to fix the issue.

### How was this patch tested?
Modified the two existing test cases.

Author: Xiao Li <gatorsmile@gmail.com>
Author: Boleyn Su <boleyn.su@gmail.com>

Closes #18863 from gatorsmile/pr-18836.
2017-08-08 00:00:01 +08:00
zhoukang 8b69b17f3f [SPARK-21544][DEPLOY][TEST-MAVEN] Tests jar of some module should not upload twice
## What changes were proposed in this pull request?

**For moudle below:**
common/network-common
streaming
sql/core
sql/catalyst
**tests.jar will install or deploy twice.Like:**
`[DEBUG] Installing org.apache.spark:spark-streaming_2.11/maven-metadata.xml to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/maven-metadata-local.xml
[INFO] Installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar
[DEBUG] Skipped re-installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar, seems unchanged`
**The reason is below:**
`[DEBUG]   (f) artifact = org.apache.spark:spark-streaming_2.11🫙2.1.0-mdh2.1.0.1-SNAPSHOT
[DEBUG]   (f) attachedArtifacts = [org.apache.spark:spark-streaming_2.11:test-jar:tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11🫙tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark
-streaming_2.11:java-source:sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:java-source:test-sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:javadoc:javadoc:2.1.0
-mdh2.1.0.1-SNAPSHOT]`

when executing 'mvn deploy' to nexus during release.I will fail since release nexus can not be overrided.

## How was this patch tested?
Execute 'mvn clean install -Pyarn -Phadoop-2.6 -Phadoop-provided -DskipTests'

Author: zhoukang <zhoukang199191@gmail.com>

Closes #18745 from caneGuy/zhoukang/fix-installtwice.
2017-08-07 12:51:39 +01:00
Takeshi Yamamuro 74b47845ea [SPARK-20963][SQL][FOLLOW-UP] Use UnresolvedSubqueryColumnAliases for visitTableName
## What changes were proposed in this pull request?
This pr (follow-up of #18772) used `UnresolvedSubqueryColumnAliases` for `visitTableName` in `AstBuilder`, which is a new unresolved `LogicalPlan` implemented in #18185.

## How was this patch tested?
Existing tests

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18857 from maropu/SPARK-20963-FOLLOWUP.
2017-08-06 10:14:45 -07:00
Yuming Wang 10b3ca3e93 [SPARK-21574][SQL] Point out user to set hive config before SparkSession is initialized
## What changes were proposed in this pull request?
Since Spark 2.0.0, SET hive config commands do not pass the values to HiveClient, this PR point out user to set hive config before SparkSession is initialized when they try to set hive config.

## How was this patch tested?
manual tests

<img width="1637" alt="spark-set" src="https://user-images.githubusercontent.com/5399861/29001141-03f943ee-7ab3-11e7-8584-ba5a5e81f6ad.png">

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18769 from wangyum/SPARK-21574.
2017-08-06 10:08:44 -07:00
vinodkc 1ba967b25e [SPARK-21588][SQL] SQLContext.getConf(key, null) should return null
## What changes were proposed in this pull request?

In SQLContext.get(key,null) for a key that is not defined in the conf, and doesn't have a default value defined, throws a NPE. Int happens only when conf has a value converter

Added null check on defaultValue inside SQLConf.getConfString to avoid calling entry.valueConverter(defaultValue)

## How was this patch tested?
Added unit test

Author: vinodkc <vinod.kc.in@gmail.com>

Closes #18852 from vinodkc/br_Fix_SPARK-21588.
2017-08-05 23:04:39 -07:00
Takeshi Yamamuro 990efad1c6 [SPARK-20963][SQL] Support column aliases for join relations in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support column aliases for join relations in FROM clause.
This pr is a sub-task of #18079.

## How was this patch tested?
Added tests in `AnalysisSuite`, `PlanParserSuite,` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18772 from maropu/SPARK-20963-2.
2017-08-05 20:35:54 -07:00
arodriguez dcac1d57f0 [SPARK-21640] Add errorifexists as a valid string for ErrorIfExists save mode
## What changes were proposed in this pull request?

This PR includes the changes to make the string "errorifexists" also valid for ErrorIfExists save mode.

## How was this patch tested?

Unit tests and manual tests

Author: arodriguez <arodriguez@arodriguez.stratio>

Closes #18844 from ardlema/SPARK-21640.
2017-08-05 11:21:51 -07:00
hyukjinkwon ba327ee54c [SPARK-21485][FOLLOWUP][SQL][DOCS] Describes examples and arguments separately, and note/since in SQL built-in function documentation
## What changes were proposed in this pull request?

This PR proposes to separate `extended` into `examples` and `arguments` internally so that both can be separately documented and add `since` and `note` for additional information.

For `since`, it looks users sometimes get confused by, up to my knowledge, missing version information. For example, see https://www.mail-archive.com/userspark.apache.org/msg64798.html

For few good examples to check the built documentation, please see both:
`from_json` - https://spark-test.github.io/sparksqldoc/#from_json
`like` - https://spark-test.github.io/sparksqldoc/#like

For `DESCRIBE FUNCTION`, `note` and `since` are added as below:

```
> DESCRIBE FUNCTION EXTENDED rlike;
...
Extended Usage:
    Arguments:
      ...

    Examples:
      ...

    Note:
      Use LIKE to match with simple string pattern
```

```
> DESCRIBE FUNCTION EXTENDED to_json;
...
    Examples:
      ...

    Since: 2.2.0
```

For the complete documentation, see https://spark-test.github.io/sparksqldoc/

## How was this patch tested?

Manual tests and existing tests. Please see https://spark-test.github.io/sparksqldoc

Jenkins tests are needed to double check

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18749 from HyukjinKwon/followup-sql-doc-gen.
2017-08-05 10:10:56 -07:00
liuxian 894d5a453a [SPARK-21580][SQL] Integers in aggregation expressions are wrongly taken as group-by ordinal
## What changes were proposed in this pull request?

create temporary view data as select * from values
(1, 1),
(1, 2),
(2, 1),
(2, 2),
(3, 1),
(3, 2)
as data(a, b);

`select 3, 4, sum(b) from data group by 1, 2;`
`select 3 as c, 4 as d, sum(b) from data group by c, d;`
When running these two cases, the following exception occurred:
`Error in query: GROUP BY position 4 is not in select list (valid range is [1, 3]); line 1 pos 10`

The cause of this failure:
If an aggregateExpression is integer, after replaced with this aggregateExpression, the
groupExpression still considered as an ordinal.

The solution:
This bug is due to re-entrance of an analyzed plan. We can solve it by using `resolveOperators` in `SubstituteUnresolvedOrdinals`.

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

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18779 from 10110346/groupby.
2017-08-04 22:55:06 -07:00
Shixiong Zhu 6cbd18c9d0 [SPARK-21374][CORE] Fix reading globbed paths from S3 into DF with disabled FS cache
## What changes were proposed in this pull request?

This PR replaces #18623 to do some clean up.

Closes #18623

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Andrey Taptunov <taptunov@amazon.com>

Closes #18848 from zsxwing/review-pr18623.
2017-08-04 22:40:04 -07:00
Reynold Xin 5ad1796b9f [SPARK-21634][SQL] Change OneRowRelation from a case object to case class
## What changes were proposed in this pull request?
OneRowRelation is the only plan that is a case object, which causes some issues with makeCopy using a 0-arg constructor. This patch changes it from a case object to a case class.

This blocks SPARK-21619.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18839 from rxin/SPARK-21634.
2017-08-04 10:36:08 -07:00
Yuming Wang 231f67247b [SPARK-21205][SQL] pmod(number, 0) should be null.
## What changes were proposed in this pull request?
Hive `pmod(3.13, 0)`:
```:sql
hive> select pmod(3.13, 0);
OK
NULL
Time taken: 2.514 seconds, Fetched: 1 row(s)
hive>
```

Spark `mod(3.13, 0)`:
```:sql
spark-sql> select mod(3.13, 0);
NULL
spark-sql>
```

But the Spark `pmod(3.13, 0)`:
```:sql
spark-sql> select pmod(3.13, 0);
17/06/25 09:35:58 ERROR SparkSQLDriver: Failed in [select pmod(3.13, 0)]
java.lang.NullPointerException
	at org.apache.spark.sql.catalyst.expressions.Pmod.pmod(arithmetic.scala:504)
	at org.apache.spark.sql.catalyst.expressions.Pmod.nullSafeEval(arithmetic.scala:432)
	at org.apache.spark.sql.catalyst.expressions.BinaryExpression.eval(Expression.scala:419)
	at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:323)
...
```
This PR make `pmod(number, 0)` to null.

## How was this patch tested?
unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18413 from wangyum/SPARK-21205.
2017-08-04 12:06:08 +02:00
Andrew Ray 25826c77dd [SPARK-21330][SQL] Bad partitioning does not allow to read a JDBC table with extreme values on the partition column
## What changes were proposed in this pull request?

An overflow of the difference of bounds on the partitioning column leads to no data being read. This
patch checks for this overflow.

## How was this patch tested?

New unit test.

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

Closes #18800 from aray/SPARK-21330.
2017-08-04 08:58:01 +01:00
Shixiong Zhu 0d26b3aa55 [SPARK-21546][SS] dropDuplicates should ignore watermark when it's not a key
## What changes were proposed in this pull request?

When the watermark is not a column of `dropDuplicates`, right now it will crash. This PR fixed this issue.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18822 from zsxwing/SPARK-21546.
2017-08-02 14:02:13 -07:00
Shixiong Zhu 7f63e85b47 [SPARK-21597][SS] Fix a potential overflow issue in EventTimeStats
## What changes were proposed in this pull request?

This PR fixed a potential overflow issue in EventTimeStats.

## How was this patch tested?

The new unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18803 from zsxwing/avg.
2017-08-02 10:59:59 -07:00
Zhan Zhang 44e501ace3 [SPARK-19839][CORE] release longArray in BytesToBytesMap
## What changes were proposed in this pull request?
When BytesToBytesMap spills, its longArray should be released. Otherwise, it may not released until the task complete. This array may take a significant amount of memory, which cannot be used by later operator, such as UnsafeShuffleExternalSorter, resulting in more frequent spill in sorter. This patch release the array as destructive iterator will not use this array anymore.

## How was this patch tested?
Manual test in production

Author: Zhan Zhang <zhanzhang@fb.com>

Closes #17180 from zhzhan/memory.
2017-07-30 18:50:19 -07:00
GuoChenzhao 51f99fb25b [SQL] Fix typo in DataframeWriter doc
## What changes were proposed in this pull request?

The format of none should be consistent with other compression codec(\`snappy\`, \`lz4\`) as \`none\`.

## How was this patch tested?

This is a typo.

Author: GuoChenzhao <chenzhao.guo@intel.com>

Closes #18758 from gczsjdy/typo.
2017-07-30 22:18:38 +09:00
Takeshi Yamamuro 6550086bbd [SPARK-20962][SQL] Support subquery column aliases in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support subquery column aliases in FROM clause.
This pr is a sub-task of #18079.

## How was this patch tested?
Added tests in `PlanParserSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18185 from maropu/SPARK-20962.
2017-07-29 10:14:47 -07:00
Xingbo Jiang 92d85637e7 [SPARK-19451][SQL] rangeBetween method should accept Long value as boundary
## What changes were proposed in this pull request?

Long values can be passed to `rangeBetween` as range frame boundaries, but we silently convert it to Int values, this can cause wrong results and we should fix this.

Further more, we should accept any legal literal values as range frame boundaries. In this PR, we make it possible for Long values, and make accepting other DataTypes really easy to add.

This PR is mostly based on Herman's previous amazing work: 596f53c339

After this been merged, we can close #16818 .

## How was this patch tested?

Add new tests in `DataFrameWindowFunctionsSuite` and `TypeCoercionSuite`.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #18540 from jiangxb1987/rangeFrame.
2017-07-29 10:11:31 -07:00
Liang-Chi Hsieh 9c8109ef41 [SPARK-21555][SQL] RuntimeReplaceable should be compared semantically by its canonicalized child
## What changes were proposed in this pull request?

When there are aliases (these aliases were added for nested fields) as parameters in `RuntimeReplaceable`, as they are not in the children expression, those aliases can't be cleaned up in analyzer rule `CleanupAliases`.

An expression `nvl(foo.foo1, "value")` can be resolved to two semantically different expressions in a group by query because they contain different aliases.

Because those aliases are not children of `RuntimeReplaceable` which is an `UnaryExpression`. So we can't trim the aliases out by simple transforming the expressions in `CleanupAliases`.

If we want to replace the non-children aliases in `RuntimeReplaceable`, we need to add more codes to `RuntimeReplaceable` and modify all expressions of `RuntimeReplaceable`. It makes the interface ugly IMO.

Consider those aliases will be replaced later at optimization and so they're no harm, this patch chooses to simply override `canonicalized` of `RuntimeReplaceable`.

One concern is about `CleanupAliases`. Because it actually cannot clean up ALL aliases inside a plan. To make caller of this rule notice that, this patch adds a comment to `CleanupAliases`.

## How was this patch tested?

Added test.

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

Closes #18761 from viirya/SPARK-21555.
2017-07-29 10:02:56 -07:00
Sean Owen 63d168cbb8 [MINOR][BUILD] Fix current lint-java failures
## What changes were proposed in this pull request?

Fixes current failures in dev/lint-java

## How was this patch tested?

Existing linter, tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #18757 from srowen/LintJava.
2017-07-28 11:31:40 +01:00
aokolnychyi f44ead89f4 [SPARK-21538][SQL] Attribute resolution inconsistency in the Dataset API
## What changes were proposed in this pull request?

This PR contains a tiny update that removes an attribute resolution inconsistency in the Dataset API. The following example is taken from the ticket description:

```
spark.range(1).withColumnRenamed("id", "x").sort(col("id"))  // works
spark.range(1).withColumnRenamed("id", "x").sort($"id")  // works
spark.range(1).withColumnRenamed("id", "x").sort('id) // works
spark.range(1).withColumnRenamed("id", "x").sort("id") // fails with:
org.apache.spark.sql.AnalysisException: Cannot resolve column name "id" among (x);
```
The above `AnalysisException` happens because the last case calls `Dataset.apply()` to convert strings into columns, which triggers attribute resolution. To make the API consistent between overloaded methods, this PR defers the resolution and constructs columns directly.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #18740 from aokolnychyi/spark-21538.
2017-07-27 16:49:42 -07:00
Wenchen Fan 9f5647d62e [SPARK-21319][SQL] Fix memory leak in sorter
## What changes were proposed in this pull request?

`UnsafeExternalSorter.recordComparator` can be either `KVComparator` or `RowComparator`, and both of them will keep the reference to the input rows they compared last time.

After sorting, we return the sorted iterator to upstream operators. However, the upstream operators may take a while to consume up the sorted iterator, and `UnsafeExternalSorter` is registered to `TaskContext` at [here](https://github.com/apache/spark/blob/v2.2.0/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java#L159-L161), which means we will keep the `UnsafeExternalSorter` instance and keep the last compared input rows in memory until the sorted iterator is consumed up.

Things get worse if we sort within partitions of a dataset and coalesce all partitions into one, as we will keep a lot of input rows in memory and the time to consume up all the sorted iterators is long.

This PR takes over https://github.com/apache/spark/pull/18543 , the idea is that, we do not keep the record comparator instance in `UnsafeExternalSorter`, but a generator of record comparator.

close #18543

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18679 from cloud-fan/memory-leak.
2017-07-27 22:56:26 +08:00
Takuya UESHIN 2ff35a057e [SPARK-21440][SQL][PYSPARK] Refactor ArrowConverters and add ArrayType and StructType support.
## What changes were proposed in this pull request?

This is a refactoring of `ArrowConverters` and related classes.

1. Refactor `ColumnWriter` as `ArrowWriter`.
2. Add `ArrayType` and `StructType` support.
3. Refactor `ArrowConverters` to skip intermediate `ArrowRecordBatch` creation.

## How was this patch tested?

Added some tests and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18655 from ueshin/issues/SPARK-21440.
2017-07-27 19:19:51 +08:00
Kazuaki Ishizaki ebbe589d12 [SPARK-21271][SQL] Ensure Unsafe.sizeInBytes is a multiple of 8
## What changes were proposed in this pull request?

This PR ensures that `Unsafe.sizeInBytes` must be a multiple of 8. It it is not satisfied. `Unsafe.hashCode` causes the assertion violation.

## How was this patch tested?

Will add test cases

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

Closes #18503 from kiszk/SPARK-21271.
2017-07-27 15:27:24 +08:00
hyukjinkwon 60472dbfd9 [SPARK-21485][SQL][DOCS] Spark SQL documentation generation for built-in functions
## What changes were proposed in this pull request?

This generates a documentation for Spark SQL built-in functions.

One drawback is, this requires a proper build to generate built-in function list.
Once it is built, it only takes few seconds by `sql/create-docs.sh`.

Please see https://spark-test.github.io/sparksqldoc/ that I hosted to show the output documentation.

There are few more works to be done in order to make the documentation pretty, for example, separating `Arguments:` and `Examples:` but I guess this should be done within `ExpressionDescription` and `ExpressionInfo` rather than manually parsing it. I will fix these in a follow up.

This requires `pip install mkdocs` to generate HTMLs from markdown files.

## How was this patch tested?

Manually tested:

```
cd docs
jekyll build
```
,

```
cd docs
jekyll serve
```

and

```
cd sql
create-docs.sh
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18702 from HyukjinKwon/SPARK-21485.
2017-07-26 09:38:51 -07:00
gatorsmile ebc24a9b7f [SPARK-20586][SQL] Add deterministic to ScalaUDF
### What changes were proposed in this pull request?
Like [Hive UDFType](https://hive.apache.org/javadocs/r2.0.1/api/org/apache/hadoop/hive/ql/udf/UDFType.html), we should allow users to add the extra flags for ScalaUDF and JavaUDF too. _stateful_/_impliesOrder_ are not applicable to our Scala UDF. Thus, we only add the following two flags.

- deterministic: Certain optimizations should not be applied if UDF is not deterministic. Deterministic UDF returns same result each time it is invoked with a particular input. This determinism just needs to hold within the context of a query.

When the deterministic flag is not correctly set, the results could be wrong.

For ScalaUDF in Dataset APIs, users can call the following extra APIs for `UserDefinedFunction` to make the corresponding changes.
- `nonDeterministic`: Updates UserDefinedFunction to non-deterministic.

Also fixed the Java UDF name loss issue.

Will submit a separate PR for `distinctLike`  for UDAF

### How was this patch tested?
Added test cases for both ScalaUDF

Author: gatorsmile <gatorsmile@gmail.com>
Author: Wenchen Fan <cloud0fan@gmail.com>

Closes #17848 from gatorsmile/udfRegister.
2017-07-25 17:19:44 -07:00
Kazuaki Ishizaki 7f295059ca [SPARK-21516][SQL][TEST] Overriding afterEach() in DatasetCacheSuite must call super.afterEach()
## What changes were proposed in this pull request?

This PR ensures to call `super.afterEach()` in overriding `afterEach()` method in `DatasetCacheSuite`. When we override `afterEach()` method in Testsuite, we have to call `super.afterEach()`.

This is a follow-up of #18719 and SPARK-21512.

## How was this patch tested?

Used the existing test suite

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

Closes #18721 from kiszk/SPARK-21516.
2017-07-25 10:51:00 +08:00
Wenchen Fan 86664338f2 [SPARK-17528][SQL][FOLLOWUP] remove unnecessary data copy in object hash aggregate
## What changes were proposed in this pull request?

In #18483 , we fixed the data copy bug when saving into `InternalRow`, and removed all workarounds for this bug in the aggregate code path. However, the object hash aggregate was missed, this PR fixes it.

This patch is also a requirement for #17419 , which shows that DataFrame version is slower than RDD version because of this issue.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18712 from cloud-fan/minor.
2017-07-24 10:18:28 -07:00
Kazuaki Ishizaki 481f079294 [SPARK-21512][SQL][TEST] DatasetCacheSuite needs to execute unpersistent after executing peristent
## What changes were proposed in this pull request?

This PR avoids to reuse unpersistent dataset among test cases by making dataset unpersistent at the end of each test case.

In `DatasetCacheSuite`, the test case `"get storage level"` does not make dataset unpersisit after make the dataset persisitent. The same dataset will be made persistent by the test case `"persist and then rebind right encoder when join 2 datasets"` Thus, we run these test cases, the second case does not perform to make dataset persistent. This is because in

When we run only the second case, it performs to make dataset persistent. It is not good to change behavior of the second test suite. The first test case should correctly make dataset unpersistent.

```
Testing started at 17:52 ...
01:52:15.053 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
01:52:48.595 WARN org.apache.spark.sql.execution.CacheManager: Asked to cache already cached data.
01:52:48.692 WARN org.apache.spark.sql.execution.CacheManager: Asked to cache already cached data.
01:52:50.864 WARN org.apache.spark.storage.RandomBlockReplicationPolicy: Expecting 1 replicas with only 0 peer/s.
01:52:50.864 WARN org.apache.spark.storage.RandomBlockReplicationPolicy: Expecting 1 replicas with only 0 peer/s.
01:52:50.868 WARN org.apache.spark.storage.BlockManager: Block rdd_8_1 replicated to only 0 peer(s) instead of 1 peers
01:52:50.868 WARN org.apache.spark.storage.BlockManager: Block rdd_8_0 replicated to only 0 peer(s) instead of 1 peers
```

After this PR, these messages do not appear
```
Testing started at 18:14 ...
02:15:05.329 WARN org.apache.hadoop.util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable

Process finished with exit code 0
```

## How was this patch tested?

Used the existing test

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

Closes #18719 from kiszk/SPARK-21512.
2017-07-23 11:31:27 -07:00
Reynold Xin a4eac8b0bb [MINOR] Remove **** in test case names in FlatMapGroupsWithStateSuite
## What changes were proposed in this pull request?
This patch removes the `****` string from test names in FlatMapGroupsWithStateSuite. `***` is a common string developers grep for when using Scala test (because it immediately shows the failing test cases). The existence of the `****` in test names disrupts that workflow.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18715 from rxin/FlatMapGroupsWithStateStar.
2017-07-23 10:41:38 -07:00
Takuya UESHIN 2f1468429f [SPARK-21472][SQL][FOLLOW-UP] Introduce ArrowColumnVector as a reader for Arrow vectors.
## What changes were proposed in this pull request?

This is a follow-up of #18680.

In some environment, a compile error happens saying:

```
.../sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ArrowColumnVector.java:243:
error: not found: type Array
  public void loadBytes(Array array) {
                        ^
```

This pr fixes it.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18701 from ueshin/issues/SPARK-21472_fup1.
2017-07-21 21:06:56 +08:00
Wenchen Fan 3ac6093086 [SPARK-10063] Follow-up: remove dead code related to an old output committer
## What changes were proposed in this pull request?

DirectParquetOutputCommitter was removed from Spark as it was deemed unsafe to use. We however still have some code to generate warning. This patch removes those code as well.

This is kind of a follow-up of https://github.com/apache/spark/pull/16796

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18689 from cloud-fan/minor.
2017-07-20 12:08:20 -07:00
Takuya UESHIN cb19880cd8 [SPARK-21472][SQL] Introduce ArrowColumnVector as a reader for Arrow vectors.
## What changes were proposed in this pull request?

Introducing `ArrowColumnVector` as a reader for Arrow vectors.
It extends `ColumnVector`, so we will be able to use it with `ColumnarBatch` and its functionalities.
Currently it supports primitive types and `StringType`, `ArrayType` and `StructType`.

## How was this patch tested?

Added tests for `ArrowColumnVector` and existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18680 from ueshin/issues/SPARK-21472.
2017-07-20 21:00:30 +08:00
gatorsmile 256358f66a [SPARK-21477][SQL][MINOR] Mark LocalTableScanExec's input data transient
## What changes were proposed in this pull request?
This PR is to mark the parameter `rows` and `unsafeRow` of LocalTableScanExec transient. It can avoid serializing the unneeded objects.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18686 from gatorsmile/LocalTableScanExec.
2017-07-20 19:16:26 +08:00
Xiang Gao b7a40f64e6 [SPARK-16542][SQL][PYSPARK] Fix bugs about types that result an array of null when creating DataFrame using python
## What changes were proposed in this pull request?
This is the reopen of https://github.com/apache/spark/pull/14198, with merge conflicts resolved.

ueshin Could you please take a look at my code?

Fix bugs about types that result an array of null when creating DataFrame using python.

Python's array.array have richer type than python itself, e.g. we can have `array('f',[1,2,3])` and `array('d',[1,2,3])`. Codes in spark-sql and pyspark didn't take this into consideration which might cause a problem that you get an array of null values when you have `array('f')` in your rows.

A simple code to reproduce this bug is:

```
from pyspark import SparkContext
from pyspark.sql import SQLContext,Row,DataFrame
from array import array

sc = SparkContext()
sqlContext = SQLContext(sc)

row1 = Row(floatarray=array('f',[1,2,3]), doublearray=array('d',[1,2,3]))
rows = sc.parallelize([ row1 ])
df = sqlContext.createDataFrame(rows)
df.show()
```

which have output

```
+---------------+------------------+
|    doublearray|        floatarray|
+---------------+------------------+
|[1.0, 2.0, 3.0]|[null, null, null]|
+---------------+------------------+
```

## How was this patch tested?

New test case added

Author: Xiang Gao <qasdfgtyuiop@gmail.com>
Author: Gao, Xiang <qasdfgtyuiop@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18444 from zasdfgbnm/fix_array_infer.
2017-07-20 12:46:06 +09:00
Burak Yavuz 2c9d5ef1f0 [SPARK-21463] Allow userSpecifiedSchema to override partition inference performed by MetadataLogFileIndex
## What changes were proposed in this pull request?

When using the MetadataLogFileIndex to read back a table, we don't respect the user provided schema as the proper column types. This can lead to issues when trying to read strings that look like dates that get truncated to DateType, or longs being truncated to IntegerType, just because a long value doesn't exist.

## How was this patch tested?

Unit tests and manual tests

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #18676 from brkyvz/stream-partitioning.
2017-07-19 15:56:26 -07:00
Corey Woodfield 8cd9cdf17a [SPARK-21333][DOCS] Removed invalid joinTypes from javadoc of Dataset#joinWith
## What changes were proposed in this pull request?

Two invalid join types were mistakenly listed in the javadoc for joinWith, in the Dataset class. I presume these were copied from the javadoc of join, but since joinWith returns a Dataset\<Tuple2\>, left_semi and left_anti are invalid, as they only return values from one of the datasets, instead of from both

## How was this patch tested?

I ran the following code :
```
public static void main(String[] args) {
	SparkSession spark = new SparkSession(new SparkContext("local[*]", "Test"));
	Dataset<Row> one = spark.createDataFrame(Arrays.asList(new Bean(1), new Bean(2), new Bean(3), new Bean(4), new Bean(5)), Bean.class);
	Dataset<Row> two = spark.createDataFrame(Arrays.asList(new Bean(4), new Bean(5), new Bean(6), new Bean(7), new Bean(8), new Bean(9)), Bean.class);

	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "inner").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "cross").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "outer").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "full").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "full_outer").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "left").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "left_outer").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "right").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "right_outer").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "left_semi").show();} catch (Exception e) {e.printStackTrace();}
	try {two.joinWith(one, one.col("x").equalTo(two.col("x")), "left_anti").show();} catch (Exception e) {e.printStackTrace();}
}
```
which tests all the different join types, and the last two (left_semi and left_anti) threw exceptions. The same code using join instead of joinWith did fine. The Bean class was just a java bean with a single int field, x.

Author: Corey Woodfield <coreywoodfield@gmail.com>

Closes #18462 from coreywoodfield/master.
2017-07-19 15:21:38 -07:00
DFFuture c9729187bc [SPARK-21446][SQL] Fix setAutoCommit never executed
## What changes were proposed in this pull request?
JIRA Issue: https://issues.apache.org/jira/browse/SPARK-21446
options.asConnectionProperties can not have fetchsize,because fetchsize belongs to Spark-only options, and Spark-only options have been excluded in connection properities.
So change properties of beforeFetch from  options.asConnectionProperties.asScala.toMap to options.asProperties.asScala.toMap

## How was this patch tested?

Author: DFFuture <albert.zhang23@gmail.com>

Closes #18665 from DFFuture/sparksql_pg.
2017-07-19 14:45:11 -07:00
Tathagata Das 70fe99dc62 [SPARK-21464][SS] Minimize deprecation warnings caused by ProcessingTime class
## What changes were proposed in this pull request?

Use of `ProcessingTime` class was deprecated in favor of `Trigger.ProcessingTime` in Spark 2.2. However interval uses to ProcessingTime causes deprecation warnings during compilation. This cannot be avoided entirely as even though it is deprecated as a public API, ProcessingTime instances are used internally in TriggerExecutor. This PR is to minimize the warning by removing its uses from tests as much as possible.

## How was this patch tested?
Existing tests.

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

Closes #18678 from tdas/SPARK-21464.
2017-07-19 11:02:07 -07:00
donnyzone 6b6dd682e8 [SPARK-21441][SQL] Incorrect Codegen in SortMergeJoinExec results failures in some cases
## What changes were proposed in this pull request?

https://issues.apache.org/jira/projects/SPARK/issues/SPARK-21441

This issue can be reproduced by the following example:

```
val spark = SparkSession
   .builder()
   .appName("smj-codegen")
   .master("local")
   .config("spark.sql.autoBroadcastJoinThreshold", "1")
   .getOrCreate()
val df1 = spark.createDataFrame(Seq((1, 1), (2, 2), (3, 3))).toDF("key", "int")
val df2 = spark.createDataFrame(Seq((1, "1"), (2, "2"), (3, "3"))).toDF("key", "str")
val df = df1.join(df2, df1("key") === df2("key"))
   .filter("int = 2 or reflect('java.lang.Integer', 'valueOf', str) = 1")
   .select("int")
   df.show()
```

To conclude, the issue happens when:
(1) SortMergeJoin condition contains CodegenFallback expressions.
(2) In PhysicalPlan tree, SortMergeJoin node  is the child of root node, e.g., the Project in above example.

This patch fixes the logic in `CollapseCodegenStages` rule.

## How was this patch tested?
Unit test and manual verification in our cluster.

Author: donnyzone <wellfengzhu@gmail.com>

Closes #18656 from DonnyZone/Fix_SortMergeJoinExec.
2017-07-19 21:48:54 +08:00
jinxing 4eb081cc87 [SPARK-21414] Refine SlidingWindowFunctionFrame to avoid OOM.
## What changes were proposed in this pull request?

In `SlidingWindowFunctionFrame`, it is now adding all rows to the buffer for which the input row value is equal to or less than the output row upper bound, then drop all rows from the buffer for which the input row value is smaller than the output row lower bound.
This could result in the buffer is very big though the window is small.
For example:
```
select a, b, sum(a)
over (partition by b order by a range between 1000000 following and 1000001 following)
from table
```
We can refine the logic and just add the qualified rows into buffer.

## How was this patch tested?
Manual test:
Run sql
`select shop, shopInfo, district, sum(revenue) over(partition by district order by revenue range between 100 following and 200 following) from revenueList limit 10`
against a table with 4  columns(shop: String, shopInfo: String, district: String, revenue: Int). The biggest partition is around 2G bytes, containing 200k lines.
Configure the executor with 2G bytes memory.
With the change in this pr, it works find. Without this change, below exception will be thrown.
```
MemoryError: Java heap space
	at org.apache.spark.sql.catalyst.expressions.UnsafeRow.copy(UnsafeRow.java:504)
	at org.apache.spark.sql.catalyst.expressions.UnsafeRow.copy(UnsafeRow.java:62)
	at org.apache.spark.sql.execution.window.SlidingWindowFunctionFrame.write(WindowFunctionFrame.scala:201)
	at org.apache.spark.sql.execution.window.WindowExec$$anonfun$14$$anon$1.next(WindowExec.scala:365)
	at org.apache.spark.sql.execution.window.WindowExec$$anonfun$14$$anon$1.next(WindowExec.scala:289)
	at org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIterator.processNext(Unknown Source)
	at org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)
	at org.apache.spark.sql.execution.WholeStageCodegenExec$$anonfun$8$$anon$1.hasNext(WholeStageCodegenExec.scala:395)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:231)
	at org.apache.spark.sql.execution.SparkPlan$$anonfun$2.apply(SparkPlan.scala:225)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
	at org.apache.spark.rdd.RDD$$anonfun$mapPartitionsInternal$1$$anonfun$apply$25.apply(RDD.scala:827)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:108)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:341)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
```

Author: jinxing <jinxing6042@126.com>

Closes #18634 from jinxing64/SPARK-21414.
2017-07-19 21:35:26 +08:00
xuanyuanking 81c99a5b95 [SPARK-21435][SQL] Empty files should be skipped while write to file
## What changes were proposed in this pull request?

Add EmptyDirectoryWriteTask for empty task while writing files. Fix the empty result for parquet format by leaving the first partition for meta writing.

## How was this patch tested?

Add new test in `FileFormatWriterSuite `

Author: xuanyuanking <xyliyuanjian@gmail.com>

Closes #18654 from xuanyuanking/SPARK-21435.
2017-07-19 10:27:42 +08:00
Tathagata Das 84f1b25f31 [SPARK-21462][SS] Added batchId to StreamingQueryProgress.json
## What changes were proposed in this pull request?

- Added batchId to StreamingQueryProgress.json as that was missing from the generated json.
- Also, removed recently added numPartitions from StatefulOperatorProgress as this value does not change through the query run, and there are other ways to find that.

## How was this patch tested?
Updated unit tests

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

Closes #18675 from tdas/SPARK-21462.
2017-07-18 16:29:45 -07:00
Sean Owen e26dac5feb [SPARK-21415] Triage scapegoat warnings, part 1
## What changes were proposed in this pull request?

Address scapegoat warnings for:
- BigDecimal double constructor
- Catching NPE
- Finalizer without super
- List.size is O(n)
- Prefer Seq.empty
- Prefer Set.empty
- reverse.map instead of reverseMap
- Type shadowing
- Unnecessary if condition.
- Use .log1p
- Var could be val

In some instances like Seq.empty, I avoided making the change even where valid in test code to keep the scope of the change smaller. Those issues are concerned with performance and it won't matter for tests.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #18635 from srowen/Scapegoat1.
2017-07-18 08:47:17 +01:00
Tathagata Das e9faae135c [SPARK-21409][SS] Follow up PR to allow different types of custom metrics to be exposed
## What changes were proposed in this pull request?

Implementation may expose both timing as well as size metrics. This PR enables that.

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

Closes #18661 from tdas/SPARK-21409-2.
2017-07-17 19:28:55 -07:00
Tathagata Das 9d8c83179a [SPARK-21409][SS] Expose state store memory usage in SQL metrics and progress updates
## What changes were proposed in this pull request?

Currently, there is no tracking of memory usage of state stores. This JIRA is to expose that through SQL metrics and StreamingQueryProgress.

Additionally, added the ability to expose implementation-specific metrics through the StateStore APIs to the SQLMetrics.

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

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

Closes #18629 from tdas/SPARK-21409.
2017-07-17 16:48:15 -07:00
gatorsmile e398c28146 [SPARK-21354][SQL] INPUT FILE related functions do not support more than one sources
### What changes were proposed in this pull request?
The build-in functions `input_file_name`, `input_file_block_start`, `input_file_block_length` do not support more than one sources, like what Hive does. Currently, Spark does not block it and the outputs are ambiguous/non-deterministic. It could be from any side.

```
hive> select *, INPUT__FILE__NAME FROM t1, t2;
FAILED: SemanticException Column INPUT__FILE__NAME Found in more than One Tables/Subqueries
```

This PR blocks it and issues an error.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18580 from gatorsmile/inputFileName.
2017-07-17 14:58:14 +08:00
Sean Owen fd52a747fd [SPARK-19810][SPARK-19810][MINOR][FOLLOW-UP] Follow-ups from to remove Scala 2.10
## What changes were proposed in this pull request?

Follow up to a few comments on https://github.com/apache/spark/pull/17150#issuecomment-315020196 that couldn't be addressed before it was merged.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #18646 from srowen/SPARK-19810.2.
2017-07-17 09:22:42 +08:00
Kazuaki Ishizaki ac5d5d7959 [SPARK-21344][SQL] BinaryType comparison does signed byte array comparison
## What changes were proposed in this pull request?

This PR fixes a wrong comparison for `BinaryType`. This PR enables unsigned comparison and unsigned prefix generation for an array for `BinaryType`. Previous implementations uses signed operations.

## How was this patch tested?

Added a test suite in `OrderingSuite`.

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

Closes #18571 from kiszk/SPARK-21344.
2017-07-14 20:16:04 -07:00
Shixiong Zhu 2d968a07d2 [SPARK-21421][SS] Add the query id as a local property to allow source and sink using it
## What changes were proposed in this pull request?

Add the query id as a local property to allow source and sink using it.

## How was this patch tested?

The new unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18638 from zsxwing/SPARK-21421.
2017-07-14 14:37:27 -07:00
Sean Owen 425c4ada4c [SPARK-19810][BUILD][CORE] Remove support for Scala 2.10
## What changes were proposed in this pull request?

- Remove Scala 2.10 build profiles and support
- Replace some 2.10 support in scripts with commented placeholders for 2.12 later
- Remove deprecated API calls from 2.10 support
- Remove usages of deprecated context bounds where possible
- Remove Scala 2.10 workarounds like ScalaReflectionLock
- Other minor Scala warning fixes

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #17150 from srowen/SPARK-19810.
2017-07-13 17:06:24 +08:00
Wenchen Fan 780586a9f2 [SPARK-17701][SQL] Refactor RowDataSourceScanExec so its sameResult call does not compare strings
## What changes were proposed in this pull request?

Currently, `RowDataSourceScanExec` and `FileSourceScanExec` rely on a "metadata" string map to implement equality comparison, since the RDDs they depend on cannot be directly compared. This has resulted in a number of correctness bugs around exchange reuse, e.g. SPARK-17673 and SPARK-16818.

To make these comparisons less brittle, we should refactor these classes to compare constructor parameters directly instead of relying on the metadata map.

This PR refactors `RowDataSourceScanExec`, `FileSourceScanExec` will be fixed in the follow-up PR.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18600 from cloud-fan/minor.
2017-07-12 09:23:54 -07:00
liuxian aaad34dc2f [SPARK-21007][SQL] Add SQL function - RIGHT && LEFT
## What changes were proposed in this pull request?
 Add  SQL function - RIGHT && LEFT, same as MySQL:
https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_left
https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_right

## How was this patch tested?
unit test

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18228 from 10110346/lx-wip-0607.
2017-07-12 18:51:19 +08:00
Burak Yavuz e0af76a36a [SPARK-21370][SS] Add test for state reliability when one read-only state store aborts after read-write state store commits
## What changes were proposed in this pull request?

During Streaming Aggregation, we have two StateStores per task, one used as read-only in
`StateStoreRestoreExec`, and one read-write used in `StateStoreSaveExec`. `StateStore.abort`
will be called for these StateStores if they haven't committed their results. We need to
make sure that `abort` in read-only store after a `commit` in the read-write store doesn't
accidentally lead to the deletion of state.

This PR adds a test for this condition.

## How was this patch tested?

This PR adds a test.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #18603 from brkyvz/ss-test.
2017-07-12 00:39:09 -07:00
Jane Wang 2cbfc975ba [SPARK-12139][SQL] REGEX Column Specification
## What changes were proposed in this pull request?
Hive interprets regular expression, e.g., `(a)?+.+` in query specification. This PR enables spark to support this feature when hive.support.quoted.identifiers is set to true.

## How was this patch tested?

- Add unittests in SQLQuerySuite.scala
- Run spark-shell tested the original failed query:
scala> hc.sql("SELECT `(a|b)?+.+` from test1").collect.foreach(println)

Author: Jane Wang <janewang@fb.com>

Closes #18023 from janewangfb/support_select_regex.
2017-07-11 22:00:36 -07:00
gatorsmile d3e071658f [SPARK-19285][SQL] Implement UDF0
### What changes were proposed in this pull request?
This PR is to implement UDF0. `UDF0` is needed when users need to implement a JAVA UDF with no argument.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18598 from gatorsmile/udf0.
2017-07-11 15:44:29 -07:00
hyukjinkwon ebc124d4c4 [SPARK-21365][PYTHON] Deduplicate logics parsing DDL type/schema definition
## What changes were proposed in this pull request?

This PR deals with four points as below:

- Reuse existing DDL parser APIs rather than reimplementing within PySpark

- Support DDL formatted string, `field type, field type`.

- Support case-insensitivity for parsing.

- Support nested data types as below:

  **Before**
  ```
  >>> spark.createDataFrame([[[1]]], "struct<a: struct<b: int>>").show()
  ...
  ValueError: The strcut field string format is: 'field_name:field_type', but got: a: struct<b: int>
  ```

  ```
  >>> spark.createDataFrame([[[1]]], "a: struct<b: int>").show()
  ...
  ValueError: The strcut field string format is: 'field_name:field_type', but got: a: struct<b: int>
  ```

  ```
  >>> spark.createDataFrame([[1]], "a int").show()
  ...
  ValueError: Could not parse datatype: a int
  ```

  **After**
  ```
  >>> spark.createDataFrame([[[1]]], "struct<a: struct<b: int>>").show()
  +---+
  |  a|
  +---+
  |[1]|
  +---+
  ```

  ```
  >>> spark.createDataFrame([[[1]]], "a: struct<b: int>").show()
  +---+
  |  a|
  +---+
  |[1]|
  +---+
  ```

  ```
  >>> spark.createDataFrame([[1]], "a int").show()
  +---+
  |  a|
  +---+
  |  1|
  +---+
  ```

## How was this patch tested?

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18590 from HyukjinKwon/deduplicate-python-ddl.
2017-07-11 22:03:10 +08:00
Xingbo Jiang 66d2168655 [SPARK-21366][SQL][TEST] Add sql test for window functions
## What changes were proposed in this pull request?

Add sql test for window functions, also remove uncecessary test cases in `WindowQuerySuite`.

## How was this patch tested?

Added `window.sql` and the corresponding output file.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #18591 from jiangxb1987/window.
2017-07-11 21:52:54 +08:00
hyukjinkwon 7514db1dec [SPARK-21263][SQL] Do not allow partially parsing double and floats via NumberFormat in CSV
## What changes were proposed in this pull request?

This PR proposes to remove `NumberFormat.parse` use to disallow a case of partially parsed data. For example,

```
scala> spark.read.schema("a DOUBLE").option("mode", "FAILFAST").csv(Seq("10u12").toDS).show()
+----+
|   a|
+----+
|10.0|
+----+
```

## How was this patch tested?

Unit tests added in `UnivocityParserSuite` and `CSVSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18532 from HyukjinKwon/SPARK-21263.
2017-07-11 11:11:08 +01:00
jinxing 97a1aa2c70 [SPARK-21315][SQL] Skip some spill files when generateIterator(startIndex) in ExternalAppendOnlyUnsafeRowArray.
## What changes were proposed in this pull request?

In current code, it is expensive to use `UnboundedFollowingWindowFunctionFrame`, because it is iterating from the start to lower bound every time calling `write` method. When traverse the iterator, it's possible to skip some spilled files thus to save some time.

## How was this patch tested?

Added unit test

Did a small test for benchmark:

Put 2000200 rows into `UnsafeExternalSorter`-- 2 spill files(each contains 1000000 rows) and inMemSorter contains 200 rows.
Move the iterator forward to index=2000001.

*With this change*:
`getIterator(2000001)`, it will cost almost 0ms~1ms;
*Without this change*:
`for(int i=0; i<2000001; i++)geIterator().loadNext()`, it will cost 300ms.

Author: jinxing <jinxing6042@126.com>

Closes #18541 from jinxing64/SPARK-21315.
2017-07-11 11:47:47 +08:00
gatorsmile 1471ee7af5 [SPARK-21350][SQL] Fix the error message when the number of arguments is wrong when invoking a UDF
### What changes were proposed in this pull request?
Users get a very confusing error when users specify a wrong number of parameters.
```Scala
    val df = spark.emptyDataFrame
    spark.udf.register("foo", (_: String).length)
    df.selectExpr("foo(2, 3, 4)")
```
```
org.apache.spark.sql.UDFSuite$$anonfun$9$$anonfun$apply$mcV$sp$12 cannot be cast to scala.Function3
java.lang.ClassCastException: org.apache.spark.sql.UDFSuite$$anonfun$9$$anonfun$apply$mcV$sp$12 cannot be cast to scala.Function3
	at org.apache.spark.sql.catalyst.expressions.ScalaUDF.<init>(ScalaUDF.scala:109)
```

This PR is to capture the exception and issue an error message that is consistent with what we did for built-in functions. After the fix, the error message is improved to
```
Invalid number of arguments for function foo; line 1 pos 0
org.apache.spark.sql.AnalysisException: Invalid number of arguments for function foo; line 1 pos 0
	at org.apache.spark.sql.catalyst.analysis.SimpleFunctionRegistry.lookupFunction(FunctionRegistry.scala:119)
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18574 from gatorsmile/statsCheck.
2017-07-11 11:19:59 +08:00
Takeshi Yamamuro a2bec6c92a [SPARK-21043][SQL] Add unionByName in Dataset
## What changes were proposed in this pull request?
This pr added `unionByName` in `DataSet`.
Here is how to use:
```
val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2")
val df2 = Seq((4, 5, 6)).toDF("col1", "col2", "col0")
df1.unionByName(df2).show

// output:
// +----+----+----+
// |col0|col1|col2|
// +----+----+----+
// |   1|   2|   3|
// |   6|   4|   5|
// +----+----+----+
```

## How was this patch tested?
Added tests in `DataFrameSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18300 from maropu/SPARK-21043-2.
2017-07-10 20:16:29 -07:00
Bryan Cutler d03aebbe65 [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  Data types except complex, date, timestamp, and decimal  are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayload` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a private method `DataFrame._collectAsArrow` is added to collect Arrow payloads and a SQLConf "spark.sql.execution.arrow.enable" can be used in `toPandas()` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #18459 from BryanCutler/toPandas_with_arrow-SPARK-13534.
2017-07-10 15:21:03 -07:00
hyukjinkwon 2bfd5accdc [SPARK-21266][R][PYTHON] Support schema a DDL-formatted string in dapply/gapply/from_json
## What changes were proposed in this pull request?

This PR supports schema in a DDL formatted string for `from_json` in R/Python and `dapply` and `gapply` in R, which are commonly used and/or consistent with Scala APIs.

Additionally, this PR exposes `structType` in R to allow working around in other possible corner cases.

**Python**

`from_json`

```python
from pyspark.sql.functions import from_json

data = [(1, '''{"a": 1}''')]
df = spark.createDataFrame(data, ("key", "value"))
df.select(from_json(df.value, "a INT").alias("json")).show()
```

**R**

`from_json`

```R
df <- sql("SELECT named_struct('name', 'Bob') as people")
df <- mutate(df, people_json = to_json(df$people))
head(select(df, from_json(df$people_json, "name STRING")))
```

`structType.character`

```R
structType("a STRING, b INT")
```

`dapply`

```R
dapply(createDataFrame(list(list(1.0)), "a"), function(x) {x}, "a DOUBLE")
```

`gapply`

```R
gapply(createDataFrame(list(list(1.0)), "a"), "a", function(key, x) { x }, "a DOUBLE")
```

## How was this patch tested?

Doc tests for `from_json` in Python and unit tests `test_sparkSQL.R` in R.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18498 from HyukjinKwon/SPARK-21266.
2017-07-10 10:40:03 -07:00
Juliusz Sompolski 18b3b00ecf [SPARK-21272] SortMergeJoin LeftAnti does not update numOutputRows
## What changes were proposed in this pull request?

Updating numOutputRows metric was missing from one return path of LeftAnti SortMergeJoin.

## How was this patch tested?

Non-zero output rows manually seen in metrics.

Author: Juliusz Sompolski <julek@databricks.com>

Closes #18494 from juliuszsompolski/SPARK-21272.
2017-07-10 09:26:42 -07:00
Takeshi Yamamuro 647963a26a [SPARK-20460][SQL] Make it more consistent to handle column name duplication
## What changes were proposed in this pull request?
This pr made it more consistent to handle column name duplication. In the current master, error handling is different when hitting column name duplication:
```
// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#12, a#13.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Duplicate column(s) : "a" found, cannot save to JSON format;
  at org.apache.spark.sql.execution.datasources.json.JsonDataSource.checkConstraints(JsonDataSource.scala:81)
  at org.apache.spark.sql.execution.datasources.json.JsonDataSource.inferSchema(JsonDataSource.scala:63)
  at org.apache.spark.sql.execution.datasources.json.JsonFileFormat.inferSchema(JsonFileFormat.scala:57)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)
  at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$7.apply(DataSource.scala:176)

// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#41, a#42.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)

// If `inferSchema` is true, a CSV format is duplicate-safe (See SPARK-16896)
scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
|  1|  1|
+---+---+

// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Reference 'a' is ambiguous, could be: a#110, a#111.;
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:287)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.resolve(LogicalPlan.scala:181)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:153)
  at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolve$1.apply(LogicalPlan.scala:152)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
  at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
```
When this patch applied, the results change to;
```

// json
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("""{"a":1, "a":1}"""""").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("json").schema(schema).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)

scala> spark.read.format("json").load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:156)

// csv
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq("a,a", "1,1").toDF().coalesce(1).write.mode("overwrite").text("/tmp/data")
scala> spark.read.format("csv").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
  at org.apache.spark.sql.DataFrameReader.load(DataFrameReader.scala:178)

scala> spark.read.format("csv").option("header", true).load("/tmp/data").show
+---+---+
| a0| a1|
+---+---+
|  1|  1|
+---+---+

// parquet
scala> val schema = StructType(StructField("a", IntegerType) :: StructField("a", IntegerType) :: Nil)
scala> Seq((1, 1)).toDF("a", "b").coalesce(1).write.mode("overwrite").parquet("/tmp/data")
scala> spark.read.format("parquet").schema(schema).option("header", false).load("/tmp/data").show
org.apache.spark.sql.AnalysisException: Found duplicate column(s) in datasource: "a";
  at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtil.scala:47)
  at org.apache.spark.sql.util.SchemaUtils$.checkSchemaColumnNameDuplication(SchemaUtil.scala:33)
  at org.apache.spark.sql.execution.datasources.DataSource.getOrInferFileFormatSchema(DataSource.scala:186)
  at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:368)
```

## How was this patch tested?
Added tests in `DataFrameReaderWriterSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17758 from maropu/SPARK-20460.
2017-07-10 15:58:34 +08:00
Wenchen Fan 0e80ecae30 [SPARK-21100][SQL][FOLLOWUP] cleanup code and add more comments for Dataset.summary
## What changes were proposed in this pull request?

Some code cleanup and adding comments to make the code more readable. Changed the way to generate result rows, to be more clear.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18570 from cloud-fan/summary.
2017-07-09 22:53:27 -07:00
Wenchen Fan 680b33f166 [SPARK-18016][SQL][FOLLOWUP] merge declareAddedFunctions, initNestedClasses and declareNestedClasses
## What changes were proposed in this pull request?

These 3 methods have to be used together, so it makes more sense to merge them into one method and then the caller side only need to call one method.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18579 from cloud-fan/minor.
2017-07-09 16:30:35 -07:00
Xiao Li c3712b77a9 [SPARK-21307][REVERT][SQL] Remove SQLConf parameters from the parser-related classes
## What changes were proposed in this pull request?
Since we do not set active sessions when parsing the plan, we are unable to correctly use SQLConf.get to find the correct active session. Since https://github.com/apache/spark/pull/18531 breaks the build, I plan to revert it at first.

## How was this patch tested?
The existing test cases

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18568 from gatorsmile/revert18531.
2017-07-08 11:56:19 -07:00
Zhenhua Wang 9fccc3627f [SPARK-21083][SQL] Store zero size and row count when analyzing empty table
## What changes were proposed in this pull request?

We should be able to store zero size and row count after analyzing empty table.

This pr also enhances the test cases for re-analyzing tables.

## How was this patch tested?

Added a new test case and enhanced some test cases.

Author: Zhenhua Wang <wangzhenhua@huawei.com>

Closes #18292 from wzhfy/analyzeNewColumn.
2017-07-08 20:44:12 +08:00
Dongjoon Hyun 0b8dd2d084 [SPARK-21345][SQL][TEST][TEST-MAVEN] SparkSessionBuilderSuite should clean up stopped sessions.
## What changes were proposed in this pull request?

`SparkSessionBuilderSuite` should clean up stopped sessions. Otherwise, it leaves behind some stopped `SparkContext`s interfereing with other test suites using `ShardSQLContext`.

Recently, master branch fails consequtively.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/

## How was this patch tested?

Pass the Jenkins with a updated suite.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18567 from dongjoon-hyun/SPARK-SESSION.
2017-07-08 20:16:47 +08:00
Michael Patterson f5f02d213d [SPARK-20456][DOCS] Add examples for functions collection for pyspark
## What changes were proposed in this pull request?

This adds documentation to many functions in pyspark.sql.functions.py:
`upper`, `lower`, `reverse`, `unix_timestamp`, `from_unixtime`, `rand`, `randn`, `collect_list`, `collect_set`, `lit`
Add units to the trigonometry functions.
Renames columns in datetime examples to be more informative.
Adds links between some functions.

## How was this patch tested?

`./dev/lint-python`
`python python/pyspark/sql/functions.py`
`./python/run-tests.py --module pyspark-sql`

Author: Michael Patterson <map222@gmail.com>

Closes #17865 from map222/spark-20456.
2017-07-07 23:59:34 -07:00
Takeshi Yamamuro 7896e7b99d [SPARK-21281][SQL] Use string types by default if array and map have no argument
## What changes were proposed in this pull request?
This pr modified code to use string types by default if `array` and `map` in functions have no argument. This behaviour is the same with Hive one;
```
hive> CREATE TEMPORARY TABLE t1 AS SELECT map();
hive> DESCRIBE t1;
_c0   map<string,string>

hive> CREATE TEMPORARY TABLE t2 AS SELECT array();
hive> DESCRIBE t2;
_c0   array<string>
```

## How was this patch tested?
Added tests in `DataFrameFunctionsSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18516 from maropu/SPARK-21281.
2017-07-07 23:05:38 -07:00
Andrew Ray e1a172c201 [SPARK-21100][SQL] Add summary method as alternative to describe that gives quartiles similar to Pandas
## What changes were proposed in this pull request?

Adds method `summary`  that allows user to specify which statistics and percentiles to calculate. By default it include the existing statistics from `describe` and quartiles (25th, 50th, and 75th percentiles) similar to Pandas. Also changes the implementation of `describe` to delegate to `summary`.

## How was this patch tested?

additional unit test

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

Closes #18307 from aray/SPARK-21100.
2017-07-08 13:47:41 +08:00
Wang Gengliang a0fe32a219 [SPARK-21336] Revise rand comparison in BatchEvalPythonExecSuite
## What changes were proposed in this pull request?

Revise rand comparison in BatchEvalPythonExecSuite

In BatchEvalPythonExecSuite, there are two cases using the case "rand() > 3"
Rand() generates a random value in [0, 1), it is wired to be compared with 3, use 0.3 instead

## How was this patch tested?

unit test

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18560 from gengliangwang/revise_BatchEvalPythonExecSuite.
2017-07-07 15:39:29 -07:00
Wenchen Fan fef081309f [SPARK-21335][SQL] support un-aliased subquery
## What changes were proposed in this pull request?

un-aliased subquery is supported by Spark SQL for a long time. Its semantic was not well defined and had confusing behaviors, and it's not a standard SQL syntax, so we disallowed it in https://issues.apache.org/jira/browse/SPARK-20690 .

However, this is a breaking change, and we do have existing queries using un-aliased subquery. We should add the support back and fix its semantic.

This PR fixes the un-aliased subquery by assigning a default alias name.

After this PR, there is no syntax change from branch 2.2 to master, but we invalid a weird use case:
`SELECT v.i from (SELECT i FROM v)`. Now this query will throw analysis exception because users should not be able to use the qualifier inside a subquery.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18559 from cloud-fan/sub-query.
2017-07-07 20:04:30 +08:00
Jacek Laskowski 7fcbb9b57f [SPARK-21313][SS] ConsoleSink's string representation
## What changes were proposed in this pull request?

Add `toString` with options for `ConsoleSink` so it shows nicely in query progress.

**BEFORE**

```
  "sink" : {
    "description" : "org.apache.spark.sql.execution.streaming.ConsoleSink4b340441"
  }
```

**AFTER**

```
  "sink" : {
    "description" : "ConsoleSink[numRows=10, truncate=false]"
  }
```

/cc zsxwing tdas

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18539 from jaceklaskowski/SPARK-21313-ConsoleSink-toString.
2017-07-07 08:31:30 +01:00
Liang-Chi Hsieh 5df99bd364 [SPARK-20703][SQL][FOLLOW-UP] Associate metrics with data writes onto DataFrameWriter operations
## What changes were proposed in this pull request?

Remove time metrics since it seems no way to measure it in non per-row tracking.

## How was this patch tested?

Existing tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18558 from viirya/SPARK-20703-followup.
2017-07-07 13:12:20 +08:00
Kazuaki Ishizaki c09b31eb8f [SPARK-21217][SQL] Support ColumnVector.Array.to<type>Array()
## What changes were proposed in this pull request?

This PR implements bulk-copy for `ColumnVector.Array.to<type>Array()` methods (e.g. `toIntArray()`) in `ColumnVector.Array` by using `System.arrayCopy()` or `Platform.copyMemory()`.

Before this PR, when one of these method is called, the generic method in `ArrayData` is called. It is not fast since element-wise copy is performed.

This PR can improve performance of a benchmark program by 1.9x and 3.2x.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_131-8u131-b11-0ubuntu1.16.04.2-b11 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz

Int Array                                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
------------------------------------------------------------------------------------------------
ON_HEAP                                        586 /  628         14.3          69.9
OFF_HEAP                                       893 /  902          9.4         106.5
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_131-8u131-b11-0ubuntu1.16.04.2-b11 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz

Int Array                                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)
------------------------------------------------------------------------------------------------
ON_HEAP                                        306 /  331         27.4          36.4
OFF_HEAP                                       282 /  287         29.8          33.6
```

Source program
```
    (MemoryMode.ON_HEAP :: MemoryMode.OFF_HEAP :: Nil).foreach { memMode => {
      val len = 8 * 1024 * 1024
      val column = ColumnVector.allocate(len * 2, new ArrayType(IntegerType, false), memMode)

      val data = column.arrayData
      var i = 0
      while (i < len) {
        data.putInt(i, i)
        i += 1
      }
      column.putArray(0, 0, len)

      val benchmark = new Benchmark("Int Array", len, minNumIters = 20)
      benchmark.addCase(s"$memMode") { iter =>
        var i = 0
        while (i < 50) {
          column.getArray(0).toIntArray
          i += 1
        }
      }
      benchmark.run
    }}
```

## How was this patch tested?

Added test suite

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

Closes #18425 from kiszk/SPARK-21217.
2017-07-07 13:09:32 +08:00
Jacek Laskowski e5bb26174d [SPARK-21329][SS] Make EventTimeWatermarkExec explicitly UnaryExecNode
## What changes were proposed in this pull request?

Making EventTimeWatermarkExec explicitly UnaryExecNode

/cc tdas zsxwing

## How was this patch tested?

Local build.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18509 from jaceklaskowski/EventTimeWatermarkExec-UnaryExecNode.
2017-07-06 18:11:41 -07:00
Wenchen Fan 40c7add3a4 [SPARK-20946][SQL] Do not update conf for existing SparkContext in SparkSession.getOrCreate
## What changes were proposed in this pull request?

SparkContext is shared by all sessions, we should not update its conf for only one session.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18536 from cloud-fan/config.
2017-07-07 08:44:31 +08:00
Tathagata Das 0217dfd26f [SPARK-21267][SS][DOCS] Update Structured Streaming Documentation
## What changes were proposed in this pull request?

Few changes to the Structured Streaming documentation
- Clarify that the entire stream input table is not materialized
- Add information for Ganglia
- Add Kafka Sink to the main docs
- Removed a couple of leftover experimental tags
- Added more associated reading material and talk videos.

In addition, https://github.com/apache/spark/pull/16856 broke the link to the RDD programming guide in several places while renaming the page. This PR fixes those sameeragarwal cloud-fan.
- Added a redirection to avoid breaking internal and possible external links.
- Removed unnecessary redirection pages that were there since the separate scala, java, and python programming guides were merged together in 2013 or 2014.

## How was this patch tested?

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

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18485 from tdas/SPARK-21267.
2017-07-06 17:28:20 -07:00
Liang-Chi Hsieh 48e44b24a7 [SPARK-21204][SQL] Add support for Scala Set collection types in serialization
## What changes were proposed in this pull request?

Currently we can't produce a `Dataset` containing `Set` in SparkSQL. This PR tries to support serialization/deserialization of `Set`.

Because there's no corresponding internal data type in SparkSQL for a `Set`, the most proper choice for serializing a set should be an array.

## How was this patch tested?

Added unit tests.

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

Closes #18416 from viirya/SPARK-21204.
2017-07-07 01:07:45 +08:00
Bogdan Raducanu 26ac085deb [SPARK-21228][SQL] InSet incorrect handling of structs
## What changes were proposed in this pull request?
When data type is struct, InSet now uses TypeUtils.getInterpretedOrdering (similar to EqualTo) to build a TreeSet. In other cases it will use a HashSet as before (which should be faster). Similarly, In.eval uses Ordering.equiv instead of equals.

## How was this patch tested?
New test in SQLQuerySuite.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18455 from bogdanrdc/SPARK-21228.
2017-07-07 01:04:57 +08:00
wangzhenhua b8e4d567a7 [SPARK-21324][TEST] Improve statistics test suites
## What changes were proposed in this pull request?

1. move `StatisticsCollectionTestBase` to a separate file.
2. move some test cases to `StatisticsCollectionSuite` so that `hive/StatisticsSuite` only keeps tests that need hive support.
3. clear up some test cases.

## How was this patch tested?

Existing tests.

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

Closes #18545 from wzhfy/cleanStatSuites.
2017-07-06 16:00:31 +08:00
Liang-Chi Hsieh 6ff05a66fe [SPARK-20703][SQL] Associate metrics with data writes onto DataFrameWriter operations
## What changes were proposed in this pull request?

Right now in the UI, after SPARK-20213, we can show the operations to write data out. However, there is no way to associate metrics with data writes. We should show relative metrics on the operations.

#### Supported commands

This change supports updating metrics for file-based data writing operations, including `InsertIntoHadoopFsRelationCommand`, `InsertIntoHiveTable`.

Supported metrics:

* number of written files
* number of dynamic partitions
* total bytes of written data
* total number of output rows
* average writing data out time (ms)
* (TODO) min/med/max number of output rows per file/partition
* (TODO) min/med/max bytes of written data per file/partition

####  Commands not supported

`InsertIntoDataSourceCommand`, `SaveIntoDataSourceCommand`:

The two commands uses DataSource APIs to write data out, i.e., the logic of writing data out is delegated to the DataSource implementations, such as  `InsertableRelation.insert` and `CreatableRelationProvider.createRelation`. So we can't obtain metrics from delegated methods for now.

`CreateHiveTableAsSelectCommand`, `CreateDataSourceTableAsSelectCommand` :

The two commands invokes other commands to write data out. The invoked commands can even write to non file-based data source. We leave them as future TODO.

#### How to update metrics of writing files out

A `RunnableCommand` which wants to update metrics, needs to override its `metrics` and provide the metrics data structure to `ExecutedCommandExec`.

The metrics are prepared during the execution of `FileFormatWriter`. The callback function passed to `FileFormatWriter` will accept the metrics and update accordingly.

There is a metrics updating function in `RunnableCommand`. In runtime, the function will be bound to the spark context and `metrics` of `ExecutedCommandExec` and pass to `FileFormatWriter`.

## How was this patch tested?

Updated unit tests.

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

Closes #18159 from viirya/SPARK-20703-2.
2017-07-06 15:47:09 +08:00
Tathagata Das 60043f2245 [SS][MINOR] Fix flaky test in DatastreamReaderWriterSuite. temp checkpoint dir should be deleted
## What changes were proposed in this pull request?

Stopping query while it is being initialized can throw interrupt exception, in which case temporary checkpoint directories will not be deleted, and the test will fail.

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

Closes #18442 from tdas/DatastreamReaderWriterSuite-fix.
2017-07-06 00:20:26 -07:00
Sumedh Wale 14a3bb3a00 [SPARK-21312][SQL] correct offsetInBytes in UnsafeRow.writeToStream
## What changes were proposed in this pull request?

Corrects offsetInBytes calculation in UnsafeRow.writeToStream. Known failures include writes to some DataSources that have own SparkPlan implementations and cause EXCHANGE in writes.

## How was this patch tested?

Extended UnsafeRowSuite.writeToStream to include an UnsafeRow over byte array having non-zero offset.

Author: Sumedh Wale <swale@snappydata.io>

Closes #18535 from sumwale/SPARK-21312.
2017-07-06 14:47:22 +08:00
gatorsmile 75b168fd30 [SPARK-21308][SQL] Remove SQLConf parameters from the optimizer
### What changes were proposed in this pull request?
This PR removes SQLConf parameters from the optimizer rules

### How was this patch tested?
The existing test cases

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18533 from gatorsmile/rmSQLConfOptimizer.
2017-07-06 14:18:50 +08:00
Shixiong Zhu ab866f1173 [SPARK-21248][SS] The clean up codes in StreamExecution should not be interrupted
## What changes were proposed in this pull request?

This PR uses `runUninterruptibly` to avoid that the clean up codes in StreamExecution is interrupted. It also removes an optimization in `runUninterruptibly` to make sure this method never throw `InterruptedException`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18461 from zsxwing/SPARK-21248.
2017-07-05 18:26:28 -07:00
gatorsmile c8e7f445b9 [SPARK-21307][SQL] Remove SQLConf parameters from the parser-related classes.
### What changes were proposed in this pull request?
This PR is to remove SQLConf parameters from the parser-related classes.

### How was this patch tested?
The existing test cases.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18531 from gatorsmile/rmSQLConfParser.
2017-07-05 11:06:15 -07:00
Jeff Zhang 742da08685 [SPARK-19439][PYSPARK][SQL] PySpark's registerJavaFunction Should Support UDAFs
## What changes were proposed in this pull request?

Support register Java UDAFs in PySpark so that user can use Java UDAF in PySpark. Besides that I also add api in `UDFRegistration`

## How was this patch tested?

Unit test is added

Author: Jeff Zhang <zjffdu@apache.org>

Closes #17222 from zjffdu/SPARK-19439.
2017-07-05 10:59:10 -07:00
ouyangxiaochen 5787ace463 [SPARK-20383][SQL] Supporting Create [temporary] Function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'
## What changes were proposed in this pull request?

support to create [temporary] function with the keyword 'OR REPLACE' and 'IF NOT EXISTS'

## How was this patch tested?
manual test and added test cases

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: ouyangxiaochen <ou.yangxiaochen@zte.com.cn>

Closes #17681 from ouyangxiaochen/spark-419.
2017-07-05 20:46:42 +08:00
gatorsmile de14086e1f [SPARK-21295][SQL] Use qualified names in error message for missing references
### What changes were proposed in this pull request?
It is strange to see the following error message. Actually, the column is from another table.
```
cannot resolve '`right.a`' given input columns: [a, c, d];
```

After the PR, the error message looks like
```
cannot resolve '`right.a`' given input columns: [left.a, right.c, right.d];
```

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18520 from gatorsmile/removeSQLConf.
2017-07-05 10:40:02 +08:00
Dongjoon Hyun 1b50e0e0d6 [SPARK-20256][SQL] SessionState should be created more lazily
## What changes were proposed in this pull request?

`SessionState` is designed to be created lazily. However, in reality, it created immediately in `SparkSession.Builder.getOrCreate` ([here](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala#L943)).

This PR aims to recover the lazy behavior by keeping the options into `initialSessionOptions`. The benefit is like the following. Users can start `spark-shell` and use RDD operations without any problems.

**BEFORE**
```scala
$ bin/spark-shell
java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder'
...
Caused by: org.apache.spark.sql.AnalysisException:
    org.apache.hadoop.hive.ql.metadata.HiveException:
       MetaException(message:java.security.AccessControlException:
          Permission denied: user=spark, access=READ,
             inode="/apps/hive/warehouse":hive:hdfs:drwx------
```
As reported in SPARK-20256, this happens when the warehouse directory is not allowed for this user.

**AFTER**
```scala
$ bin/spark-shell
...
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 2.3.0-SNAPSHOT
      /_/

Using Scala version 2.11.8 (Java HotSpot(TM) 64-Bit Server VM, Java 1.8.0_112)
Type in expressions to have them evaluated.
Type :help for more information.

scala> sc.range(0, 10, 1).count()
res0: Long = 10
```

## How was this patch tested?

Manual.

This closes #18512 .

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18501 from dongjoon-hyun/SPARK-20256.
2017-07-04 09:48:40 -07:00
YIHAODIAN\wangshuangshuang a3c29fcbbd [SPARK-19726][SQL] Faild to insert null timestamp value to mysql using spark jdbc
## What changes were proposed in this pull request?

when creating table like following:
> create table timestamp_test(id int(11), time_stamp timestamp not null default current_timestamp);

The result of Excuting "insert into timestamp_test values (111, null)" is different between Spark and JDBC.
```
mysql> select * from timestamp_test;
+------+---------------------+
| id   | time_stamp          |
+------+---------------------+
|  111 | 1970-01-01 00:00:00 | -> spark
|  111 | 2017-06-27 19:32:38 | -> mysql
+------+---------------------+
2 rows in set (0.00 sec)
```
   Because in such case ```StructField.nullable``` is false, so the generated codes of ```InvokeLike``` and ```BoundReference``` don't check whether the field is null or not. Instead, they directly use ```CodegenContext.INPUT_ROW.getLong(1)```, however, ```UnsafeRow.setNullAt(1)``` will put 0 in the underlying memory.

   The PR will ```always``` set ```StructField.nullable```  true after obtaining metadata from jdbc connection, Since we can insert null to not null timestamp column in MySQL. In this way, spark will propagate null to underlying DB engine, and let DB to choose how to process NULL.

## How was this patch tested?

Added tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: YIHAODIAN\wangshuangshuang <wangshuangshuang@yihaodian.com>
Author: Shuangshuang Wang <wsszone@gmail.com>

Closes #18445 from shuangshuangwang/SPARK-19726.
2017-07-04 09:44:27 -07:00
gatorsmile 29b1f6b09f [SPARK-21256][SQL] Add withSQLConf to Catalyst Test
### What changes were proposed in this pull request?
SQLConf is moved to Catalyst. We are adding more and more test cases for verifying the conf-specific behaviors. It is nice to add a helper function to simplify the test cases.

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

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18469 from gatorsmile/withSQLConf.
2017-07-04 08:54:07 -07:00
gatorsmile c79c10ebaf [TEST] Different behaviors of SparkContext Conf when building SparkSession
## What changes were proposed in this pull request?
If the created ACTIVE sparkContext is not EXPLICITLY passed through the Builder's API `sparkContext()`, the conf of this sparkContext will also contain the conf set through the API `config()`; otherwise, the conf of this sparkContext will NOT contain the conf set through the API `config()`

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18517 from gatorsmile/fixTestCase2.
2017-07-03 16:18:54 -07:00
Wenchen Fan f953ca56ec [SPARK-21284][SQL] rename SessionCatalog.registerFunction parameter name
## What changes were proposed in this pull request?

Looking at the code in `SessionCatalog.registerFunction`, the parameter `ignoreIfExists` is a wrong name. When `ignoreIfExists` is true, we will override the function if it already exists. So `overrideIfExists` should be the corrected name.

## How was this patch tested?

N/A

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18510 from cloud-fan/minor.
2017-07-03 10:51:44 -07:00
Takeshi Yamamuro 363bfe30ba [SPARK-20073][SQL] Prints an explicit warning message in case of NULL-safe equals
## What changes were proposed in this pull request?
This pr added code to print the same warning messages with `===` cases when using NULL-safe equals (`<=>`).

## How was this patch tested?
Existing tests.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18436 from maropu/SPARK-20073.
2017-07-03 10:14:03 -07:00
aokolnychyi 17bdc36ef1 [SPARK-21102][SQL] Refresh command is too aggressive in parsing
### Idea

This PR adds validation to REFRESH sql statements. Currently, users can specify whatever they want as resource path. For example, spark.sql("REFRESH ! $ !") will be executed without any exceptions.

### Implementation

I am not sure that my current implementation is the most optimal, so any feedback is appreciated. My first idea was to make the grammar as strict as possible. Unfortunately, there were some problems. I tried the approach below:

SqlBase.g4
```
...
    | REFRESH TABLE tableIdentifier                                    #refreshTable
    | REFRESH resourcePath                                             #refreshResource
...

resourcePath
    : STRING
    | (IDENTIFIER | number | nonReserved | '/' | '-')+ // other symbols can be added if needed
    ;
```
It is not flexible enough and requires to explicitly mention all possible symbols. Therefore, I came up with the current approach that is implemented in the code.

Let me know your opinion on which one is better.

Author: aokolnychyi <anton.okolnychyi@sap.com>

Closes #18368 from aokolnychyi/spark-21102.
2017-07-03 09:35:49 -07:00
Rui Zha d4107196d5 [SPARK-18004][SQL] Make sure the date or timestamp related predicate can be pushed down to Oracle correctly
## What changes were proposed in this pull request?

Move `compileValue` method in JDBCRDD to JdbcDialect, and override the `compileValue` method in OracleDialect to rewrite the Oracle-specific timestamp and date literals in where clause.

## How was this patch tested?

An integration test has been added.

Author: Rui Zha <zrdt713@gmail.com>
Author: Zharui <zrdt713@gmail.com>

Closes #18451 from SharpRay/extend-compileValue-to-dialects.
2017-07-02 17:37:47 -07:00
Xingbo Jiang c605fee01f [SPARK-21260][SQL][MINOR] Remove the unused OutputFakerExec
## What changes were proposed in this pull request?

OutputFakerExec was added long ago and is not used anywhere now so we should remove it.

## How was this patch tested?
N/A

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #18473 from jiangxb1987/OutputFakerExec.
2017-07-02 08:50:48 +01:00
Reynold Xin b1d719e7c9 [SPARK-21273][SQL] Propagate logical plan stats using visitor pattern and mixin
## What changes were proposed in this pull request?
We currently implement statistics propagation directly in logical plan. Given we already have two different implementations, it'd make sense to actually decouple the two and add stats propagation using mixin. This would reduce the coupling between logical plan and statistics handling.

This can also be a powerful pattern in the future to add additional properties (e.g. constraints).

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18479 from rxin/stats-trait.
2017-06-30 21:10:23 -07:00
wangzhenhua 61b5df567e [SPARK-21127][SQL] Update statistics after data changing commands
## What changes were proposed in this pull request?

Update stats after the following data changing commands:

- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand

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

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

Closes #18334 from wzhfy/changeStatsForOperation.
2017-07-01 10:01:44 +08:00
Wenchen Fan 4eb41879ce [SPARK-17528][SQL] data should be copied properly before saving into InternalRow
## What changes were proposed in this pull request?

For performance reasons, `UnsafeRow.getString`, `getStruct`, etc. return a "pointer" that points to a memory region of this unsafe row. This makes the unsafe projection a little dangerous, because all of its output rows share one instance.

When we implement SQL operators, we should be careful to not cache the input rows because they may be produced by unsafe projection from child operator and thus its content may change overtime.

However, when we updating values of InternalRow(e.g. in mutable projection and safe projection), we only copy UTF8String, we should also copy InternalRow, ArrayData and MapData. This PR fixes this, and also fixes the copy of vairous InternalRow, ArrayData and MapData implementations.

## How was this patch tested?

new regression tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18483 from cloud-fan/fix-copy.
2017-07-01 09:25:29 +08:00
Liang-Chi Hsieh fd13255225 [SPARK-21052][SQL][FOLLOW-UP] Add hash map metrics to join
## What changes were proposed in this pull request?

Remove `numHashCollisions` in `BytesToBytesMap`. And change `getAverageProbesPerLookup()` to `getAverageProbesPerLookup` as suggested.

## How was this patch tested?

Existing tests.

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

Closes #18480 from viirya/SPARK-21052-followup.
2017-06-30 15:11:27 -07:00
Xiao Li eed9c4ef85 [SPARK-21129][SQL] Arguments of SQL function call should not be named expressions
### What changes were proposed in this pull request?

Function argument should not be named expressions. It could cause two issues:
- Misleading error message
- Unexpected query results when the column name is `distinct`, which is not a reserved word in our parser.

```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query: cannot resolve '`distinct`' given input columns: [c1, c2]; line 1 pos 26;
'Project [unresolvedalias('count(c1#30, 'distinct), None)]
+- SubqueryAlias t1
   +- CatalogRelation `default`.`t1`, org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, [c1#30, c2#31]
```

After the fix, the error message becomes
```
spark-sql> select count(distinct c1, distinct c2) from t1;
Error in query:
extraneous input 'c2' expecting {')', ',', '.', '[', 'OR', 'AND', 'IN', NOT, 'BETWEEN', 'LIKE', RLIKE, 'IS', EQ, '<=>', '<>', '!=', '<', LTE, '>', GTE, '+', '-', '*', '/', '%', 'DIV', '&', '|', '||', '^'}(line 1, pos 35)

== SQL ==
select count(distinct c1, distinct c2) from t1
-----------------------------------^^^
```

### How was this patch tested?
Added a test case to parser suite.

Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #18338 from gatorsmile/parserDistinctAggFunc.
2017-06-30 14:23:56 -07:00
Herman van Hovell e2f32ee45a [SPARK-21258][SQL] Fix WindowExec complex object aggregation with spilling
## What changes were proposed in this pull request?
`WindowExec` currently improperly stores complex objects (UnsafeRow, UnsafeArrayData, UnsafeMapData, UTF8String) during aggregation by keeping a reference in the buffer used by `GeneratedMutableProjections` to the actual input data. Things go wrong when the input object (or the backing bytes) are reused for other things. This could happen in window functions when it starts spilling to disk. When reading the back the spill files the `UnsafeSorterSpillReader` reuses the buffer to which the `UnsafeRow` points, leading to weird corruption scenario's. Note that this only happens for aggregate functions that preserve (parts of) their input, for example `FIRST`, `LAST`, `MIN` & `MAX`.

This was not seen before, because the spilling logic was not doing actual spills as much and actually used an in-memory page. This page was not cleaned up during window processing and made sure unsafe objects point to their own dedicated memory location. This was changed by https://github.com/apache/spark/pull/16909, after this PR Spark spills more eagerly.

This PR provides a surgical fix because we are close to releasing Spark 2.2. This change just makes sure that there cannot be any object reuse at the expensive of a little bit of performance. We will follow-up with a more subtle solution at a later point.

## How was this patch tested?
Added a regression test to `DataFrameWindowFunctionsSuite`.

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

Closes #18470 from hvanhovell/SPARK-21258.
2017-06-30 12:34:09 +08:00
Liang-Chi Hsieh 18066f2e61 [SPARK-21052][SQL] Add hash map metrics to join
## What changes were proposed in this pull request?

This adds the average hash map probe metrics to join operator such as `BroadcastHashJoin` and `ShuffledHashJoin`.

This PR adds the API to `HashedRelation` to get average hash map probe.

## How was this patch tested?

Related test cases are added.

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

Closes #18301 from viirya/SPARK-21052.
2017-06-29 21:28:48 +08:00
jinxing d106a74c53 [SPARK-21240] Fix code style for constructing and stopping a SparkContext in UT.
## What changes were proposed in this pull request?

Same with SPARK-20985.
Fix code style for constructing and stopping a `SparkContext`. Assure the context is stopped to avoid other tests complain that there's only one `SparkContext` can exist.

Author: jinxing <jinxing6042@126.com>

Closes #18454 from jinxing64/SPARK-21240.
2017-06-29 09:59:36 +01:00
Wenchen Fan 9f6b3e65cc [SPARK-21238][SQL] allow nested SQL execution
## What changes were proposed in this pull request?

This is kind of another follow-up for https://github.com/apache/spark/pull/18064 .

In #18064 , we wrap every SQL command with SQL execution, which makes nested SQL execution very likely to happen. #18419 trid to improve it a little bit, by introduing `SQLExecition.ignoreNestedExecutionId`. However, this is not friendly to data source developers, they may need to update their code to use this `ignoreNestedExecutionId` API.

This PR proposes a new solution, to just allow nested execution. The downside is that, we may have multiple executions for one query. We can improve this by updating the data organization in SQLListener, to have 1-n mapping from query to execution, instead of 1-1 mapping. This can be done in a follow-up.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18450 from cloud-fan/execution-id.
2017-06-29 14:37:42 +08:00
wangzhenhua 82e24912d6 [SPARK-21237][SQL] Invalidate stats once table data is changed
## What changes were proposed in this pull request?

Invalidate spark's stats after data changing commands:

- InsertIntoHadoopFsRelationCommand
- InsertIntoHiveTable
- LoadDataCommand
- TruncateTableCommand
- AlterTableSetLocationCommand
- AlterTableDropPartitionCommand

## How was this patch tested?

Added test cases.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #18449 from wzhfy/removeStats.
2017-06-29 11:32:29 +08:00
Wenchen Fan 25c2edf6f9 [SPARK-21229][SQL] remove QueryPlan.preCanonicalized
## What changes were proposed in this pull request?

`QueryPlan.preCanonicalized` is only overridden in a few places, and it does introduce an extra concept to `QueryPlan` which may confuse people.

This PR removes it and override `canonicalized` in these places

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18440 from cloud-fan/minor.
2017-06-29 11:21:50 +08:00
hyukjinkwon db44f5f3e8 [SPARK-21224][R] Specify a schema by using a DDL-formatted string when reading in R
## What changes were proposed in this pull request?

This PR proposes to support a DDL-formetted string as schema as below:

```r
mockLines <- c("{\"name\":\"Michael\"}",
               "{\"name\":\"Andy\", \"age\":30}",
               "{\"name\":\"Justin\", \"age\":19}")
jsonPath <- tempfile(pattern = "sparkr-test", fileext = ".tmp")
writeLines(mockLines, jsonPath)
df <- read.df(jsonPath, "json", "name STRING, age DOUBLE")
collect(df)
```

## How was this patch tested?

Tests added in `test_streaming.R` and `test_sparkSQL.R` and manual tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18431 from HyukjinKwon/r-ddl-schema.
2017-06-28 19:36:00 -07:00
Burak Yavuz e68aed70fb [SPARK-21216][SS] Hive strategies missed in Structured Streaming IncrementalExecution
## What changes were proposed in this pull request?

If someone creates a HiveSession, the planner in `IncrementalExecution` doesn't take into account the Hive scan strategies. This causes joins of Streaming DataFrame's with Hive tables to fail.

## How was this patch tested?

Regression test

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #18426 from brkyvz/hive-join.
2017-06-28 10:45:45 -07:00
Wenchen Fan 838effb98a Revert "[SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas"
This reverts commit e44697606f.
2017-06-28 14:28:40 +08:00
Liang-Chi Hsieh fd8c931a30 [SPARK-19104][SQL] Lambda variables in ExternalMapToCatalyst should be global
## What changes were proposed in this pull request?

The issue happens in `ExternalMapToCatalyst`. For example, the following codes create `ExternalMapToCatalyst` to convert Scala Map to catalyst map format.

    val data = Seq.tabulate(10)(i => NestedData(1, Map("key" -> InnerData("name", i + 100))))
    val ds = spark.createDataset(data)

The `valueConverter` in `ExternalMapToCatalyst` looks like:

    if (isnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true))) null else named_struct(name, staticinvoke(class org.apache.spark.unsafe.types.UTF8String, StringType, fromString, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).name, true), value, assertnotnull(lambdavariable(ExternalMapToCatalyst_value52, ExternalMapToCatalyst_value_isNull52, ObjectType(class org.apache.spark.sql.InnerData), true)).value)

There is a `CreateNamedStruct` expression (`named_struct`) to create a row of `InnerData.name` and `InnerData.value` that are referred by `ExternalMapToCatalyst_value52`.

Because `ExternalMapToCatalyst_value52` are local variable, when `CreateNamedStruct` splits expressions to individual functions, the local variable can't be accessed anymore.

## How was this patch tested?

Jenkins tests.

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

Closes #18418 from viirya/SPARK-19104.
2017-06-28 00:57:05 +08:00
Wang Gengliang 3cb3ccce12 [SPARK-21196] Split codegen info of query plan into sequence
codegen info of query plan can be very long.
In debugging console / web page, it would be more readable if the subtrees and corresponding codegen are split into sequence.

Example:

```java
codegenStringSeq(sql("select 1").queryExecution.executedPlan)
```
The example will return Seq[(String, String)] of length 1, containing the subtree as string and the corresponding generated code.

The subtree as string:

> (*Project [1 AS 1#0]
> +- Scan OneRowRelation[]

The generated code:
```java
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private UnsafeRow project_result;
/* 010 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder project_holder;
/* 011 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter project_rowWriter;
/* 012 */
/* 013 */   public GeneratedIterator(Object[] references) {
/* 014 */     this.references = references;
/* 015 */   }
/* 016 */
/* 017 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 018 */     partitionIndex = index;
/* 019 */     this.inputs = inputs;
/* 020 */     inputadapter_input = inputs[0];
/* 021 */     project_result = new UnsafeRow(1);
/* 022 */     project_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(project_result, 0);
/* 023 */     project_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(project_holder, 1);
/* 024 */
/* 025 */   }
/* 026 */
/* 027 */   protected void processNext() throws java.io.IOException {
/* 028 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 029 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 030 */       project_rowWriter.write(0, 1);
/* 031 */       append(project_result);
/* 032 */       if (shouldStop()) return;
/* 033 */     }
/* 034 */   }
/* 035 */
/* 036 */ }
```
## What changes were proposed in this pull request?
add method codegenToSeq: split codegen info of query plan into sequence

## How was this patch tested?
unit test

cloud-fan gatorsmile
Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18409 from gengliangwang/codegen.
2017-06-27 17:24:46 +08:00
Wenchen Fan c22810004f [SPARK-20213][SQL][FOLLOW-UP] introduce SQLExecution.ignoreNestedExecutionId
## What changes were proposed in this pull request?

in https://github.com/apache/spark/pull/18064, to work around the nested sql execution id issue, we introduced several internal methods in `Dataset`, like `collectInternal`, `countInternal`, `showInternal`, etc., to avoid nested execution id.

However, this approach has poor expansibility. When we hit other nested execution id cases, we may need to add more internal methods in `Dataset`.

Our goal is to ignore the nested execution id in some cases, and we can have a better approach to achieve this goal, by introducing `SQLExecution.ignoreNestedExecutionId`. Whenever we find a place which needs to ignore the nested execution, we can just wrap the action with `SQLExecution.ignoreNestedExecutionId`, and this is more expansible than the previous approach.

The idea comes from https://github.com/apache/spark/pull/17540/files#diff-ab49028253e599e6e74cc4f4dcb2e3a8R57 by rdblue

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18419 from cloud-fan/follow.
2017-06-27 02:35:51 +08:00
Burak Yavuz 5282bae040 [SPARK-21153] Use project instead of expand in tumbling windows
## What changes were proposed in this pull request?

Time windowing in Spark currently performs an Expand + Filter, because there is no way to guarantee the amount of windows a timestamp will fall in, in the general case. However, for tumbling windows, a record is guaranteed to fall into a single bucket. In this case, doubling the number of records with Expand is wasteful, and can be improved by using a simple Projection instead.

Benchmarks show that we get an order of magnitude performance improvement after this patch.

## How was this patch tested?

Existing unit tests. Benchmarked using the following code:

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

spark.time {
  spark.range(numRecords)
    .select(from_unixtime((current_timestamp().cast("long") * 1000 + 'id / 1000) / 1000) as 'time)
    .select(window('time, "10 seconds"))
    .count()
}
```

Setup:
 - 1 c3.2xlarge worker (8 cores)

![image](https://user-images.githubusercontent.com/5243515/27348748-ed991b84-55a9-11e7-8f8b-6e7abc524417.png)

1 B rows ran in 287 seconds after this optimization. I didn't wait for it to finish without the optimization. Shows about 5x improvement for large number of records.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #18364 from brkyvz/opt-tumble.
2017-06-26 01:26:32 -07:00
Masha Basmanova b449a1d6aa [SPARK-21079][SQL] Calculate total size of a partition table as a sum of individual partitions
## What changes were proposed in this pull request?

Storage URI of a partitioned table may or may not point to a directory under which individual partitions are stored. In fact, individual partitions may be located in totally unrelated directories. Before this change, ANALYZE TABLE table COMPUTE STATISTICS command calculated total size of a table by adding up sizes of files found under table's storage URI. This calculation could produce 0 if partitions are stored elsewhere.

This change uses storage URIs of individual partitions to calculate the sizes of all partitions of a table and adds these up to produce the total size of a table.

CC: wzhfy

## How was this patch tested?

Added unit test.

Ran ANALYZE TABLE xxx COMPUTE STATISTICS on a partitioned Hive table and verified that sizeInBytes is calculated correctly. Before this change, the size would be zero.

Author: Masha Basmanova <mbasmanova@fb.com>

Closes #18309 from mbasmanova/mbasmanova-analyze-part-table.
2017-06-24 22:49:35 -07:00
gatorsmile 2e1586f60a [SPARK-21203][SQL] Fix wrong results of insertion of Array of Struct
### What changes were proposed in this pull request?
```SQL
CREATE TABLE `tab1`
(`custom_fields` ARRAY<STRUCT<`id`: BIGINT, `value`: STRING>>)
USING parquet

INSERT INTO `tab1`
SELECT ARRAY(named_struct('id', 1, 'value', 'a'), named_struct('id', 2, 'value', 'b'))

SELECT custom_fields.id, custom_fields.value FROM tab1
```

The above query always return the last struct of the array, because the rule `SimplifyCasts` incorrectly rewrites the query. The underlying cause is we always use the same `GenericInternalRow` object when doing the cast.

### How was this patch tested?

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18412 from gatorsmile/castStruct.
2017-06-24 22:35:59 +08:00
Gabor Feher b837bf9ae9 [SPARK-20555][SQL] Fix mapping of Oracle DECIMAL types to Spark types in read path
## What changes were proposed in this pull request?

This PR is to revert some code changes in the read path of https://github.com/apache/spark/pull/14377. The original fix is https://github.com/apache/spark/pull/17830

When merging this PR, please give the credit to gaborfeher

## How was this patch tested?

Added a test case to OracleIntegrationSuite.scala

Author: Gabor Feher <gabor.feher@lynxanalytics.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #18408 from gatorsmile/OracleType.
2017-06-23 21:53:38 -07:00
hyukjinkwon 7525ce98b4 [SPARK-20431][SS][FOLLOWUP] Specify a schema by using a DDL-formatted string in DataStreamReader
## What changes were proposed in this pull request?

This pr supported a DDL-formatted string in `DataStreamReader.schema`.
This fix could make users easily define a schema without importing the type classes.

For example,

```scala
scala> spark.readStream.schema("col0 INT, col1 DOUBLE").load("/tmp/abc").printSchema()
root
 |-- col0: integer (nullable = true)
 |-- col1: double (nullable = true)
```

## How was this patch tested?

Added tests in `DataStreamReaderWriterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18373 from HyukjinKwon/SPARK-20431.
2017-06-24 11:39:41 +08:00
Xiao Li 03eb6117af [SPARK-21164][SQL] Remove isTableSample from Sample and isGenerated from Alias and AttributeReference
## What changes were proposed in this pull request?
`isTableSample` and `isGenerated ` were introduced for SQL Generation respectively by https://github.com/apache/spark/pull/11148 and https://github.com/apache/spark/pull/11050

Since SQL Generation is removed, we do not need to keep `isTableSample`.

## How was this patch tested?
The existing test cases

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18379 from gatorsmile/CleanSample.
2017-06-23 14:48:33 -07:00
Dilip Biswal 13c2a4f2f8 [SPARK-20417][SQL] Move subquery error handling to checkAnalysis from Analyzer
## What changes were proposed in this pull request?
Currently we do a lot of validations for subquery in the Analyzer. We should move them to CheckAnalysis which is the framework to catch and report Analysis errors. This was mentioned as a review comment in SPARK-18874.

## How was this patch tested?
Exists tests + A few tests added to SQLQueryTestSuite.

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

Closes #17713 from dilipbiswal/subquery_checkanalysis.
2017-06-23 11:02:54 -07:00
Ong Ming Yang 4cc62951a2 [MINOR][DOCS] Docs in DataFrameNaFunctions.scala use wrong method
## What changes were proposed in this pull request?

* Following the first few examples in this file, the remaining methods should also be methods of `df.na` not `df`.
* Filled in some missing parentheses

## How was this patch tested?

N/A

Author: Ong Ming Yang <me@ongmingyang.com>

Closes #18398 from ongmingyang/master.
2017-06-23 10:56:59 -07:00
Tathagata Das 2ebd0838d1 [SPARK-21192][SS] Preserve State Store provider class configuration across StreamingQuery restarts
## What changes were proposed in this pull request?

If the SQL conf for StateStore provider class is changed between restarts (i.e. query started with providerClass1 and attempted to restart using providerClass2), then the query will fail in a unpredictable way as files saved by one provider class cannot be used by the newer one.

Ideally, the provider class used to start the query should be used to restart the query, and the configuration in the session where it is being restarted should be ignored.

This PR saves the provider class config to OffsetSeqLog, in the same way # shuffle partitions is saved and recovered.

## How was this patch tested?
new unit tests

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

Closes #18402 from tdas/SPARK-21192.
2017-06-23 10:55:02 -07:00
wangzhenhua b803b66a81 [SPARK-21180][SQL] Remove conf from stats functions since now we have conf in LogicalPlan
## What changes were proposed in this pull request?

After wiring `SQLConf` in logical plan ([PR 18299](https://github.com/apache/spark/pull/18299)), we can remove the need of passing `conf` into `def stats` and `def computeStats`.

## How was this patch tested?

Covered by existing tests, plus some modified existing tests.

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

Closes #18391 from wzhfy/removeConf.
2017-06-23 10:33:53 -07:00
Takeshi Yamamuro f3dea60793 [SPARK-21144][SQL] Print a warning if the data schema and partition schema have the duplicate columns
## What changes were proposed in this pull request?
The current master outputs unexpected results when the data schema and partition schema have the duplicate columns:
```
withTempPath { dir =>
  val basePath = dir.getCanonicalPath
  spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=1").toString)
  spark.range(0, 3).toDF("foo").write.parquet(new Path(basePath, "foo=a").toString)
  spark.read.parquet(basePath).show()
}

+---+
|foo|
+---+
|  1|
|  1|
|  a|
|  a|
|  1|
|  a|
+---+
```
This patch added code to print a warning when the duplication found.

## How was this patch tested?
Manually checked.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18375 from maropu/SPARK-21144-3.
2017-06-23 09:28:02 -07:00
jinxing 153dd49b74 [SPARK-21047] Add test suites for complicated cases in ColumnarBatchSuite
## What changes were proposed in this pull request?
Current ColumnarBatchSuite has very simple test cases for `Array` and `Struct`. This pr wants to add  some test suites for complicated cases in ColumnVector.

Author: jinxing <jinxing6042@126.com>

Closes #18327 from jinxing64/SPARK-21047.
2017-06-23 20:41:17 +08:00
Tathagata Das fe24634d14 [SPARK-21145][SS] Added StateStoreProviderId with queryRunId to reload StateStoreProviders when query is restarted
## What changes were proposed in this pull request?
StateStoreProvider instances are loaded on-demand in a executor when a query is started. When a query is restarted, the loaded provider instance will get reused. Now, there is a non-trivial chance, that the task of the previous query run is still running, while the tasks of the restarted run has started. So for a stateful partition, there may be two concurrent tasks related to the same stateful partition, and there for using the same provider instance. This can lead to inconsistent results and possibly random failures, as state store implementations are not designed to be thread-safe.

To fix this, I have introduced a `StateStoreProviderId`, that unique identifies a provider loaded in an executor. It has the query run id in it, thus making sure that restarted queries will force the executor to load a new provider instance, thus avoiding two concurrent tasks (from two different runs) from reusing the same provider instance.

Additional minor bug fixes
- All state stores related to query run is marked as deactivated in the `StateStoreCoordinator` so that the executors can unload them and clear resources.
- Moved the code that determined the checkpoint directory of a state store from implementation-specific code (`HDFSBackedStateStoreProvider`) to non-specific code (StateStoreId), so that implementation do not accidentally get it wrong.
  - Also added store name to the path, to support multiple stores per sql operator partition.

*Note:* This change does not address the scenario where two tasks of the same run (e.g. speculative tasks) are concurrently running in the same executor. The chance of this very small, because ideally speculative tasks should never run in the same executor.

## How was this patch tested?
Existing unit tests + new unit test.

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

Closes #18355 from tdas/SPARK-21145.
2017-06-23 00:43:21 -07:00
Wang Gengliang b8a743b6a5 [SPARK-21174][SQL] Validate sampling fraction in logical operator level
## What changes were proposed in this pull request?

Currently the validation of sampling fraction in dataset is incomplete.
As an improvement, validate sampling fraction in logical operator level:
1) if with replacement: fraction should be nonnegative
2) else: fraction should be on interval [0, 1]
Also add test cases for the validation.

## How was this patch tested?
integration tests

gatorsmile cloud-fan
Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Wang Gengliang <ltnwgl@gmail.com>

Closes #18387 from gengliangwang/sample_ratio_validate.
2017-06-23 09:27:35 +08:00
Bryan Cutler e44697606f [SPARK-13534][PYSPARK] Using Apache Arrow to increase performance of DataFrame.toPandas
## What changes were proposed in this pull request?
Integrate Apache Arrow with Spark to increase performance of `DataFrame.toPandas`.  This has been done by using Arrow to convert data partitions on the executor JVM to Arrow payload byte arrays where they are then served to the Python process.  The Python DataFrame can then collect the Arrow payloads where they are combined and converted to a Pandas DataFrame.  All non-complex data types are currently supported, otherwise an `UnsupportedOperation` exception is thrown.

Additions to Spark include a Scala package private method `Dataset.toArrowPayloadBytes` that will convert data partitions in the executor JVM to `ArrowPayload`s as byte arrays so they can be easily served.  A package private class/object `ArrowConverters` that provide data type mappings and conversion routines.  In Python, a public method `DataFrame.collectAsArrow` is added to collect Arrow payloads and an optional flag in `toPandas(useArrow=False)` to enable using Arrow (uses the old conversion by default).

## How was this patch tested?
Added a new test suite `ArrowConvertersSuite` that will run tests on conversion of Datasets to Arrow payloads for supported types.  The suite will generate a Dataset and matching Arrow JSON data, then the dataset is converted to an Arrow payload and finally validated against the JSON data.  This will ensure that the schema and data has been converted correctly.

Added PySpark tests to verify the `toPandas` method is producing equal DataFrames with and without pyarrow.  A roundtrip test to ensure the pandas DataFrame produced by pyspark is equal to a one made directly with pandas.

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Li Jin <ice.xelloss@gmail.com>
Author: Li Jin <li.jin@twosigma.com>
Author: Wes McKinney <wes.mckinney@twosigma.com>

Closes #15821 from BryanCutler/wip-toPandas_with_arrow-SPARK-13534.
2017-06-23 09:01:13 +08:00
Lubo Zhang e55a105ae0 [SPARK-20599][SS] ConsoleSink should work with (batch)
## What changes were proposed in this pull request?

Currently, if we read a batch and want to display it on the console sink, it will lead a runtime exception.

Changes:

- In this PR, we add a match rule to check whether it is a ConsoleSinkProvider, we will display the Dataset
 if using console format.

## How was this patch tested?

spark.read.schema().json(path).write.format("console").save

Author: Lubo Zhang <lubo.zhang@intel.com>
Author: lubozhan <lubo.zhang@intel.com>

Closes #18347 from lubozhan/dev.
2017-06-22 11:18:58 -07:00
actuaryzhang 97b307c87c [SQL][DOC] Fix documentation of lpad
## What changes were proposed in this pull request?
Fix incomplete documentation for `lpad`.

Author: actuaryzhang <actuaryzhang10@gmail.com>

Closes #18367 from actuaryzhang/SQLDoc.
2017-06-22 10:12:33 +01:00
Shixiong Zhu d66b143eec [SPARK-21167][SS] Decode the path generated by File sink to handle special characters
## What changes were proposed in this pull request?

Decode the path generated by File sink to handle special characters.

## How was this patch tested?

The added unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18381 from zsxwing/SPARK-21167.
2017-06-21 23:43:21 -07:00
hyukjinkwon 7a00c658d4 [SPARK-21147][SS] Throws an analysis exception when a user-specified schema is given in socket/rate sources
## What changes were proposed in this pull request?

This PR proposes to throw an exception if a schema is provided by user to socket source as below:

**socket source**

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

val userSpecifiedSchema = StructType(
  StructField("name", StringType) ::
  StructField("area", StringType) :: Nil)
val df = spark.readStream.format("socket").option("host", "localhost").option("port", 9999).schema(userSpecifiedSchema).load
df.printSchema
```

Before

```
root
 |-- value: string (nullable = true)
```

After

```
org.apache.spark.sql.AnalysisException: The socket source does not support a user-specified schema.;
  at org.apache.spark.sql.execution.streaming.TextSocketSourceProvider.sourceSchema(socket.scala:199)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:192)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:87)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:87)
  at org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:30)
  at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:150)
  ... 50 elided
```

**rate source**

```scala
spark.readStream.format("rate").schema(spark.range(1).schema).load().printSchema()
```

Before

```
root
 |-- timestamp: timestamp (nullable = true)
 |-- value: long (nullable = true)`
```

After

```
org.apache.spark.sql.AnalysisException: The rate source does not support a user-specified schema.;
  at org.apache.spark.sql.execution.streaming.RateSourceProvider.sourceSchema(RateSourceProvider.scala:57)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceSchema(DataSource.scala:192)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo$lzycompute(DataSource.scala:87)
  at org.apache.spark.sql.execution.datasources.DataSource.sourceInfo(DataSource.scala:87)
  at org.apache.spark.sql.execution.streaming.StreamingRelation$.apply(StreamingRelation.scala:30)
  at org.apache.spark.sql.streaming.DataStreamReader.load(DataStreamReader.scala:150)
  ... 48 elided
```

## How was this patch tested?

Unit test in `TextSocketStreamSuite` and `RateSourceSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18365 from HyukjinKwon/SPARK-21147.
2017-06-21 10:51:17 -07:00
Xingbo Jiang cad88f17e8 [SPARK-17851][SQL][TESTS] Make sure all test sqls in catalyst pass checkAnalysis
## What changes were proposed in this pull request?

Currently we have several tens of test sqls in catalyst will fail at `SimpleAnalyzer.checkAnalysis`, we should make sure they are valid.

This PR makes the following changes:
1. Apply `checkAnalysis` on plans that tests `Optimizer` rules, but don't require the testcases for `Parser`/`Analyzer` pass `checkAnalysis`;
2. Fix testcases for `Optimizer` that would have fall.
## How was this patch tested?

Apply `SimpleAnalyzer.checkAnalysis` on plans in `PlanTest.comparePlans`, update invalid test cases.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>
Author: jiangxingbo <jiangxb1987@gmail.com>

Closes #15417 from jiangxb1987/cptest.
2017-06-21 09:40:06 -07:00
Marcos P e92befcb4b [MINOR][DOC] modified issue link and updated status
## What changes were proposed in this pull request?

This PR aims to clarify some outdated comments that i found at **spark-catalyst** and **spark-sql** pom files. Maven bug still happening and in order to track it I have updated the issue link and also the status of the issue.

Author: Marcos P <mpenate@stratio.com>

Closes #18374 from mpenate/fix/mng-3559-comment.
2017-06-21 15:34:10 +01:00
sureshthalamati 9ce714dca2 [SPARK-10655][SQL] Adding additional data type mappings to jdbc DB2dialect.
This patch adds DB2 specific data type mappings for decfloat, real, xml , and timestamp with time zone (DB2Z specific type)  types on read and for byte, short data types  on write to the to jdbc data source DB2 dialect. Default mapping does not work for these types when reading/writing from DB2 database.

Added docker test, and a JDBC unit test case.

Author: sureshthalamati <suresh.thalamati@gmail.com>

Closes #9162 from sureshthalamati/db2dialect_enhancements-spark-10655.
2017-06-20 22:35:42 -07:00
Wenchen Fan e862dc9049 [SPARK-21150][SQL] Persistent view stored in Hive metastore should be case preserving
## What changes were proposed in this pull request?

This is a regression in Spark 2.2. In Spark 2.2, we introduced a new way to resolve persisted view: https://issues.apache.org/jira/browse/SPARK-18209 , but this makes the persisted view non case-preserving because we store the schema in hive metastore directly. We should follow data source table and store schema in table properties.

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18360 from cloud-fan/view.
2017-06-20 09:15:33 -07:00
Xianyang Liu 0a4b7e4f81 [MINOR] Fix some typo of the document
## What changes were proposed in this pull request?

Fix some typo of the document.

## How was this patch tested?

Existing tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #18350 from ConeyLiu/fixtypo.
2017-06-19 20:35:58 +01:00
Yong Tang e5387018e7 [SPARK-19975][PYTHON][SQL] Add map_keys and map_values functions to Python
## What changes were proposed in this pull request?

This fix tries to address the issue in SPARK-19975 where we
have `map_keys` and `map_values` functions in SQL yet there
is no Python equivalent functions.

This fix adds `map_keys` and `map_values` functions to Python.

## How was this patch tested?

This fix is tested manually (See Python docs for examples).

Author: Yong Tang <yong.tang.github@outlook.com>

Closes #17328 from yongtang/SPARK-19975.
2017-06-19 11:40:07 -07:00
Yuming Wang f913f158ec [SPARK-20948][SQL] Built-in SQL Function UnaryMinus/UnaryPositive support string type
## What changes were proposed in this pull request?

Built-in SQL Function UnaryMinus/UnaryPositive support string type, if it's string type, convert it to double type, after this PR:
```sql
spark-sql> select positive('-1.11'), negative('-1.11');
-1.11   1.11
spark-sql>
```

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18173 from wangyum/SPARK-20948.
2017-06-18 20:14:05 -07:00
Yuming Wang ce49428ef7 [SPARK-20749][SQL][FOLLOWUP] Support character_length
## What changes were proposed in this pull request?

The function `char_length` is shorthand for `character_length` function. Both Hive and Postgresql support `character_length`,  This PR add support for `character_length`.

Ref:
https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-StringFunctions
https://www.postgresql.org/docs/current/static/functions-string.html

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18330 from wangyum/SPARK-20749-character_length.
2017-06-18 18:56:53 -07:00
Yuming Wang 53e48f73e4 [SPARK-20931][SQL] ABS function support string type.
## What changes were proposed in this pull request?

ABS function support string type. Hive/MySQL support this feature.

Ref: 4ba713ccd8/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDFAbs.java (L93)

## How was this patch tested?
 unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18153 from wangyum/SPARK-20931.
2017-06-16 09:40:58 -07:00
Wenchen Fan d1c333ac77 [SPARK-21119][SQL] unset table properties should keep the table comment
## What changes were proposed in this pull request?

Previous code mistakenly use `table.properties.get("comment")` to read the existing table comment, we should use `table.comment`

## How was this patch tested?

new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18325 from cloud-fan/unset.
2017-06-16 08:05:43 -07:00
Kazuaki Ishizaki 7a3e5dc28b [SPARK-20749][SQL] Built-in SQL Function Support - all variants of LEN[GTH]
## What changes were proposed in this pull request?

This PR adds built-in SQL function `BIT_LENGTH()`, `CHAR_LENGTH()`, and `OCTET_LENGTH()` functions.

`BIT_LENGTH()` returns the bit length of the given string or binary expression.
`CHAR_LENGTH()` returns the length of the given string or binary expression. (i.e. equal to `LENGTH()`)
`OCTET_LENGTH()` returns the byte length of the given string or binary expression.

## How was this patch tested?

Added new test suites for these three functions

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

Closes #18046 from kiszk/SPARK-20749.
2017-06-15 23:06:58 -07:00
Xiao Li 5d35d5c15c [SPARK-21112][SQL] ALTER TABLE SET TBLPROPERTIES should not overwrite COMMENT
### What changes were proposed in this pull request?
`ALTER TABLE SET TBLPROPERTIES` should not overwrite `COMMENT` even if the input property does not have the property of `COMMENT`. This PR is to fix the issue.

### How was this patch tested?
Covered by the existing tests.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18318 from gatorsmile/fixTableComment.
2017-06-16 10:11:23 +08:00
ALeksander Eskilson b32b2123dd [SPARK-18016][SQL][CATALYST] Code Generation: Constant Pool Limit - Class Splitting
## What changes were proposed in this pull request?

This pull-request exclusively includes the class splitting feature described in #16648. When code for a given class would grow beyond 1600k bytes, a private, nested sub-class is generated into which subsequent functions are inlined. Additional sub-classes are generated as the code threshold is met subsequent times. This code includes 3 changes:

1. Includes helper maps, lists, and functions for keeping track of sub-classes during code generation (included in the `CodeGenerator` class). These helper functions allow nested classes and split functions to be initialized/declared/inlined to the appropriate locations in the various projection classes.
2. Changes `addNewFunction` to return a string to support instances where a split function is inlined to a nested class and not the outer class (and so must be invoked using the class-qualified name). Uses of `addNewFunction` throughout the codebase are modified so that the returned name is properly used.
3. Removes instances of the `this` keyword when used on data inside generated classes. All state declared in the outer class is by default global and accessible to the nested classes. However, if a reference to global state in a nested class is prepended with the `this` keyword, it would attempt to reference state belonging to the nested class (which would not exist), rather than the correct variable belonging to the outer class.

## How was this patch tested?

Added a test case to the `GeneratedProjectionSuite` that increases the number of columns tested in various projections to a threshold that would previously have triggered a `JaninoRuntimeException` for the Constant Pool.

Note: This PR does not address the second Constant Pool issue with code generation (also mentioned in #16648): excess global mutable state. A second PR may be opened to resolve that issue.

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

Closes #18075 from bdrillard/class_splitting_only.
2017-06-15 13:45:08 +08:00
Xiao Li 2051428173 [SPARK-20980][SQL] Rename wholeFile to multiLine for both CSV and JSON
### What changes were proposed in this pull request?
The current option name `wholeFile` is misleading for CSV users. Currently, it is not representing a record per file. Actually, one file could have multiple records. Thus, we should rename it. Now, the proposal is `multiLine`.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18202 from gatorsmile/renameCVSOption.
2017-06-15 13:18:19 +08:00
Reynold Xin fffeb6d7c3 [SPARK-21092][SQL] Wire SQLConf in logical plan and expressions
## What changes were proposed in this pull request?
It is really painful to not have configs in logical plan and expressions. We had to add all sorts of hacks (e.g. pass SQLConf explicitly in functions). This patch exposes SQLConf in logical plan, using a thread local variable and a getter closure that's set once there is an active SparkSession.

The implementation is a bit of a hack, since we didn't anticipate this need in the beginning (config was only exposed in physical plan). The implementation is described in `SQLConf.get`.

In terms of future work, we should follow up to clean up CBO (remove the need for passing in config).

## How was this patch tested?
Updated relevant tests for constraint propagation.

Author: Reynold Xin <rxin@databricks.com>

Closes #18299 from rxin/SPARK-21092.
2017-06-14 22:11:41 -07:00
Xiao Li 77a2fc5b52 Revert "[SPARK-20941][SQL] Fix SubqueryExec Reuse"
This reverts commit f7cf2096fd.
2017-06-14 11:48:32 -07:00
Xiao Li df766a4714 [SPARK-21089][SQL] Fix DESC EXTENDED/FORMATTED to Show Table Properties
### What changes were proposed in this pull request?

Since both table properties and storage properties share the same key values, table properties are not shown in the output of DESC EXTENDED/FORMATTED when the storage properties are not empty.

This PR is to fix the above issue by renaming them to different keys.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18294 from gatorsmile/tableProperties.
2017-06-14 11:13:16 -07:00
Yuming Wang 4d01aa4648 [SPARK-20754][SQL][FOLLOWUP] Add Function Alias For MOD/POSITION.
## What changes were proposed in this pull request?

https://github.com/apache/spark/pull/18106 Support TRUNC (number),  We should also add function alias for `MOD `and `POSITION`.

`POSITION(substr IN str) `is a synonym for `LOCATE(substr,str)`. same as MySQL: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_position

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18206 from wangyum/SPARK-20754-mod&position.
2017-06-13 23:39:06 -07:00
lianhuiwang 8b5b2e272f [SPARK-20986][SQL] Reset table's statistics after PruneFileSourcePartitions rule.
## What changes were proposed in this pull request?
After PruneFileSourcePartitions rule, It needs reset table's statistics because PruneFileSourcePartitions can filter some unnecessary partitions. So the statistics need to be changed.

## How was this patch tested?
add unit test.

Author: lianhuiwang <lianhuiwang09@gmail.com>

Closes #18205 from lianhuiwang/SPARK-20986.
2017-06-14 09:57:56 +08:00
liuxian 7ba8bf288d [SPARK-21016][CORE] Improve code fault tolerance for converting string to number
## What changes were proposed in this pull request?
When converting `string` to `number`(int, long or double),  if the string has a space before or after,will lead to unnecessary mistakes.

## How was this patch tested?
unit test

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18238 from 10110346/lx-wip-0608.
2017-06-13 10:12:28 -07:00
Liang-Chi Hsieh bcf3643f94 [SPARK-21051][SQL] Add hash map metrics to aggregate
## What changes were proposed in this pull request?

This adds the average hash map probe metrics to hash aggregate.

`BytesToBytesMap` already has API to get the metrics, this PR adds an API to `UnsafeFixedWidthAggregationMap` to access it.

Preparing a test for this metrics seems tricky, because we don't know what collision keys are. For now, the test case generates random data large enough to have desired probe.

TODO in later PR: add hash map metrics to join.

## How was this patch tested?

Added test to SQLMetricsSuite.

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

Closes #18258 from viirya/SPARK-20953.
2017-06-13 10:10:35 -07:00
Rishabh Bhardwaj 9b2c877bec [SPARK-21039][SPARK CORE] Use treeAggregate instead of aggregate in DataFrame.stat.bloomFilter
## What changes were proposed in this pull request?
To use treeAggregate instead of aggregate in DataFrame.stat.bloomFilter to parallelize the operation of merging the bloom filters
(Please fill in changes proposed in this fix)

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

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Rishabh Bhardwaj <rbnext29@gmail.com>
Author: Rishabh Bhardwaj <admin@rishabh.local>
Author: Rishabh Bhardwaj <r0b00ko@rishabh.Dlink>
Author: Rishabh Bhardwaj <admin@Admins-MacBook-Pro.local>
Author: Rishabh Bhardwaj <r0b00ko@rishabh.local>

Closes #18263 from rishabhbhardwaj/SPARK-21039.
2017-06-13 15:09:12 +01:00
Sean Owen 7b7c85ede3 [SPARK-20920][SQL] ForkJoinPool pools are leaked when writing hive tables with many partitions
## What changes were proposed in this pull request?

Don't leave thread pool running from AlterTableRecoverPartitionsCommand DDL command

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #18216 from srowen/SPARK-20920.
2017-06-13 10:48:07 +01:00
Dongjoon Hyun 2639c3ed03 [SPARK-19910][SQL] stack should not reject NULL values due to type mismatch
## What changes were proposed in this pull request?

Since `stack` function generates a table with nullable columns, it should allow mixed null values.

```scala
scala> sql("select stack(3, 1, 2, 3)").printSchema
root
 |-- col0: integer (nullable = true)

scala> sql("select stack(3, 1, 2, null)").printSchema
org.apache.spark.sql.AnalysisException: cannot resolve 'stack(3, 1, 2, NULL)' due to data type mismatch: Argument 1 (IntegerType) != Argument 3 (NullType); line 1 pos 7;
```

## How was this patch tested?

Pass the Jenkins with a new test case.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17251 from dongjoon-hyun/SPARK-19910.
2017-06-12 21:18:43 -07:00
Wenchen Fan fc0e6944a5 Revert "[SPARK-21046][SQL] simplify the array offset and length in ColumnVector"
This reverts commit 22dd65f58e.
2017-06-13 09:15:14 +08:00
Shixiong Zhu 74a432d3a3 [SPARK-20979][SS] Add RateSource to generate values for tests and benchmark
## What changes were proposed in this pull request?

This PR adds RateSource for Structured Streaming so that the user can use it to generate data for tests and benchmark easily.

This source generates increment long values with timestamps. Each generated row has two columns: a timestamp column for the generated time and an auto increment long column starting with 0L.

It supports the following options:
- `rowsPerSecond` (e.g. 100, default: 1): How many rows should be generated per second.
- `rampUpTime` (e.g. 5s, default: 0s): How long to ramp up before the generating speed becomes `rowsPerSecond`. Using finer granularities than seconds will be truncated to integer seconds.
- `numPartitions` (e.g. 10, default: Spark's default parallelism): The partition number for the generated rows. The source will try its best to reach `rowsPerSecond`, but the query may be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed.

Here is a simple example that prints 10 rows per seconds:
```
    spark.readStream
      .format("rate")
      .option("rowsPerSecond", "10")
      .load()
      .writeStream
      .format("console")
      .start()
```

The idea came from marmbrus and he did the initial work.

## How was this patch tested?

The added tests.

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Michael Armbrust <michael@databricks.com>

Closes #18199 from zsxwing/rate.
2017-06-12 14:58:08 -07:00
Wenchen Fan 22dd65f58e [SPARK-21046][SQL] simplify the array offset and length in ColumnVector
## What changes were proposed in this pull request?

Currently when a `ColumnVector` stores array type elements, we will use 2 arrays for lengths and offsets and implement them individually in on-heap and off-heap column vector.

In this PR, we use one array to represent both offsets and lengths, so that we can treat it as `ColumnVector` and all the logic can go to the base class `ColumnVector`

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18260 from cloud-fan/put.
2017-06-13 00:12:34 +08:00
Dongjoon Hyun a92e095e70 [SPARK-21041][SQL] SparkSession.range should be consistent with SparkContext.range
## What changes were proposed in this pull request?

This PR fixes the inconsistency in `SparkSession.range`.

**BEFORE**
```scala
scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
res2: Array[Long] = Array(9223372036854775804, 9223372036854775805, 9223372036854775806)
```

**AFTER**
```scala
scala> spark.range(java.lang.Long.MAX_VALUE - 3, java.lang.Long.MIN_VALUE + 2, 1).collect
res2: Array[Long] = Array()
```

## How was this patch tested?

Pass the Jenkins with newly added test cases.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18257 from dongjoon-hyun/SPARK-21041.
2017-06-12 20:58:27 +08:00
Michal Senkyr f48273c13c [SPARK-18891][SQL] Support for specific Java List subtypes
## What changes were proposed in this pull request?

Add support for specific Java `List` subtypes in deserialization as well as a generic implicit encoder.

All `List` subtypes are supported by using either the size-specifying constructor (one `int` parameter) or the default constructor.

Interfaces/abstract classes use the following implementations:

* `java.util.List`, `java.util.AbstractList` or `java.util.AbstractSequentialList` => `java.util.ArrayList`

## How was this patch tested?

```bash
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> val jlist = new java.util.LinkedList[Int]; jlist.add(1)
jlist: java.util.LinkedList[Int] = [1]
res0: Boolean = true

scala> Seq(jlist).toDS().map(_.element()).collect()
res1: Array[Int] = Array(1)
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #18009 from michalsenkyr/dataset-java-lists.
2017-06-12 08:53:23 +08:00
Michal Senkyr 0538f3b0ae [SPARK-18891][SQL] Support for Scala Map collection types
## What changes were proposed in this pull request?

Add support for arbitrary Scala `Map` types in deserialization as well as a generic implicit encoder.

Used the builder approach as in #16541 to construct any provided `Map` type upon deserialization.

Please note that this PR also adds (ignored) tests for issue [SPARK-19104 CompileException with Map and Case Class in Spark 2.1.0](https://issues.apache.org/jira/browse/SPARK-19104) but doesn't solve it.

Added support for Java Maps in codegen code (encoders will be added in a different PR) with the following default implementations for interfaces/abstract classes:

* `java.util.Map`, `java.util.AbstractMap` => `java.util.HashMap`
* `java.util.SortedMap`, `java.util.NavigableMap` => `java.util.TreeMap`
* `java.util.concurrent.ConcurrentMap` => `java.util.concurrent.ConcurrentHashMap`
* `java.util.concurrent.ConcurrentNavigableMap` => `java.util.concurrent.ConcurrentSkipListMap`

Resulting codegen for `Seq(Map(1 -> 2)).toDS().map(identity).queryExecution.debug.codegen`:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
/* 010 */   private int CollectObjectsToMap_loopValue0;
/* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
/* 012 */   private int CollectObjectsToMap_loopValue2;
/* 013 */   private UnsafeRow deserializetoobject_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 016 */   private scala.collection.immutable.Map mapelements_argValue;
/* 017 */   private UnsafeRow mapelements_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 020 */   private UnsafeRow serializefromobject_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 031 */     partitionIndex = index;
/* 032 */     this.inputs = inputs;
/* 033 */     wholestagecodegen_init_0();
/* 034 */     wholestagecodegen_init_1();
/* 035 */
/* 036 */   }
/* 037 */
/* 038 */   private void wholestagecodegen_init_0() {
/* 039 */     inputadapter_input = inputs[0];
/* 040 */
/* 041 */     deserializetoobject_result = new UnsafeRow(1);
/* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 044 */
/* 045 */     mapelements_result = new UnsafeRow(1);
/* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 048 */     serializefromobject_result = new UnsafeRow(1);
/* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 052 */
/* 053 */   }
/* 054 */
/* 055 */   private void wholestagecodegen_init_1() {
/* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 057 */
/* 058 */   }
/* 059 */
/* 060 */   protected void processNext() throws java.io.IOException {
/* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
/* 065 */
/* 066 */       boolean deserializetoobject_isNull1 = true;
/* 067 */       ArrayData deserializetoobject_value1 = null;
/* 068 */       if (!inputadapter_isNull) {
/* 069 */         deserializetoobject_isNull1 = false;
/* 070 */         if (!deserializetoobject_isNull1) {
/* 071 */           Object deserializetoobject_funcResult = null;
/* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
/* 073 */           if (deserializetoobject_funcResult == null) {
/* 074 */             deserializetoobject_isNull1 = true;
/* 075 */           } else {
/* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
/* 077 */           }
/* 078 */
/* 079 */         }
/* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 081 */       }
/* 082 */
/* 083 */       boolean deserializetoobject_isNull3 = true;
/* 084 */       ArrayData deserializetoobject_value3 = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull3 = false;
/* 087 */         if (!deserializetoobject_isNull3) {
/* 088 */           Object deserializetoobject_funcResult1 = null;
/* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
/* 090 */           if (deserializetoobject_funcResult1 == null) {
/* 091 */             deserializetoobject_isNull3 = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
/* 098 */       }
/* 099 */       scala.collection.immutable.Map deserializetoobject_value = null;
/* 100 */
/* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
/* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
/* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
/* 104 */       }
/* 105 */
/* 106 */       if (!deserializetoobject_isNull1) {
/* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
/* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
/* 109 */         }
/* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
/* 111 */
/* 112 */         scala.collection.mutable.Builder CollectObjectsToMap_builderValue5 = scala.collection.immutable.Map$.MODULE$.newBuilder();
/* 113 */         CollectObjectsToMap_builderValue5.sizeHint(deserializetoobject_dataLength);
/* 114 */
/* 115 */         int deserializetoobject_loopIndex = 0;
/* 116 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 117 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
/* 118 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
/* 119 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
/* 120 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
/* 121 */
/* 122 */           if (CollectObjectsToMap_loopIsNull1) {
/* 123 */             throw new RuntimeException("Found null in map key!");
/* 124 */           }
/* 125 */
/* 126 */           scala.Tuple2 CollectObjectsToMap_loopValue4;
/* 127 */
/* 128 */           if (CollectObjectsToMap_loopIsNull3) {
/* 129 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, null);
/* 130 */           } else {
/* 131 */             CollectObjectsToMap_loopValue4 = new scala.Tuple2(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
/* 132 */           }
/* 133 */
/* 134 */           CollectObjectsToMap_builderValue5.$plus$eq(CollectObjectsToMap_loopValue4);
/* 135 */
/* 136 */           deserializetoobject_loopIndex += 1;
/* 137 */         }
/* 138 */
/* 139 */         deserializetoobject_value = (scala.collection.immutable.Map) CollectObjectsToMap_builderValue5.result();
/* 140 */       }
/* 141 */
/* 142 */       boolean mapelements_isNull = true;
/* 143 */       scala.collection.immutable.Map mapelements_value = null;
/* 144 */       if (!false) {
/* 145 */         mapelements_argValue = deserializetoobject_value;
/* 146 */
/* 147 */         mapelements_isNull = false;
/* 148 */         if (!mapelements_isNull) {
/* 149 */           Object mapelements_funcResult = null;
/* 150 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 151 */           if (mapelements_funcResult == null) {
/* 152 */             mapelements_isNull = true;
/* 153 */           } else {
/* 154 */             mapelements_value = (scala.collection.immutable.Map) mapelements_funcResult;
/* 155 */           }
/* 156 */
/* 157 */         }
/* 158 */         mapelements_isNull = mapelements_value == null;
/* 159 */       }
/* 160 */
/* 161 */       MapData serializefromobject_value = null;
/* 162 */       if (!mapelements_isNull) {
/* 163 */         final int serializefromobject_length = mapelements_value.size();
/* 164 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
/* 165 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
/* 166 */         int serializefromobject_index = 0;
/* 167 */         final scala.collection.Iterator serializefromobject_entries = mapelements_value.iterator();
/* 168 */         while(serializefromobject_entries.hasNext()) {
/* 169 */           final scala.Tuple2 serializefromobject_entry = (scala.Tuple2) serializefromobject_entries.next();
/* 170 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry._1();
/* 171 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry._2();
/* 172 */
/* 173 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
/* 174 */
/* 175 */           if (false) {
/* 176 */             throw new RuntimeException("Cannot use null as map key!");
/* 177 */           } else {
/* 178 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
/* 179 */           }
/* 180 */
/* 181 */           if (false) {
/* 182 */             serializefromobject_convertedValues[serializefromobject_index] = null;
/* 183 */           } else {
/* 184 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
/* 185 */           }
/* 186 */
/* 187 */           serializefromobject_index++;
/* 188 */         }
/* 189 */
/* 190 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
/* 191 */       }
/* 192 */       serializefromobject_holder.reset();
/* 193 */
/* 194 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 195 */
/* 196 */       if (mapelements_isNull) {
/* 197 */         serializefromobject_rowWriter.setNullAt(0);
/* 198 */       } else {
/* 199 */         // Remember the current cursor so that we can calculate how many bytes are
/* 200 */         // written later.
/* 201 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 202 */
/* 203 */         if (serializefromobject_value instanceof UnsafeMapData) {
/* 204 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
/* 205 */           // grow the global buffer before writing data.
/* 206 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 207 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 208 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 209 */
/* 210 */         } else {
/* 211 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
/* 212 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
/* 213 */
/* 214 */           // preserve 8 bytes to write the key array numBytes later.
/* 215 */           serializefromobject_holder.grow(8);
/* 216 */           serializefromobject_holder.cursor += 8;
/* 217 */
/* 218 */           // Remember the current cursor so that we can write numBytes of key array later.
/* 219 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
/* 220 */
/* 221 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
/* 222 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
/* 223 */             // grow the global buffer before writing data.
/* 224 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
/* 225 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 226 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
/* 227 */
/* 228 */           } else {
/* 229 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
/* 230 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 231 */
/* 232 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
/* 233 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
/* 234 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
/* 235 */               } else {
/* 236 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
/* 237 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
/* 238 */               }
/* 239 */             }
/* 240 */           }
/* 241 */
/* 242 */           // Write the numBytes of key array into the first 8 bytes.
/* 243 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
/* 244 */
/* 245 */           if (serializefromobject_values instanceof UnsafeArrayData) {
/* 246 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
/* 247 */             // grow the global buffer before writing data.
/* 248 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
/* 249 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 250 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
/* 251 */
/* 252 */           } else {
/* 253 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
/* 254 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
/* 255 */
/* 256 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
/* 257 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
/* 258 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
/* 259 */               } else {
/* 260 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
/* 261 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
/* 262 */               }
/* 263 */             }
/* 264 */           }
/* 265 */
/* 266 */         }
/* 267 */
/* 268 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 269 */       }
/* 270 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 271 */       append(serializefromobject_result);
/* 272 */       if (shouldStop()) return;
/* 273 */     }
/* 274 */   }
/* 275 */ }
```

Codegen for `java.util.Map`:

```
/* 001 */ public Object generate(Object[] references) {
/* 002 */   return new GeneratedIterator(references);
/* 003 */ }
/* 004 */
/* 005 */ final class GeneratedIterator extends org.apache.spark.sql.execution.BufferedRowIterator {
/* 006 */   private Object[] references;
/* 007 */   private scala.collection.Iterator[] inputs;
/* 008 */   private scala.collection.Iterator inputadapter_input;
/* 009 */   private boolean CollectObjectsToMap_loopIsNull1;
/* 010 */   private int CollectObjectsToMap_loopValue0;
/* 011 */   private boolean CollectObjectsToMap_loopIsNull3;
/* 012 */   private int CollectObjectsToMap_loopValue2;
/* 013 */   private UnsafeRow deserializetoobject_result;
/* 014 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder deserializetoobject_holder;
/* 015 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter deserializetoobject_rowWriter;
/* 016 */   private java.util.HashMap mapelements_argValue;
/* 017 */   private UnsafeRow mapelements_result;
/* 018 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder mapelements_holder;
/* 019 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter mapelements_rowWriter;
/* 020 */   private UnsafeRow serializefromobject_result;
/* 021 */   private org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder serializefromobject_holder;
/* 022 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter serializefromobject_rowWriter;
/* 023 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter;
/* 024 */   private org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter serializefromobject_arrayWriter1;
/* 025 */
/* 026 */   public GeneratedIterator(Object[] references) {
/* 027 */     this.references = references;
/* 028 */   }
/* 029 */
/* 030 */   public void init(int index, scala.collection.Iterator[] inputs) {
/* 031 */     partitionIndex = index;
/* 032 */     this.inputs = inputs;
/* 033 */     wholestagecodegen_init_0();
/* 034 */     wholestagecodegen_init_1();
/* 035 */
/* 036 */   }
/* 037 */
/* 038 */   private void wholestagecodegen_init_0() {
/* 039 */     inputadapter_input = inputs[0];
/* 040 */
/* 041 */     deserializetoobject_result = new UnsafeRow(1);
/* 042 */     this.deserializetoobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(deserializetoobject_result, 32);
/* 043 */     this.deserializetoobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(deserializetoobject_holder, 1);
/* 044 */
/* 045 */     mapelements_result = new UnsafeRow(1);
/* 046 */     this.mapelements_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(mapelements_result, 32);
/* 047 */     this.mapelements_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(mapelements_holder, 1);
/* 048 */     serializefromobject_result = new UnsafeRow(1);
/* 049 */     this.serializefromobject_holder = new org.apache.spark.sql.catalyst.expressions.codegen.BufferHolder(serializefromobject_result, 32);
/* 050 */     this.serializefromobject_rowWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter(serializefromobject_holder, 1);
/* 051 */     this.serializefromobject_arrayWriter = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 052 */
/* 053 */   }
/* 054 */
/* 055 */   private void wholestagecodegen_init_1() {
/* 056 */     this.serializefromobject_arrayWriter1 = new org.apache.spark.sql.catalyst.expressions.codegen.UnsafeArrayWriter();
/* 057 */
/* 058 */   }
/* 059 */
/* 060 */   protected void processNext() throws java.io.IOException {
/* 061 */     while (inputadapter_input.hasNext() && !stopEarly()) {
/* 062 */       InternalRow inputadapter_row = (InternalRow) inputadapter_input.next();
/* 063 */       boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 064 */       MapData inputadapter_value = inputadapter_isNull ? null : (inputadapter_row.getMap(0));
/* 065 */
/* 066 */       boolean deserializetoobject_isNull1 = true;
/* 067 */       ArrayData deserializetoobject_value1 = null;
/* 068 */       if (!inputadapter_isNull) {
/* 069 */         deserializetoobject_isNull1 = false;
/* 070 */         if (!deserializetoobject_isNull1) {
/* 071 */           Object deserializetoobject_funcResult = null;
/* 072 */           deserializetoobject_funcResult = inputadapter_value.keyArray();
/* 073 */           if (deserializetoobject_funcResult == null) {
/* 074 */             deserializetoobject_isNull1 = true;
/* 075 */           } else {
/* 076 */             deserializetoobject_value1 = (ArrayData) deserializetoobject_funcResult;
/* 077 */           }
/* 078 */
/* 079 */         }
/* 080 */         deserializetoobject_isNull1 = deserializetoobject_value1 == null;
/* 081 */       }
/* 082 */
/* 083 */       boolean deserializetoobject_isNull3 = true;
/* 084 */       ArrayData deserializetoobject_value3 = null;
/* 085 */       if (!inputadapter_isNull) {
/* 086 */         deserializetoobject_isNull3 = false;
/* 087 */         if (!deserializetoobject_isNull3) {
/* 088 */           Object deserializetoobject_funcResult1 = null;
/* 089 */           deserializetoobject_funcResult1 = inputadapter_value.valueArray();
/* 090 */           if (deserializetoobject_funcResult1 == null) {
/* 091 */             deserializetoobject_isNull3 = true;
/* 092 */           } else {
/* 093 */             deserializetoobject_value3 = (ArrayData) deserializetoobject_funcResult1;
/* 094 */           }
/* 095 */
/* 096 */         }
/* 097 */         deserializetoobject_isNull3 = deserializetoobject_value3 == null;
/* 098 */       }
/* 099 */       java.util.HashMap deserializetoobject_value = null;
/* 100 */
/* 101 */       if ((deserializetoobject_isNull1 && !deserializetoobject_isNull3) ||
/* 102 */         (!deserializetoobject_isNull1 && deserializetoobject_isNull3)) {
/* 103 */         throw new RuntimeException("Invalid state: Inconsistent nullability of key-value");
/* 104 */       }
/* 105 */
/* 106 */       if (!deserializetoobject_isNull1) {
/* 107 */         if (deserializetoobject_value1.numElements() != deserializetoobject_value3.numElements()) {
/* 108 */           throw new RuntimeException("Invalid state: Inconsistent lengths of key-value arrays");
/* 109 */         }
/* 110 */         int deserializetoobject_dataLength = deserializetoobject_value1.numElements();
/* 111 */         java.util.Map CollectObjectsToMap_builderValue5 = new java.util.HashMap(deserializetoobject_dataLength);
/* 112 */
/* 113 */         int deserializetoobject_loopIndex = 0;
/* 114 */         while (deserializetoobject_loopIndex < deserializetoobject_dataLength) {
/* 115 */           CollectObjectsToMap_loopValue0 = (int) (deserializetoobject_value1.getInt(deserializetoobject_loopIndex));
/* 116 */           CollectObjectsToMap_loopValue2 = (int) (deserializetoobject_value3.getInt(deserializetoobject_loopIndex));
/* 117 */           CollectObjectsToMap_loopIsNull1 = deserializetoobject_value1.isNullAt(deserializetoobject_loopIndex);
/* 118 */           CollectObjectsToMap_loopIsNull3 = deserializetoobject_value3.isNullAt(deserializetoobject_loopIndex);
/* 119 */
/* 120 */           if (CollectObjectsToMap_loopIsNull1) {
/* 121 */             throw new RuntimeException("Found null in map key!");
/* 122 */           }
/* 123 */
/* 124 */           CollectObjectsToMap_builderValue5.put(CollectObjectsToMap_loopValue0, CollectObjectsToMap_loopValue2);
/* 125 */
/* 126 */           deserializetoobject_loopIndex += 1;
/* 127 */         }
/* 128 */
/* 129 */         deserializetoobject_value = (java.util.HashMap) CollectObjectsToMap_builderValue5;
/* 130 */       }
/* 131 */
/* 132 */       boolean mapelements_isNull = true;
/* 133 */       java.util.HashMap mapelements_value = null;
/* 134 */       if (!false) {
/* 135 */         mapelements_argValue = deserializetoobject_value;
/* 136 */
/* 137 */         mapelements_isNull = false;
/* 138 */         if (!mapelements_isNull) {
/* 139 */           Object mapelements_funcResult = null;
/* 140 */           mapelements_funcResult = ((scala.Function1) references[0]).apply(mapelements_argValue);
/* 141 */           if (mapelements_funcResult == null) {
/* 142 */             mapelements_isNull = true;
/* 143 */           } else {
/* 144 */             mapelements_value = (java.util.HashMap) mapelements_funcResult;
/* 145 */           }
/* 146 */
/* 147 */         }
/* 148 */         mapelements_isNull = mapelements_value == null;
/* 149 */       }
/* 150 */
/* 151 */       MapData serializefromobject_value = null;
/* 152 */       if (!mapelements_isNull) {
/* 153 */         final int serializefromobject_length = mapelements_value.size();
/* 154 */         final Object[] serializefromobject_convertedKeys = new Object[serializefromobject_length];
/* 155 */         final Object[] serializefromobject_convertedValues = new Object[serializefromobject_length];
/* 156 */         int serializefromobject_index = 0;
/* 157 */         final java.util.Iterator serializefromobject_entries = mapelements_value.entrySet().iterator();
/* 158 */         while(serializefromobject_entries.hasNext()) {
/* 159 */           final java.util.Map$Entry serializefromobject_entry = (java.util.Map$Entry) serializefromobject_entries.next();
/* 160 */           int ExternalMapToCatalyst_key1 = (Integer) serializefromobject_entry.getKey();
/* 161 */           int ExternalMapToCatalyst_value1 = (Integer) serializefromobject_entry.getValue();
/* 162 */
/* 163 */           boolean ExternalMapToCatalyst_value_isNull1 = false;
/* 164 */
/* 165 */           if (false) {
/* 166 */             throw new RuntimeException("Cannot use null as map key!");
/* 167 */           } else {
/* 168 */             serializefromobject_convertedKeys[serializefromobject_index] = (Integer) ExternalMapToCatalyst_key1;
/* 169 */           }
/* 170 */
/* 171 */           if (false) {
/* 172 */             serializefromobject_convertedValues[serializefromobject_index] = null;
/* 173 */           } else {
/* 174 */             serializefromobject_convertedValues[serializefromobject_index] = (Integer) ExternalMapToCatalyst_value1;
/* 175 */           }
/* 176 */
/* 177 */           serializefromobject_index++;
/* 178 */         }
/* 179 */
/* 180 */         serializefromobject_value = new org.apache.spark.sql.catalyst.util.ArrayBasedMapData(new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedKeys), new org.apache.spark.sql.catalyst.util.GenericArrayData(serializefromobject_convertedValues));
/* 181 */       }
/* 182 */       serializefromobject_holder.reset();
/* 183 */
/* 184 */       serializefromobject_rowWriter.zeroOutNullBytes();
/* 185 */
/* 186 */       if (mapelements_isNull) {
/* 187 */         serializefromobject_rowWriter.setNullAt(0);
/* 188 */       } else {
/* 189 */         // Remember the current cursor so that we can calculate how many bytes are
/* 190 */         // written later.
/* 191 */         final int serializefromobject_tmpCursor = serializefromobject_holder.cursor;
/* 192 */
/* 193 */         if (serializefromobject_value instanceof UnsafeMapData) {
/* 194 */           final int serializefromobject_sizeInBytes = ((UnsafeMapData) serializefromobject_value).getSizeInBytes();
/* 195 */           // grow the global buffer before writing data.
/* 196 */           serializefromobject_holder.grow(serializefromobject_sizeInBytes);
/* 197 */           ((UnsafeMapData) serializefromobject_value).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 198 */           serializefromobject_holder.cursor += serializefromobject_sizeInBytes;
/* 199 */
/* 200 */         } else {
/* 201 */           final ArrayData serializefromobject_keys = serializefromobject_value.keyArray();
/* 202 */           final ArrayData serializefromobject_values = serializefromobject_value.valueArray();
/* 203 */
/* 204 */           // preserve 8 bytes to write the key array numBytes later.
/* 205 */           serializefromobject_holder.grow(8);
/* 206 */           serializefromobject_holder.cursor += 8;
/* 207 */
/* 208 */           // Remember the current cursor so that we can write numBytes of key array later.
/* 209 */           final int serializefromobject_tmpCursor1 = serializefromobject_holder.cursor;
/* 210 */
/* 211 */           if (serializefromobject_keys instanceof UnsafeArrayData) {
/* 212 */             final int serializefromobject_sizeInBytes1 = ((UnsafeArrayData) serializefromobject_keys).getSizeInBytes();
/* 213 */             // grow the global buffer before writing data.
/* 214 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes1);
/* 215 */             ((UnsafeArrayData) serializefromobject_keys).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 216 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes1;
/* 217 */
/* 218 */           } else {
/* 219 */             final int serializefromobject_numElements = serializefromobject_keys.numElements();
/* 220 */             serializefromobject_arrayWriter.initialize(serializefromobject_holder, serializefromobject_numElements, 4);
/* 221 */
/* 222 */             for (int serializefromobject_index1 = 0; serializefromobject_index1 < serializefromobject_numElements; serializefromobject_index1++) {
/* 223 */               if (serializefromobject_keys.isNullAt(serializefromobject_index1)) {
/* 224 */                 serializefromobject_arrayWriter.setNullInt(serializefromobject_index1);
/* 225 */               } else {
/* 226 */                 final int serializefromobject_element = serializefromobject_keys.getInt(serializefromobject_index1);
/* 227 */                 serializefromobject_arrayWriter.write(serializefromobject_index1, serializefromobject_element);
/* 228 */               }
/* 229 */             }
/* 230 */           }
/* 231 */
/* 232 */           // Write the numBytes of key array into the first 8 bytes.
/* 233 */           Platform.putLong(serializefromobject_holder.buffer, serializefromobject_tmpCursor1 - 8, serializefromobject_holder.cursor - serializefromobject_tmpCursor1);
/* 234 */
/* 235 */           if (serializefromobject_values instanceof UnsafeArrayData) {
/* 236 */             final int serializefromobject_sizeInBytes2 = ((UnsafeArrayData) serializefromobject_values).getSizeInBytes();
/* 237 */             // grow the global buffer before writing data.
/* 238 */             serializefromobject_holder.grow(serializefromobject_sizeInBytes2);
/* 239 */             ((UnsafeArrayData) serializefromobject_values).writeToMemory(serializefromobject_holder.buffer, serializefromobject_holder.cursor);
/* 240 */             serializefromobject_holder.cursor += serializefromobject_sizeInBytes2;
/* 241 */
/* 242 */           } else {
/* 243 */             final int serializefromobject_numElements1 = serializefromobject_values.numElements();
/* 244 */             serializefromobject_arrayWriter1.initialize(serializefromobject_holder, serializefromobject_numElements1, 4);
/* 245 */
/* 246 */             for (int serializefromobject_index2 = 0; serializefromobject_index2 < serializefromobject_numElements1; serializefromobject_index2++) {
/* 247 */               if (serializefromobject_values.isNullAt(serializefromobject_index2)) {
/* 248 */                 serializefromobject_arrayWriter1.setNullInt(serializefromobject_index2);
/* 249 */               } else {
/* 250 */                 final int serializefromobject_element1 = serializefromobject_values.getInt(serializefromobject_index2);
/* 251 */                 serializefromobject_arrayWriter1.write(serializefromobject_index2, serializefromobject_element1);
/* 252 */               }
/* 253 */             }
/* 254 */           }
/* 255 */
/* 256 */         }
/* 257 */
/* 258 */         serializefromobject_rowWriter.setOffsetAndSize(0, serializefromobject_tmpCursor, serializefromobject_holder.cursor - serializefromobject_tmpCursor);
/* 259 */       }
/* 260 */       serializefromobject_result.setTotalSize(serializefromobject_holder.totalSize());
/* 261 */       append(serializefromobject_result);
/* 262 */       if (shouldStop()) return;
/* 263 */     }
/* 264 */   }
/* 265 */ }
```

## How was this patch tested?

```
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> Seq(collection.mutable.HashMap(1 -> 2, 2 -> 3)).toDS().map(_ += (3 -> 4)).collect()
res0: Array[scala.collection.mutable.HashMap[Int,Int]] = Array(Map(2 -> 3, 1 -> 2, 3 -> 4))
```

Author: Michal Senkyr <mike.senkyr@gmail.com>
Author: Michal Šenkýř <mike.senkyr@gmail.com>

Closes #16986 from michalsenkyr/dataset-map-builder.
2017-06-12 08:47:01 +08:00
Zhenhua Wang a7c61c100b [SPARK-21031][SQL] Add alterTableStats to store spark's stats and let alterTable keep existing stats
## What changes were proposed in this pull request?

Currently, hive's stats are read into `CatalogStatistics`, while spark's stats are also persisted through `CatalogStatistics`. As a result, hive's stats can be unexpectedly propagated into spark' stats.

For example, for a catalog table, we read stats from hive, e.g. "totalSize" and put it into `CatalogStatistics`. Then, by using "ALTER TABLE" command, we will store the stats in `CatalogStatistics` into metastore as spark's stats (because we don't know whether it's from spark or not). But spark's stats should be only generated by "ANALYZE" command. This is unexpected from this command.

Secondly, now that we have spark's stats in metastore, after inserting new data, although hive updated "totalSize" in metastore, we still cannot get the right `sizeInBytes` in `CatalogStatistics`, because we respect spark's stats (should not exist) over hive's stats.

A running example is shown in [JIRA](https://issues.apache.org/jira/browse/SPARK-21031).

To fix this, we add a new method `alterTableStats` to store spark's stats, and let `alterTable` keep existing stats.

## How was this patch tested?

Added new tests.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18248 from wzhfy/separateHiveStats.
2017-06-12 08:23:04 +08:00
sujithjay 3a840048ed Fixed typo in sql.functions
## What changes were proposed in this pull request?

I fixed a typo in the Scaladoc for the method `def struct(cols: Column*): Column`. 'retained' was misspelt as 'remained'.

## How was this patch tested?
Before:

Creates a new struct column.
   If the input column is a column in a `DataFrame`, or a derived column expression
   that is named (i.e. aliased), its name would be **remained** as the StructField's name,
   otherwise, the newly generated StructField's name would be auto generated as
   `col` with a suffix `index + 1`, i.e. col1, col2, col3, ...

After:

   Creates a new struct column.
   If the input column is a column in a `DataFrame`, or a derived column expression
   that is named (i.e. aliased), its name would be **retained** as the StructField's name,
   otherwise, the newly generated StructField's name would be auto generated as
   `col` with a suffix `index + 1`, i.e. col1, col2, col3, ...

Author: sujithjay <sujith@logistimo.com>

Closes #18254 from sujithjay/fix-typo.
2017-06-11 18:23:57 +01:00
Xiao Li 8e96acf71c [SPARK-20211][SQL] Fix the Precision and Scale of Decimal Values when the Input is BigDecimal between -1.0 and 1.0
### What changes were proposed in this pull request?
The precision and scale of decimal values are wrong when the input is BigDecimal between -1.0 and 1.0.

The BigDecimal's precision is the digit count starts from the leftmost nonzero digit based on the [JAVA's BigDecimal definition](https://docs.oracle.com/javase/7/docs/api/java/math/BigDecimal.html). However, our Decimal decision follows the database decimal standard, which is the total number of digits, including both to the left and the right of the decimal point. Thus, this PR is to fix the issue by doing the conversion.

Before this PR, the following queries failed:
```SQL
select 1 > 0.0001
select floor(0.0001)
select ceil(0.0001)
```

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18244 from gatorsmile/bigdecimal.
2017-06-10 10:28:14 -07:00
Reynold Xin b78e3849b2 [SPARK-21042][SQL] Document Dataset.union is resolution by position
## What changes were proposed in this pull request?
Document Dataset.union is resolution by position, not by name, since this has been a confusing point for a lot of users.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18256 from rxin/SPARK-21042.
2017-06-09 18:29:33 -07:00
Xiao Li 571635488d [SPARK-20918][SQL] Use FunctionIdentifier as function identifiers in FunctionRegistry
### What changes were proposed in this pull request?
Currently, the unquoted string of a function identifier is being used as the function identifier in the function registry. This could cause the incorrect the behavior when users use `.` in the function names. This PR is to take the `FunctionIdentifier` as the identifier in the function registry.

- Add one new function `createOrReplaceTempFunction` to `FunctionRegistry`
```Scala
final def createOrReplaceTempFunction(name: String, builder: FunctionBuilder): Unit
```

### How was this patch tested?
Add extra test cases to verify the inclusive bug fixes.

Author: Xiao Li <gatorsmile@gmail.com>
Author: gatorsmile <gatorsmile@gmail.com>

Closes #18142 from gatorsmile/fuctionRegistry.
2017-06-09 10:16:30 -07:00
Dongjoon Hyun 6e95897e88 [SPARK-20954][SQL] DESCRIBE [EXTENDED] result should be compatible with previous Spark
## What changes were proposed in this pull request?

After [SPARK-20067](https://issues.apache.org/jira/browse/SPARK-20067), `DESCRIBE` and `DESCRIBE EXTENDED` shows the following result. This is incompatible with Spark 2.1.1. This PR removes the column header line in case of those command.

**MASTER** and **BRANCH-2.2**
```scala
scala> sql("desc t").show(false)
+----------+---------+-------+
|col_name  |data_type|comment|
+----------+---------+-------+
|# col_name|data_type|comment|
|a         |int      |null   |
+----------+---------+-------+
```

**SPARK 2.1.1** and **this PR**
```scala
scala> sql("desc t").show(false)
+--------+---------+-------+
|col_name|data_type|comment|
+--------+---------+-------+
|a       |int      |null   |
+--------+---------+-------+
```

## How was this patch tested?

Pass the Jenkins with the updated test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #18203 from dongjoon-hyun/SPARK-20954.
2017-06-08 16:46:56 -07:00
Xiao Li 1a527bde49 [SPARK-20976][SQL] Unify Error Messages for FAILFAST mode
### What changes were proposed in this pull request?
Before 2.2, we indicate the job was terminated because of `FAILFAST` mode.
```
Malformed line in FAILFAST mode: {"a":{, b:3}
```
If possible, we should keep it. This PR is to unify the error messages.

### How was this patch tested?
Modified the existing messages.

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18196 from gatorsmile/messFailFast.
2017-06-08 12:10:31 -07:00
Sean Owen 847efe1265 [SPARK-20914][DOCS] Javadoc contains code that is invalid
## What changes were proposed in this pull request?

Fix Java, Scala Dataset examples in scaladoc, which didn't compile.

## How was this patch tested?

Existing compilation/test

Author: Sean Owen <sowen@cloudera.com>

Closes #18215 from srowen/SPARK-20914.
2017-06-08 10:56:23 +01:00
Wenchen Fan c92949ac23 [SPARK-20972][SQL] rename HintInfo.isBroadcastable to broadcast
## What changes were proposed in this pull request?

`HintInfo.isBroadcastable` is actually not an accurate name, it's used to force the planner to broadcast a plan no matter what the data size is, via the hint mechanism. I think `forceBroadcast` is a better name.

And `isBroadcastable` only have 2 possible values: `Some(true)` and `None`, so we can just use boolean type for it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18189 from cloud-fan/stats.
2017-06-06 22:50:06 -07:00
Reza Safi b61a401da8 [SPARK-20926][SQL] Removing exposures to guava library caused by directly accessing SessionCatalog's tableRelationCache
There could be test failures because DataStorageStrategy, HiveMetastoreCatalog and also HiveSchemaInferenceSuite were exposed to guava library by directly accessing SessionCatalog's tableRelationCacheg. These failures occur when guava shading is in place.

## What changes were proposed in this pull request?
This change removes those guava exposures by introducing new methods in SessionCatalog and also changing DataStorageStrategy, HiveMetastoreCatalog and HiveSchemaInferenceSuite so that they use those proxy methods.

## How was this patch tested?

Unit tests passed after applying these changes.

Author: Reza Safi <rezasafi@cloudera.com>

Closes #18148 from rezasafi/branch-2.2.

(cherry picked from commit 1388fdd707)
2017-06-06 09:54:13 -07:00
Shixiong Zhu bc537e40ad [SPARK-20957][SS][TESTS] Fix o.a.s.sql.streaming.StreamingQueryManagerSuite listing
## What changes were proposed in this pull request?

When stopping StreamingQuery, StreamExecution will set `streamDeathCause` then notify StreamingQueryManager to remove this query. So it's possible that when `q2.exception.isDefined` returns `true`, StreamingQueryManager's active list still has `q2`.

This PR just puts the checks into `eventually` to fix the flaky test.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18180 from zsxwing/SPARK-20957.
2017-06-05 14:34:10 -07:00
Wenchen Fan dec9aa3b37 [SPARK-20961][SQL] generalize the dictionary in ColumnVector
## What changes were proposed in this pull request?

As the first step of https://issues.apache.org/jira/browse/SPARK-20960 , to make `ColumnVector` public, this PR generalize `ColumnVector.dictionary` to not couple with parquet.

## How was this patch tested?

existing tests

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18183 from cloud-fan/dictionary.
2017-06-04 13:43:51 -07:00
Ruben Berenguel Montoro 6cbc61d107 [SPARK-19732][SQL][PYSPARK] Add fill functions for nulls in bool fields of datasets
## What changes were proposed in this pull request?

Allow fill/replace of NAs with booleans, both in Python and Scala

## How was this patch tested?

Unit tests, doctests

This PR is original work from me and I license this work to the Spark project

Author: Ruben Berenguel Montoro <ruben@mostlymaths.net>
Author: Ruben Berenguel <ruben@mostlymaths.net>

Closes #18164 from rberenguel/SPARK-19732-fillna-bools.
2017-06-03 14:56:42 +09:00
Zhenhua Wang 6de41e951f [SPARK-17078][SQL][FOLLOWUP] Simplify explain cost command
## What changes were proposed in this pull request?

Usually when using explain cost command, users want to see the stats of plan. Since stats is only showed in optimized plan, it is more direct and convenient to include only optimized plan and physical plan in the output.

## How was this patch tested?

Enhanced existing test.

Author: Zhenhua Wang <wzh_zju@163.com>

Closes #18190 from wzhfy/simplifyExplainCost.
2017-06-02 17:36:00 -07:00
Yin Huai 0eb1fc6cd5 Revert "[SPARK-20946][SQL] simplify the config setting logic in SparkSession.getOrCreate"
This reverts commit e11d90bf8d.
2017-06-02 15:36:21 -07:00
Wenchen Fan e11d90bf8d [SPARK-20946][SQL] simplify the config setting logic in SparkSession.getOrCreate
## What changes were proposed in this pull request?

The current conf setting logic is a little complex and has duplication, this PR simplifies it.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18172 from cloud-fan/session.
2017-06-02 10:05:05 -07:00
Wenchen Fan d1b80ab922 [SPARK-20967][SQL] SharedState.externalCatalog is not really lazy
## What changes were proposed in this pull request?

`SharedState.externalCatalog` is marked as a `lazy val` but actually it's not lazy. We access `externalCatalog` while initializing `SharedState` and thus eliminate the effort of `lazy val`. When creating `ExternalCatalog` we will try to connect to the metastore and may throw an error, so it makes sense to make it a `lazy val` in `SharedState`.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18187 from cloud-fan/minor.
2017-06-02 09:58:01 -07:00
Bogdan Raducanu 2134196a9c [SPARK-20854][SQL] Extend hint syntax to support expressions
## What changes were proposed in this pull request?

SQL hint syntax:
* support expressions such as strings, numbers, etc. instead of only identifiers as it is currently.
* support multiple hints, which was missing compared to the DataFrame syntax.

DataFrame API:
* support any parameters in DataFrame.hint instead of just strings

## How was this patch tested?
Existing tests. New tests in PlanParserSuite. New suite DataFrameHintSuite.

Author: Bogdan Raducanu <bogdan@databricks.com>

Closes #18086 from bogdanrdc/SPARK-20854.
2017-06-01 15:50:40 -07:00
Xiao Li f7cf2096fd [SPARK-20941][SQL] Fix SubqueryExec Reuse
### What changes were proposed in this pull request?
Before this PR, Subquery reuse does not work. Below are three issues:
- Subquery reuse does not work.
- It is sharing the same `SQLConf` (`spark.sql.exchange.reuse`) with the one for Exchange Reuse.
- No test case covers the rule Subquery reuse.

This PR is to fix the above three issues.
- Ignored the physical operator `SubqueryExec` when comparing two plans.
- Added a dedicated conf `spark.sql.subqueries.reuse` for controlling Subquery Reuse
- Added a test case for verifying the behavior

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #18169 from gatorsmile/subqueryReuse.
2017-06-01 09:52:18 -07:00
Yuming Wang 6d05c1c1da [SPARK-20910][SQL] Add build-in SQL function - UUID
## What changes were proposed in this pull request?

Add build-int SQL function - UUID.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18136 from wangyum/SPARK-20910.
2017-06-01 16:15:24 +09:00
Shixiong Zhu 2bc3272880 [SPARK-20894][SS] Resolve the checkpoint location in driver and use the resolved path in state store
## What changes were proposed in this pull request?

When the user runs a Structured Streaming query in a cluster, if the driver uses the local file system, StateStore running in executors will throw a file-not-found exception. However, the current error is not obvious.

This PR makes StreamExecution resolve the path in driver and uses the full path including the scheme part (such as `hdfs:/`, `file:/`) in StateStore.

Then if the above error happens, StateStore will throw an error with this full path which starts with `file:/`, and it makes this error obvious: the checkpoint location is on the local file system.

One potential minor issue is that the user cannot use different default file system settings in driver and executors (e.g., use a public HDFS address in driver and a private HDFS address in executors) after this change. However, since the batch query also has this issue (See 4bb6a53ebd/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L402)), it doesn't make things worse.

## How was this patch tested?

The new added test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #18149 from zsxwing/SPARK-20894.
2017-05-31 17:24:37 -07:00
gatorsmile de934e6718 [SPARK-19236][SQL][FOLLOW-UP] Added createOrReplaceGlobalTempView method
### What changes were proposed in this pull request?
This PR does the following tasks:
- Added  since
- Added the Python API
- Added test cases

### How was this patch tested?
Added test cases to both Scala and Python

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18147 from gatorsmile/createOrReplaceGlobalTempView.
2017-05-31 11:38:43 -07:00
Liu Shaohui d0f36bcb10 [SPARK-20633][SQL] FileFormatWriter should not wrap FetchFailedException
## What changes were proposed in this pull request?

Explicitly handle the FetchFailedException in FileFormatWriter, so it does not get wrapped.

Note that this is no longer strictly necessary after SPARK-19276, but it improves error messages and also will help avoid others stumbling across this in the future.

## How was this patch tested?

Existing unit tests.

Closes https://github.com/apache/spark/pull/17893

Author: Liu Shaohui <liushaohui@xiaomi.com>

Closes #18145 from squito/SPARK-20633.
2017-05-31 10:53:31 -05:00
Jacek Laskowski beed5e20af [DOCS][MINOR] Scaladoc fixes (aka typo hunting)
## What changes were proposed in this pull request?

Minor changes to scaladoc

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18074 from jaceklaskowski/scaladoc-fixes.
2017-05-31 11:24:37 +01:00
Wenchen Fan 1f5dddffa3 Revert "[SPARK-20392][SQL] Set barrier to prevent re-entering a tree"
This reverts commit 8ce0d8ffb6.
2017-05-30 21:14:55 -07:00
Wenchen Fan 10e526e7e6 [SPARK-20213][SQL] Fix DataFrameWriter operations in SQL UI tab
## What changes were proposed in this pull request?

Currently the `DataFrameWriter` operations have several problems:

1. non-file-format data source writing action doesn't show up in the SQL tab in Spark UI
2. file-format data source writing action shows a scan node in the SQL tab, without saying anything about writing. (streaming also have this issue, but not fixed in this PR)
3. Spark SQL CLI actions don't show up in the SQL tab.

This PR fixes all of them, by refactoring the `ExecuteCommandExec` to make it have children.

 close https://github.com/apache/spark/pull/17540

## How was this patch tested?

existing tests.

Also test the UI manually. For a simple command: `Seq(1 -> "a").toDF("i", "j").write.parquet("/tmp/qwe")`

before this PR:
<img width="266" alt="qq20170523-035840 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326050/24e18ba2-3f6c-11e7-8817-6dd275bf6ac5.png">
after this PR:
<img width="287" alt="qq20170523-035708 2x" src="https://cloud.githubusercontent.com/assets/3182036/26326054/2ad7f460-3f6c-11e7-8053-d68325beb28f.png">

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18064 from cloud-fan/execution.
2017-05-30 20:12:32 -07:00
Tathagata Das fa757ee1d4 [SPARK-20883][SPARK-20376][SS] Refactored StateStore APIs and added conf to choose implementation
## What changes were proposed in this pull request?

A bunch of changes to the StateStore APIs and implementation.
Current state store API has a bunch of problems that causes too many transient objects causing memory pressure.

- `StateStore.get(): Option` forces creation of Some/None objects for every get. Changed this to return the row or null.
- `StateStore.iterator(): (UnsafeRow, UnsafeRow)` forces creation of new tuple for each record returned. Changed this to return a UnsafeRowTuple which can be reused across records.
- `StateStore.updates()` requires the implementation to keep track of updates, while this is used minimally (only by Append mode in streaming aggregations). Removed updates() and updated StateStoreSaveExec accordingly.
- `StateStore.filter(condition)` and `StateStore.remove(condition)` has been merge into a single API `getRange(start, end)` which allows a state store to do optimized range queries (i.e. avoid full scans). Stateful operators have been updated accordingly.
- Removed a lot of unnecessary row copies Each operator copied rows before calling StateStore.put() even if the implementation does not require it to be copied. It is left up to the implementation on whether to copy the row or not.

Additionally,
- Added a name to the StateStoreId so that each operator+partition can use multiple state stores (different names)
- Added a configuration that allows the user to specify which implementation to use.
- Added new metrics to understand the time taken to update keys, remove keys and commit all changes to the state store. These metrics will be visible on the plan diagram in the SQL tab of the UI.
- Refactored unit tests such that they can be reused to test any implementation of StateStore.

## How was this patch tested?
Old and new unit tests

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

Closes #18107 from tdas/SPARK-20376.
2017-05-30 15:33:06 -07:00
Josh Rosen 798a04fd76 HOTFIX: fix Scalastyle break introduced in 4d57981cfb 2017-05-30 12:22:23 -07:00
Arman 4d57981cfb [SPARK-19236][CORE] Added createOrReplaceGlobalTempView method
## What changes were proposed in this pull request?

Added the createOrReplaceGlobalTempView method for dataset

Author: Arman <arman.yazdani.10@gmail.com>

Closes #16598 from arman1371/patch-1.
2017-05-30 11:09:21 -07:00
Liang-Chi Hsieh 35b644bd03 [SPARK-20916][SQL] Improve error message for unaliased subqueries in FROM clause
## What changes were proposed in this pull request?

We changed the parser to reject unaliased subqueries in the FROM clause in SPARK-20690. However, the error message that we now give isn't very helpful:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    mismatched input 'FROM' expecting {<EOF>, 'WHERE', 'GROUP', 'ORDER', 'HAVING', 'LIMIT', 'LATERAL', 'WINDOW', 'UNION', 'EXCEPT', 'MINUS', 'INTERSECT', 'SORT', 'CLUSTER', 'DISTRIBUTE'}(line 1, pos 9)

We should modify the parser to throw a more clear error for such queries:

    scala> sql("""SELECT x FROM (SELECT 1 AS x)""")
    org.apache.spark.sql.catalyst.parser.ParseException:
    The unaliased subqueries in the FROM clause are not supported.(line 1, pos 14)

## How was this patch tested?

Modified existing tests to reflect this change.

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

Closes #18141 from viirya/SPARK-20916.
2017-05-30 06:28:43 -07:00
Yuming Wang d797ed0ef1 [SPARK-20909][SQL] Add build-int SQL function - DAYOFWEEK
## What changes were proposed in this pull request?

Add build-int SQL function - DAYOFWEEK

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #18134 from wangyum/SPARK-20909.
2017-05-30 15:40:50 +09:00
Kazuaki Ishizaki c9749068ec [SPARK-20907][TEST] Use testQuietly for test suites that generate long log output
## What changes were proposed in this pull request?

Supress console output by using `testQuietly` in test suites

## How was this patch tested?

Tested by `"SPARK-19372: Filter can be executed w/o generated code due to JVM code size limit"` in `DataFrameSuite`

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

Closes #18135 from kiszk/SPARK-20907.
2017-05-29 12:17:14 -07:00
Kazuaki Ishizaki ef9fd920c3 [SPARK-20750][SQL] Built-in SQL Function Support - REPLACE
## What changes were proposed in this pull request?

This PR adds built-in SQL function `(REPLACE(<string_expression>, <search_string> [, <replacement_string>])`

`REPLACE()` return that string that is replaced all occurrences with given string.

## How was this patch tested?

added new test suites

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

Closes #18047 from kiszk/SPARK-20750.
2017-05-29 11:47:31 -07:00
Tejas Patil f9b59abeae [SPARK-20758][SQL] Add Constant propagation optimization
## What changes were proposed in this pull request?

See class doc of `ConstantPropagation` for the approach used.

## How was this patch tested?

- Added unit tests

Author: Tejas Patil <tejasp@fb.com>

Closes #17993 from tejasapatil/SPARK-20758_const_propagation.
2017-05-29 12:21:34 +02:00
Takeshi Yamamuro 24d34281d7 [SPARK-20841][SQL] Support table column aliases in FROM clause
## What changes were proposed in this pull request?
This pr added parsing rules to support table column aliases in FROM clause.

## How was this patch tested?
Added tests in `PlanParserSuite`,  `SQLQueryTestSuite`, and `PlanParserSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #18079 from maropu/SPARK-20841.
2017-05-28 13:23:18 -07:00
liuxian 3969a8078e [SPARK-20876][SQL] If the input parameter is float type for ceil or floor,the result is not we expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12345
For this case, the result we expected is `12346`
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12345
For this case, the result we expected is `-12346`

Because in `Ceil` or `Floor`, `inputTypes` has no FloatType, so it is converted to LongType.
## How was this patch tested?

After the modification:
spark-sql>SELECT ceil(cast(12345.1233 as float));
spark-sql>12346
spark-sql>SELECT floor(cast(-12345.1233 as float));
spark-sql>-12346

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18103 from 10110346/wip-lx-0525-1.
2017-05-27 16:23:45 -07:00
Wenchen Fan 08ede46b89 [SPARK-20897][SQL] cached self-join should not fail
## What changes were proposed in this pull request?

The failed test case is, we have a `SortMergeJoinExec` for a self-join, which means we have a `ReusedExchange` node in the query plan. It works fine without caching, but throws an exception in `SortMergeJoinExec.outputPartitioning` if we cache it.

The root cause is, `ReusedExchange` doesn't propagate the output partitioning from its child, so in `SortMergeJoinExec.outputPartitioning` we create `PartitioningCollection` with a hash partitioning and an unknown partitioning, and fail.

This bug is mostly fine, because inserting the `ReusedExchange` is the last step to prepare the physical plan, we won't call `SortMergeJoinExec.outputPartitioning` anymore after this.

However, if the dataframe is cached, the physical plan of it becomes `InMemoryTableScanExec`, which contains another physical plan representing the cached query, and it has gone through the entire planning phase and may have `ReusedExchange`. Then the planner call `InMemoryTableScanExec.outputPartitioning`, which then calls `SortMergeJoinExec.outputPartitioning` and trigger this bug.

## How was this patch tested?

a new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #18121 from cloud-fan/bug.
2017-05-27 16:16:51 -07:00
setjet c491e2ed90 [SPARK-20873][SQL] Improve the error message for unsupported Column Type
## What changes were proposed in this pull request?
Upon encountering an invalid columntype, the column type object is printed, rather than the type.
This  change improves this by outputting its name.

## How was this patch tested?
Added a simple  unit test to verify the contents of the raised exception

Author: setjet <rubenljanssen@gmail.com>

Closes #18097 from setjet/spark-20873.
2017-05-26 15:07:28 -07:00
Michael Armbrust d935e0a9d9 [SPARK-20844] Remove experimental from Structured Streaming APIs
Now that Structured Streaming has been out for several Spark release and has large production use cases, the `Experimental` label is no longer appropriate.  I've left `InterfaceStability.Evolving` however, as I think we may make a few changes to the pluggable Source & Sink API in Spark 2.3.

Author: Michael Armbrust <michael@databricks.com>

Closes #18065 from marmbrus/streamingGA.
2017-05-26 13:33:23 -07:00
Liang-Chi Hsieh 8ce0d8ffb6 [SPARK-20392][SQL] Set barrier to prevent re-entering a tree
## What changes were proposed in this pull request?

It is reported that there is performance downgrade when applying ML pipeline for dataset with many columns but few rows.

A big part of the performance downgrade comes from some operations (e.g., `select`) on DataFrame/Dataset which re-create new DataFrame/Dataset with a new `LogicalPlan`. The cost can be ignored in the usage of SQL, normally.

However, it's not rare to chain dozens of pipeline stages in ML. When the query plan grows incrementally during running those stages, the total cost spent on re-creation of DataFrame grows too. In particular, the `Analyzer` will go through the big query plan even most part of it is analyzed.

By eliminating part of the cost, the time to run the example code locally is reduced from about 1min to about 30 secs.

In particular, the time applying the pipeline locally is mostly spent on calling transform of the 137 `Bucketizer`s. Before the change, each call of `Bucketizer`'s transform can cost about 0.4 sec. So the total time spent on all `Bucketizer`s' transform is about 50 secs. After the change, each call only costs about 0.1 sec.

<del>We also make `boundEnc` as lazy variable to reduce unnecessary running time.</del>

### Performance improvement

The codes and datasets provided by Barry Becker to re-produce this issue and benchmark can be found on the JIRA.

Before this patch: about 1 min
After this patch: about 20 secs

## How was this patch tested?

Existing tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17770 from viirya/SPARK-20392.
2017-05-26 13:45:55 +08:00
setjet 2dbe0c5288 [SPARK-20775][SQL] Added scala support from_json
## What changes were proposed in this pull request?

from_json function required to take in a java.util.Hashmap. For other functions, a java wrapper is provided which casts a java hashmap to a scala map. Only a java function is provided in this case, forcing scala users to pass in a java.util.Hashmap.

Added the missing wrapper.

## How was this patch tested?
Added a unit test for passing in a scala map

Author: setjet <rubenljanssen@gmail.com>

Closes #18094 from setjet/spark-20775.
2017-05-26 10:21:39 +08:00
hyukjinkwon e9f983df27 [SPARK-19707][SPARK-18922][TESTS][SQL][CORE] Fix test failures/the invalid path check for sc.addJar on Windows
## What changes were proposed in this pull request?

This PR proposes two things:

- A follow up for SPARK-19707 (Improving the invalid path check for sc.addJar on Windows as well).

```
org.apache.spark.SparkContextSuite:
 - add jar with invalid path *** FAILED *** (32 milliseconds)
   2 was not equal to 1 (SparkContextSuite.scala:309)
   ...
```

- Fix path vs URI related test failures on Windows.

```
org.apache.spark.storage.LocalDirsSuite:
 - SPARK_LOCAL_DIRS override also affects driver *** FAILED *** (0 milliseconds)
   new java.io.File("/NONEXISTENT_PATH").exists() was true (LocalDirsSuite.scala:50)
   ...

 - Utils.getLocalDir() throws an exception if any temporary directory cannot be retrieved *** FAILED *** (15 milliseconds)
   Expected exception java.io.IOException to be thrown, but no exception was thrown. (LocalDirsSuite.scala:64)
   ...
```

```
org.apache.spark.sql.hive.HiveSchemaInferenceSuite:
 - orc: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-dae61ab3-a851-4dd3-bf4e-be97c501f254
   ...

 - parquet: schema should be inferred and saved when INFER_AND_SAVE is specified *** FAILED *** (203 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fa3aff89-a66e-4376-9a37-2a9b87596939
   ...

 - orc: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (141 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-fb464e59-b049-481b-9c75-f53295c9fc2c
   ...

 - parquet: schema should be inferred but not stored when INFER_ONLY is specified *** FAILED *** (125 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-9487568e-80a4-42b3-b0a5-d95314c4ccbc
   ...

 - orc: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (156 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-0d2dfa45-1b0f-4958-a8be-1074ed0135a
   ...

 - parquet: schema should not be inferred when NEVER_INFER is specified *** FAILED *** (547 milliseconds)
   java.net.URISyntaxException: Illegal character in opaque part at index 2: C:\projects\spark\target\tmp\spark-6d95d64e-613e-4a59-a0f6-d198c5aa51ee
   ...
```

```
org.apache.spark.sql.execution.command.DDLSuite:
 - create temporary view using *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-3881d9ca-561b-488d-90b9-97587472b853	mp;
   ...

 - insert data to a data source table which has a non-existing location should succeed *** FAILED *** (109 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4cad3d19-6085-4b75-b407-fe5e9d21df54 did not equal file:///C:/projects/spark/target/tmp/spark-4cad3d19-6085-4b75-b407-fe5e9d21df54 (DDLSuite.scala:1869)
   ...

 - insert into a data source table with a non-existing partition location should succeed *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d did not equal file:///C:/projects/spark/target/tmp/spark-4b52e7de-e3aa-42fd-95d4-6d4d58d1d95d (DDLSuite.scala:1910)
   ...

 - read data from a data source table which has a non-existing location should succeed *** FAILED *** (93 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-f8c281e2-08c2-4f73-abbf-f3865b702c34 did not equal file:///C:/projects/spark/target/tmp/spark-f8c281e2-08c2-4f73-abbf-f3865b702c34 (DDLSuite.scala:1937)
   ...

 - read data from a data source table with non-existing partition location should succeed *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - create datasource table with a non-existing location *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-387316ae-070c-4e78-9b78-19ebf7b29ec8 did not equal file:///C:/projects/spark/target/tmp/spark-387316ae-070c-4e78-9b78-19ebf7b29ec8 (DDLSuite.scala:1982)
   ...

 - CTAS for external data source table with a non-existing location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - CTAS for external data source table with a existed location *** FAILED *** (15 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a:b *** FAILED *** (143 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a%b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - data source table:partition column name containing a,b *** FAILED *** (109 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - location uri contains a b for datasource table *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-5739cda9-b702-4e14-932c-42e8c4174480a%20b did not equal file:///C:/projects/spark/target/tmp/spark-5739cda9-b702-4e14-932c-42e8c4174480/a%20b (DDLSuite.scala:2084)
   ...

 - location uri contains a:b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-9bdd227c-840f-4f08-b7c5-4036638f098da:b did not equal file:///C:/projects/spark/target/tmp/spark-9bdd227c-840f-4f08-b7c5-4036638f098d/a:b (DDLSuite.scala:2084)
   ...

 - location uri contains a%b for datasource table *** FAILED *** (78 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-62bb5f1d-fa20-460a-b534-cb2e172a3640a%25b did not equal file:///C:/projects/spark/target/tmp/spark-62bb5f1d-fa20-460a-b534-cb2e172a3640/a%25b (DDLSuite.scala:2084)
   ...

 - location uri contains a b for database *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a:b for database *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - location uri contains a%b for database *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.hive.execution.HiveDDLSuite:
 - create hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a non-existing location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - CTAS for external hive table with a existed location *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of parquet table containing a b *** FAILED *** (156 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a:b *** FAILED *** (94 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a%b *** FAILED *** (125 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of parquet table containing a,b *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...

 - partition column name of hive table containing a b *** FAILED *** (15 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a:b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a%b *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - partition column name of hive table containing a,b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a:b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...

 - hive table: location uri contains a%b *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
   ...
```

```
org.apache.spark.sql.sources.PathOptionSuite:
 - path option also exist for write path *** FAILED *** (94 milliseconds)
   file:/C:projectsspark%09arget%09mpspark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc did not equal file:///C:/projects/spark/target/tmp/spark-2870b281-7ac0-43d6-b6b6-134e01ab6fdc (PathOptionSuite.scala:98)
   ...
```

```
org.apache.spark.sql.CachedTableSuite:
 - SPARK-19765: UNCACHE TABLE should un-cache all cached plans that refer to this table *** FAILED *** (110 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
   ...
```

```
org.apache.spark.sql.execution.DataSourceScanExecRedactionSuite:
 - treeString is redacted *** FAILED *** (250 milliseconds)
   "file:/C:/projects/spark/target/tmp/spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" did not contain "C:\projects\spark\target\tmp\spark-3ecc1fa4-3e76-489c-95f4-f0b0500eae28" (DataSourceScanExecRedactionSuite.scala:46)
   ...
```

## How was this patch tested?

Tested via AppVeyor for each and checked it passed once each. These should be retested via AppVeyor in this PR.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17987 from HyukjinKwon/windows-20170515.
2017-05-25 17:10:30 +01:00
Liang-Chi Hsieh 6b68d61cf3 [SPARK-20848][SQL][FOLLOW-UP] Shutdown the pool after reading parquet files
## What changes were proposed in this pull request?

This is a follow-up to #18073. Taking a safer approach to shutdown the pool to prevent possible issue. Also using `ThreadUtils.newForkJoinPool` instead to set a better thread name.

## How was this patch tested?

Manually test.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18100 from viirya/SPARK-20848-followup.
2017-05-25 09:55:45 +08:00
liuxian 197f9018a4 [SPARK-20403][SQL] Modify the instructions of some functions
## What changes were proposed in this pull request?
1.    add  instructions of  'cast'  function When using 'show functions'  and 'desc function cast'
       command in spark-sql
2.    Modify the  instructions of functions,such as
     boolean,tinyint,smallint,int,bigint,float,double,decimal,date,timestamp,binary,string

## How was this patch tested?
Before modification:
spark-sql>desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr AS type) - Casts the value `expr` to the target data type `type`.

After modification:
spark-sql> desc function boolean;
Function: boolean
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`.

spark-sql> desc function cast
Function: cast
Class: org.apache.spark.sql.catalyst.expressions.Cast
Usage: cast(expr AS type) - Casts the value `expr` to the target data type `type`.

Author: liuxian <liu.xian3@zte.com.cn>

Closes #17698 from 10110346/wip_lx_0418.
2017-05-24 17:32:02 -07:00
Jacek Laskowski 5f8ff2fc9a [SPARK-16202][SQL][DOC] Follow-up to Correct The Description of CreatableRelationProvider's createRelation
## What changes were proposed in this pull request?

Follow-up to SPARK-16202:

1. Remove the duplication of the meaning of `SaveMode` (as one was in fact missing that had proven that the duplication may be incomplete in the future again)

2. Use standard scaladoc tags

/cc gatorsmile rxin yhuai (as they were involved previously)

## How was this patch tested?

local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #18026 from jaceklaskowski/CreatableRelationProvider-SPARK-16202.
2017-05-24 17:24:23 -07:00
Kris Mok c0b3e45e3b [SPARK-20872][SQL] ShuffleExchange.nodeName should handle null coordinator
## What changes were proposed in this pull request?

A one-liner change in `ShuffleExchange.nodeName` to cover the case when `coordinator` is `null`, so that the match expression is exhaustive.

Please refer to [SPARK-20872](https://issues.apache.org/jira/browse/SPARK-20872) for a description of the symptoms.
TL;DR is that inspecting a `ShuffleExchange` (directly or transitively) on the Executor side can hit a case where the `coordinator` field of a `ShuffleExchange` is null, and thus will trigger a `MatchError` in `ShuffleExchange.nodeName()`'s inexhaustive match expression.

Also changed two other match conditions in `ShuffleExchange` on the `coordinator` field to be consistent.

## How was this patch tested?

Manually tested this change with a case where the `coordinator` is null to make sure `ShuffleExchange.nodeName` doesn't throw a `MatchError` any more.

Author: Kris Mok <kris.mok@databricks.com>

Closes #18095 from rednaxelafx/shuffleexchange-nodename.
2017-05-24 17:19:35 -07:00
Reynold Xin a64746677b [SPARK-20867][SQL] Move hints from Statistics into HintInfo class
## What changes were proposed in this pull request?
This is a follow-up to SPARK-20857 to move the broadcast hint from Statistics into a new HintInfo class, so we can be more flexible in adding new hints in the future.

## How was this patch tested?
Updated test cases to reflect the change.

Author: Reynold Xin <rxin@databricks.com>

Closes #18087 from rxin/SPARK-20867.
2017-05-24 13:57:19 -07:00
Liang-Chi Hsieh f72ad303f0 [SPARK-20848][SQL] Shutdown the pool after reading parquet files
## What changes were proposed in this pull request?

From JIRA: On each call to spark.read.parquet, a new ForkJoinPool is created. One of the threads in the pool is kept in the WAITING state, and never stopped, which leads to unbounded growth in number of threads.

We should shutdown the pool after reading parquet files.

## How was this patch tested?

Added a test to ParquetFileFormatSuite.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #18073 from viirya/SPARK-20848.
2017-05-25 00:35:40 +08:00
Kirby Linvill 4816c2ef5e [SPARK-15648][SQL] Add teradataDialect for JDBC connection to Teradata
The contribution is my original work and I license the work to the project under the project’s open source license.

Note: the Teradata JDBC connector limits the row size to 64K. The default string datatype equivalent I used is a 255 character/byte length varchar. This effectively limits the max number of string columns to 250 when using the Teradata jdbc connector.

## What changes were proposed in this pull request?

Added a teradataDialect for JDBC connection to Teradata. The Teradata dialect uses VARCHAR(255) in place of TEXT for string datatypes, and CHAR(1) in place of BIT(1) for boolean datatypes.

## How was this patch tested?

I added two unit tests to double check that the types get set correctly for a teradata jdbc url. I also ran a couple manual tests to make sure the jdbc connector worked with teradata and to make sure that an error was thrown if a row could potentially exceed 64K (this error comes from the teradata jdbc connector, not from the spark code). I did not check how string columns longer than 255 characters are handled.

Author: Kirby Linvill <kirby.linvill@teradata.com>
Author: klinvill <kjlinvill@gmail.com>

Closes #16746 from klinvill/master.
2017-05-23 12:00:58 -07:00
Reynold Xin 0d589ba00b [SPARK-20857][SQL] Generic resolved hint node
## What changes were proposed in this pull request?
This patch renames BroadcastHint to ResolvedHint (and Hint to UnresolvedHint) so the hint framework is more generic and would allow us to introduce other hint types in the future without introducing new hint nodes.

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

Author: Reynold Xin <rxin@databricks.com>

Closes #18072 from rxin/SPARK-20857.
2017-05-23 18:44:49 +02:00
gatorsmile f3ed62a381 [SPARK-20831][SQL] Fix INSERT OVERWRITE data source tables with IF NOT EXISTS
### What changes were proposed in this pull request?
Currently, we have a bug when we specify `IF NOT EXISTS` in `INSERT OVERWRITE` data source tables. For example, given a query:
```SQL
INSERT OVERWRITE TABLE $tableName partition (b=2, c=3) IF NOT EXISTS SELECT 9, 10
```
we will get the following error:
```
unresolved operator 'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true;;
'InsertIntoTable Relation[a#425,d#426,b#427,c#428] parquet, Map(b -> Some(2), c -> Some(3)), true, true
+- Project [cast(9#423 as int) AS a#429, cast(10#424 as int) AS d#430]
   +- Project [9 AS 9#423, 10 AS 10#424]
      +- OneRowRelation$
```

This PR is to fix the issue to follow the behavior of Hive serde tables
> INSERT OVERWRITE will overwrite any existing data in the table or partition unless IF NOT EXISTS is provided for a partition

### How was this patch tested?
Modified an existing test case

Author: gatorsmile <gatorsmile@gmail.com>

Closes #18050 from gatorsmile/insertPartitionIfNotExists.
2017-05-22 22:24:50 +08:00
Michal Senkyr a2b3b67624 [SPARK-19089][SQL] Add support for nested sequences
## What changes were proposed in this pull request?

Replaced specific sequence encoders with generic sequence encoder to enable nesting of sequences.

Does not add support for nested arrays as that cannot be solved in this way.

## How was this patch tested?

```bash
build/mvn -DskipTests clean package && dev/run-tests
```

Additionally in Spark shell:

```
scala> Seq(Seq(Seq(1))).toDS.collect()
res0: Array[Seq[Seq[Int]]] = Array(List(List(1)))
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #18011 from michalsenkyr/dataset-seq-nested.
2017-05-22 16:49:19 +08:00
Kazuaki Ishizaki 833c8d4152 [SPARK-20770][SQL] Improve ColumnStats
## What changes were proposed in this pull request?

This PR improves the implementation of `ColumnStats` by using the following appoaches.

1. Declare subclasses of `ColumnStats` as `final`
2. Remove unnecessary call of `row.isNullAt(ordinal)`
3. Remove the dependency on `GenericInternalRow`

For 1., this declaration encourages method inlining and other optimizations of JIT compiler
For 2., in `gatherStats()`, while previous code in subclasses of `ColumnStats` always calls `row.isNullAt()` twice, the PR just calls `row.isNullAt()` only once.
For 3., `collectedStatistics()` returns `Array[Any]` instead of `GenericInternalRow`. This removes the dependency of unnecessary package and reduces the number of allocations of `GenericInternalRow`.

In addition to that, in the future, `gatherValueStats()`, which is specialized for each data type, can be effectively called from the generated code without using generic data structure `InternalRow`.

## How was this patch tested?

Tested by existing test suite

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

Closes #18002 from kiszk/SPARK-20770.
2017-05-22 16:23:23 +08:00
caoxuewen 3c9eef35a8 [SPARK-20786][SQL] Improve ceil and floor handle the value which is not expected
## What changes were proposed in this pull request?

spark-sql>SELECT ceil(1234567890123456);
1234567890123456

spark-sql>SELECT ceil(12345678901234567);
12345678901234568

spark-sql>SELECT ceil(123456789012345678);
123456789012345680

when the length of the getText is greater than 16. long to double will be precision loss.

but mysql handle the value is ok.

mysql> SELECT ceil(1234567890123456);
+------------------------+
| ceil(1234567890123456) |
+------------------------+
|       1234567890123456 |
+------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(12345678901234567);
+-------------------------+
| ceil(12345678901234567) |
+-------------------------+
|       12345678901234567 |
+-------------------------+
1 row in set (0.00 sec)

mysql> SELECT ceil(123456789012345678);
+--------------------------+
| ceil(123456789012345678) |
+--------------------------+
|       123456789012345678 |
+--------------------------+
1 row in set (0.00 sec)

## How was this patch tested?

Supplement the unit test.

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #18016 from heary-cao/ceil_long.
2017-05-21 22:39:07 -07:00
Tathagata Das 9d6661c829 [SPARK-20792][SS] Support same timeout operations in mapGroupsWithState function in batch queries as in streaming queries
## What changes were proposed in this pull request?

Currently, in the batch queries, timeout is disabled (i.e. GroupStateTimeout.NoTimeout) which means any GroupState.setTimeout*** operation would throw UnsupportedOperationException. This makes it weird when converting a streaming query into a batch query by changing the input DF from streaming to a batch DF. If the timeout was enabled and used, then the batch query will start throwing UnsupportedOperationException.

This PR creates the dummy state in batch queries with the provided timeoutConf so that it behaves in the same way. The code has been refactored to make it obvious when the state is being created for a batch query or a streaming query.

## How was this patch tested?
Additional tests

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

Closes #18024 from tdas/SPARK-20792.
2017-05-21 13:07:25 -07:00
Yuming Wang bff021dfaf [SPARK-20751][SQL] Add built-in SQL Function - COT
## What changes were proposed in this pull request?

Add built-in SQL Function - COT.

## How was this patch tested?

unit tests

Author: Yuming Wang <wgyumg@gmail.com>

Closes #17999 from wangyum/SPARK-20751.
2017-05-19 09:40:22 -07:00
tpoterba 3f2cd51ee0 [SPARK-20773][SQL] ParquetWriteSupport.writeFields is quadratic in number of fields
Fix quadratic List indexing in ParquetWriteSupport.

I noticed this function while profiling some code with today. It showed up as a significant factor in a table with twenty columns; with hundreds of columns, it could dominate any other function call.

## What changes were proposed in this pull request?

The writeFields method iterates from 0 until number of fields, indexing into rootFieldWriters for each element. rootFieldWriters is a List, so indexing is a linear operation. The complexity of the writeFields method is thus quadratic in the number of fields.

Solution: explicitly convert rootFieldWriters to Array (implicitly converted to WrappedArray) for constant-time indexing.

## How was this patch tested?

This is a one-line change for performance reasons.

Author: tpoterba <tpoterba@broadinstitute.org>
Author: Tim Poterba <tpoterba@gmail.com>

Closes #18005 from tpoterba/tpoterba-patch-1.
2017-05-19 14:17:12 +02:00
Ala Luszczak ce8edb8bf4 [SPARK-20798] GenerateUnsafeProjection should check if a value is null before calling the getter
## What changes were proposed in this pull request?

GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior.

Example of code generated before:
```scala
/* 059 */         final UTF8String fieldName = value.getUTF8String(0);
/* 060 */         if (value.isNullAt(0)) {
/* 061 */           rowWriter1.setNullAt(0);
/* 062 */         } else {
/* 063 */           rowWriter1.write(0, fieldName);
/* 064 */         }
```

Example of code generated now:
```scala
/* 060 */         boolean isNull1 = value.isNullAt(0);
/* 061 */         UTF8String value1 = isNull1 ? null : value.getUTF8String(0);
/* 062 */         if (isNull1) {
/* 063 */           rowWriter1.setNullAt(0);
/* 064 */         } else {
/* 065 */           rowWriter1.write(0, value1);
/* 066 */         }
```

## How was this patch tested?

Adds GenerateUnsafeProjectionSuite.

Author: Ala Luszczak <ala@databricks.com>

Closes #18030 from ala/fix-generate-unsafe-projection.
2017-05-19 13:18:48 +02:00
hyukjinkwon 8fb3d5c6da [SPARK-20364][SQL] Disable Parquet predicate pushdown for fields having dots in the names
## What changes were proposed in this pull request?

This is an alternative workaround by simply avoiding the predicate pushdown for columns having dots in the names. This is an approach different with https://github.com/apache/spark/pull/17680.

The downside of this PR is, literally it does not push down filters on the column having dots in Parquet files at all (both no record level and no rowgroup level) whereas the downside of the approach in that PR, it does not use the Parquet's API properly but in a hacky way to support this case.

I assume we prefer a safe way here by using the Parquet API properly but this does close that PR as we are basically just avoiding here.

This way looks a simple workaround and probably it is fine given the problem looks arguably rather corner cases (although it might end up with reading whole row groups under the hood but either looks not the best).

Currently, if there are dots in the column name, predicate pushdown seems being failed in Parquet.

**With dots**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("col.dots").write.parquet(path)
spark.read.parquet(path).where("`col.dots` IS NOT NULL").show()
```

```
+--------+
|col.dots|
+--------+
+--------+
```

**Without dots**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("coldots").write.parquet(path)
spark.read.parquet(path).where("`coldots` IS NOT NULL").show()
```

```
+-------+
|coldots|
+-------+
|      1|
+-------+
```

**After**

```scala
val path = "/tmp/abcde"
Seq(Some(1), None).toDF("col.dots").write.parquet(path)
spark.read.parquet(path).where("`col.dots` IS NOT NULL").show()
```

```
+--------+
|col.dots|
+--------+
|       1|
+--------+
```

## How was this patch tested?

Unit tests added in `ParquetFilterSuite`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18000 from HyukjinKwon/SPARK-20364-workaround.
2017-05-18 10:52:23 -07:00
Liang-Chi Hsieh 7463a88be6 [SPARK-20690][SQL] Subqueries in FROM should have alias names
## What changes were proposed in this pull request?

We add missing attributes into Filter in Analyzer. But we shouldn't do it through subqueries like this:

    select 1 from  (select 1 from onerow t1 LIMIT 1) where  t1.c1=1

This query works in current codebase. However, the outside where clause shouldn't be able to refer `t1.c1` attribute.

The root cause is we allow subqueries in FROM have no alias names previously, it is confusing and isn't supported by various databases such as MySQL, Postgres, Oracle. We shouldn't support it too.

## How was this patch tested?

Jenkins tests.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #17935 from viirya/SPARK-20690.
2017-05-17 12:57:35 +08:00
Kazuaki Ishizaki 6f62e9d9b9 [SPARK-19372][SQL] Fix throwing a Java exception at df.fliter() due to 64KB bytecode size limit
## What changes were proposed in this pull request?

When an expression for `df.filter()` has many nodes (e.g. 400), the size of Java bytecode for the generated Java code is more than 64KB. It produces an Java exception. As a result, the execution fails.
This PR continues to execute by calling `Expression.eval()` disabling code generation if an exception has been caught.

## How was this patch tested?

Add a test suite into `DataFrameSuite`

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

Closes #17087 from kiszk/SPARK-19372.
2017-05-16 14:47:21 -07:00
Takuya UESHIN c8c878a416 [SPARK-20588][SQL] Cache TimeZone instances.
## What changes were proposed in this pull request?

Because the method `TimeZone.getTimeZone(String ID)` is synchronized on the TimeZone class, concurrent call of this method will become a bottleneck.
This especially happens when casting from string value containing timezone info to timestamp value, which uses `DateTimeUtils.stringToTimestamp()` and gets TimeZone instance on the site.

This pr makes a cache of the generated TimeZone instances to avoid the synchronization.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #17933 from ueshin/issues/SPARK-20588.
2017-05-15 16:52:22 -07:00
Dongjoon Hyun bbd163d589 [SPARK-20735][SQL][TEST] Enable cross join in TPCDSQueryBenchmark
## What changes were proposed in this pull request?

Since [SPARK-17298](https://issues.apache.org/jira/browse/SPARK-17298), some queries (q28, q61, q77, q88, q90) in the test suites fail with a message "_Use the CROSS JOIN syntax to allow cartesian products between these relations_".

This benchmark is used as a reference model for Spark TPC-DS, so this PR aims to enable the correct configuration in `TPCDSQueryBenchmark.scala`.

## How was this patch tested?

Manual. (Run TPCDSQueryBenchmark)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #17977 from dongjoon-hyun/SPARK-20735.
2017-05-15 11:24:30 -07:00
Tathagata Das 499ba2cb47 [SPARK-20717][SS] Minor tweaks to the MapGroupsWithState behavior
## What changes were proposed in this pull request?

Timeout and state data are two independent entities and should be settable independently. Therefore, in the same call of the user-defined function, one should be able to set the timeout before initializing the state and also after removing the state. Whether timeouts can be set or not, should not depend on the current state, and vice versa.

However, a limitation of the current implementation is that state cannot be null while timeout is set. This is checked lazily after the function call has completed.

## How was this patch tested?
- Updated existing unit tests that test the behavior of GroupState.setTimeout*** wrt to the current state
- Added new tests that verify the disallowed cases where state is undefined but timeout is set.

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

Closes #17957 from tdas/SPARK-20717.
2017-05-15 10:48:10 -07:00
Tejas Patil d2416925c4 [SPARK-17729][SQL] Enable creating hive bucketed tables
## What changes were proposed in this pull request?

Hive allows inserting data to bucketed table without guaranteeing bucketed and sorted-ness based on these two configs : `hive.enforce.bucketing` and `hive.enforce.sorting`.

What does this PR achieve ?
- Spark will disallow users from writing outputs to hive bucketed tables by default (given that output won't adhere with Hive's semantics).
- IF user still wants to write to hive bucketed table, the only resort is to use `hive.enforce.bucketing=false` and `hive.enforce.sorting=false` which means user does NOT care about bucketing guarantees.

Changes done in this PR:
- Extract table's bucketing information in `HiveClientImpl`
- While writing table info to metastore, `HiveClientImpl` now populates the bucketing information in the hive `Table` object
- `InsertIntoHiveTable` allows inserts to bucketed table only if both `hive.enforce.bucketing` and `hive.enforce.sorting` are `false`

Ability to create bucketed tables will enable adding test cases to Spark while I add more changes related to hive bucketing support. Design doc for hive hive bucketing support : https://docs.google.com/document/d/1a8IDh23RAkrkg9YYAeO51F4aGO8-xAlupKwdshve2fc/edit#

## How was this patch tested?
- Added test for creating bucketed and sorted table.
- Added test to ensure that INSERTs fail if strict bucket / sort is enforced
- Added test to ensure that INSERTs can go through if strict bucket / sort is NOT enforced
- Added test to validate that bucketing information shows up in output of DESC FORMATTED
- Added test to ensure that `SHOW CREATE TABLE` works for hive bucketed tables

Author: Tejas Patil <tejasp@fb.com>

Closes #17644 from tejasapatil/SPARK-17729_create_bucketed_table.
2017-05-16 01:47:23 +08:00
Tathagata Das 271175e2bd [SPARK-20716][SS] StateStore.abort() should not throw exceptions
## What changes were proposed in this pull request?

StateStore.abort() should do a best effort attempt to clean up temporary resources. It should not throw errors, especially because its called in a TaskCompletionListener, because this error could hide previous real errors in the task.

## How was this patch tested?
No unit test.

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

Closes #17958 from tdas/SPARK-20716.
2017-05-15 10:46:38 -07:00
Takeshi Yamamuro b0888d1ac3 [SPARK-20730][SQL] Add an optimizer rule to combine nested Concat
## What changes were proposed in this pull request?
This pr added a new Optimizer rule to combine nested Concat. The master supports a pipeline operator '||' to concatenate strings in #17711 (This pr is follow-up). Since the parser currently generates nested Concat expressions, the optimizer needs to combine the nested expressions.

## How was this patch tested?
Added tests in `CombineConcatSuite` and `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17970 from maropu/SPARK-20730.
2017-05-15 16:24:55 +08:00
Wenchen Fan 1283c3d11a [SPARK-20725][SQL] partial aggregate should behave correctly for sameResult
## What changes were proposed in this pull request?

For aggregate function with `PartialMerge` or `Final` mode, the input is aggregate buffers instead of the actual children expressions. So the actual children expressions won't affect the result, we should normalize the expr id for them.

## How was this patch tested?

a new regression test

Author: Wenchen Fan <wenchen@databricks.com>

Closes #17964 from cloud-fan/tmp.
2017-05-13 12:09:06 -07:00
hyukjinkwon 3f98375d8a [SPARK-18772][SQL] Avoid unnecessary conversion try for special floats in JSON
## What changes were proposed in this pull request?

This PR is based on  https://github.com/apache/spark/pull/16199 and extracts the valid change from https://github.com/apache/spark/pull/9759 to resolve SPARK-18772

This avoids additional conversion try with `toFloat` and `toDouble`.

For avoiding additional conversions, please refer the codes below:

**Before**

```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:30:41 ERROR Executor: Exception in task 0.0 in stage 2.0 (TID 2)
java.lang.NumberFormatException: For input string: "nan"
...
```

**After**

```scala
scala> import org.apache.spark.sql.types._
import org.apache.spark.sql.types._

scala> spark.read.schema(StructType(Seq(StructField("a", DoubleType)))).option("mode", "FAILFAST").json(Seq("""{"a": "nan"}""").toDS).show()
17/05/12 11:44:30 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.RuntimeException: Cannot parse nan as DoubleType.
...
```

## How was this patch tested?

Unit tests added in `JsonSuite`.

Closes #16199

Author: hyukjinkwon <gurwls223@gmail.com>
Author: Nathan Howell <nhowell@godaddy.com>

Closes #17956 from HyukjinKwon/SPARK-18772.
2017-05-13 20:56:04 +08:00
Xiao Li b84ff7eb62 [SPARK-20719][SQL] Support LIMIT ALL
### What changes were proposed in this pull request?
`LIMIT ALL` is the same as omitting the `LIMIT` clause. It is supported by both PrestgreSQL and Presto. This PR is to support it by adding it in the parser.

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

Author: Xiao Li <gatorsmile@gmail.com>

Closes #17960 from gatorsmile/LimitAll.
2017-05-12 15:26:10 -07:00
Tathagata Das 0d3a63193c [SPARK-20714][SS] Fix match error when watermark is set with timeout = no timeout / processing timeout
## What changes were proposed in this pull request?

When watermark is set, and timeout conf is NoTimeout or ProcessingTimeTimeout (both do not need the watermark), the query fails at runtime with the following exception.
```
MatchException: Some(org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate1a9b798e) (of class scala.Some)
    org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:120)
    org.apache.spark.sql.execution.streaming.FlatMapGroupsWithStateExec$$anonfun$doExecute$1.apply(FlatMapGroupsWithStateExec.scala:116)
    org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:70)
    org.apache.spark.sql.execution.streaming.state.package$StateStoreOps$$anonfun$1.apply(package.scala:65)
    org.apache.spark.sql.execution.streaming.state.StateStoreRDD.compute(StateStoreRDD.scala:64)
```

The match did not correctly handle cases where watermark was defined by the timeout was different from EventTimeTimeout.

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

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

Closes #17954 from tdas/SPARK-20714.
2017-05-12 10:49:50 -07:00
Takeshi Yamamuro b526f70c16 [SPARK-19951][SQL] Add string concatenate operator || to Spark SQL
## What changes were proposed in this pull request?
This pr added code to support `||` for string concatenation. This string operation is supported in PostgreSQL and MySQL.

## How was this patch tested?
Added tests in `SparkSqlParserSuite`

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17711 from maropu/SPARK-19951.
2017-05-12 09:55:51 -07:00
Takeshi Yamamuro 92ea7fd7b6 [SPARK-20710][SQL] Support aliases in CUBE/ROLLUP/GROUPING SETS
## What changes were proposed in this pull request?
This pr added  `Analyzer` code for supporting aliases in CUBE/ROLLUP/GROUPING SETS (This is follow-up of #17191).

## How was this patch tested?
Added tests in `SQLQueryTestSuite`.

Author: Takeshi Yamamuro <yamamuro@apache.org>

Closes #17948 from maropu/SPARK-20710.
2017-05-12 20:48:30 +08:00
wangzhenhua 54b4f2ad43 [SPARK-20718][SQL][FOLLOWUP] Fix canonicalization for HiveTableScanExec
## What changes were proposed in this pull request?

Fix canonicalization for different filter orders in `HiveTableScanExec`.

## How was this patch tested?

Added a new test case.

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #17962 from wzhfy/canonicalizeHiveTableScanExec.
2017-05-12 20:43:22 +08:00
Sean Owen fc8a2b6ee6 [SPARK-20554][BUILD] Remove usage of scala.language.reflectiveCalls
## What changes were proposed in this pull request?

Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17949 from srowen/SPARK-20554.
2017-05-12 09:55:04 +01:00
hyukjinkwon 720708ccdd [SPARK-20639][SQL] Add single argument support for to_timestamp in SQL with documentation improvement
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Use casting rules to a timestamp in `to_timestamp` by default (it was `yyyy-MM-dd HH:mm:ss`).

- Support single argument for `to_timestamp` similarly with APIs in other languages.

  For example, the one below works

  ```
  import org.apache.spark.sql.functions._
  Seq("2016-12-31 00:12:00.00").toDF("a").select(to_timestamp(col("a"))).show()
  ```

  prints

  ```
  +----------------------------------------+
  |to_timestamp(`a`, 'yyyy-MM-dd HH:mm:ss')|
  +----------------------------------------+
  |                     2016-12-31 00:12:00|
  +----------------------------------------+
  ```

  whereas this does not work in SQL.

  **Before**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  Error in query: Invalid number of arguments for function to_timestamp; line 1 pos 7
  ```

  **After**

  ```
  spark-sql> SELECT to_timestamp('2016-12-31 00:12:00');
  2016-12-31 00:12:00
  ```

- Related document improvement for SQL function descriptions and other API descriptions accordingly.

  **Before**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage: to_date(date_str, fmt) - Parses the `left` expression with the `fmt` expression. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
  Usage: to_timestamp(timestamp, fmt) - Parses the `left` expression with the `format` expression to a timestamp. Returns null with invalid input.
  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00.0
  ```

  **After**

  ```
  spark-sql> DESCRIBE FUNCTION extended to_date;
  ...
  Usage:
      to_date(date_str[, fmt]) - Parses the `date_str` expression with the `fmt` expression to
        a date. Returns null with invalid input. By default, it follows casting rules to a date if
        the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_date('2009-07-30 04:17:52');
         2009-07-30
        > SELECT to_date('2016-12-31', 'yyyy-MM-dd');
         2016-12-31
  ```

  ```
  spark-sql> DESCRIBE FUNCTION extended to_timestamp;
  ...
   Usage:
      to_timestamp(timestamp[, fmt]) - Parses the `timestamp` expression with the `fmt` expression to
        a timestamp. Returns null with invalid input. By default, it follows casting rules to
        a timestamp if the `fmt` is omitted.

  Extended Usage:
      Examples:
        > SELECT to_timestamp('2016-12-31 00:12:00');
         2016-12-31 00:12:00
        > SELECT to_timestamp('2016-12-31', 'yyyy-MM-dd');
         2016-12-31 00:00:00
  ```

## How was this patch tested?

Added tests in `datetime.sql`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17901 from HyukjinKwon/to_timestamp_arg.
2017-05-12 16:42:58 +08:00