Commit graph

2227 commits

Author SHA1 Message Date
Gengliang Wang b5e183cdc7 [SPARK-28108][SQL][test-hadoop3.2] Simplify OrcFilters
## What changes were proposed in this pull request?

In #24068, IvanVergiliev fixes the issue that OrcFilters.createBuilder has exponential complexity in the height of the filter tree due to the way the check-and-build pattern is implemented.

Comparing to the approach in #24068, I propose a simple solution for the issue:
1. separate the logic of building a convertible filter tree and the actual SearchArgument builder, since the two procedures are different and their return types are different. Thus the new introduced class `ActionType`,`TrimUnconvertibleFilters` and `BuildSearchArgument`  in #24068 can be dropped. The code is more readable.
2. For most of the leaf nodes, the convertible result is always Some(node), we can abstract it like this PR.
3. The code is actually small changes on the previous code. See https://github.com/apache/spark/pull/24783

## How was this patch tested?
Run the benchmark provided in #24068:
```
val schema = StructType.fromDDL("col INT")
(20 to 30).foreach { width =>
  val whereFilter = (1 to width).map(i => EqualTo("col", i)).reduceLeft(Or)
  val start = System.currentTimeMillis()
  OrcFilters.createFilter(schema, Seq(whereFilter))
  println(s"With $width filters, conversion takes ${System.currentTimeMillis() - start} ms")
}
```
Result:
```
With 20 filters, conversion takes 6 ms
With 21 filters, conversion takes 0 ms
With 22 filters, conversion takes 0 ms
With 23 filters, conversion takes 0 ms
With 24 filters, conversion takes 0 ms
With 25 filters, conversion takes 0 ms
With 26 filters, conversion takes 0 ms
With 27 filters, conversion takes 0 ms
With 28 filters, conversion takes 0 ms
With 29 filters, conversion takes 0 ms
With 30 filters, conversion takes 0 ms
```

Also verified with Unit tests.

Closes #24910 from gengliangwang/refactorOrcFilters.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-24 12:23:52 +08:00
Liang-Chi Hsieh a00774afea [SPARK-28054][SQL] Fix error when insert Hive partitioned table dynamically where partition name is upper case
## What changes were proposed in this pull request?

When we use upper case partition name in Hive table, like:

```
CREATE TABLE src (KEY STRING, VALUE STRING) PARTITIONED BY (DS STRING)
```

Then, `insert into table` query doesn't work
```
INSERT INTO TABLE src PARTITION(ds) SELECT 'k' key, 'v' value, '1' ds
// or
INSERT INTO TABLE src PARTITION(DS) SELECT 'k' KEY, 'v' VALUE, '1' DS
```

```
[info]   org.apache.spark.sql.AnalysisException:
org.apache.hadoop.hive.ql.metadata.Table.ValidationFailureSemanticException: Partition spec {ds=, DS=1} contains non-partition columns;
```

As Hive metastore is not case preserving and keeps partition columns with lower cased names, we lowercase column names in partition spec before passing to Hive client. But we write upper case column names in partition paths.

However, when calling `loadDynamicPartitions` to do `insert into table` for dynamic partition, Hive calculates full path spec for partition paths. So it calculates a partition spec like `{ds=, DS=1}` in above case and fails partition column validation. This patch is proposed to fix the issue by lowercasing the column names in written partition paths for Hive partitioned table.

This fix touchs `saveAsHiveFile` method, which is used in `InsertIntoHiveDirCommand` and `InsertIntoHiveTable` commands. Among them, only `InsertIntoHiveTable` passes `partitionAttributes` parameter. So I think this change only affects `InsertIntoHiveTable` command.

## How was this patch tested?

Added test.

Closes #24886 from viirya/SPARK-28054.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-24 09:44:38 +09:00
sychen d9697fedf5 [SPARK-28012][SQL] Hive UDF supports struct type foldable expression
## What changes were proposed in this pull request?

Currently using hive udf, the parameter is struct type, there will be an exception thrown.

No handler for Hive UDF 'xxxUDF': java.lang.RuntimeException: Hive doesn't support the constant type [StructType(StructField(name,StringType,true), StructField(value,DecimalType(3,1),true))]

## How was this patch tested?
added new UT

Closes #24846 from cxzl25/hive_udf_literal_struct_type.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-20 14:36:01 +09:00
Yuming Wang 4968f87168 [SPARK-23263][TEST] CTAS should update stat if autoUpdate statistics is enabled
## What changes were proposed in this pull request?
The [SPARK-27403](https://issues.apache.org/jira/browse/SPARK-27403) fixed CTAS cannot update statistics even if `spark.sql.statistics.size.autoUpdate.enabled` is enabled, as mentioned in [SPARK-23263](https://issues.apache.org/jira/browse/SPARK-23263).

This pr adds tests for that fix.

## How was this patch tested?

N/A

Closes #20430 from wangyum/SPARK-23263.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-20 14:19:10 +09:00
Ivan Vergiliev a5dcb82b5a [SPARK-27105][SQL] Optimize away exponential complexity in ORC predicate conversion
## What changes were proposed in this pull request?

`OrcFilters.createBuilder` has exponential complexity in the height of the filter tree due to the way the check-and-build pattern is implemented. We've hit this in production by passing a `Column` filter to Spark directly, with a job taking multiple hours for a simple set of ~30 filters. This PR changes the checking logic so that the conversion has linear complexity in the size of the tree instead of exponential in its height.

Right now, due to the way ORC `SearchArgument` works, the code is forced to do two separate phases when converting a given Spark filter to an ORC filter:
1. Check if the filter is convertible.
2. Only if the check in 1. succeeds, perform the actual conversion into the resulting ORC filter.

However, there's one detail which is the culprit in the exponential complexity: phases 1. and 2. are both done using the exact same method. The resulting exponential complexity is easiest to see in the `NOT` case - consider the following code:

```
val f1 = col("id") === lit(5)
val f2 = !f1
val f3 = !f2
val f4 = !f3
val f5 = !f4
```

Now, when we run `createBuilder` on `f5`, we get the following behaviour:
1. call `createBuilder(f4)` to check if the child `f4` is convertible
2. call `createBuilder(f4)` to actually convert it

This seems fine when looking at a single level, but what actually ends up happening is:
- `createBuilder(f3)` will then recursively be called 4 times - 2 times in step 1., and two times in step 2.
- `createBuilder(f2)` will be called 8 times - 4 times in each top-level step, 2 times in each sub-step.
- `createBuilder(f1)` will be called 16 times.

As a result, having a tree of height > 30 leads to billions of calls to `createBuilder`, heap allocations, and so on and can take multiple hours.

The way this PR solves this problem is by separating the `check` and `convert` functionalities into separate functions. This way, the call to `createBuilder` on `f5` above would look like this:
1. call `isConvertible(f4)` to check if the child `f4` is convertible - amortized constant complexity
2. call `createBuilder(f4)` to actually convert it - linear complexity in the size of the subtree.

This way, we get an overall complexity that's linear in the size of the filter tree, allowing us to convert tree with 10s of thousands of nodes in milliseconds.

The reason this split (`check` and `build`) is possible is that the checking never actually depends on the actual building of the filter. The `check` part of `createBuilder` depends mainly on:
- `isSearchableType` for leaf nodes, and
- `check`-ing the child filters for composite nodes like NOT, AND and OR.
Situations like the `SearchArgumentBuilder` throwing an exception while building the resulting ORC filter are not handled right now - they just get thrown out of the class, and this change preserves this behaviour.

This PR extracts this part of the code to a separate class which allows the conversion to make very efficient checks to confirm that a given child is convertible before actually converting it.

Results:
Before:
- converting a skewed tree with a height of ~35 took about 6-7 hours.
- converting a skewed tree with hundreds or thousands of nodes would be completely impossible.

Now:
- filtering against a skewed tree with a height of 1500 in the benchmark suite finishes in less than 10 seconds.

## Steps to reproduce
```scala
val schema = StructType.fromDDL("col INT")
(20 to 30).foreach { width =>
  val whereFilter = (1 to width).map(i => EqualTo("col", i)).reduceLeft(Or)
  val start = System.currentTimeMillis()
  OrcFilters.createFilter(schema, Seq(whereFilter))
  println(s"With $width filters, conversion takes ${System.currentTimeMillis() - start} ms")
}
```

### Before this PR
```
With 20 filters, conversion takes 363 ms
With 21 filters, conversion takes 496 ms
With 22 filters, conversion takes 939 ms
With 23 filters, conversion takes 1871 ms
With 24 filters, conversion takes 3756 ms
With 25 filters, conversion takes 7452 ms
With 26 filters, conversion takes 14978 ms
With 27 filters, conversion takes 30519 ms
With 28 filters, conversion takes 60361 ms // 1 minute
With 29 filters, conversion takes 126575 ms // 2 minutes 6 seconds
With 30 filters, conversion takes 257369 ms // 4 minutes 17 seconds
```

### After this PR
```
With 20 filters, conversion takes 12 ms
With 21 filters, conversion takes 0 ms
With 22 filters, conversion takes 1 ms
With 23 filters, conversion takes 0 ms
With 24 filters, conversion takes 1 ms
With 25 filters, conversion takes 1 ms
With 26 filters, conversion takes 0 ms
With 27 filters, conversion takes 1 ms
With 28 filters, conversion takes 0 ms
With 29 filters, conversion takes 1 ms
With 30 filters, conversion takes 0 ms
```

## How was this patch tested?

There are no changes in behaviour, and the existing tests pass. Added new benchmarks that expose the problematic behaviour and they finish quickly with the changes applied.

Closes #24068 from IvanVergiliev/optimize-orc-filters.

Authored-by: Ivan Vergiliev <ivan.vergiliev@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-19 10:44:58 +08:00
Gengliang Wang 23ebd389b5 [SPARK-27418][SQL] Migrate Parquet to File Data Source V2
## What changes were proposed in this pull request?

 Migrate Parquet to File Data Source V2

## How was this patch tested?

Unit test

Closes #24327 from gengliangwang/parquetV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-06-15 20:52:50 +09:00
Zhu, Lipeng 5700c39c89 [SPARK-27578][SQL] Support INTERVAL ... HOUR TO SECOND syntax
## What changes were proposed in this pull request?

Currently, SparkSQL can support interval format like this.
```sql
SELECT INTERVAL '0 23:59:59.155' DAY TO SECOND
 ```

Like Presto/Teradata, this PR aims to support grammar like below.
```sql
SELECT INTERVAL '23:59:59.155' HOUR TO SECOND
```

Although we can add a new function for this pattern, we had better extend the existing code to handle a missing day case. So, the following is also supported.
```sql
SELECT INTERVAL '23:59:59.155' DAY TO SECOND
SELECT INTERVAL '1 23:59:59.155' HOUR TO SECOND
```
Currently Vertica/Teradata/Postgresql/SQL Server have fully support of below interval functions.
- interval ... year to month
- interval ... day to hour
- interval ... day to minute
- interval ... day to second
- interval ... hour to minute
- interval ... hour to second
- interval ... minute to second

https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/LanguageElements/Literals/interval-qualifier.htm
df1a699e5b/src/test/regress/sql/interval.sql (L180-L203)
https://docs.teradata.com/reader/S0Fw2AVH8ff3MDA0wDOHlQ/KdCtT3pYFo~_enc8~kGKVw
https://docs.microsoft.com/en-us/sql/odbc/reference/appendixes/interval-literals?view=sql-server-2017

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24472 from lipzhu/SPARK-27578.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-13 10:12:55 -07:00
John Zhuge abe370f971 [SPARK-27322][SQL] DataSourceV2 table relation
## What changes were proposed in this pull request?

Support multi-catalog in the following SELECT code paths:

- SELECT * FROM catalog.db.tbl
- TABLE catalog.db.tbl
- JOIN or UNION tables from different catalogs
- SparkSession.table("catalog.db.tbl")
- CTE relation
- View text

## How was this patch tested?

New unit tests.
All existing unit tests in catalyst and sql core.

Closes #24741 from jzhuge/SPARK-27322-pr.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-06-13 13:48:40 +08:00
LantaoJin 63e0711524 [SPARK-27899][SQL] Make HiveMetastoreClient.getTableObjectsByName available in ExternalCatalog/SessionCatalog API
## What changes were proposed in this pull request?

The new Spark ThriftServer SparkGetTablesOperation implemented in https://github.com/apache/spark/pull/22794 does a catalog.getTableMetadata request for every table. This can get very slow for large schemas (~50ms per table with an external Hive metastore).
Hive ThriftServer GetTablesOperation uses HiveMetastoreClient.getTableObjectsByName to get table information in bulk, but we don't expose that through our APIs that go through Hive -> HiveClientImpl (HiveClient) -> HiveExternalCatalog (ExternalCatalog) -> SessionCatalog.

If we added and exposed getTableObjectsByName through our catalog APIs, we could resolve that performance problem in SparkGetTablesOperation.

## How was this patch tested?

Add UT

Closes #24774 from LantaoJin/SPARK-27899.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-11 15:32:59 +08:00
Yuming Wang 2926890ffb [SPARK-27970][SQL] Support Hive 3.0 metastore
## What changes were proposed in this pull request?

It seems that some users are using Hive 3.0.0. This pr makes it support Hive 3.0 metastore.

## How was this patch tested?

unit tests

Closes #24688 from wangyum/SPARK-26145.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-07 15:24:07 -07:00
LantaoJin 18834e85d0 [SPARK-27899][SQL] Refactor getTableOption() to extract a common method
## What changes were proposed in this pull request?

This is a part of #24774, to reduce the code changes made by that.

## How was this patch tested?

Exist UTs.

Closes #24803 from LantaoJin/SPARK-27899_refactor.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:36:25 -07:00
Dongjoon Hyun 8486680b34 [SPARK-24544][SQL][FOLLOWUP] Remove a wrong warning on Hive fallback lookup
## What changes were proposed in this pull request?

This PR is a follow-up of https://github.com/apache/spark/pull/21790 which causes a regression to show misleading warnings always at first invocation for all Hive function. Hive fallback lookup should not be warned. It's a normal process in function lookups.

**CURRENT (Showing `NoSuchFunctionException` and working)**
```scala
scala> sql("select histogram_numeric(a,2) from values(1) T(a)").show
19/06/02 22:02:10 WARN HiveSessionCatalog: Encountered a failure during looking up
function: org.apache.spark.sql.catalyst.analysis.NoSuchFunctionException:
Undefined function: 'histogram_numeric'. This function is neither a registered temporary
function nor a permanent function registered in the database 'default'.;
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.failFunctionLookup(SessionCatalog.scala:1234)
  at org.apache.spark.sql.catalyst.catalog.SessionCatalog.lookupFunction(SessionCatalog.scala:1302)
...
+------------------------+
|histogram_numeric( a, 2)|
+------------------------+
|            [[1.0, 1.0]]|
+------------------------+
```

## How was this patch tested?

Manually execute the above query.

Closes #24773 from dongjoon-hyun/SPARK-24544.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-03 00:04:00 -07:00
Yuming Wang d53b61c311 [SPARK-27831][SQL][TEST] Move Hive test jars to maven dependency
## What changes were proposed in this pull request?

This pr moves Hive test jars(`hive-contrib-0.13.1.jar`, `hive-hcatalog-core-0.13.1.jar`, `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.5.jar`) to maven dependency.

## How was this patch tested?

Existing test

Please note that this pr need test with `maven` and `sbt`.

Closes #24751 from wangyum/SPARK-27831.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 20:23:08 -07:00
Ajith 3806887afb [SPARK-27907][SQL] HiveUDAF should return NULL in case of 0 rows
## What changes were proposed in this pull request?

When query returns zero rows, the HiveUDAFFunction throws NPE

## CASE 1:
create table abc(a int)
select histogram_numeric(a,2) from abc // NPE
```
Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 0, localhost, executor driver): java.lang.NullPointerException
	at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:471)
	at org.apache.spark.sql.hive.HiveUDAFFunction.eval(hiveUDFs.scala:315)
	at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.eval(interfaces.scala:543)
	at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$5(AggregationIterator.scala:231)
	at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132)
	at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839)
	at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
	at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
	at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:122)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
```

## CASE 2:
create table abc(a int)
insert into abc values (1)
select histogram_numeric(a,2) from abc where a=3 // NPE

```
Job aborted due to stage failure: Task 0 in stage 4.0 failed 1 times, most recent failure: Lost task 0.0 in stage 4.0 (TID 5, localhost, executor driver): java.lang.NullPointerException
at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:477)
at org.apache.spark.sql.hive.HiveUDAFFunction.serialize(hiveUDFs.scala:315)
at org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate.serializeAggregateBufferInPlace(interfaces.scala:570)
at org.apache.spark.sql.execution.aggregate.AggregationIterator.$anonfun$generateResultProjection$6(AggregationIterator.scala:254)
at org.apache.spark.sql.execution.aggregate.ObjectAggregationIterator.outputForEmptyGroupingKeyWithoutInput(ObjectAggregationIterator.scala:97)
at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2(ObjectHashAggregateExec.scala:132)
at org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec.$anonfun$doExecute$2$adapted(ObjectHashAggregateExec.scala:107)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2(RDD.scala:839)
at org.apache.spark.rdd.RDD.$anonfun$mapPartitionsWithIndexInternal$2$adapted(RDD.scala:839)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)
at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:327)
at org.apache.spark.rdd.RDD.iterator(RDD.scala:291)
at org.apache.spark.shuffle.ShuffleWriteProcessor.write(ShuffleWriteProcessor.scala:59)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:94)
at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:52)
at org.apache.spark.scheduler.Task.run(Task.scala:122)
at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:425)
at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1350)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:428)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```

Hence add a check not avoid NPE

## How was this patch tested?

Added new UT case

Closes #24762 from ajithme/hiveudaf.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-02 10:54:21 -07:00
Dongjoon Hyun 955eef95b3 Revert "[SPARK-27831][SQL][TEST][test-hadoop3.2] Move Hive test jars to maven dependency"
This reverts commit 24180c00e0.
2019-05-30 10:06:55 -07:00
Yuming Wang db3e746b64 [SPARK-27875][CORE][SQL][ML][K8S] Wrap all PrintWriter with Utils.tryWithResource
## What changes were proposed in this pull request?

This pr wrap all `PrintWriter` with `Utils.tryWithResource` to prevent resource leak.

## How was this patch tested?

Existing test

Closes #24739 from wangyum/SPARK-27875.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-30 19:54:32 +09:00
Yuming Wang 4e61de4380 [SPARK-27863][SQL] Metadata files and temporary files should not be counted as data files
## What changes were proposed in this pull request?
[`DataSourceUtils.isDataPath(path)`](https://github.com/apache/spark/blob/v2.4.3/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CommandUtils.scala#L95) should be `DataSourceUtils.isDataPath(status.getPath)`.

This pr fix this issue.

## How was this patch tested?

unit tests

Closes #24725 from wangyum/SPARK-27863.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-28 09:28:35 -07:00
Yuming Wang 193304b51b [SPARK-27441][SQL][TEST] Add read/write tests to Hive serde tables
## What changes were proposed in this pull request?

The versions between Hive, Parquet and ORC after the built-in Hive upgraded to 2.3.5 for Hadoop 3.2:

- built-in Hive is 1.2.1.spark2:

  | ORC | Parquet
-- | -- | --
Spark datasource table | 1.5.5 | 1.10.1
Spark hive table | Hive built-in | 1.6.0
Apache Hive 1.2.1 | Hive built-in | 1.6.0

- built-in Hive is 2.3.5:

  | ORC | Parquet
-- | -- | --
Spark datasource table | 1.5.5 | 1.10.1
Spark hive table | 1.5.5 | [1.10.1](https://github.com/apache/spark/pull/24346)
Apache Hive 2.3.5 | 1.3.4 | 1.8.1

We should add a test for Hive Serde table. This pr adds tests to test read/write of all supported data types using Parquet and ORC.

## How was this patch tested?

unit tests

Closes #24345 from wangyum/SPARK-27441.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 08:35:58 -07:00
Yuming Wang dcacfc5da6 [SPARK-27074][SQL][test-hadoop3.2][test-maven] Hive 3.1 metastore support HiveClientImpl.runHive
## What changes were proposed in this pull request?

Hive 3.1.1's `CommandProcessor` have 2 changes:
1. [HIVE-17626](https://issues.apache.org/jira/browse/HIVE-17626)(Hive 3.0.0) add ReExecDriver. So the current code path is: 02bbe977ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L736-L742)
We can disable `hive.query.reexecution.enabled` to workaround this change.
2. [HIVE-18238](http://issues.apache.org/jira/browse/HIVE-18238)(Hive 3.0.0) changed the `Driver.close()` function return type. We can workaround it by ` driver.getClass.getMethod("close").invoke(driver)`

So Hive 3.1 metastore could support `HiveClientImpl.runHive` after this pr.

## How was this patch tested?

unit tests

Closes #23992 from wangyum/SPARK-27074.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-26 08:24:41 -07:00
Yuming Wang 24180c00e0 [SPARK-27831][SQL][TEST][test-hadoop3.2] Move Hive test jars to maven dependency
## What changes were proposed in this pull request?

This pr moves Hive test jars(`hive-contrib-0.13.1.jar`, `hive-hcatalog-core-0.13.1.jar`, `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.5.jar`) to maven dependency.

## How was this patch tested?

Existing test

Closes #24695 from wangyum/SPARK-27831.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-24 10:33:34 -07:00
Yuming Wang 76988dd4a2 [SPARK-27737][FOLLOW-UP][SQL][test-hadoop3.2] Update Hive test jars from 2.3.4 to 2.3.5
## What changes were proposed in this pull request?

This pr update `hive-contrib-2.3.4.jar` to `hive-contrib-2.3.5.jar` and `hive-hcatalog-core-2.3.4.jar` to `hive-hcatalog-core-2.3.5.jar`.

## How was this patch tested?

Existing test

Closes #24673 from wangyum/SPARK-27737-hive.jar.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-22 08:29:06 -07:00
Dongjoon Hyun a24cdc00bf [SPARK-27800][SQL][HOTFIX][FOLLOWUP] Fix wrong answer on BitwiseXor test cases
This PR is a follow up of https://github.com/apache/spark/pull/24669 to fix the wrong answers used in test cases.

Closes #24674 from dongjoon-hyun/SPARK-27800.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-22 03:11:29 -07:00
Wenchen Fan 03c9e8adee [SPARK-24586][SQL] Upcast should not allow casting from string to other types
## What changes were proposed in this pull request?

When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet.

However, the current upcast behavior is a little weird, we don't allow up casting from string to numeric, but allow non-numeric types as the target, like boolean, date, etc.

As a result, `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution.

The motivation of the up cast is to prevent things like runtime NPE, it's more reasonable to make up cast stricter.

This PR does 2 things:
1. rename `Cast.canSafeCast` to `Cast.canUpcast`, and support complex typres
2. remove `Cast.mayTruncate` and replace it with `!Cast.canUpcast`

Note that, the up cast change also affects persistent view resolution. But since we don't support changing column types of an existing table, there is no behavior change here.

## How was this patch tested?

new tests

Closes #21586 from cloud-fan/cast.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-22 11:35:51 +08:00
Yuming Wang 6cd1efd0ae [SPARK-27737][SQL] Upgrade to Hive 2.3.5 for Hive Metastore Client and Hadoop-3.2 profile
## What changes were proposed in this pull request?

This PR aims to upgrade to Hive 2.3.5 for Hive Metastore Client and Hadoop-3.2 profile.

Release Notes - Hive - Version 2.3.5

- [[HIVE-21536](https://issues.apache.org/jira/browse/HIVE-21536)] - Backport HIVE-17764 to branch-2.3
- [[HIVE-21585](https://issues.apache.org/jira/browse/HIVE-21585)] - Upgrade branch-2.3 to ORC 1.3.4
- [[HIVE-21639](https://issues.apache.org/jira/browse/HIVE-21639)] - Spark test failed since HIVE-10632
- [[HIVE-21680](https://issues.apache.org/jira/browse/HIVE-21680)] - Backport HIVE-17644 to branch-2 and branch-2.3

https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12345394&styleName=Text&projectId=12310843

## How was this patch tested?

This PR is tested in two ways.
- Pass the Jenkins with the default configuration for `Hive Metastore Client` testing.
- Pass the Jenkins with `test-hadoop3.2` configuration for `Hadoop 3.2` testing.

Closes #24620 from wangyum/SPARK-27737.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-22 10:24:17 +09:00
williamwong 8442d94fb1 [SPARK-27248][SQL] refreshTable should recreate cache with same cache name and storage level
If we refresh a cached table, the table cache will be first uncached and then recache (lazily). Currently, the logic is embedded in CatalogImpl.refreshTable method.
The current implementation does not preserve the cache name and storage level. As a result, cache name and cache level could be changed after a REFERSH. IMHO, it is not what a user would expect.
I would like to fix this behavior by first save the cache name and storage level for recaching the table.

Two unit tests are added to make sure cache name is unchanged upon table refresh. Before applying this patch, the test created for qualified case would fail.

Closes #24221 from William1104/feature/SPARK-27248.

Lead-authored-by: williamwong <william1104@gmail.com>
Co-authored-by: William Wong <william1104@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-21 11:37:16 -07:00
Yuming Wang 5dda1fe296 [SPARK-27699][FOLLOW-UP][SQL][test-hadoop3.2][test-maven] Fix hadoop-3.2 test error
## What changes were proposed in this pull request?

This pr fix `hadoop-3.2` test error:
```
- SPARK-27699 Converting disjunctions into ORC SearchArguments *** FAILED ***
  Expected "...SS_THAN_EQUALS a 10)[
  leaf-1 = (LESS_THAN a 1)
  ]expr = (or (not leaf...", but got "...SS_THAN_EQUALS a 10)[, leaf-1 = (LESS_THAN a 1), ]expr = (or (not leaf..." (HiveOrcFilterSuite.scala:445)
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105514/consoleFull

## How was this patch tested?

N/A

Closes #24639 from wangyum/SPARK-27699.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-20 13:04:05 -07:00
liuxian 9bca99b29b [SPARK-27552][SQL] The configuration hive.exec.stagingdir is invalid on Windows OS
## What changes were proposed in this pull requesst?
If we set `hive.exec.stagingdir=.test-staging\tmp`,
But the staging directory is still `.hive-staging` on Windows OS.

Reasons for failure:
Test code:
```
 val path = new Path("C:\\test\\hivetable")
  println("path.toString: " + path.toString)
  println("path.toUri.getPath: " + path.toUri.getPath)
```

Output:
```
path.toString: C:/test/hivetable
path.toUri.getPath: /C:/test/hivetable
```
 We can see that `path.toUri.getPath` has one more separator than `path.toString`,  and the separator is   ' / ',  not  ' \ '
So `stagingPathName.stripPrefix(inputPathName).stripPrefix(File.separator).startsWith(".")` will return false
## How was this patch tested?
1. Existed tests
2. Manual testing on Windows OS

Closes #24446 from 10110346/stagingdir.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-17 14:00:17 -05:00
Gengliang Wang e39e97b73a [SPARK-27699][SQL] Partially push down disjunctive predicated in Parquet/ORC
## What changes were proposed in this pull request?

Currently, in `ParquetFilters` and `OrcFilters`, if the child predicate of `Or` operator can't be entirely pushed down, the predicates will be thrown away.
In fact, the conjunctive predicates under `Or` operators can be partially pushed down.
For example, says `a` and `b` are convertible, while `c` can't be pushed down, the predicate
`a or (b and c)`
can be converted as
`(a or b) and (a or c)`
We can still push down `(a or b)`.
We can't push down disjunctive predicates only when one of its children is not partially convertible.

This PR also improve the filter pushing down logic in `DataSourceV2Strategy`. With partial filter push down in `Or` operator, the result of `pushedFilters()` might not exist in the mapping `translatedFilterToExpr`.  To fix it, this PR changes the mapping `translatedFilterToExpr` as leaf filter expression to `sources.filter`, and later on rebuild the whole expression with the mapping.
## How was this patch tested?

Unit test

Closes #24598 from gengliangwang/pushdownDisjunctivePredicates.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-17 19:25:24 +08:00
Ryan Blue 2da5b21834 [SPARK-24923][SQL] Implement v2 CreateTableAsSelect
## What changes were proposed in this pull request?

This adds a v2 implementation for CTAS queries

* Update the SQL parser to parse CREATE queries using multi-part identifiers
* Update `CheckAnalysis` to validate partitioning references with the CTAS query schema
* Add `CreateTableAsSelect` v2 logical plan and `CreateTableAsSelectExec` v2 physical plan
* Update create conversion from `CreateTableAsSelectStatement` to support the new v2 logical plan
* Update `DataSourceV2Strategy` to convert v2 CTAS logical plan to the new physical plan
* Add `findNestedField` to `StructType` to support reference validation

## How was this patch tested?

We have been running these changes in production for several months. Also:

* Add a test suite `CreateTablePartitioningValidationSuite` for new analysis checks
* Add a test suite for v2 SQL, `DataSourceV2SQLSuite`
* Update catalyst `DDLParserSuite` to use multi-part identifiers (`Seq[String]`)
* Add test cases to `PlanResolutionSuite` for v2 CTAS: known catalog and v2 source implementation

Closes #24570 from rdblue/SPARK-24923-add-v2-ctas.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-15 11:24:03 +08:00
Yuming Wang fee695d0cf [SPARK-27690][SQL] Remove materialized views first in HiveClientImpl.reset
## What changes were proposed in this pull request?

We should remove materialized view first otherwise(note that Hive 3.1 could reproduce this issue):
```scala
Cause: org.apache.derby.shared.common.error.DerbySQLIntegrityConstraintViolationException: DELETE on table 'TBLS' caused a violation of foreign key constraint 'MV_TABLES_USED_FK2' for key (4).  The statement has been rolled back.
at org.apache.derby.impl.jdbc.SQLExceptionFactory.getSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.Util.generateCsSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.wrapInSQLException(Unknown Source)
at org.apache.derby.impl.jdbc.TransactionResourceImpl.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedConnection.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.ConnectionChild.handleException(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedStatement.executeStatement(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedPreparedStatement.executeBatchElement(Unknown Source)
at org.apache.derby.impl.jdbc.EmbedStatement.executeLargeBatch(Unknown Source)
```

## How was this patch tested?

Existing test

Closes #24592 from wangyum/SPARK-27690.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-14 09:05:22 -07:00
Yuming Wang f3ddd6f9da [SPARK-27402][SQL][TEST-HADOOP3.2][TEST-MAVEN] Fix hadoop-3.2 test issue(except the hive-thriftserver module)
## What changes were proposed in this pull request?

This pr fix hadoop-3.2 test issues(except the `hive-thriftserver` module):
1. Add `hive.metastore.schema.verification` and `datanucleus.schema.autoCreateAll` to HiveConf.
2. hadoop-3.2 support access the Hive metastore from 0.12 to 2.2

After [SPARK-27176](https://issues.apache.org/jira/browse/SPARK-27176) and this PR, we upgraded the built-in Hive to 2.3 when enabling the Hadoop 3.2+ profile. This upgrade fixes the following issues:
- [HIVE-6727](https://issues.apache.org/jira/browse/HIVE-6727): Table level stats for external tables are set incorrectly.
- [HIVE-15653](https://issues.apache.org/jira/browse/HIVE-15653): Some ALTER TABLE commands drop table stats.
- [SPARK-12014](https://issues.apache.org/jira/browse/SPARK-12014): Spark SQL query containing semicolon is broken in Beeline.
- [SPARK-25193](https://issues.apache.org/jira/browse/SPARK-25193): insert overwrite doesn't throw exception when drop old data fails.
- [SPARK-25919](https://issues.apache.org/jira/browse/SPARK-25919): Date value corrupts when tables are "ParquetHiveSerDe" formatted and target table is Partitioned.
- [SPARK-26332](https://issues.apache.org/jira/browse/SPARK-26332): Spark sql write orc table on viewFS throws exception.
- [SPARK-26437](https://issues.apache.org/jira/browse/SPARK-26437): Decimal data becomes bigint to query, unable to query.

## How was this patch tested?
This pr test Spark’s Hadoop 3.2 profile on jenkins and #24591 test Spark’s Hadoop 2.7 profile on jenkins

This PR close #24591

Closes #24391 from wangyum/SPARK-27402.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-13 10:35:26 -07:00
Yuming Wang b5ffec12eb [SPARK-27563][FOLLOWUP] Fix to download new release from dist.apache.org
## What changes were proposed in this pull request?

`https://archive.apache.org/dist/spark/` does not have latest Spark 2.4.3:
<img src="https://user-images.githubusercontent.com/5399861/57288553-4264b600-70ec-11e9-8dcc-71b7589f5ad0.png" width="400">

This pr add `https://dist.apache.org/repos/dist/release/spark/` to mirrors list to download latest Spark.

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105202/testReport/org.apache.spark.sql.hive/HiveExternalCatalogVersionsSuite/_It_is_not_a_test_it_is_a_sbt_testing_SuiteSelector_/

## How was this patch tested?

manual tests:
```
build/sbt "hive/testOnly *.HiveExternalCatalogVersionsSuite"  -Phive
```

Closes #24544 from wangyum/Unable-to-download-Spark-2.4.3.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-05-09 08:45:52 -05:00
sandeep katta c66ec43945 [SPARK-27555][SQL] HiveSerDe should fall back to hadoopconf if hive.default.fileformat is not found in SQLConf
## What changes were proposed in this pull request?

SQLConf does not load hive-site.xml.So HiveSerDe should fall back to hadoopconf if  hive.default.fileformat is not found in SQLConf

## How was this patch tested?

Tested manually.
Added UT

Closes #24489 from sandeep-katta/spark-27555.

Authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-05-04 09:02:12 +09:00
Sean Owen 25ee0474f4 [SPARK-26936][MINOR][FOLLOWUP] Don't need the JobConf anymore, it seems
## What changes were proposed in this pull request?

On a second look in comments, seems like the JobConf isn't needed anymore here. It was used inconsistently before, it seems, and I don't see any reason a Hadoop Job config is required here anyway.

## How was this patch tested?

Existing tests.

Closes #24491 from srowen/SPARK-26936.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-29 19:47:20 -07:00
Wenchen Fan 7432e7ded4 [SPARK-24935][SQL][FOLLOWUP] support INIT -> UPDATE -> MERGE -> FINISH in Hive UDAF adapter
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24144 . #24144 missed one case: when hash aggregate fallback to sort aggregate, the life cycle of UDAF is: INIT -> UPDATE -> MERGE -> FINISH.

However, not all Hive UDAF can support it. Hive UDAF knows the aggregation mode when creating the aggregation buffer, so that it can create different buffers for different inputs: the original data or the aggregation buffer. Please see an example in the [sketches library](7f9e76e9e0/src/main/java/com/yahoo/sketches/hive/cpc/DataToSketchUDAF.java (L107)). The buffer for UPDATE may not support MERGE.

This PR updates the Hive UDAF adapter in Spark to support INIT -> UPDATE -> MERGE -> FINISH, by turning it to  INIT -> UPDATE -> FINISH + IINIT -> MERGE -> FINISH.

## How was this patch tested?

a new test case

Closes #24459 from cloud-fan/hive-udaf.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-30 10:35:23 +08:00
Sean Owen 8a17d26784 [SPARK-27536][CORE][ML][SQL][STREAMING] Remove most use of scala.language.existentials
## What changes were proposed in this pull request?

I want to get rid of as much use of `scala.language.existentials` as possible for 3.0. It's a complicated language feature that generates warnings unless this value is imported. It might even be on the way out of Scala: https://contributors.scala-lang.org/t/proposal-to-remove-existential-types-from-the-language/2785

For Spark, it comes up mostly where the code plays fast and loose with generic types, not the advanced situations you'll often see referenced where this feature is explained. For example, it comes up in cases where a function returns something like `(String, Class[_])`. Scala doesn't like matching this to any other instance of `(String, Class[_])` because doing so requires inferring the existence of some type that satisfies both. Seems obvious if the generic type is a wildcard, but, not technically something Scala likes to let you get away with.

This is a large PR, and it only gets rid of _most_ instances of `scala.language.existentials`. The change should be all compile-time and shouldn't affect APIs or logic.

Many of the changes simply touch up sloppiness about generic types, making the known correct value explicit in the code.

Some fixes involve being more explicit about the existence of generic types in methods. For instance, `def foo(arg: Class[_])` seems innocent enough but should really be declared `def foo[T](arg: Class[T])` to let Scala select and fix a single type when evaluating calls to `foo`.

For kind of surprising reasons, this comes up in places where code evaluates a tuple of things that involve a generic type, but is OK if the two parts of the tuple are evaluated separately.

One key change was altering `Utils.classForName(...): Class[_]` to the more correct `Utils.classForName[T](...): Class[T]`. This caused a number of small but positive changes to callers that otherwise had to cast the result.

In several tests, `Dataset[_]` was used where `DataFrame` seems to be the clear intent.

Finally, in a few cases in MLlib, the return type `this.type` was used where there are no subclasses of the class that uses it. This really isn't needed and causes issues for Scala reasoning about the return type. These are just changed to be concrete classes as return types.

After this change, we have only a few classes that still import `scala.language.existentials` (because modifying them would require extensive rewrites to fix) and no build warnings.

## How was this patch tested?

Existing tests.

Closes #24431 from srowen/SPARK-27536.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-29 11:02:01 -05:00
Wenchen Fan 85fd552ed6 [SPARK-27190][SQL] add table capability for streaming
## What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/24012 , to add the corresponding capabilities for streaming.

## How was this patch tested?

existing tests

Closes #24129 from cloud-fan/capability.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-26 15:44:23 +08:00
Wenchen Fan 2234667b15 [SPARK-27563][SQL][TEST] automatically get the latest Spark versions in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

We can get the latest downloadable Spark versions from https://dist.apache.org/repos/dist/release/spark/

## How was this patch tested?

manually.

Closes #24454 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-26 16:37:43 +09:00
Wenchen Fan b7f9830670 [MINOR][TEST] switch from 2.4.1 to 2.4.2 in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

update `HiveExternalCatalogVersionsSuite` to test 2.4.2, as 2.4.1 will be removed from Mirror Network soon.

## How was this patch tested?

N/A

Closes #24452 from cloud-fan/release.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-25 10:26:40 +08:00
Sean Owen 596a5ff273 [MINOR][BUILD] Update genjavadoc to 0.13
## What changes were proposed in this pull request?

Kind of related to https://github.com/gatorsmile/spark/pull/5 - let's update genjavadoc to see if it generates fewer spurious javadoc errors to begin with.

## How was this patch tested?

Existing docs build

Closes #24443 from srowen/genjavadoc013.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-24 13:44:48 +09:00
Gengliang Wang 00f2f311f7 [SPARK-27128][SQL] Migrate JSON to File Data Source V2
## What changes were proposed in this pull request?
Migrate JSON to File Data Source V2

## How was this patch tested?

Unit test

Closes #24058 from gengliangwang/jsonV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-23 22:39:59 +08:00
Eric Liang 5172190da1 [SPARK-27392][SQL] TestHive test tables should be placed in shared test state, not per session
## What changes were proposed in this pull request?

Otherwise, tests that use tables from multiple sessions will run into issues if they access the same table. The correct location is in shared state.

A couple other minor test improvements.

cc gatorsmile srinathshankar

## How was this patch tested?

Existing unit tests.

Closes #24302 from ericl/test-conflicts.

Lead-authored-by: Eric Liang <ekl@databricks.com>
Co-authored-by: Eric Liang <ekhliang@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-22 11:05:31 -07:00
Yuming Wang 8f82237a5b [SPARK-27501][SQL][TEST] Add test for HIVE-13083: Writing HiveDecimal to ORC can wrongly suppress present stream
## What changes were proposed in this pull request?

This PR add test for [HIVE-13083](https://issues.apache.org/jira/browse/HIVE-13083): Writing HiveDecimal to ORC can wrongly suppress present stream.

## How was this patch tested?
manual tests:
```
build/sbt  "hive/testOnly *HiveOrcQuerySuite" -Phive -Phadoop-3.2
```

Closes #24397 from wangyum/SPARK-26437.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-04-19 10:12:21 +09:00
Gengliang Wang 3748b381df [SPARK-27460][TESTS][FOLLOWUP] Add HiveClientVersions to parallel test suite list
## What changes were proposed in this pull request?

The test time of `HiveClientVersions` is around 3.5 minutes.
This PR is to add it into the parallel test suite list. To make sure there is no colliding warehouse location,  we can change the warehouse path to a temporary directory.

## How was this patch tested?

Unit test

Closes #24404 from gengliangwang/parallelTestFollowUp.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-18 15:37:55 -07:00
Gengliang Wang f9837d3bf6 [SPARK-27448][SQL] File source V2 table provider should be compatible with V1 provider
## What changes were proposed in this pull request?

In the rule `PreprocessTableCreation`, if an existing table is appended with a different provider, the action will fail.
Currently, there are two implementations for file sources and creating a table with file source V2 will always fall back to V1 FileFormat. We should consider the following cases as valid:
1. Appending a table with file source V2 provider using the v1 file format
2. Appending a table with v1 file format provider using file source V2 format

## How was this patch tested?

Unit test

Closes #24356 from gengliangwang/fixTableProvider.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-04-16 14:26:38 +08:00
Gengliang Wang d50603a37c [SPARK-27271][SQL] Migrate Text to File Data Source V2
## What changes were proposed in this pull request?

Migrate Text source to File Data Source V2

## How was this patch tested?

Unit test

Closes #24207 from gengliangwang/textV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-08 10:15:22 -07:00
Yuming Wang 33f3c48cac [SPARK-27176][SQL] Upgrade hadoop-3's built-in Hive maven dependencies to 2.3.4
## What changes were proposed in this pull request?

This PR mainly contains:
1. Upgrade hadoop-3's built-in Hive maven dependencies to 2.3.4.
2. Resolve compatibility issues between Hive 1.2.1 and Hive 2.3.4 in the `sql/hive` module.

## How was this patch tested?
jenkins test hadoop-2.7
manual test hadoop-3:
```shell
build/sbt clean package -Phadoop-3.2 -Phive
export SPARK_PREPEND_CLASSES=true

# rm -rf metastore_db

cat <<EOF > test_hadoop3.scala
spark.range(10).write.saveAsTable("test_hadoop3")
spark.table("test_hadoop3").show
EOF

bin/spark-shell --conf spark.hadoop.hive.metastore.schema.verification=false --conf spark.hadoop.datanucleus.schema.autoCreateAll=true -i test_hadoop3.scala
```

Closes #23788 from wangyum/SPARK-23710-hadoop3.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-08 08:42:21 -07:00
Yuming Wang 017919b636 [SPARK-27383][SQL][TEST] Avoid using hard-coded jar names in Hive tests
## What changes were proposed in this pull request?

This pr avoid using hard-coded jar names(`hive-contrib-0.13.1.jar` and `hive-hcatalog-core-0.13.1.jar`) in Hive tests. This change makes it easy to change when upgrading the built-in Hive to 2.3.4.

## How was this patch tested?

Existing test

Closes #24294 from wangyum/SPARK-27383.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-06 18:06:52 -05:00
gengjiaan 979bb905b7 [SPARK-26936][SQL] Fix bug of insert overwrite local dir can not create temporary path in local staging directory
## What changes were proposed in this pull request?
Th environment of my cluster as follows:
```
OS:Linux version 2.6.32-220.7.1.el6.x86_64 (mockbuildc6b18n3.bsys.dev.centos.org) (gcc version 4.4.6 20110731 (Red Hat 4.4.6-3) (GCC) ) #1 SMP Wed Mar 7 00:52:02 GMT 2012
Hadoop: 2.7.2
Spark: 2.3.0 or 3.0.0(master branch)
Hive: 1.2.1
```

My spark run on deploy mode yarn-client.

If I execute the SQL `insert overwrite local directory '/home/test/call_center/' select * from call_center`, a HiveException will appear as follows:
`Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: Mkdirs failed to create file:/home/xitong/hive/stagingdir_hive_2019-02-19_17-31-00_678_1816816774691551856-1/-ext-10000/_temporary/0/_temporary/attempt_20190219173233_0002_m_000000_3 (exists=false, cwd=file:/data10/yarn/nm-local-dir/usercache/xitong/appcache/application_1543893582405_6126857/container_e124_1543893582405_6126857_01_000011)
at org.apache.hadoop.hive.ql.io.HiveFileFormatUtils.getHiveRecordWriter(HiveFileFormatUtils.java:249)`
Current spark sql generate a local temporary path in local staging directory.The schema of local temporary path start with `file`, so the HiveException appears.
This PR change the local temporary path to HDFS temporary path, and use DistributedFileSystem instance copy the data from HDFS temporary path to local directory.
If Spark run on local deploy mode, 'insert overwrite local directory' works fine.
## How was this patch tested?

UT cannot support yarn-client mode.The test is in my product environment.

Closes #23841 from beliefer/fix-bug-of-insert-overwrite-local-dir.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-05 14:02:46 -05:00
Dongjoon Hyun 938d954375 [SPARK-27382][SQL][TEST] Update Spark 2.4.x testing in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

Since Apache Spark 2.4.1 vote passed and is distributed into mirrors, we need to test 2.4.1. This should land on both `master` and `branch-2.4`.

## How was this patch tested?

Pass the Jenkins.

Closes #24292 from dongjoon-hyun/SPARK-27382.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-04 13:49:56 -07:00
Yuming Wang 1d95dea307 [SPARK-27349][SQL] Dealing with TimeVars removed in Hive 2.x
## What changes were proposed in this pull request?
`hive.stats.jdbc.timeout` and `hive.stats.retries.wait` were removed by [HIVE-12164](https://issues.apache.org/jira/browse/HIVE-12164).
This pr to deal with this change.

## How was this patch tested?

unit tests

Closes #24277 from wangyum/SPARK-27349.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-04-03 22:52:37 -07:00
Gengliang Wang 5dab5f651f [SPARK-27326][SQL] Fall back all v2 file sources in InsertIntoTable to V1 FileFormat
## What changes were proposed in this pull request?

In the first PR for file source V2, there was a rule for falling back Orc V2 table to OrcFileFormat: https://github.com/apache/spark/pull/23383/files#diff-57e8244b6964e4f84345357a188421d5R34

As we are migrating more file sources to data source V2, we should make the rule more generic. This PR proposes to:
1. Rename the rule `FallbackOrcDataSourceV2 ` to `FallBackFileSourceV2`.The name is more generic. And we use "fall back" as verb, while "fallback" is noun.
2. Rename the method `fallBackFileFormat` in `FileDataSourceV2` to `fallbackFileFormat`. Here we should use "fallback" as noun.
3. Add new method `fallbackFileFormat` in `FileTable`. This is for falling back to V1 in rule `FallbackOrcDataSourceV2 `.

## How was this patch tested?

Existing Unit tests.

Closes #24251 from gengliangwang/fallbackV1Rule.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-30 14:38:26 -07:00
Takeshi Yamamuro 956b52b167 [SPARK-26771][SQL][FOLLOWUP] Make all the uncache operations non-blocking by default
## What changes were proposed in this pull request?
To make the blocking behaviour consistent, this pr made catalog table/view `uncacheQuery` non-blocking by default. If this pr merged, all the behaviours in spark are non-blocking by default.

## How was this patch tested?
Pass Jenkins.

Closes #24212 from maropu/SPARK-26771-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-03-27 21:01:36 +09:00
Maxim Gekk 69035684d4 [SPARK-27242][SQL] Make formatting TIMESTAMP/DATE literals independent from the default time zone
## What changes were proposed in this pull request?

In the PR, I propose to use the SQL config `spark.sql.session.timeZone` in formatting `TIMESTAMP` literals, and make formatting `DATE` literals independent from time zone. The changes make parsing and formatting `TIMESTAMP`/`DATE` literals consistent, and independent from the default time zone of current JVM.

Also this PR ports `TIMESTAMP`/`DATE` literals formatting on Proleptic Gregorian Calendar via using `TimestampFormatter`/`DateFormatter`.

## How was this patch tested?

Added new tests to `LiteralExpressionSuite`

Closes #24181 from MaxGekk/timezone-aware-literals.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 15:29:59 -07:00
Sean Owen 8bc304f97e [SPARK-26132][BUILD][CORE] Remove support for Scala 2.11 in Spark 3.0.0
## What changes were proposed in this pull request?

Remove Scala 2.11 support in build files and docs, and in various parts of code that accommodated 2.11. See some targeted comments below.

## How was this patch tested?

Existing tests.

Closes #23098 from srowen/SPARK-26132.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-25 10:46:42 -05:00
pgandhi a6c207c9c0 [SPARK-24935][SQL] fix Hive UDAF with two aggregation buffers
## What changes were proposed in this pull request?

Hive UDAF knows the aggregation mode when creating the aggregation buffer, so that it can create different buffers for different inputs: the original data or the aggregation buffer. Please see an example in the [sketches library](7f9e76e9e0/src/main/java/com/yahoo/sketches/hive/cpc/DataToSketchUDAF.java (L107)).

However, the Hive UDAF adapter in Spark always creates the buffer with partial1 mode, which can only deal with one input: the original data. This PR fixes it.

All credits go to pgandhi999 , who investigate the problem and study the Hive UDAF behaviors, and write the tests.

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

## How was this patch tested?

a new test

Closes #24144 from cloud-fan/hive.

Lead-authored-by: pgandhi <pgandhi@verizonmedia.com>
Co-authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-03-24 16:07:35 -07:00
Ryan Blue 34e3cc7060 [SPARK-27108][SQL] Add parsed SQL plans for create, CTAS.
## What changes were proposed in this pull request?

This moves parsing `CREATE TABLE ... USING` statements into catalyst. Catalyst produces logical plans with the parsed information and those plans are converted to v1 `DataSource` plans in `DataSourceAnalysis`.

This prepares for adding v2 create plans that should receive the information parsed from SQL without being translated to v1 plans first.

This also makes it possible to parse in catalyst instead of breaking the parser across the abstract `AstBuilder` in catalyst and `SparkSqlParser` in core.

For more information, see the [mailing list thread](https://lists.apache.org/thread.html/54f4e1929ceb9a2b0cac7cb058000feb8de5d6c667b2e0950804c613%3Cdev.spark.apache.org%3E).

## How was this patch tested?

This uses existing tests to catch regressions. This introduces no behavior changes.

Closes #24029 from rdblue/SPARK-27108-add-parsed-create-logical-plans.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-22 13:58:54 -07:00
Sean Owen c65f9b2bc3 [SPARK-26839][SQL] Work around classloader changes in Java 9 for Hive isolation
Note, this doesn't really resolve the JIRA, but makes the changes we can make so far that would be required to solve it.

## What changes were proposed in this pull request?

Java 9+ changed how ClassLoaders work. The two most salient points:
- The boot classloader no longer 'sees' the platform classes. A new 'platform classloader' does and should be the parent of new ClassLoaders
- The system classloader is no longer a URLClassLoader, so we can't get the URLs of JARs in its classpath

## How was this patch tested?

We'll see whether Java 8 tests still pass here. Java 11 tests do not fully pass at this point; more notes below. This does make progress on the failures though.

(NB: to test with Java 11, you need to build with Java 8 first, setting JAVA_HOME and java's executable correctly, then switch both to Java 11 for testing.)

Closes #24057 from srowen/SPARK-26839.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-20 09:12:52 -05:00
s71955 e402de5fd0 [SPARK-26176][SQL] Verify column names for CTAS with STORED AS
## What changes were proposed in this pull request?
Currently, users meet job abortions while creating a table using the Hive serde "STORED AS" with invalid column names. We had better prevent this by raising **AnalysisException** with a guide to use aliases instead like Paquet data source tables.
thus making compatible with error message shown while creating Parquet/ORC native table.

**BEFORE**
```scala
scala> sql("set spark.sql.hive.convertMetastoreParquet=false")
scala> sql("CREATE TABLE a STORED AS PARQUET AS SELECT 1 AS `COUNT(ID)`")
Caused by: java.lang.IllegalArgumentException: No enum constant parquet.schema.OriginalType.col1
```

**AFTER**
```scala
scala> sql("CREATE TABLE a STORED AS PARQUET AS SELECT 1 AS `COUNT(ID)`")
 Please use alias to rename it.;eption: Attribute name "count(ID)" contains invalid character(s) among " ,;{}()\n\t=".
```

## How was this patch tested?
Pass the Jenkins with the newly added test case.

Closes #24075 from sujith71955/master_serde.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-19 20:29:47 +08:00
Ryan Blue e348f14259 [SPARK-26811][SQL] Add capabilities to v2.Table
## What changes were proposed in this pull request?

This adds a new method, `capabilities` to `v2.Table` that returns a set of `TableCapability`. Capabilities are used to fail queries during analysis checks, `V2WriteSupportCheck`, when the table does not support operations, like truncation.

## How was this patch tested?

Existing tests for regressions, added new analysis suite, `V2WriteSupportCheckSuite`, for new capability checks.

Closes #24012 from rdblue/SPARK-26811-add-capabilities.

Authored-by: Ryan Blue <blue@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-18 18:25:11 +08:00
Yuming Wang da7db9abf6 [SPARK-23749][SQL] Replace built-in Hive API (isSub/toKryo) and remove OrcProto.Type usage
## What changes were proposed in this pull request?

In order to make the upgrade built-in Hive changes smaller.
This pr workaround the simplest 3 API changes first.

## How was this patch tested?

manual tests

Closes #24018 from wangyum/SPARK-23749.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-14 11:41:40 -07:00
Yuming Wang 470313e660 [SPARK-27118][SQL] Upgrade Hive Metastore Client to the latest versions for Hive 1.0.x/1.1.x
## What changes were proposed in this pull request?

Hive 1.1.1 and Hive 1.0.1 released. We should upgrade Hive Metastore Client version.

https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329444&styleName=Text&projectId=12310843
https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12329557&styleName=Text&projectId=12310843

## How was this patch tested?

N/A

Closes #24040 from wangyum/SPARK-27118.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:50:10 -08:00
Yuming Wang f732647ae4 [SPARK-27054][BUILD][SQL] Remove the Calcite dependency
## What changes were proposed in this pull request?

Calcite is only used for [runSqlHive](02bbe977ab/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L699-L705)) when `hive.cbo.enable=true`([SemanticAnalyzer](https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java#L278-L280)).
So we can disable `hive.cbo.enable` and remove Calcite dependency.

## How was this patch tested?

Exist tests

Closes #23970 from wangyum/SPARK-27054.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Yuming Wang <wgyumg@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-09 16:34:24 -08:00
CodeGod a29df5fa02 [SPARK-27080][SQL] bug fix: mergeWithMetastoreSchema with uniform lower case comparison
## What changes were proposed in this pull request?
When reading parquet file with merging metastore schema and file schema, we should compare field names using uniform case. In current implementation, lowercase is used but one omission. And this patch fix it.

## How was this patch tested?
Unit test

Closes #24001 from codeborui/mergeSchemaBugFix.

Authored-by: CodeGod <>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-09 21:28:10 +08:00
Yuming Wang 43dcb91a4c [SPARK-19678][FOLLOW-UP][SQL] Add behavior change test when table statistics are incorrect
## What changes were proposed in this pull request?

Since Spark 2.2.0 ([SPARK-19678](https://issues.apache.org/jira/browse/SPARK-19678)), the below SQL changed from `broadcast join` to `sort merge join`:
```sql
-- small external table with incorrect statistics
CREATE EXTERNAL TABLE t1(c1 int)
ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe'
WITH SERDEPROPERTIES (
  'serialization.format' = '1'
)
STORED AS
  INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat'
  OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat'
LOCATION 'file:///tmp/t1'
TBLPROPERTIES (
'rawDataSize'='-1', 'numFiles'='0', 'totalSize'='0', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='-1'
);

-- big table
CREATE TABLE t2 (c1 int)
LOCATION 'file:///tmp/t2'
TBLPROPERTIES (
'rawDataSize'='23437737', 'numFiles'='12222', 'totalSize'='333442230', 'COLUMN_STATS_ACCURATE'='false', 'numRows'='443442223'
);

explain SELECT t1.c1 FROM t1 INNER JOIN t2 ON t1.c1 = t2.c1;
```
This pr add a test case for this behavior change.

## How was this patch tested?

unit tests

Closes #24003 from wangyum/SPARK-19678.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-08 11:47:49 +08:00
Yuming Wang d70b6a39e1 [MINOR][BUILD] Add 2 maven properties(hive.classifier and hive.parquet.group)
## What changes were proposed in this pull request?

This pr adds 2 maven properties to help us upgrade the built-in Hive.

| Property Name | Default | In future |
| ------ | ------ | ------ |
| hive.classifier | (none) | core |
| hive.parquet.group | com.twitter | org.apache.parquet |

## How was this patch tested?

existing tests

Closes #23996 from wangyum/add_2_maven_properties.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-07 16:46:07 -06:00
Yuming Wang 32848eecc5 [SPARK-27078][SQL] Fix NoSuchFieldError when read Hive materialized views
## What changes were proposed in this pull request?

This pr fix `NoSuchFieldError` when reading Hive materialized views from Hive 2.3.4.

How to reproduce:
Hive side:
```sql
CREATE TABLE materialized_view_tbl (key INT);
CREATE MATERIALIZED VIEW view_1 DISABLE REWRITE AS SELECT * FROM materialized_view_tbl;
```
Spark side:
```java
bin/spark-sql --conf spark.sql.hive.metastore.version=2.3.4 --conf spark.sql.hive.metastore.jars=maven

spark-sql> select * from view_1;
19/03/05 19:55:37 ERROR SparkSQLDriver: Failed in [select * from view_1]
java.lang.NoSuchFieldError: INDEX_TABLE
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$3(HiveClientImpl.scala:438)
	at scala.Option.map(Option.scala:163)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$getTableOption$1(HiveClientImpl.scala:370)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:277)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:215)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:214)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:260)
	at org.apache.spark.sql.hive.client.HiveClientImpl.getTableOption(HiveClientImpl.scala:368)
```

## How was this patch tested?

unit tests

Closes #23984 from wangyum/SPARK-24360.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-06 16:56:32 -08:00
Dilip Biswal 4a486d6716 [SPARK-26982][SQL] Enhance describe framework to describe the output of a query.
## What changes were proposed in this pull request?
Currently we can use `df.printSchema` to discover the schema information for a query. We should have a way to describe the output schema of a query using SQL interface.

Example:

DESCRIBE SELECT * FROM desc_table
DESCRIBE QUERY SELECT * FROM desc_table
```SQL

spark-sql> create table desc_table (c1 int comment 'c1-comment', c2 decimal comment 'c2-comment', c3 string);

spark-sql> desc select * from desc_table;
c1	int	        c1-comment
c2	decimal(10,0)	c2-comment
c3	string	        NULL

```
## How was this patch tested?
Added a new test under SQLQueryTestSuite and SparkSqlParserSuite

Closes #23883 from dilipbiswal/dkb_describe_query.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-02 11:21:23 +08:00
liuxian 02bbe977ab [MINOR] Remove unnecessary gets when getting a value from map.
## What changes were proposed in this pull request?

Redundant `get`  when getting a value from `Map` given a key.

## How was this patch tested?

N/A

Closes #23901 from 10110346/removegetfrommap.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-01 11:48:07 -06:00
liuxian 7912dbb88f [MINOR] Simplify boolean expression
## What changes were proposed in this pull request?

Comparing whether Boolean expression is equal to true is redundant
For example:
The datatype of `a` is boolean.
Before:
if (a == true)
After:
if (a)

## How was this patch tested?
N/A

Closes #23884 from 10110346/simplifyboolean.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-27 08:38:00 -06:00
Maxim Gekk a2a41b7bf2 [SPARK-26978][CORE][SQL] Avoid magic time constants
## What changes were proposed in this pull request?

In the PR, I propose to refactor existing code related to date/time conversions, and replace constants like `1000` and `1000000` by `DateTimeUtils` constants and transformation functions from `java.util.concurrent.TimeUnit._`.

## How was this patch tested?

The changes are tested by existing test suites.

Closes #23878 from MaxGekk/magic-time-constants.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-26 09:08:12 -06:00
gengjiaan f9776e3892 [MINOR][SQL] Fix typo in exception about set table properties.
## What changes were proposed in this pull request?

The function of the method named verifyTableProperties is

`If the given table properties contains datasource properties, throw an exception. We will do this check when create or alter a table, i.e. when we try to write table metadata to Hive metastore.`

But the message of AnalysisException in verifyTableProperties contains one typo and one unsuited word.
So I change the exception from

`Cannot persistent ${table.qualifiedName} into hive metastore`

to

`Cannot persist ${table.qualifiedName} into Hive metastore`

## How was this patch tested?

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

Closes #23574 from beliefer/incorrect-analysis-exception.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-21 22:13:47 -06:00
Takeshi Yamamuro e2b8cc65cd [SPARK-26897][SQL][TEST][FOLLOW-UP] Remove workaround for 2.2.0 and 2.1.x in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?
This pr just removed workaround for 2.2.0 and 2.1.x in HiveExternalCatalogVersionsSuite.

## How was this patch tested?
Pass the Jenkins.

Closes #23817 from maropu/SPARK-26607-FOLLOWUP.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-02-18 11:24:36 +08:00
Takeshi Yamamuro dcdbd06b68 [SPARK-26897][SQL][TEST] Update Spark 2.3.x testing from HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?
The maintenance release of `branch-2.3` (v2.3.3) vote passed, so this issue updates PROCESS_TABLES.testingVersions in HiveExternalCatalogVersionsSuite

## How was this patch tested?
Pass the Jenkins.

Closes #23807 from maropu/SPARK-26897.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-02-18 08:05:49 +09:00
Gabor Somogyi 28ced387b9 [SPARK-26772][YARN] Delete ServiceCredentialProvider and make HadoopDelegationTokenProvider a developer API
## What changes were proposed in this pull request?

`HadoopDelegationTokenProvider` has basically the same functionality just like `ServiceCredentialProvider` so the interfaces can be merged.

`YARNHadoopDelegationTokenManager` now loads `ServiceCredentialProvider`s in one step. The drawback of this if one provider fails all others are not loaded. `HadoopDelegationTokenManager` loads `HadoopDelegationTokenProvider`s independently so it provides more robust behaviour.

In this PR I've I've made the following changes:
* Deleted `YARNHadoopDelegationTokenManager` and `ServiceCredentialProvider`
* Made `HadoopDelegationTokenProvider` a `DeveloperApi`

## How was this patch tested?

Existing unit tests.

Closes #23686 from gaborgsomogyi/SPARK-26772.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-15 14:43:13 -08:00
Kent Yao ac9c0536bc [SPARK-26794][SQL] SparkSession enableHiveSupport does not point to hive but in-memory while the SparkContext exists
## What changes were proposed in this pull request?

```java
public class SqlDemo {
    public static void main(final String[] args) throws Exception {
        SparkConf conf = new SparkConf().setAppName("spark-sql-demo");
        JavaSparkContext sc = new JavaSparkContext(conf);
        SparkSession ss = SparkSession.builder().enableHiveSupport().getOrCreate();
        ss.sql("show databases").show();
    }
}
```
Before https://issues.apache.org/jira/browse/SPARK-20946, the demo above point to the right hive metastore if the hive-site.xml is present. But now it can only point to the default in-memory one.

Catalog is now as a variable shared across SparkSessions, it is instantiated with SparkContext's conf. After https://issues.apache.org/jira/browse/SPARK-20946, Session level configs are not pass to SparkContext's conf anymore, so the enableHiveSupport API takes no affect on the catalog instance.

You can set spark.sql.catalogImplementation=hive application wide to solve the problem, or never create a sc before you call SparkSession.builder().enableHiveSupport().getOrCreate()

Here we respect the SparkSession level configuration at the first time to generate catalog within SharedState

## How was this patch tested?

1. add ut
2. manually
```scala
test("enableHiveSupport has right to determine the catalog while using an existing sc") {
    val conf = new SparkConf().setMaster("local").setAppName("SharedState Test")
    val sc = SparkContext.getOrCreate(conf)
    val ss = SparkSession.builder().enableHiveSupport().getOrCreate()
    assert(ss.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog],
      "The catalog should be hive ")

    val ss2 = SparkSession.builder().getOrCreate()
    assert(ss2.sharedState.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog],
      "The catalog should be shared across sessions")
  }
```

Without this fix, the above test will fail.
You can apply it to `org.apache.spark.sql.hive.HiveSharedStateSuite`,
and run,
```sbt
./build/sbt  -Phadoop-2.7 -Phive  "hive/testOnly org.apache.spark.sql.hive.HiveSharedStateSuite"
```
to verify.

Closes #23709 from yaooqinn/SPARK-26794.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-14 15:07:22 +08:00
yangjie01 5864e8e474 [SPARK-25158][SQL] Executor accidentally exit because ScriptTransformationWriterThread throw Exception.
## What changes were proposed in this pull request?

Run Spark-Sql job use transform features(`ScriptTransformationExec`) with config `spark.speculation = true`, sometimes job fails and we found many Executor Dead through `Executor Tab`, through analysis log and code we found :

`ScriptTransformationExec` start a new thread(`ScriptTransformationWriterThread`), the new thread is very likely to throw `TaskKilledException`(from iter.map.foreach part) when speculation is on, this exception will captured by `SparkUncaughtExceptionHandler` which registered during Executor start, `SparkUncaughtExceptionHandler` will call `System.exit (SparkExitCode.UNCAUGHT_EXCEPTION)` to shutdown `Executor`, this is unexpected.

We should not kill the executor just because `ScriptTransformationWriterThread` fails. log the error(not only `TaskKilledException`) instead of throwing it is enough, Exception already pass to `ScriptTransformationExec` and handle by `TaskRunner`.

## How was this patch tested?

Register `TestUncaughtExceptionHandler` to test case in `ScriptTransformationSuite`, then assert there is no Uncaught Exception handled.

Before this patch "script transformation should not swallow errors from upstream operators (no serde)" and "script transformation should not swallow errors from upstream operators (with serde)"  throwing `IllegalArgumentException` and handle by `TestUncaughtExceptionHandler` .

Closes #22149 from LuciferYang/fix-transformation-task-kill.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-02-12 12:16:33 +08:00
Gabor Somogyi d0443a74d1 [SPARK-26766][CORE] Remove the list of filesystems from HadoopDelegationTokenProvider.obtainDelegationTokens
## What changes were proposed in this pull request?

Delegation token providers interface now has a parameter `fileSystems` but this is needed only for `HadoopFSDelegationTokenProvider`.

In this PR I've addressed this issue in the following way:
* Removed `fileSystems` parameter from `HadoopDelegationTokenProvider`
* Moved `YarnSparkHadoopUtil.hadoopFSsToAccess` into `HadoopFSDelegationTokenProvider`
* Moved `spark.yarn.stagingDir` into core
* Moved `spark.yarn.access.namenodes` into core and renamed to `spark.kerberos.access.namenodes`
* Moved `spark.yarn.access.hadoopFileSystems` into core and renamed to `spark.kerberos.access.hadoopFileSystems`

## How was this patch tested?

Existing unit tests.

Closes #23698 from gaborgsomogyi/SPARK-26766.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-08 13:41:52 -08:00
Sean Owen 8171b156eb [SPARK-26771][CORE][GRAPHX] Make .unpersist(), .destroy() consistently non-blocking by default
## What changes were proposed in this pull request?

Make .unpersist(), .destroy() non-blocking by default and adjust callers to request blocking only where important.

This also adds an optional blocking argument to Pyspark's RDD.unpersist(), which never had one.

## How was this patch tested?

Existing tests.

Closes #23685 from srowen/SPARK-26771.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-01 18:29:55 -06:00
Dongjoon Hyun aeff69bd87
[SPARK-24360][SQL] Support Hive 3.1 metastore
## What changes were proposed in this pull request?

Hive 3.1.1 is released. This PR aims to support Hive 3.1.x metastore.
Please note that Hive 3.0.0 Metastore is skipped intentionally.

## How was this patch tested?

Pass the Jenkins with the updated test cases including 3.1.

Closes #23694 from dongjoon-hyun/SPARK-24360-3.1.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-30 20:33:21 -08:00
Gengliang Wang 36a2e6371b
[SPARK-26716][SQL] FileFormat: the supported types of read/write should be consistent
## What changes were proposed in this pull request?

1. Remove parameter `isReadPath`. The supported types of read/write should be the same.

2. Disallow reading `NullType` for ORC data source. In #21667 and #21389, it was supposed that ORC supports reading `NullType`, but can't write it. This doesn't make sense. I read docs and did some tests. ORC doesn't support `NullType`.

## How was this patch tested?

Unit tset

Closes #23639 from gengliangwang/supportDataType.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-27 10:11:42 -08:00
heguozi e71acd9a23 [SPARK-26630][SQL] Support reading Hive-serde tables whose INPUTFORMAT is org.apache.hadoop.mapreduce
## What changes were proposed in this pull request?

When we read a hive table and create RDDs in `TableReader`, it'll throw exception `java.lang.ClassCastException: org.apache.hadoop.mapreduce.lib.input.TextInputFormat cannot be cast to org.apache.hadoop.mapred.InputFormat` if the input format class of the table is from mapreduce package.

Now we use NewHadoopRDD to deal with the new input format and keep HadoopRDD to the old one.

This PR is from #23506. We can reproduce this issue by executing the new test with the code in old version. When create a table with `org.apache.hadoop.mapreduce.....` input format, we will find the exception thrown in `org.apache.spark.rdd.HadoopRDD.getInputFormat(HadoopRDD.scala:190)`

## How was this patch tested?

Added a new test.

Closes #23559 from Deegue/fix-hadoopRDD.

Lead-authored-by: heguozi <zyzzxycj@gmail.com>
Co-authored-by: Yizhong Zhang <zyzzxycj@163.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-26 10:17:03 -08:00
Gabor Somogyi 773efede20 [SPARK-26254][CORE] Extract Hive + Kafka dependencies from Core.
## What changes were proposed in this pull request?

There are ugly provided dependencies inside core for the following:
* Hive
* Kafka

In this PR I've extracted them out. This PR contains the following:
* Token providers are now loaded with service loader
* Hive token provider moved to hive project
* Kafka token provider extracted into a new project

## How was this patch tested?

Existing + newly added unit tests.
Additionally tested on cluster.

Closes #23499 from gaborgsomogyi/SPARK-26254.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-25 10:36:00 -08:00
Gengliang Wang f5b9370da2 [SPARK-26709][SQL] OptimizeMetadataOnlyQuery does not handle empty records correctly
## What changes were proposed in this pull request?

When reading from empty tables, the optimization `OptimizeMetadataOnlyQuery` may return wrong results:
```
sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)")
sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)")
sql("SELECT MAX(p1) FROM t")
```
The result is supposed to be `null`. However, with the optimization the result is `5`.

The rule is originally ported from https://issues.apache.org/jira/browse/HIVE-1003 in #13494. In Hive, the rule is disabled by default in a later release(https://issues.apache.org/jira/browse/HIVE-15397), due to the same problem.

It is hard to completely avoid the correctness issue. Because data sources like Parquet can be metadata-only. Spark can't tell whether it is empty or not without actually reading it. This PR disable the optimization by default.

## How was this patch tested?

Unit test

Closes #23635 from gengliangwang/optimizeMetadata.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-24 18:24:49 -08:00
Kris Mok 02d8ae3d59
[SPARK-26661][SQL] Show actual class name of the writing command in CTAS explain
## What changes were proposed in this pull request?

The explain output of the Hive CTAS command, regardless of whether it's actually writing via Hive's SerDe or converted into using Spark's data source, would always show that it's using `InsertIntoHiveTable` because it's hardcoded.

e.g.
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHiveTable]
```
This CTAS is converted into using Spark's data source, but it still says `InsertIntoHiveTable` in the explain output.

It's better to show the actual class name of the writing command used. For the example above, it'd be:
```
Execute OptimizedCreateHiveTableAsSelectCommand [Database:default, TableName: foo, InsertIntoHadoopFsRelationCommand]
```

## How was this patch tested?

Added test case in `HiveExplainSuite`

Closes #23582 from rednaxelafx/fix-explain-1.

Authored-by: Kris Mok <kris.mok@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-22 13:55:41 -08:00
Kazuaki Ishizaki 7bf0794651 [SPARK-26463][CORE] Use ConfigEntry for hardcoded configs for scheduler categories.
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.dynamicAllocation`, `spark.scheduler`, `spark.rpc`, `spark.task`, `spark.speculation`, and `spark.cleaner` configs to use `ConfigEntry`.

## How was this patch tested?

Existing tests

Closes #23416 from kiszk/SPARK-26463.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 07:44:36 -06:00
Kazuaki Ishizaki 64cc9e572e
[SPARK-26477][CORE] Use ConfigEntry for hardcoded configs for unsafe category
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.unsafe` configs to use ConfigEntry and put them in the `config` package.

## How was this patch tested?

Existing UTs

Closes #23412 from kiszk/SPARK-26477.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-18 23:57:04 -08:00
Gengliang Wang c0632cec04 [SPARK-23817][SQL] Create file source V2 framework and migrate ORC read path
## What changes were proposed in this pull request?
Create a framework for file source V2 based on data source V2 API.
As a good example for demonstrating the framework, this PR also migrate ORC source. This is because ORC file source supports both row scan and columnar scan, and the implementation is simpler comparing with Parquet.

Note: Currently only read path of V2 API is done, this framework and migration are only for the read path.
Supports the following scan:
- Scan ColumnarBatch
- Scan UnsafeRow
- Push down filters
- Push down required columns

Not supported( due to the limitation of data source V2 API):
- Stats metrics
- Catalog table
- Writes

## How was this patch tested?

Unit test

Closes #23383 from gengliangwang/latest_orcV2.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-17 23:33:29 +08:00
John Zhuge 3f8007102a [SPARK-26576][SQL] Broadcast hint not applied to partitioned table
## What changes were proposed in this pull request?

Make sure broadcast hint is applied to partitioned tables.

## How was this patch tested?

- A new unit test in PruneFileSourcePartitionsSuite
- Unit test suites touched by SPARK-14581: JoinOptimizationSuite, FilterPushdownSuite, ColumnPruningSuite, and PruneFiltersSuite

Closes #23507 from jzhuge/SPARK-26576.

Closes #23530 from jzhuge/SPARK-26576-master.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-13 15:36:40 -08:00
Dongjoon Hyun 3587a9a227
[SPARK-26607][SQL][TEST] Remove Spark 2.2.x testing from HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

The vote of final release of `branch-2.2` passed and the branch goes EOL. This PR removes Spark 2.2.x from the testing coverage.

## How was this patch tested?

Pass the Jenkins.

Closes #23526 from dongjoon-hyun/SPARK-26607.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-11 22:53:58 -08:00
Jungtaek Lim (HeartSaVioR) d9e4cf67c0 [SPARK-26482][CORE] Use ConfigEntry for hardcoded configs for ui categories
## What changes were proposed in this pull request?

The PR makes hardcoded configs below to use `ConfigEntry`.

* spark.ui
* spark.ssl
* spark.authenticate
* spark.master.rest
* spark.master.ui
* spark.metrics
* spark.admin
* spark.modify.acl

This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).

## How was this patch tested?

Existing tests.

Closes #23423 from HeartSaVioR/SPARK-26466.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-11 10:18:07 -08:00
Sean Owen 51a6ba0181 [SPARK-26503][CORE] Get rid of spark.sql.legacy.timeParser.enabled
## What changes were proposed in this pull request?

Per discussion in #23391 (comment) this proposes to just remove the old pre-Spark-3 time parsing behavior.

This is a rebase of https://github.com/apache/spark/pull/23411

## How was this patch tested?

Existing tests.

Closes #23495 from srowen/SPARK-26503.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-11 08:53:12 -06:00
Dongjoon Hyun 270916f8cd
[SPARK-26584][SQL] Remove spark.sql.orc.copyBatchToSpark internal conf
## What changes were proposed in this pull request?

This PR aims to remove internal ORC configuration to simplify the code path for Spark 3.0.0. This removes the configuration `spark.sql.orc.copyBatchToSpark` and related ORC codes including tests and benchmarks.

## How was this patch tested?

Pass the Jenkins with the reduced test coverage.

Closes #23503 from dongjoon-hyun/SPARK-26584.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-10 08:42:23 -08:00
Wenchen Fan e853afb416 [SPARK-26448][SQL] retain the difference between 0.0 and -0.0
## What changes were proposed in this pull request?

In https://github.com/apache/spark/pull/23043 , we introduced a behavior change: Spark users are not able to distinguish 0.0 and -0.0 anymore.

This PR proposes an alternative fix to the original bug, to retain the difference between 0.0 and -0.0 inside Spark.

The idea is, we can rewrite the window partition key, join key and grouping key during logical phase, to normalize the special floating numbers. Thus only operators care about special floating numbers need to pay the perf overhead, and end users can distinguish -0.0.

## How was this patch tested?

existing test

Closes #23388 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-01-09 13:50:32 -08:00
Gengliang Wang 311f32f37f [SPARK-26571][SQL] Update Hive Serde mapping with canonical name of Parquet and Orc FileFormat
## What changes were proposed in this pull request?

Currently Spark table maintains Hive catalog storage format, so that Hive client can read it.  In `HiveSerDe.scala`, Spark uses a mapping from its data source to HiveSerde. The mapping is old, we need to update with latest canonical name of Parquet and Orc FileFormat.

Otherwise the following queries will result in wrong Serde value in Hive table(default value `org.apache.hadoop.mapred.SequenceFileInputFormat`), and Hive client will fail to read the output table:
```
df.write.format("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat").saveAsTable(..)
```

```
df.write.format("org.apache.spark.sql.execution.datasources.orc.OrcFileFormat").saveAsTable(..)
```

This minor PR is to fix the mapping.

## How was this patch tested?

Unit test.

Closes #23491 from gengliangwang/fixHiveSerdeMap.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-09 10:18:33 +08:00
Maxim Gekk b305d71625
[SPARK-26547][SQL] Remove duplicate toHiveString from HiveUtils
## What changes were proposed in this pull request?

The `toHiveString()` and `toHiveStructString` methods were removed from `HiveUtils` because they have been already implemented in `HiveResult`. One related test was moved to `HiveResultSuite`.

## How was this patch tested?

By tests from `hive-thriftserver`.

Closes #23466 from MaxGekk/dedup-hive-result-string.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-06 17:36:06 -08:00
Sean Owen 36440e6447 [SPARK-26306][TEST][BUILD] More memory to de-flake SorterSuite
## What changes were proposed in this pull request?

Increase test memory to avoid OOM in TimSort-related tests.

## How was this patch tested?

Existing tests.

Closes #23425 from srowen/SPARK-26306.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-04 15:35:23 -06:00
Maxim Gekk 2a30deb85a [SPARK-26502][SQL] Move hiveResultString() from QueryExecution to HiveResult
## What changes were proposed in this pull request?

In the PR, I propose to move `hiveResultString()` out of `QueryExecution` and put it to a separate object.

Closes #23409 from MaxGekk/hive-result-string.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2019-01-03 11:27:40 +01:00
Maxim Gekk 5da55873fa [SPARK-26374][TEST][SQL] Enable TimestampFormatter in HadoopFsRelationTest
## What changes were proposed in this pull request?

Default timestamp pattern defined in `JSONOptions` doesn't allow saving/loading timestamps with time zones of seconds precision. Because of that, the round trip test failed for timestamps before 1582. In the PR, I propose to extend zone offset section from `XXX` to `XXXXX` which should allow to save/load zone offsets like `-07:52:48`.

## How was this patch tested?

It was tested by `JsonHadoopFsRelationSuite` and `TimestampFormatterSuite`.

Closes #23417 from MaxGekk/hadoopfsrelationtest-new-formatter.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-02 07:59:32 +08:00
zhoukang 2bf4d97118 [SPARK-24544][SQL] Print actual failure cause when look up function failed
## What changes were proposed in this pull request?

When we operate as below:
`
0: jdbc:hive2://xxx/> create  function funnel_analysis as 'com.xxx.hive.extend.udf.UapFunnelAnalysis';
`

`
0: jdbc:hive2://xxx/> select funnel_analysis(1,",",1,'');
Error: org.apache.spark.sql.AnalysisException: Undefined function: 'funnel_analysis'. This function is neither a registered temporary function nor a permanent function registered in the database 'xxx'.; line 1 pos 7 (state=,code=0)
`

`
0: jdbc:hive2://xxx/> describe function funnel_analysis;
+-----------------------------------------------------------+--+
|                       function_desc                       |
+-----------------------------------------------------------+--+
| Function: xxx.funnel_analysis                            |
| Class: com.xxx.hive.extend.udf.UapFunnelAnalysis  |
| Usage: N/A.                                               |
+-----------------------------------------------------------+--+
`
We can see describe funtion will get right information,but when we actually use this funtion,we will get an undefined exception.
Which is really misleading,the real cause is below:
 `
No handler for Hive UDF 'com.xxx.xxx.hive.extend.udf.UapFunnelAnalysis': java.lang.IllegalStateException: Should not be called directly;
	at org.apache.hadoop.hive.ql.udf.generic.GenericUDTF.initialize(GenericUDTF.java:72)
	at org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector$lzycompute(hiveUDFs.scala:204)
	at org.apache.spark.sql.hive.HiveGenericUDTF.outputInspector(hiveUDFs.scala:204)
	at org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema$lzycompute(hiveUDFs.scala:212)
	at org.apache.spark.sql.hive.HiveGenericUDTF.elementSchema(hiveUDFs.scala:212)
`
This patch print the actual failure for quick debugging.
## How was this patch tested?
UT

Closes #21790 from caneGuy/zhoukang/print-warning1.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-01 09:13:13 -06:00
Maxim Gekk a1c1dd3484 [SPARK-26191][SQL] Control truncation of Spark plans via maxFields parameter
## What changes were proposed in this pull request?

In the PR, I propose to add `maxFields` parameter to all functions involved in creation of textual representation of spark plans such as `simpleString` and `verboseString`. New parameter restricts number of fields converted to truncated strings. Any elements beyond the limit will be dropped and replaced by a `"... N more fields"` placeholder. The threshold is bumped up to `Int.MaxValue` for `toFile()`.

## How was this patch tested?

Added a test to `QueryExecutionSuite` which checks `maxFields` impacts on number of truncated fields in `LocalRelation`.

Closes #23159 from MaxGekk/to-file-max-fields.

Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Herman van Hovell <hvanhovell@databricks.com>
2018-12-27 11:13:16 +01:00
Liang-Chi Hsieh f89cdec8b9 [SPARK-26435][SQL] Support creating partitioned table using Hive CTAS by specifying partition column names
## What changes were proposed in this pull request?

Spark SQL doesn't support creating partitioned table using Hive CTAS in SQL syntax. However it is supported by using DataFrameWriter API.

```scala
val df = Seq(("a", 1)).toDF("part", "id")
df.write.format("hive").partitionBy("part").saveAsTable("t")
```
Hive begins to support this syntax in newer version: https://issues.apache.org/jira/browse/HIVE-20241:

```
CREATE TABLE t PARTITIONED BY (part) AS SELECT 1 as id, "a" as part
```

This patch adds this support to SQL syntax.

## How was this patch tested?

Added tests.

Closes #23376 from viirya/hive-ctas-partitioned-table.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-27 16:03:14 +08:00
Liang-Chi Hsieh 5ad03607d1 [SPARK-25271][SQL] Hive ctas commands should use data source if it is convertible
## What changes were proposed in this pull request?

In Spark 2.3.0 and previous versions, Hive CTAS command will convert to use data source to write data into the table when the table is convertible. This behavior is controlled by the configs like HiveUtils.CONVERT_METASTORE_ORC and HiveUtils.CONVERT_METASTORE_PARQUET.

In 2.3.1, we drop this optimization by mistake in the PR [SPARK-22977](https://github.com/apache/spark/pull/20521/files#r217254430). Since that Hive CTAS command only uses Hive Serde to write data.

This patch adds this optimization back to Hive CTAS command. This patch adds OptimizedCreateHiveTableAsSelectCommand which uses data source to write data.

## How was this patch tested?

Added test.

Closes #22514 from viirya/SPARK-25271-2.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-20 10:47:24 +08:00
Maxim Gekk 8a27952cdb [SPARK-26243][SQL] Use java.time API for parsing timestamps and dates from JSON
## What changes were proposed in this pull request?

In the PR, I propose to switch on **java.time API** for parsing timestamps and dates from JSON inputs with microseconds precision. The SQL config `spark.sql.legacy.timeParser.enabled` allow to switch back to previous behavior with using `java.text.SimpleDateFormat`/`FastDateFormat` for parsing/generating timestamps/dates.

## How was this patch tested?

It was tested by `JsonExpressionsSuite`, `JsonFunctionsSuite` and `JsonSuite`.

Closes #23196 from MaxGekk/json-time-parser.

Lead-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-16 09:32:13 +08:00
Darcy Shen c8ac6ae84c [SPARK-26319][SQL][TEST] Add appendReadColumns Unit Test for HiveShimSuite
## What changes were proposed in this pull request?

Add appendReadColumns Unit Test for HiveShimSuite.

## How was this patch tested?
```
$ build/sbt
> project hive
> testOnly *HiveShimSuite
```

Closes #23268 from sadhen/refactor/hiveshim.

Authored-by: Darcy Shen <sadhen@zoho.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-10 22:26:28 +08:00
gatorsmile 3bc83de3cc [SPARK-26307][SQL] Fix CTAS when INSERT a partitioned table using Hive serde
## What changes were proposed in this pull request?

This is a  Spark 2.3 regression introduced in https://github.com/apache/spark/pull/20521. We should add the partition info for InsertIntoHiveTable in CreateHiveTableAsSelectCommand. Otherwise, we will hit the following error by running the newly added test case:

```
[info] - CTAS: INSERT a partitioned table using Hive serde *** FAILED *** (829 milliseconds)
[info]   org.apache.spark.SparkException: Requested partitioning does not match the tab1 table:
[info] Requested partitions:
[info] Table partitions: part
[info]   at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.processInsert(InsertIntoHiveTable.scala:179)
[info]   at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.run(InsertIntoHiveTable.scala:107)
```

## How was this patch tested?

Added a test case.

Closes #23255 from gatorsmile/fixCTAS.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-10 14:57:20 +08:00
Takeshi Yamamuro 2612848422 [SPARK-25374][SQL] SafeProjection supports fallback to an interpreted mode
## What changes were proposed in this pull request?
In SPARK-23711, we have implemented the expression fallback logic to an interpreted mode. So, this pr fixed code to support the same fallback mode in `SafeProjection` based on `CodeGeneratorWithInterpretedFallback`.

## How was this patch tested?
Add tests in `CodeGeneratorWithInterpretedFallbackSuite` and `UnsafeRowConverterSuite`.

Closes #22468 from maropu/SPARK-25374-3.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-04 20:20:29 +08:00
Daoyuan Wang 8534d753ec [SPARK-26181][SQL] the hasMinMaxStats method of ColumnStatsMap is not correct
## What changes were proposed in this pull request?

For now the `hasMinMaxStats` will return the same as `hasCountStats`, which is obviously not as expected.

## How was this patch tested?

Existing tests.

Closes #23152 from adrian-wang/minmaxstats.

Authored-by: Daoyuan Wang <me@daoyuan.wang>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-12-03 23:54:26 +08:00
caoxuewen 327ac83f5c [SPARK-26180][CORE][TEST] Reuse withTempDir function to the SparkCore test case
## What changes were proposed in this pull request?

Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore  to clean up SparkCore test cases. thanks.

## How was this patch tested?

N / A

Closes #23151 from heary-cao/withCreateTempDir.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-01 16:34:11 +08:00
Takanobu Asanuma 15c0384977
[SPARK-26134][CORE] Upgrading Hadoop to 2.7.4 to fix java.version problem
## What changes were proposed in this pull request?

When I ran spark-shell on JDK11+28(2018-09-25), It failed with the error below.

```
Exception in thread "main" java.lang.ExceptionInInitializerError
	at org.apache.hadoop.util.StringUtils.<clinit>(StringUtils.java:80)
	at org.apache.hadoop.security.SecurityUtil.getAuthenticationMethod(SecurityUtil.java:611)
	at org.apache.hadoop.security.UserGroupInformation.initialize(UserGroupInformation.java:273)
	at org.apache.hadoop.security.UserGroupInformation.ensureInitialized(UserGroupInformation.java:261)
	at org.apache.hadoop.security.UserGroupInformation.loginUserFromSubject(UserGroupInformation.java:791)
	at org.apache.hadoop.security.UserGroupInformation.getLoginUser(UserGroupInformation.java:761)
	at org.apache.hadoop.security.UserGroupInformation.getCurrentUser(UserGroupInformation.java:634)
	at org.apache.spark.util.Utils$.$anonfun$getCurrentUserName$1(Utils.scala:2427)
	at scala.Option.getOrElse(Option.scala:121)
	at org.apache.spark.util.Utils$.getCurrentUserName(Utils.scala:2427)
	at org.apache.spark.SecurityManager.<init>(SecurityManager.scala:79)
	at org.apache.spark.deploy.SparkSubmit.secMgr$lzycompute$1(SparkSubmit.scala:359)
	at org.apache.spark.deploy.SparkSubmit.secMgr$1(SparkSubmit.scala:359)
	at org.apache.spark.deploy.SparkSubmit.$anonfun$prepareSubmitEnvironment$9(SparkSubmit.scala:367)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment(SparkSubmit.scala:367)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:143)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:86)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:927)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:936)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.lang.StringIndexOutOfBoundsException: begin 0, end 3, length 2
	at java.base/java.lang.String.checkBoundsBeginEnd(String.java:3319)
	at java.base/java.lang.String.substring(String.java:1874)
	at org.apache.hadoop.util.Shell.<clinit>(Shell.java:52)
```
This is a Hadoop issue that fails to parse some java.version. It has been fixed from Hadoop-2.7.4(see [HADOOP-14586](https://issues.apache.org/jira/browse/HADOOP-14586)).

Note, Hadoop-2.7.5 or upper have another problem with Spark ([SPARK-25330](https://issues.apache.org/jira/browse/SPARK-25330)). So upgrading to 2.7.4 would be fine for now.

## How was this patch tested?
Existing tests.

Closes #23101 from tasanuma/SPARK-26134.

Authored-by: Takanobu Asanuma <tasanuma@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-21 23:09:57 -08:00
Reynold Xin 07a700b371 [SPARK-26129][SQL] Instrumentation for per-query planning time
## What changes were proposed in this pull request?
We currently don't have good visibility into query planning time (analysis vs optimization vs physical planning). This patch adds a simple utility to track the runtime of various rules and various planning phases.

## How was this patch tested?
Added unit tests and end-to-end integration tests.

Closes #23096 from rxin/SPARK-26129.

Authored-by: Reynold Xin <rxin@databricks.com>
Signed-off-by: Reynold Xin <rxin@databricks.com>
2018-11-21 16:41:12 +01:00
Sean Owen 630e25e355 [SPARK-26026][BUILD] Published Scaladoc jars missing from Maven Central
## What changes were proposed in this pull request?

This restores scaladoc artifact generation, which got dropped with the Scala 2.12 update. The change looks large, but is almost all due to needing to make the InterfaceStability annotations top-level classes (i.e. `InterfaceStability.Stable` -> `Stable`), unfortunately. A few inner class references had to be qualified too.

Lots of scaladoc warnings now reappear. We can choose to disable generation by default and enable for releases, later.

## How was this patch tested?

N/A; build runs scaladoc now.

Closes #23069 from srowen/SPARK-26026.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 08:06:33 -06:00
Dongjoon Hyun ed46ac9f47
[SPARK-26091][SQL] Upgrade to 2.3.4 for Hive Metastore Client 2.3
## What changes were proposed in this pull request?

[Hive 2.3.4 is released on Nov. 7th](https://hive.apache.org/downloads.html#7-november-2018-release-234-available). This PR aims to support that version.

## How was this patch tested?

Pass the Jenkins with the updated version

Closes #23059 from dongjoon-hyun/SPARK-26091.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-17 03:28:43 -08:00
Dongjoon Hyun b538c442cb [MINOR][SQL] Fix typo in CTAS plan database string
## What changes were proposed in this pull request?

Since [Spark 1.6.0](56d7da14ab (diff-6f38a103058a6e233b7ad80718452387R96)), there was a redundant '}' character in CTAS string plan's database argument string; `default}`. This PR aims to fix it.

**BEFORE**
```scala
scala> sc.version
res1: String = 1.6.0

scala> sql("create table t as select 1").explain
== Physical Plan ==
ExecutedCommand CreateTableAsSelect [Database:default}, TableName: t, InsertIntoHiveTable]
+- Project [1 AS _c0#3]
   +- OneRowRelation$
```

**AFTER**
```scala
scala> sql("create table t as select 1").explain
== Physical Plan ==
Execute CreateHiveTableAsSelectCommand CreateHiveTableAsSelectCommand [Database:default, TableName: t, InsertIntoHiveTable]
+- *(1) Project [1 AS 1#4]
   +- Scan OneRowRelation[]
```

## How was this patch tested?

Manual.

Closes #23064 from dongjoon-hyun/SPARK-FIX.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-11-17 18:18:41 +08:00
DB Tsai ad853c5678
[SPARK-25956] Make Scala 2.12 as default Scala version in Spark 3.0
## What changes were proposed in this pull request?

This PR makes Spark's default Scala version as 2.12, and Scala 2.11 will be the alternative version. This implies that Scala 2.12 will be used by our CI builds including pull request builds.

We'll update the Jenkins to include a new compile-only jobs for Scala 2.11 to ensure the code can be still compiled with Scala 2.11.

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Sean Owen 2d085c13b7 [SPARK-25984][CORE][SQL][STREAMING] Remove deprecated .newInstance(), primitive box class constructor calls
## What changes were proposed in this pull request?

Deprecated in Java 11, replace Class.newInstance with Class.getConstructor.getInstance, and primtive wrapper class constructors with valueOf or equivalent

## How was this patch tested?

Existing tests.

Closes #22988 from srowen/SPARK-25984.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-10 09:52:14 -06:00
Dongjoon Hyun d66a4e82ec [SPARK-25102][SQL] Write Spark version to ORC/Parquet file metadata
## What changes were proposed in this pull request?

Currently, Spark writes Spark version number into Hive Table properties with `spark.sql.create.version`.
```
parameters:{
  spark.sql.sources.schema.part.0={
    "type":"struct",
    "fields":[{"name":"a","type":"integer","nullable":true,"metadata":{}}]
  },
  transient_lastDdlTime=1541142761,
  spark.sql.sources.schema.numParts=1,
  spark.sql.create.version=2.4.0
}
```

This PR aims to write Spark versions to ORC/Parquet file metadata with `org.apache.spark.sql.create.version` because we used `org.apache.` prefix in Parquet metadata already. It's different from Hive Table property key `spark.sql.create.version`, but it seems that we cannot change Hive Table property for backward compatibility.

After this PR, ORC and Parquet file generated by Spark will have the following metadata.

**ORC (`native` and `hive` implmentation)**
```
$ orc-tools meta /tmp/o
File Version: 0.12 with ...
...
User Metadata:
  org.apache.spark.sql.create.version=3.0.0
```

**PARQUET**
```
$ parquet-tools meta /tmp/p
...
creator:     parquet-mr version 1.10.0 (build 031a6654009e3b82020012a18434c582bd74c73a)
extra:       org.apache.spark.sql.create.version = 3.0.0
extra:       org.apache.spark.sql.parquet.row.metadata = {"type":"struct","fields":[{"name":"id","type":"long","nullable":false,"metadata":{}}]}
```

## How was this patch tested?

Pass the Jenkins with newly added test cases.

This closes #22255.

Closes #22932 from dongjoon-hyun/SPARK-25102.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-11-09 22:42:48 -08:00
Wenchen Fan 973f7c01df
[MINOR] update HiveExternalCatalogVersionsSuite to test 2.4.0
## What changes were proposed in this pull request?

Since Spark 2.4.0 is released, we should test it in HiveExternalCatalogVersionsSuite

## How was this patch tested?

N/A

Closes #22984 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-08 15:49:36 -08:00
Gengliang Wang 7bb901aa28
[SPARK-25964][SQL][MINOR] Revise OrcReadBenchmark/DataSourceReadBenchmark case names and execution instructions
## What changes were proposed in this pull request?

1. OrcReadBenchmark is under hive module, so the way to run it should be
```
build/sbt "hive/test:runMain <this class>"
```

2. The benchmark "String with Nulls Scan" should be with case "String with Nulls Scan(5%/50%/95%)", not "(0.05%/0.5%/0.95%)"

3. Add the null value percentages in the test case names of DataSourceReadBenchmark, for the benchmark "String with Nulls Scan" .

## How was this patch tested?

Re-run benchmarks

Closes #22965 from gengliangwang/fixHiveOrcReadBenchmark.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Gengliang Wang <ltnwgl@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-08 10:08:14 -08:00
hyukjinkwon 0a2e45fdb8 Revert "[SPARK-23831][SQL] Add org.apache.derby to IsolatedClientLoader"
This reverts commit a75571b46f.
2018-11-08 16:32:25 +08:00
Sean Owen c0d1bf0322 [MINOR] Fix typos and misspellings
## What changes were proposed in this pull request?

Fix typos and misspellings, per https://github.com/apache/spark-website/pull/158#issuecomment-435790366

## How was this patch tested?

Existing tests.

Closes #22950 from srowen/Typos.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-05 17:34:23 -06:00
Takuya UESHIN 4afb350334 [SPARK-25884][SQL][FOLLOW-UP] Add sample.json back.
## What changes were proposed in this pull request?

This is a follow-up pr of #22892 which moved `sample.json` from hive module to sql module, but we still need the file in hive module.

## How was this patch tested?

Existing tests.

Closes #22942 from ueshin/issues/SPARK-25884/sample.json.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-05 15:53:06 +08:00
Dongjoon Hyun e91b607719
[SPARK-25918][SQL] LOAD DATA LOCAL INPATH should handle a relative path
## What changes were proposed in this pull request?

Unfortunately, it seems that we missed this in 2.4.0. In Spark 2.4, if the default file system is not the local file system, `LOAD DATA LOCAL INPATH` only works in case of absolute paths. This PR aims to fix it to support relative paths. This is a regression in 2.4.0.

```scala
$ ls kv1.txt
kv1.txt

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

## How was this patch tested?

Pass the Jenkins

Closes #22927 from dongjoon-hyun/SPARK-LOAD.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-01 23:18:20 -07:00
Takuya UESHIN cc82b9fed8 [SPARK-25884][SQL] Add TBLPROPERTIES and COMMENT, and use LOCATION when SHOW CREATE TABLE.
## What changes were proposed in this pull request?

When `SHOW CREATE TABLE` for Datasource tables, we are missing `TBLPROPERTIES` and `COMMENT`, and we should use `LOCATION` instead of path in `OPTION`.

## How was this patch tested?

Splitted `ShowCreateTableSuite` to confirm to work with both `InMemoryCatalog` and `HiveExternalCatalog`, and  added some tests.

Closes #22892 from ueshin/issues/SPARK-25884/show_create_table.

Authored-by: Takuya UESHIN <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-01 10:00:14 +08:00
Dongjoon Hyun b3af917e76
[SPARK-25893][SQL] Show a directional error message for unsupported Hive Metastore versions
## What changes were proposed in this pull request?

When `spark.sql.hive.metastore.version` is misconfigured, we had better give a directional error message.

**BEFORE**
```scala
scala> sql("show databases").show
scala.MatchError: 2.4 (of class java.lang.String)
```

**AFTER**
```scala
scala> sql("show databases").show
java.lang.UnsupportedOperationException: Unsupported Hive Metastore version (2.4).
Please set spark.sql.hive.metastore.version with a valid version.
```

## How was this patch tested?

Manual.

Closes #22902 from dongjoon-hyun/SPARK-25893.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-31 09:20:19 -07:00
yucai 409d688fb6 [SPARK-25864][SQL][TEST] Make main args accessible for BenchmarkBase's subclass
## What changes were proposed in this pull request?

Set main args correctly in BenchmarkBase, to make it accessible for its subclass.
It will benefit:
- BuiltInDataSourceWriteBenchmark
- AvroWriteBenchmark

## How was this patch tested?

manual tests

Closes #22872 from yucai/main_args.

Authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-29 20:00:31 +08:00
liuxian 4427a96bce [SPARK-25806][SQL] The instance of FileSplit is redundant
## What changes were proposed in this pull request?

 The instance of `FileSplit` is redundant for   `ParquetFileFormat` and `hive\orc\OrcFileFormat` class.

## How was this patch tested?
Existing unit tests in `ParquetQuerySuite.scala` and `HiveOrcQuerySuite.scala`

Closes #22802 from 10110346/FileSplitnotneed.

Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-28 17:39:16 -05:00
laskfla 6f05669e4e [MINOR][DOC] Fix comment error of HiveUtils
## What changes were proposed in this pull request?
Change the version number in comment of `HiveUtils.newClientForExecution` from `13` to `1.2.1` .

## How was this patch tested?

N/A

Closes #22850 from laskfla/HiveUtils-Comment.

Authored-by: laskfla <wwlsax11@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-27 08:09:59 -05:00
Peter Toth ccd07b7366
[SPARK-25665][SQL][TEST] Refactor ObjectHashAggregateExecBenchmark to…
## What changes were proposed in this pull request?

Refactor ObjectHashAggregateExecBenchmark to use main method

## How was this patch tested?

Manually tested:
```
bin/spark-submit --class org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark --jars sql/catalyst/target/spark-catalyst_2.11-3.0.0-SNAPSHOT-tests.jar,core/target/spark-core_2.11-3.0.0-SNAPSHOT-tests.jar,sql/hive/target/spark-hive_2.11-3.0.0-SNAPSHOT.jar --packages org.spark-project.hive:hive-exec:1.2.1.spark2 sql/hive/target/spark-hive_2.11-3.0.0-SNAPSHOT-tests.jar
```
Generated results with:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain org.apache.spark.sql.execution.benchmark.ObjectHashAggregateExecBenchmark"
```

Closes #22804 from peter-toth/SPARK-25665.

Lead-authored-by: Peter Toth <peter.toth@gmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-25 12:42:31 -07:00
Yuming Wang 9ad0f6ea89
[SPARK-25269][SQL] SQL interface support specify StorageLevel when cache table
## What changes were proposed in this pull request?

SQL interface support specify `StorageLevel` when cache table. The semantic is:
```sql
CACHE TABLE tableName OPTIONS('storageLevel' 'DISK_ONLY');
```
All supported `StorageLevel` are:
eefdf9f9dd/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala (L172-L183)

## How was this patch tested?

unit tests and manual tests.

manual tests configuration:
```
--executor-memory 15G --executor-cores 5 --num-executors 50
```
Data:
Input Size / Records: 1037.7 GB / 11732805788

Result:
![image](https://user-images.githubusercontent.com/5399861/47213362-56a1c980-d3cd-11e8-82e7-28d7abc5923e.png)

Closes #22263 from wangyum/SPARK-25269.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-19 09:15:55 -07:00
Peter Toth f38594fc56 [SPARK-25768][SQL] fix constant argument expecting UDAFs
## What changes were proposed in this pull request?

Without this PR some UDAFs like `GenericUDAFPercentileApprox` can throw an exception because expecting a constant parameter (object inspector) as a particular argument.

The exception is thrown because `toPrettySQL` call in `ResolveAliases` analyzer rule transforms a `Literal` parameter to a `PrettyAttribute` which is then transformed to an `ObjectInspector` instead of a `ConstantObjectInspector`.
The exception comes from `getEvaluator` method of `GenericUDAFPercentileApprox` that actually shouldn't be called during `toPrettySQL` transformation. The reason why it is called are the non lazy fields in `HiveUDAFFunction`.

This PR makes all fields of `HiveUDAFFunction` lazy.

## How was this patch tested?

added new UT

Closes #22766 from peter-toth/SPARK-25768.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-19 21:17:14 +08:00
彭灿00244106 e9332f600e [SQL][CATALYST][MINOR] update some error comments
## What changes were proposed in this pull request?

this PR correct some comment error:
1. change from "as low a possible" to "as low as possible" in RewriteDistinctAggregates.scala
2. delete redundant word “with” in HiveTableScanExec’s  doExecute()  method

## How was this patch tested?

Existing unit tests.

Closes #22694 from CarolinePeng/update_comment.

Authored-by: 彭灿00244106 <00244106@zte.intra>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-17 12:45:13 +08:00
Imran Rashid fdaa99897a [SPARK-25738][SQL] Fix LOAD DATA INPATH for hdfs port
## What changes were proposed in this pull request?

LOAD DATA INPATH didn't work if the defaultFS included a port for hdfs.
Handling this just requires a small change to use the correct URI
constructor.

## How was this patch tested?

Added a unit test, ran all tests via jenkins

Closes #22733 from squito/SPARK-25738.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-10-15 18:34:30 -07:00
Wenchen Fan 78e133141c [SPARK-25708][SQL] HAVING without GROUP BY means global aggregate
## What changes were proposed in this pull request?

According to the SQL standard, when a query contains `HAVING`, it indicates an aggregate operator. For more details please refer to https://blog.jooq.org/2014/12/04/do-you-really-understand-sqls-group-by-and-having-clauses/

However, in Spark SQL parser, we treat HAVING as a normal filter when there is no GROUP BY, which breaks SQL semantic and lead to wrong result. This PR fixes the parser.

## How was this patch tested?

new test

Closes #22696 from cloud-fan/having.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-12 00:24:06 -07:00
Kazuaki Ishizaki c9d7d83ed5 [SPARK-25388][TEST][SQL] Detect incorrect nullable of DataType in the result
## What changes were proposed in this pull request?

This PR can correctly cause assertion failure when incorrect nullable of DataType in the result is generated by a target function to be tested.

Let us think the following example. In the future, a developer would write incorrect code that returns unexpected result. We have to correctly cause fail in this test since `valueContainsNull=false` while `expr` includes `null`. However, without this PR, this test passes. This PR can correctly cause fail.

```
test("test TARGETFUNCTON") {
  val expr = TARGETMAPFUNCTON()
  // expr = UnsafeMap(3 -> 6, 7 -> null)
  // expr.dataType = (IntegerType, IntegerType, false)

  expected = Map(3 -> 6, 7 -> null)
  checkEvaluation(expr, expected)
```

In [`checkEvaluationWithUnsafeProjection`](https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala#L208-L235), the results are compared using `UnsafeRow`. When the given `expected` is [converted](https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala#L226-L227)) to `UnsafeRow` using the `DataType` of `expr`.
```
val expectedRow = UnsafeProjection.create(Array(expression.dataType, expression.dataType)).apply(lit)
```

In summary, `expr` is `[0,1800000038,5000000038,18,2,0,700000003,2,0,6,18,2,0,700000003,2,0,6]` with and w/o this PR. `expected` is converted to

* w/o  this PR, `[0,1800000038,5000000038,18,2,0,700000003,2,0,6,18,2,0,700000003,2,0,6]`
* with this PR, `[0,1800000038,5000000038,18,2,0,700000003,2,2,6,18,2,0,700000003,2,2,6]`

As a result, w/o this PR, the test unexpectedly passes.

This is because, w/o this PR, based on given `dataType`, generated code of projection for `expected` avoids to set nullbit.
```
                    // tmpInput_2 is expected
/* 155 */           for (int index_1 = 0; index_1 < numElements_1; index_1++) {
/* 156 */             mutableStateArray_1[1].write(index_1, tmpInput_2.getInt(index_1));
/* 157 */           }
```

With this PR, generated code of projection for `expected` always checks whether nullbit should be set by `isNullAt`
```
                    // tmpInput_2 is expected
/* 161 */           for (int index_1 = 0; index_1 < numElements_1; index_1++) {
/* 162 */
/* 163 */             if (tmpInput_2.isNullAt(index_1)) {
/* 164 */               mutableStateArray_1[1].setNull4Bytes(index_1);
/* 165 */             } else {
/* 166 */               mutableStateArray_1[1].write(index_1, tmpInput_2.getInt(index_1));
/* 167 */             }
/* 168 */
/* 169 */           }
```

## How was this patch tested?

Existing UTs

Closes #22375 from kiszk/SPARK-25388.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-10-12 11:14:35 +08:00
Sean Owen 80813e1980 [SPARK-25016][BUILD][CORE] Remove support for Hadoop 2.6
## What changes were proposed in this pull request?

Remove Hadoop 2.6 references and make 2.7 the default.
Obviously, this is for master/3.0.0 only.
After this we can also get rid of the separate test jobs for Hadoop 2.6.

## How was this patch tested?

Existing tests

Closes #22615 from srowen/SPARK-25016.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-10 12:07:53 -07:00
Gengliang Wang 6df2345794
[SPARK-25699][SQL] Partially push down conjunctive predicated in ORC
## What changes were proposed in this pull request?

Inspired by https://github.com/apache/spark/pull/22574 .
We can partially push down top level conjunctive predicates to Orc.
This PR improves Orc predicate push down in both SQL and Hive module.

## How was this patch tested?

New unit test.

Closes #22684 from gengliangwang/pushOrcFilters.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-10-10 18:18:56 +00:00
Dilip Biswal 3528c08beb [SPARK-25611][SPARK-25612][SQL][TESTS] Improve test run time of CompressionCodecSuite
## What changes were proposed in this pull request?
Reduced the combination of codecs from 9 to 3 to improve the test runtime.

## How was this patch tested?
This is a test fix.

Closes #22641 from dilipbiswal/SPARK-25611.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-10 08:51:16 -07:00
Gengliang Wang 6a60fb0aad [SPARK-25630][TEST] Reduce test time of HadoopFsRelationTest
## What changes were proposed in this pull request?
There was 5 suites extends `HadoopFsRelationTest`,  for testing "orc"/"parquet"/"text"/"json" data sources.
This PR refactor the base trait `HadoopFsRelationTest`:
1. Rename unnecessary loop for setting parquet conf
2. The test case `SPARK-8406: Avoids name collision while writing files` takes about 14 to 20 seconds. As now all the file format data source are using common code, for creating result files, we can test one data source(Parquet) only to reduce test time.

To run related 5 suites:
```
./build/sbt "hive/testOnly *HadoopFsRelationSuite"
```
The total test run time is reduced from 5 minutes 40 seconds to 3 minutes 50 seconds.

## How was this patch tested?

Unit test

Closes #22643 from gengliangwang/refactorHadoopFsRelationTest.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-08 13:05:53 -07:00
Gengliang Wang bbd038d243 [SPARK-25653][TEST] Add tag ExtendedHiveTest for HiveSparkSubmitSuite
## What changes were proposed in this pull request?

The total run time of `HiveSparkSubmitSuite` is about 10 minutes.
While the related code is stable, add tag `ExtendedHiveTest` for it.
## How was this patch tested?

Unit test.

Closes #22642 from gengliangwang/addTagForHiveSparkSubmitSuite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-05 17:03:24 -07:00
Dongjoon Hyun 1c9486c1ac [SPARK-25635][SQL][BUILD] Support selective direct encoding in native ORC write
## What changes were proposed in this pull request?

Before ORC 1.5.3, `orc.dictionary.key.threshold` and `hive.exec.orc.dictionary.key.size.threshold` are applied for all columns. This has been a big huddle to enable dictionary encoding. From ORC 1.5.3, `orc.column.encoding.direct` is added to enforce direct encoding selectively in a column-wise manner. This PR aims to add that feature by upgrading ORC from 1.5.2 to 1.5.3.

The followings are the patches in ORC 1.5.3 and this feature is the only one related to Spark directly.
```
ORC-406: ORC: Char(n) and Varchar(n) writers truncate to n bytes & corrupts multi-byte data (gopalv)
ORC-403: [C++] Add checks to avoid invalid offsets in InputStream
ORC-405: Remove calcite as a dependency from the benchmarks.
ORC-375: Fix libhdfs on gcc7 by adding #include <functional> two places.
ORC-383: Parallel builds fails with ConcurrentModificationException
ORC-382: Apache rat exclusions + add rat check to travis
ORC-401: Fix incorrect quoting in specification.
ORC-385: Change RecordReader to extend Closeable.
ORC-384: [C++] fix memory leak when loading non-ORC files
ORC-391: [c++] parseType does not accept underscore in the field name
ORC-397: Allow selective disabling of dictionary encoding. Original patch was by Mithun Radhakrishnan.
ORC-389: Add ability to not decode Acid metadata columns
```

## How was this patch tested?

Pass the Jenkins with newly added test cases.

Closes #22622 from dongjoon-hyun/SPARK-25635.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-05 16:42:06 -07:00
Dilip Biswal a433fbcee6 [SPARK-25626][SQL][TEST] Improve the test execution time of HiveClientSuites
## What changes were proposed in this pull request?
Improve the runtime by reducing the number of partitions created in the test. The number of partitions are reduced from 280 to 60.

Here are the test times for the `getPartitionsByFilter returns all partitions` test  on my laptop.

```
[info] - 0.13: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (4 seconds, 230 milliseconds)
[info] - 0.14: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (3 seconds, 576 milliseconds)
[info] - 1.0: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (3 seconds, 495 milliseconds)
[info] - 1.1: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (6 seconds, 728 milliseconds)
[info] - 1.2: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (7 seconds, 260 milliseconds)
[info] - 2.0: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (8 seconds, 270 milliseconds)
[info] - 2.1: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (6 seconds, 856 milliseconds)
[info] - 2.2: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (7 seconds, 587 milliseconds)
[info] - 2.3: getPartitionsByFilter returns all partitions when hive.metastore.try.direct.sql=false (7 seconds, 230 milliseconds)
## How was this patch tested?
Test only.

Closes #22644 from dilipbiswal/SPARK-25626.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-05 14:39:30 -07:00
Gengliang Wang 7b4e94f160
[SPARK-25581][SQL] Rename method benchmark as runBenchmarkSuite in BenchmarkBase
## What changes were proposed in this pull request?

Rename method `benchmark` in `BenchmarkBase` as `runBenchmarkSuite `. Also add comments.
Currently the method name `benchmark` is a bit confusing. Also the name is the same as instances of `Benchmark`:

f246813afb/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala (L330-L339)

## How was this patch tested?

Unit test.

Closes #22599 from gengliangwang/renameBenchmarkSuite.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-10-02 10:04:47 -07:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

This patch is to bump the master branch version to 3.0.0-SNAPSHOT.

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
hyukjinkwon a2f502cf53 [SPARK-25565][BUILD] Add scalastyle rule to check add Locale.ROOT to .toLowerCase and .toUpperCase for internal calls
## What changes were proposed in this pull request?

This PR adds a rule to force `.toLowerCase(Locale.ROOT)` or `toUpperCase(Locale.ROOT)`.

It produces an error as below:

```
[error]       Are you sure that you want to use toUpperCase or toLowerCase without the root locale? In most cases, you
[error]       should use toUpperCase(Locale.ROOT) or toLowerCase(Locale.ROOT) instead.
[error]       If you must use toUpperCase or toLowerCase without the root locale, wrap the code block with
[error]       // scalastyle:off caselocale
[error]       .toUpperCase
[error]       .toLowerCase
[error]       // scalastyle:on caselocale
```

This PR excludes the cases above for SQL code path for external calls like table name, column name and etc.

For test suites, or when it's clear there's no locale problem like Turkish locale problem, it uses `Locale.ROOT`.

One minor problem is, `UTF8String` has both methods, `toLowerCase` and `toUpperCase`, and the new rule detects them as well. They are ignored.

## How was this patch tested?

Manually tested, and Jenkins tests.

Closes #22581 from HyukjinKwon/SPARK-25565.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-30 14:31:04 +08:00
yucai f246813afb
[SPARK-25508][SQL][TEST] Refactor OrcReadBenchmark to use main method
## What changes were proposed in this pull request?

Refactor OrcReadBenchmark to use main method.
Generate benchmark result:
```
SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain org.apache.spark.sql.hive.orc.OrcReadBenchmark"
```
## How was this patch tested?

manual tests

Closes #22580 from yucai/SPARK-25508.

Lead-authored-by: yucai <yyu1@ebay.com>
Co-authored-by: Yucai Yu <yucai.yu@foxmail.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-29 09:48:03 -07:00
Dongjoon Hyun 1e437835e9 [SPARK-25570][SQL][TEST] Replace 2.3.1 with 2.3.2 in HiveExternalCatalogVersionsSuite
## What changes were proposed in this pull request?

This PR aims to prevent test slowdowns at `HiveExternalCatalogVersionsSuite` by using the latest Apache Spark 2.3.2 link because the Apache mirrors will remove the old Spark 2.3.1 binaries eventually. `HiveExternalCatalogVersionsSuite` will not fail because [SPARK-24813](https://issues.apache.org/jira/browse/SPARK-24813) implements a fallback logic. However, it will cause many trials and fallbacks in all builds over `branch-2.3/branch-2.4/master`. We had better fix this issue.

## How was this patch tested?

Pass the Jenkins with the updated version.

Closes #22587 from dongjoon-hyun/SPARK-25570.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-29 11:43:58 +08:00
Dilip Biswal 7deef7a49b [SPARK-25458][SQL] Support FOR ALL COLUMNS in ANALYZE TABLE
## What changes were proposed in this pull request?
**Description from the JIRA :**
Currently, to collect the statistics of all the columns, users need to specify the names of all the columns when calling the command "ANALYZE TABLE ... FOR COLUMNS...". This is not user friendly. Instead, we can introduce the following SQL command to achieve it without specifying the column names.

```
   ANALYZE TABLE [db_name.]tablename COMPUTE STATISTICS FOR ALL COLUMNS;
```

## How was this patch tested?
Added new tests in SparkSqlParserSuite and StatisticsSuite

Closes #22566 from dilipbiswal/SPARK-25458.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-28 15:03:06 -07:00
Chris Zhao 3b7395fe02
[SPARK-25459][SQL] Add viewOriginalText back to CatalogTable
## What changes were proposed in this pull request?

The `show create table` will show a lot of generated attributes for views that created by older Spark version. This PR will basically revert https://issues.apache.org/jira/browse/SPARK-19272 back, so when you `DESC [FORMATTED|EXTENDED] view` will show the original view DDL text.

## How was this patch tested?
Unit test.

Closes #22458 from zheyuan28/testbranch.

Lead-authored-by: Chris Zhao <chris.zhao@databricks.com>
Co-authored-by: Christopher Zhao <chris.zhao@databricks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-27 17:55:08 -07:00
Dongjoon Hyun 81cbcca600
[SPARK-25534][SQL] Make SQLHelper trait
## What changes were proposed in this pull request?

Currently, Spark has 7 `withTempPath` and 6 `withSQLConf` functions. This PR aims to remove duplicated and inconsistent code and reduce them to the following meaningful implementations.

**withTempPath**
- `SQLHelper.withTempPath`: The one which was used in `SQLTestUtils`.

**withSQLConf**
- `SQLHelper.withSQLConf`: The one which was used in `PlanTest`.
- `ExecutorSideSQLConfSuite.withSQLConf`: The one which doesn't throw `AnalysisException` on StaticConf changes.
- `SQLTestUtils.withSQLConf`: The one which overrides intentionally to change the active session.
```scala
  protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = {
    SparkSession.setActiveSession(spark)
    super.withSQLConf(pairs: _*)(f)
  }
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #22548 from dongjoon-hyun/SPARK-25534.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-09-25 23:03:54 -07:00