Commit graph

24922 commits

Author SHA1 Message Date
Unknown 3f35440304 [SPARK-28543][DOCS][WEBUI] Document Spark Jobs page
## What changes were proposed in this pull request?

New documentation to explain in detail Web UI Jobs page and link it to monitoring page. New images are included to better explanation

![image](https://user-images.githubusercontent.com/12819544/62898145-2741bc00-bd55-11e9-89f7-175a4fd81009.png)
![image](https://user-images.githubusercontent.com/12819544/62898187-39235f00-bd55-11e9-9f03-a4d179e197fe.png)

## How was this patch tested?

This pull request contains only documentation. I have generated it using "jekyll build" to ensure that it's ok

Closes #25424 from planga82/feature/SPARK-28543_ImproveWebUIDocs.

Lead-authored-by: Unknown <soypab@gmail.com>
Co-authored-by: Pablo <soypab@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-15 08:52:23 -05:00
Yuming Wang 1b416a0c77 [SPARK-27592][SQL] Set the bucketed data source table SerDe correctly
## What changes were proposed in this pull request?

Hive using incorrect **InputFormat**(`org.apache.hadoop.mapred.SequenceFileInputFormat`) to read Spark's **Parquet** bucketed data source table.
Spark side:
```sql
spark-sql> CREATE TABLE t (c1 INT, c2 INT) USING parquet CLUSTERED BY (c1) SORTED BY (c1) INTO 2 BUCKETS;
2019-04-29 17:52:05 WARN  HiveExternalCatalog:66 - Persisting bucketed data source table `default`.`t` into Hive metastore in Spark SQL specific format, which is NOT compatible with Hive.
spark-sql> DESC FORMATTED t;
c1	int	NULL
c2	int	NULL

# Detailed Table Information
Database	default
Table	t
Owner	yumwang
Created Time	Mon Apr 29 17:52:05 CST 2019
Last Access	Thu Jan 01 08:00:00 CST 1970
Created By	Spark 2.4.0
Type	MANAGED
Provider	parquet
Num Buckets	2
Bucket Columns	[`c1`]
Sort Columns	[`c1`]
Table Properties	[transient_lastDdlTime=1556531525]
Location	file:/user/hive/warehouse/t
Serde Library	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat	org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties	[serialization.format=1]
```
Hive side:
```sql
hive> DESC FORMATTED t;
OK
# col_name            	data_type           	comment

c1                  	int
c2                  	int

# Detailed Table Information
Database:           	default
Owner:              	root
CreateTime:         	Wed May 08 03:38:46 GMT-07:00 2019
LastAccessTime:     	UNKNOWN
Retention:          	0
Location:           	file:/user/hive/warehouse/t
Table Type:         	MANAGED_TABLE
Table Parameters:
	bucketing_version   	spark
	spark.sql.create.version	3.0.0-SNAPSHOT
	spark.sql.sources.provider	parquet
	spark.sql.sources.schema.bucketCol.0	c1
	spark.sql.sources.schema.numBucketCols	1
	spark.sql.sources.schema.numBuckets	2
	spark.sql.sources.schema.numParts	1
	spark.sql.sources.schema.numSortCols	1
	spark.sql.sources.schema.part.0	{\"type\":\"struct\",\"fields\":[{\"name\":\"c1\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"c2\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}}]}
	spark.sql.sources.schema.sortCol.0	c1
	transient_lastDdlTime	1557311926

# Storage Information
SerDe Library:      	org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe
InputFormat:        	org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat
OutputFormat:       	org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat
Compressed:         	No
Num Buckets:        	-1
Bucket Columns:     	[]
Sort Columns:       	[]
Storage Desc Params:
	path                	file:/user/hive/warehouse/t
	serialization.format	1
```

So it's non-bucketed table at Hive side. This pr set the `SerDe` correctly so Hive can read these tables.

Related code:
33f3c48cac/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala (L976-L990)
f9776e3892/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala (L444-L459)

## How was this patch tested?

unit tests

Closes #24486 from wangyum/SPARK-27592.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-15 17:21:13 +08:00
Gabor Somogyi a493031e2e [SPARK-28695][SS] Use CaseInsensitiveMap in KafkaSourceProvider to make source param handling more robust
## What changes were proposed in this pull request?

[SPARK-28163](https://issues.apache.org/jira/browse/SPARK-28163) fixed a bug and during the analysis we've concluded it would be more robust to use `CaseInsensitiveMap` inside Kafka source. This case less lower/upper case problem would rise in the future.

Please note this PR doesn't intend to solve any kind of actual problem but finish the concept added in [SPARK-28163](https://issues.apache.org/jira/browse/SPARK-28163) (in a fix PR I didn't want to add too invasive changes). In this PR I've changed `Map[String, String]` to `CaseInsensitiveMap[String]` to enforce the usage. These are the main use-cases:
* `contains` => `CaseInsensitiveMap` solves it
* `get...` => `CaseInsensitiveMap` solves it
* `filter` => keys must be converted to lowercase because there is no guarantee that the incoming map has such key set
* `find` => keys must be converted to lowercase because there is no guarantee that the incoming map has such key set
* passing parameters to Kafka consumer/producer => keys must be converted to lowercase because there is no guarantee that the incoming map has such key set

## How was this patch tested?

Existing unit tests.

Closes #25418 from gaborgsomogyi/SPARK-28695.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-15 14:43:52 +08:00
Burak Yavuz 0526529b31 [SPARK-28666] Support saveAsTable for V2 tables through Session Catalog
## What changes were proposed in this pull request?

We add support for the V2SessionCatalog for saveAsTable, such that V2 tables can plug in and leverage existing DataFrameWriter.saveAsTable APIs to write and create tables through the session catalog.

## How was this patch tested?

Unit tests. A lot of tests broke under hive when things were not working properly under `ResolveTables`, therefore I believe the current set of tests should be sufficient in testing the table resolution and read code paths.

Closes #25402 from brkyvz/saveAsV2.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-15 12:29:34 +08:00
Xianjin YE 3ec24fd128 [SPARK-28203][CORE][PYTHON] PythonRDD should respect SparkContext's hadoop configuration
## What changes were proposed in this pull request?
1. PythonHadoopUtil.mapToConf generates a Configuration with loadDefaults disabled
2. merging hadoop conf in several places of PythonRDD is consistent.

## How was this patch tested?
Added a new test and existed tests

Closes #25002 from advancedxy/SPARK-28203.

Authored-by: Xianjin YE <advancedxy@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-15 10:39:33 +09:00
Maxim Gekk 3a4afce96c [SPARK-28687][SQL] Support epoch, isoyear, milliseconds and microseconds at extract()
## What changes were proposed in this pull request?

In the PR, I propose new expressions `Epoch`, `IsoYear`, `Milliseconds` and `Microseconds`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT):

1. `epoch` - the number of seconds since 1970-01-01 00:00:00 local time in microsecond precision.
2. `isoyear` - the ISO 8601 week-numbering year that the date falls in. Each ISO 8601 week-numbering year begins with the Monday of the week containing the 4th of January.
3. `milliseconds` - the seconds field including fractional parts multiplied by 1,000.
4. `microseconds` - the seconds field including fractional parts multiplied by 1,000,000.

Here are examples:
```sql
spark-sql> SELECT EXTRACT(EPOCH FROM TIMESTAMP '2019-08-11 19:07:30.123456');
1565550450.123456
spark-sql> SELECT EXTRACT(ISOYEAR FROM DATE '2006-01-01');
2005
spark-sql> SELECT EXTRACT(MILLISECONDS FROM TIMESTAMP '2019-08-11 19:07:30.123456');
30123.456
spark-sql> SELECT EXTRACT(MICROSECONDS FROM TIMESTAMP '2019-08-11 19:07:30.123456');
30123456
```

## How was this patch tested?

Added new tests to `DateExpressionsSuite`, and uncommented existing tests in `extract.sql` and `pgSQL/date.sql`.

Closes #25408 from MaxGekk/extract-ext3.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-14 08:44:44 -07:00
xy_xin 2eeb25e52d [SPARK-28351][SQL] Support DELETE in DataSource V2
## What changes were proposed in this pull request?

This pr adds DELETE support for V2 datasources. As a first step, this pr only support delete by source filters:
```scala
void delete(Filter[] filters);
```
which could not deal with complicated cases like subqueries.

Since it's uncomfortable to embed the implementation of DELETE in the current V2 APIs, a new mix-in of datasource is added, which is called `SupportsMaintenance`, similar to `SupportsRead` and `SupportsWrite`.  A datasource which can be maintained means we can perform DELETE/UPDATE/MERGE/OPTIMIZE on the datasource, as long as the datasource implements the necessary mix-ins.

## How was this patch tested?

new test case.

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

Closes #25115 from xianyinxin/SPARK-28351.

Authored-by: xy_xin <xianyin.xxy@alibaba-inc.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-14 23:38:45 +08:00
John Zhuge 391c7e8f2e [SPARK-27739][SQL] df.persist should save stats from optimized plan
## What changes were proposed in this pull request?

CacheManager.cacheQuery saves the stats from the optimized plan to cache.

## How was this patch tested?

Existing testss.

Closes #24623 from jzhuge/SPARK-27739.

Authored-by: John Zhuge <jzhuge@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-14 19:49:53 +08:00
Edgar Rodriguez 598fcbe5ed [SPARK-28265][SQL] Add renameTable to TableCatalog API
## What changes were proposed in this pull request?

This PR adds the `renameTable` call to the `TableCatalog` API, as described in the [Table Metadata API SPIP](https://docs.google.com/document/d/1zLFiA1VuaWeVxeTDXNg8bL6GP3BVoOZBkewFtEnjEoo/edit#heading=h.m45webtwxf2d).

This PR is related to: https://github.com/apache/spark/pull/24246

## How was this patch tested?

Added  unit tests and contract tests.

Closes #25206 from edgarRd/SPARK-28265-add-rename-table-catalog-api.

Authored-by: Edgar Rodriguez <edgar.rd@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-14 14:24:13 +08:00
DB Tsai 3302042ec4 [SPARK-28719][BUILD] [FOLLOWUP] Add JDK11 for Github Actions
## What changes were proposed in this pull request?

Add JDK11 for Github Actions

Closes #25444 from dbtsai/jdk11.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-08-14 03:14:07 +00:00
Marcelo Vanzin 0343854f54 [SPARK-28487][K8S] More responsive dynamic allocation with K8S
This change implements a few changes to the k8s pod allocator so
that it behaves a little better when dynamic allocation is on.

(i) Allow the application to ramp up immediately when there's a
change in the target number of executors. Without this change,
scaling would only trigger when a change happened in the state of
the cluster, e.g. an executor going down, or when the periodical
snapshot was taken (default every 30s).

(ii) Get rid of pending pod requests, both acknowledged (i.e. Spark
knows that a pod is pending resource allocation) and unacknowledged
(i.e. Spark has requested the pod but the API server hasn't created it
yet), when they're not needed anymore. This avoids starting those
executors to just remove them after the idle timeout, wasting resources
in the meantime.

(iii) Re-work some of the code to avoid unnecessary logging. While not
bad without dynamic allocation, the existing logging was very chatty
when dynamic allocation was on. With the changes, all the useful
information is still there, but only when interesting changes happen.

(iv) Gracefully shut down executors when they become idle. Just deleting
the pod causes a lot of ugly logs to show up, so it's better to ask pods
to exit nicely. That also allows Spark to respect the "don't delete
pods" option when dynamic allocation is on.

Tested on a small k8s cluster running different TPC-DS workloads.

Closes #25236 from vanzin/SPARK-28487.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-08-13 17:29:54 -07:00
Dilip Biswal 331f2657d9 [SPARK-27768][SQL] Support Infinity/NaN-related float/double literals case-insensitively
## What changes were proposed in this pull request?
Here is the problem description from the JIRA.
```
When the inputs contain the constant 'infinity', Spark SQL does not generate the expected results.

SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('1'), (CAST('infinity' AS DOUBLE))) v(x);
SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('infinity'), ('1')) v(x);
SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('infinity'), ('infinity')) v(x);
SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE))
FROM (VALUES ('-infinity'), ('infinity')) v(x);
 The root cause: Spark SQL does not recognize the special constants in a case insensitive way. In PostgreSQL, they are recognized in a case insensitive way.

Link: https://www.postgresql.org/docs/9.3/datatype-numeric.html
```

In this PR, the casting code is enhanced to handle these `special` string literals in case insensitive manner.

## How was this patch tested?
Added tests in CastSuite and modified existing test suites.

Closes #25331 from dilipbiswal/double_infinity.

Authored-by: Dilip Biswal <dbiswal@us.ibm.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 16:48:30 -07:00
Dongjoon Hyun f1d6b19de5 [SPARK-28720][BUILD][R] Update AppVeyor R version to 3.6.1
## What changes were proposed in this pull request?

R version 3.6.1 (Action of the Toes) was released on 2019-07-05. This PR aims to upgrade R installation for AppVeyor CI environment.

## How was this patch tested?

Pass the AppVeyor CI.

Closes #25441 from dongjoon-hyun/SPARK-28720.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-08-13 22:56:53 +00:00
DB Tsai 601fd45814 [SPARK-28719][BUILD] Enable Github Actions for master
## What changes were proposed in this pull request?

Github now provides free CI/CD for build, test, and deploy. This PR enables a simple Github Actions to build master with JDK8 with latest Ubuntu. We can extend it with different versions of JDK, and even build Spark with docker images in the future.

Closes #25440 from dbtsai/actions.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2019-08-13 22:55:02 +00:00
Maxim Gekk 3d85c54895 [SPARK-28700][SQL] Use DECIMAL type for sec in make_timestamp()
## What changes were proposed in this pull request?

Changed type of `sec` argument in the `make_timestamp()` function from `DOUBLE` to `DECIMAL(8, 6)`. The scale is set to 6 to cover microsecond fractions, and the precision is 2 digits for seconds + 6 digits for microsecond fraction. New type prevents losing precision in some cases, for example:

Before:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 58.000001);
2019-08-12 00:00:58
```

After:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 58.000001);
2019-08-12 00:00:58.000001
```

Also switching to `DECIMAL` fixes rounding `sec` towards "nearest neighbor" unless both neighbors are equidistant, in which case round up. For example:

Before:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 0.1234567);
2019-08-12 00:00:00.123456
```

After:
```sql
spark-sql> select make_timestamp(2019, 8, 12, 0, 0, 0.1234567);
2019-08-12 00:00:00.123457
```

## How was this patch tested?

This was tested by `DateExpressionsSuite` and `pgSQL/timestamp.sql`.

Closes #25421 from MaxGekk/make_timestamp-decimal.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 15:51:50 -07:00
Maxim Gekk f04a766946 [SPARK-28718][SQL] Support field synonyms at extract
## What changes were proposed in this pull request?

In the PR, I propose additional synonyms for the `field` argument of `extract` supported by PostgreSQL. The `extract.sql` is updated to check all supported values of the `field` argument. The list of synonyms was taken from https://github.com/postgres/postgres/blob/master/src/backend/utils/adt/datetime.c .

## How was this patch tested?

By running `extract.sql` via:
```
$ build/sbt "sql/test-only *SQLQueryTestSuite -- -z extract.sql"
```

Closes #25438 from MaxGekk/extract-field-synonyms.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 15:36:28 -07:00
Yuming Wang 13b62f31cd [SPARK-28708][SQL] IsolatedClientLoader will not load hive classes from application jars on JDK9+
## What changes were proposed in this pull request?

We have 8 test cases in `HiveSparkSubmitSuite` still fail with `java.lang.ClassNotFoundException` when running on JDK9+:
```
[info] - SPARK-18989: DESC TABLE should not fail with format class not found *** FAILED *** (9 seconds, 927 milliseconds)
[info]   spark-submit returned with exit code 1.
[info]   Command line: './bin/spark-submit' '--class' 'org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE' '--name' 'SPARK-18947' '--master' 'local-cluster[2,1,1024]' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' '--jars' '/root/.m2/repository/org/apache/hive/hive-contrib/2.3.6-SNAPSHOT/hive-contrib-2.3.6-SNAPSHOT.jar' 'file:/root/opensource/spark/target/tmp/spark-36d27542-7b82-4962-a362-bb51ef3e457d/testJar-1565682620744.jar'
[info]
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: An illegal reflective access operation has occurred
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/root/opensource/spark/common/unsafe/target/scala-2.12/classes/) to constructor java.nio.DirectByteBuffer(long,int)
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
[info]   2019-08-13 00:50:22.073 - stderr> WARNING: All illegal access operations will be denied in a future release
[info]   2019-08-13 00:50:28.31 - stderr> Exception in thread "main" java.lang.NoClassDefFoundError: org/apache/hadoop/hive/ql/metadata/HiveException
[info]   2019-08-13 00:50:28.31 - stderr> 	at java.base/java.lang.Class.getDeclaredConstructors0(Native Method)
[info]   2019-08-13 00:50:28.31 - stderr> 	at java.base/java.lang.Class.privateGetDeclaredConstructors(Class.java:3138)
[info]   2019-08-13 00:50:28.31 - stderr> 	at java.base/java.lang.Class.getConstructors(Class.java:1944)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:294)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:410)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:305)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:68)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:67)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$databaseExists$1(HiveExternalCatalog.scala:221)
[info]   2019-08-13 00:50:28.31 - stderr> 	at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:221)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:139)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:129)
[info]   2019-08-13 00:50:28.31 - stderr> 	at org.apache.spark.sql.hive.HiveSessionStateBuilder.externalCatalog(HiveSessionStateBuilder.scala:42)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.HiveSessionStateBuilder.$anonfun$catalog$1(HiveSessionStateBuilder.scala:57)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog$lzycompute(SessionCatalog.scala:91)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.externalCatalog(SessionCatalog.scala:91)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.databaseExists(SessionCatalog.scala:244)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.requireDbExists(SessionCatalog.scala:178)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.catalyst.catalog.SessionCatalog.createTable(SessionCatalog.scala:317)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.CreateTableCommand.run(tables.scala:132)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:70)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:68)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:79)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:213)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3431)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$4(SQLExecution.scala:100)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:160)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:87)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3427)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:213)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:95)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:653)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE$.main(HiveSparkSubmitSuite.scala:829)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE.main(HiveSparkSubmitSuite.scala)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/jdk.internal.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.lang.reflect.Method.invoke(Method.java:566)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:920)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:179)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:202)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:89)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:999)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1008)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
[info]   2019-08-13 00:50:28.311 - stderr> Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.ql.metadata.HiveException
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.net.URLClassLoader.findClass(URLClassLoader.java:471)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:588)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.doLoadClass(IsolatedClientLoader.scala:250)
[info]   2019-08-13 00:50:28.311 - stderr> 	at org.apache.spark.sql.hive.client.IsolatedClientLoader$$anon$1.loadClass(IsolatedClientLoader.scala:239)
[info]   2019-08-13 00:50:28.311 - stderr> 	at java.base/java.lang.ClassLoader.loadClass(ClassLoader.java:521)
[info]   2019-08-13 00:50:28.311 - stderr> 	... 48 more
```

Note that this pr fixes `java.lang.ClassNotFoundException`, but the test will fail again with a different reason, the Hive-side `java.lang.ClassCastException` which will be resolved in the official Hive 2.3.6 release.
```
[info] - SPARK-18989: DESC TABLE should not fail with format class not found *** FAILED *** (7 seconds, 649 milliseconds)
[info]   spark-submit returned with exit code 1.
[info]   Command line: './bin/spark-submit' '--class' 'org.apache.spark.sql.hive.SPARK_18989_CREATE_TABLE' '--name' 'SPARK-18947' '--master' 'local-cluster[2,1,1024]' '--conf' 'spark.ui.enabled=false' '--conf' 'spark.master.rest.enabled=false' '--jars' '/Users/dongjoon/.ivy2/cache/org.apache.hive/hive-contrib/jars/hive-contrib-2.3.5.jar' 'file:/Users/dongjoon/PRS/PR-25429/target/tmp/spark-48b7c936-0ec2-4311-9fb5-0de4bf86a0eb/testJar-1565710418275.jar'
[info]
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: An illegal reflective access operation has occurred
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dongjoon/PRS/PR-25429/common/unsafe/target/scala-2.12/classes/) to constructor java.nio.DirectByteBuffer(long,int)
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: Please consider reporting this to the maintainers of org.apache.spark.unsafe.Platform
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations
[info]   2019-08-13 08:33:39.221 - stderr> WARNING: All illegal access operations will be denied in a future release
[info]   2019-08-13 08:33:43.59 - stderr> Exception in thread "main" org.apache.spark.sql.AnalysisException: java.lang.ClassCastException: class jdk.internal.loader.ClassLoaders$AppClassLoader cannot be cast to class java.net.URLClassLoader (jdk.internal.loader.ClassLoaders$AppClassLoader and java.net.URLClassLoader are in module java.base of loader 'bootstrap');
[info]   2019-08-13 08:33:43.59 - stderr> 	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:109)
```

## How was this patch tested?

manual tests:

1. Install [Hive 2.3.6-SNAPSHOT](https://github.com/wangyum/hive/tree/HIVE-21584-branch-2.3) to local maven repository:
```
mvn clean install -DskipTests=true
```
2. Upgrade our built-in Hive to 2.3.6-SNAPSHOT, you can checkout [this branch](https://github.com/wangyum/spark/tree/SPARK-28708-Hive-2.3.6) to test.
3. Test with hadoop-3.2:
```
build/sbt "hive/test-only *. HiveSparkSubmitSuite" -Phive -Phadoop-3.2 -Phive-thriftserver
...
[info] Run completed in 3 minutes, 8 seconds.
[info] Total number of tests run: 11
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 11, failed 0, canceled 3, ignored 0, pending 0
[info] All tests passed.
```

Closes #25429 from wangyum/SPARK-28708.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 11:21:19 -07:00
Fokko Driesprong d8dd5719b4 [SPARK-28713][BUILD] Bump checkstyle from 8.14 to 8.23
## What changes were proposed in this pull request?

Fixes a vulnerability from the GitHub Security Advisory Database:

_Moderate severity vulnerability that affects com.puppycrawl.tools:checkstyle_
Checkstyle prior to 8.18 loads external DTDs by default, which can potentially lead to denial of service attacks or the leaking of confidential information.

https://github.com/checkstyle/checkstyle/issues/6474

Affected versions: < 8.18

## How was this patch tested?

Ran checkstyle locally.

Closes #25432 from Fokko/SPARK-28713.

Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 11:09:14 -07:00
Yuming Wang c81da276ba [SPARK-28714][SQL][TEST] Add hive.aux.jars.path test for spark-sql shell
## What changes were proposed in this pull request?

`Utilities.addToClassPath` has been changed since [HIVE-22096](https://issues.apache.org/jira/browse/HIVE-22096), but we use it to add plugin jars:
128ea37bda/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala (L144-L147)

This PR add test for `spark-sql` adding plugin jars.

## How was this patch tested?

N/A

Closes #25435 from wangyum/SPARK-28714.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-13 09:19:58 -07:00
Liang-Chi Hsieh e6a0385289 [SPARK-28422][SQL][PYTHON] GROUPED_AGG pandas_udf should work without group by clause
## What changes were proposed in this pull request?

A GROUPED_AGG pandas python udf can't work, if without group by clause, like `select udf(id) from table`.

This doesn't match with aggregate function like sum, count..., and also dataset API like `df.agg(udf(df['id']))`.

When we parse a udf (or an aggregate function) like that from SQL syntax, it is known as a function in a project. `GlobalAggregates` rule in analysis makes such project as aggregate, by looking for aggregate expressions. At the moment, we should also look for GROUPED_AGG pandas python udf.

## How was this patch tested?

Added tests.

Closes #25352 from viirya/SPARK-28422.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-14 00:32:33 +09:00
Gengliang Wang 48adc91057 [SPARK-28698][SQL] Support user-specified output schema in to_avro
## What changes were proposed in this pull request?

The mapping of Spark schema to Avro schema is many-to-many. (See https://spark.apache.org/docs/latest/sql-data-sources-avro.html#supported-types-for-spark-sql---avro-conversion)
The default schema mapping might not be exactly what users want. For example, by default, a "string" column is always written as "string" Avro type, but users might want to output the column as "enum" Avro type.
With PR https://github.com/apache/spark/pull/21847, Spark supports user-specified schema in the batch writer.
For the function `to_avro`, we should support user-specified output schema as well.

## How was this patch tested?

Unit test.

Closes #25419 from gengliangwang/to_avro.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-13 20:52:16 +08:00
Xingbo Jiang 3249c7ab49 [SPARK-28706][SQL] Allow cast null type to any types
## What changes were proposed in this pull request?

#25242 proposed to disallow upcasting complex data types to string type, however, upcasting from null type to any types should still be safe.

## How was this patch tested?

Add corresponding case in `CastSuite`.

Closes #25425 from jiangxb1987/nullToString.

Authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-13 19:02:04 +08:00
Kousuke Saruta 247bebcf94 [SPARK-28561][WEBUI] DAG viz for barrier-execution mode
## What changes were proposed in this pull request?

In the current UI, we cannot identify which RDDs are barrier. Visualizing it will make easy to debug.
Following images are shown after this change.

![Screenshot from 2019-07-30 16-30-35](https://user-images.githubusercontent.com/4736016/62110508-83cec100-b2e9-11e9-83b9-bc2e485a4cbe.png)
![Screenshot from 2019-07-30 16-31-09](https://user-images.githubusercontent.com/4736016/62110509-83cec100-b2e9-11e9-9e2e-47c4dae23a52.png)

The boxes in pale green mean barrier (We might need to discuss which color is proper).

## How was this patch tested?

Tested manually.
The images above are shown by following operations.

```
val rdd1 = sc.parallelize(1 to 10)
val rdd2 = sc.parallelize(1 to 10)
val rdd3 = rdd1.zip(rdd2).barrier.mapPartitions(identity(_))
val rdd4 = rdd3.map(identity(_))
val rdd5 = rdd4.reduceByKey(_+_)
rdd5.collect
```

Closes #25296 from sarutak/barrierexec-dagviz.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-08-12 22:38:10 -07:00
Yuming Wang 9a7f29023e [SPARK-28383][SQL] SHOW CREATE TABLE is not supported on a temporary view
## What changes were proposed in this pull request?
It throws `Table or view not found` when showing  temporary views:
```sql
spark-sql> CREATE TEMPORARY VIEW temp_view AS SELECT 1 AS a;
spark-sql> show create table temp_view;
Error in query: Table or view 'temp_view' not found in database 'default';
```
It's not easy to support temporary views. This pr changed it to throws `SHOW CREATE TABLE is not supported on a temporary view`:
```sql
spark-sql> CREATE TEMPORARY VIEW temp_view AS SELECT 1 AS a;
spark-sql> show create table temp_view;
Error in query: SHOW CREATE TABLE is not supported on a temporary view: temp_view;
```

## How was this patch tested?

unit tests

Closes #25149 from wangyum/SPARK-28383.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 21:01:19 -07:00
Yuming Wang 016e1b491c [SPARK-28703][SQL][TEST] Skip HiveExternalCatalogVersionsSuite and 3 tests in HiveSparkSubmitSuite at JDK9+
## What changes were proposed in this pull request?
This PR skip more test when testing with `JAVA_9` or later:
1. Skip `HiveExternalCatalogVersionsSuite` when testing with `JAVA_9` or later because our previous version does not support `JAVA_9` or later.

2. Skip 3 tests in `HiveSparkSubmitSuite` because the `spark.sql.hive.metastore.version` of these tests is lower than `2.0`, however Datanucleus 3.x seem does not support `JAVA_9` or later. Hive upgrade Datanucleus to 4.x from Hive 2.0([HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113)):

```
[info]   Cause: org.datanucleus.exceptions.NucleusException: The java type java.lang.Long (jdbc-type="", sql-type="") cant be mapped for this datastore. No mapping is available.
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.getDatastoreMappingClass(RDBMSMappingManager.java:1215)
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.createDatastoreMapping(RDBMSMappingManager.java:1378)
[info]   at org.datanucleus.store.rdbms.table.AbstractClassTable.addDatastoreId(AbstractClassTable.java:392)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.initializePK(ClassTable.java:1087)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.preInitialize(ClassTable.java:247)
```

Please note that this exclude only the tests related to the old metastore library, some other tests of `HiveSparkSubmitSuite` still fail on JDK9+.

## How was this patch tested?

manual tests:

Test with JDK 11:
```
[info] HiveExternalCatalogVersionsSuite:
[info] - backward compatibility !!! CANCELED !!! (37 milliseconds)

[info] HiveSparkSubmitSuite:
...
[info] - SPARK-8020: set sql conf in spark conf !!! CANCELED !!! (30 milliseconds)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:130)
...
[info] - SPARK-9757 Persist Parquet relation with decimal column !!! CANCELED !!! (1 millisecond)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:168)
...
[info] - SPARK-16901: set javax.jdo.option.ConnectionURL !!! CANCELED !!! (1 millisecond)
[info]   org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (HiveSparkSubmitSuite.scala:260)
...
```

Closes #25426 from wangyum/SPARK-28703.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 20:42:06 -07:00
Stavros Kontopoulos ec84415358 [SPARK-28280][PYTHON][SQL][TESTS][FOLLOW-UP] Add UDF cases into group by clause in 'udf-group-by.sql'
## What changes were proposed in this pull request?
This PR is a followup of a fix as described in here: https://github.com/apache/spark/pull/25215#issuecomment-517659981

<details><summary>Diff comparing to 'group-by.sql'</summary>
<p>

```diff
diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
index 3a5df254f2..febe47b5ba 100644
--- a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-by.sql.out
 -13,26 +13,26  struct<>

 -- !query 1
-SELECT a, COUNT(b) FROM testData
+SELECT udf(a), udf(COUNT(b)) FROM testData
 -- !query 1 schema
 struct<>
 -- !query 1 output
 org.apache.spark.sql.AnalysisException
-grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(count(testdata.`b`) AS `count(b)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.;
+grouping expressions sequence is empty, and 'testdata.`a`' is not an aggregate function. Wrap '(CAST(udf(cast(count(b) as string)) AS BIGINT) AS `CAST(udf(cast(count(b) as string)) AS BIGINT)`)' in windowing function(s) or wrap 'testdata.`a`' in first() (or first_value) if you don't care which value you get.;

 -- !query 2
-SELECT COUNT(a), COUNT(b) FROM testData
+SELECT COUNT(udf(a)), udf(COUNT(b)) FROM testData
 -- !query 2 schema
-struct<count(a):bigint,count(b):bigint>
+struct<count(CAST(udf(cast(a as string)) AS INT)):bigint,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 2 output
 7	7

 -- !query 3
-SELECT a, COUNT(b) FROM testData GROUP BY a
+SELECT udf(a), COUNT(udf(b)) FROM testData GROUP BY a
 -- !query 3 schema
-struct<a:int,count(b):bigint>
+struct<CAST(udf(cast(a as string)) AS INT):int,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 3 output
 1	2
 2	2
 -41,7 +41,7  NULL	1

 -- !query 4
-SELECT a, COUNT(b) FROM testData GROUP BY b
+SELECT udf(a), udf(COUNT(udf(b))) FROM testData GROUP BY b
 -- !query 4 schema
 struct<>
 -- !query 4 output
 -50,9 +50,9  expression 'testdata.`a`' is neither present in the group by, nor is it an aggre

 -- !query 5
-SELECT COUNT(a), COUNT(b) FROM testData GROUP BY a
+SELECT COUNT(udf(a)), COUNT(udf(b)) FROM testData GROUP BY udf(a)
 -- !query 5 schema
-struct<count(a):bigint,count(b):bigint>
+struct<count(CAST(udf(cast(a as string)) AS INT)):bigint,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 5 output
 0	1
 2	2
 -61,15 +61,15  struct<count(a):bigint,count(b):bigint>

 -- !query 6
-SELECT 'foo', COUNT(a) FROM testData GROUP BY 1
+SELECT 'foo', COUNT(udf(a)) FROM testData GROUP BY 1
 -- !query 6 schema
-struct<foo:string,count(a):bigint>
+struct<foo:string,count(CAST(udf(cast(a as string)) AS INT)):bigint>
 -- !query 6 output
 foo	7

 -- !query 7
-SELECT 'foo' FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo' FROM testData WHERE a = 0 GROUP BY udf(1)
 -- !query 7 schema
 struct<foo:string>
 -- !query 7 output
 -77,25 +77,25  struct<foo:string>

 -- !query 8
-SELECT 'foo', APPROX_COUNT_DISTINCT(a) FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo', udf(APPROX_COUNT_DISTINCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1)
 -- !query 8 schema
-struct<foo:string,approx_count_distinct(a):bigint>
+struct<foo:string,CAST(udf(cast(approx_count_distinct(cast(udf(cast(a as string)) as int), 0.05, 0, 0) as string)) AS BIGINT):bigint>
 -- !query 8 output

 -- !query 9
-SELECT 'foo', MAX(STRUCT(a)) FROM testData WHERE a = 0 GROUP BY 1
+SELECT 'foo', MAX(STRUCT(udf(a))) FROM testData WHERE a = 0 GROUP BY udf(1)
 -- !query 9 schema
-struct<foo:string,max(named_struct(a, a)):struct<a:int>>
+struct<foo:string,max(named_struct(col1, CAST(udf(cast(a as string)) AS INT))):struct<col1:int>>
 -- !query 9 output

 -- !query 10
-SELECT a + b, COUNT(b) FROM testData GROUP BY a + b
+SELECT udf(a + b), udf(COUNT(b)) FROM testData GROUP BY a + b
 -- !query 10 schema
-struct<(a + b):int,count(b):bigint>
+struct<CAST(udf(cast((a + b) as string)) AS INT):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 10 output
 2	1
 3	2
 -105,7 +105,7  NULL	1

 -- !query 11
-SELECT a + 2, COUNT(b) FROM testData GROUP BY a + 1
+SELECT udf(a + 2), udf(COUNT(b)) FROM testData GROUP BY a + 1
 -- !query 11 schema
 struct<>
 -- !query 11 output
 -114,9 +114,9  expression 'testdata.`a`' is neither present in the group by, nor is it an aggre

 -- !query 12
-SELECT a + 1 + 1, COUNT(b) FROM testData GROUP BY a + 1
+SELECT udf(a + 1) + 1, udf(COUNT(b)) FROM testData GROUP BY udf(a + 1)
 -- !query 12 schema
-struct<((a + 1) + 1):int,count(b):bigint>
+struct<(CAST(udf(cast((a + 1) as string)) AS INT) + 1):int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 12 output
 3	2
 4	2
 -125,26 +125,26  NULL	1

 -- !query 13
-SELECT SKEWNESS(a), KURTOSIS(a), MIN(a), MAX(a), AVG(a), VARIANCE(a), STDDEV(a), SUM(a), COUNT(a)
+SELECT SKEWNESS(udf(a)), udf(KURTOSIS(a)), udf(MIN(a)), MAX(udf(a)), udf(AVG(udf(a))), udf(VARIANCE(a)), STDDEV(udf(a)), udf(SUM(a)), udf(COUNT(a))
 FROM testData
 -- !query 13 schema
-struct<skewness(CAST(a AS DOUBLE)):double,kurtosis(CAST(a AS DOUBLE)):double,min(a):int,max(a):int,avg(a):double,var_samp(CAST(a AS DOUBLE)):double,stddev_samp(CAST(a AS DOUBLE)):double,sum(a):bigint,count(a):bigint>
+struct<skewness(CAST(CAST(udf(cast(a as string)) AS INT) AS DOUBLE)):double,CAST(udf(cast(kurtosis(cast(a as double)) as string)) AS DOUBLE):double,CAST(udf(cast(min(a) as string)) AS INT):int,max(CAST(udf(cast(a as string)) AS INT)):int,CAST(udf(cast(avg(cast(cast(udf(cast(a as string)) as int) as bigint)) as string)) AS DOUBLE):double,CAST(udf(cast(var_samp(cast(a as double)) as string)) AS DOUBLE):double,stddev_samp(CAST(CAST(udf(cast(a as string)) AS INT) AS DOUBLE)):double,CAST(udf(cast(sum(cast(a as bigint)) as string)) AS BIGINT):bigint,CAST(udf(cast(count(a) as string)) AS BIGINT):bigint>
 -- !query 13 output
 -0.2723801058145729	-1.5069204152249134	1	3	2.142857142857143	0.8095238095238094	0.8997354108424372	15	7

 -- !query 14
-SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a
+SELECT COUNT(DISTINCT udf(b)), udf(COUNT(DISTINCT b, c)) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY udf(a)
 -- !query 14 schema
-struct<count(DISTINCT b):bigint,count(DISTINCT b, c):bigint>
+struct<count(DISTINCT CAST(udf(cast(b as string)) AS INT)):bigint,CAST(udf(cast(count(distinct b, c) as string)) AS BIGINT):bigint>
 -- !query 14 output
 1	1

 -- !query 15
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k
+SELECT udf(a) AS k, COUNT(udf(b)) FROM testData GROUP BY k
 -- !query 15 schema
-struct<k:int,count(b):bigint>
+struct<k:int,count(CAST(udf(cast(b as string)) AS INT)):bigint>
 -- !query 15 output
 1	2
 2	2
 -153,21 +153,21  NULL	1

 -- !query 16
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k HAVING k > 1
+SELECT a AS k, udf(COUNT(b)) FROM testData GROUP BY k HAVING k > 1
 -- !query 16 schema
-struct<k:int,count(b):bigint>
+struct<k:int,CAST(udf(cast(count(b) as string)) AS BIGINT):bigint>
 -- !query 16 output
 2	2
 3	2

 -- !query 17
-SELECT COUNT(b) AS k FROM testData GROUP BY k
+SELECT udf(COUNT(b)) AS k FROM testData GROUP BY k
 -- !query 17 schema
 struct<>
 -- !query 17 output
 org.apache.spark.sql.AnalysisException
-aggregate functions are not allowed in GROUP BY, but found count(testdata.`b`);
+aggregate functions are not allowed in GROUP BY, but found CAST(udf(cast(count(b) as string)) AS BIGINT);

 -- !query 18
 -180,7 +180,7  struct<>

 -- !query 19
-SELECT k AS a, COUNT(v) FROM testDataHasSameNameWithAlias GROUP BY a
+SELECT k AS a, udf(COUNT(udf(v))) FROM testDataHasSameNameWithAlias GROUP BY udf(a)
 -- !query 19 schema
 struct<>
 -- !query 19 output
 -197,32 +197,32  spark.sql.groupByAliases	false

 -- !query 21
-SELECT a AS k, COUNT(b) FROM testData GROUP BY k
+SELECT a AS k, udf(COUNT(udf(b))) FROM testData GROUP BY k
 -- !query 21 schema
 struct<>
 -- !query 21 output
 org.apache.spark.sql.AnalysisException
-cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 47
+cannot resolve '`k`' given input columns: [testdata.a, testdata.b]; line 1 pos 57

 -- !query 22
-SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a
+SELECT udf(a), COUNT(udf(1)) FROM testData WHERE false GROUP BY udf(a)
 -- !query 22 schema
-struct<a:int,count(1):bigint>
+struct<CAST(udf(cast(a as string)) AS INT):int,count(CAST(udf(cast(1 as string)) AS INT)):bigint>
 -- !query 22 output

 -- !query 23
-SELECT COUNT(1) FROM testData WHERE false
+SELECT udf(COUNT(1)) FROM testData WHERE false
 -- !query 23 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 23 output
 0

 -- !query 24
-SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t
+SELECT 1 FROM (SELECT udf(COUNT(1)) FROM testData WHERE false) t
 -- !query 24 schema
 struct<1:int>
 -- !query 24 output
 -232,7 +232,7  struct<1:int>
 -- !query 25
 SELECT 1 from (
   SELECT 1 AS z,
-  MIN(a.x)
+  udf(MIN(a.x))
   FROM (select 1 as x) a
   WHERE false
 ) b
 -244,32 +244,32  struct<1:int>

 -- !query 26
-SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*)
+SELECT corr(DISTINCT x, y), udf(corr(DISTINCT y, x)), count(*)
   FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y)
 -- !query 26 schema
-struct<corr(DISTINCT CAST(x AS DOUBLE), CAST(y AS DOUBLE)):double,corr(DISTINCT CAST(y AS DOUBLE), CAST(x AS DOUBLE)):double,count(1):bigint>
+struct<corr(DISTINCT CAST(x AS DOUBLE), CAST(y AS DOUBLE)):double,CAST(udf(cast(corr(distinct cast(y as double), cast(x as double)) as string)) AS DOUBLE):double,count(1):bigint>
 -- !query 26 output
 1.0	1.0	3

 -- !query 27
-SELECT 1 FROM range(10) HAVING true
+SELECT udf(1) FROM range(10) HAVING true
 -- !query 27 schema
-struct<1:int>
+struct<CAST(udf(cast(1 as string)) AS INT):int>
 -- !query 27 output
 1

 -- !query 28
-SELECT 1 FROM range(10) HAVING MAX(id) > 0
+SELECT udf(udf(1)) FROM range(10) HAVING MAX(id) > 0
 -- !query 28 schema
-struct<1:int>
+struct<CAST(udf(cast(cast(udf(cast(1 as string)) as int) as string)) AS INT):int>
 -- !query 28 output
 1

 -- !query 29
-SELECT id FROM range(10) HAVING id > 0
+SELECT udf(id) FROM range(10) HAVING id > 0
 -- !query 29 schema
 struct<>
 -- !query 29 output
 -291,33 +291,33  struct<>

 -- !query 31
-SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0
+SELECT udf(every(v)), udf(some(v)), any(v) FROM test_agg WHERE 1 = 0
 -- !query 31 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(every(v) as string)) AS BOOLEAN):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 31 output
 NULL	NULL	NULL

 -- !query 32
-SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4
+SELECT udf(every(udf(v))), some(v), any(v) FROM test_agg WHERE k = 4
 -- !query 32 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(every(cast(udf(cast(v as string)) as boolean)) as string)) AS BOOLEAN):boolean,some(v):boolean,any(v):boolean>
 -- !query 32 output
 NULL	NULL	NULL

 -- !query 33
-SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5
+SELECT every(v), udf(some(v)), any(v) FROM test_agg WHERE k = 5
 -- !query 33 schema
-struct<every(v):boolean,some(v):boolean,any(v):boolean>
+struct<every(v):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 33 output
 false	true	true

 -- !query 34
-SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k
+SELECT udf(k), every(v), udf(some(v)), any(v) FROM test_agg GROUP BY udf(k)
 -- !query 34 schema
-struct<k:int,every(v):boolean,some(v):boolean,any(v):boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,every(v):boolean,CAST(udf(cast(some(v) as string)) AS BOOLEAN):boolean,any(v):boolean>
 -- !query 34 output
 1	false	true	true
 2	true	true	true
 -327,9 +327,9  struct<k:int,every(v):boolean,some(v):boolean,any(v):boolean>

 -- !query 35
-SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false
+SELECT udf(k), every(v) FROM test_agg GROUP BY k HAVING every(v) = false
 -- !query 35 schema
-struct<k:int,every(v):boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,every(v):boolean>
 -- !query 35 output
 1	false
 3	false
 -337,77 +337,77  struct<k:int,every(v):boolean>

 -- !query 36
-SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL
+SELECT udf(k), udf(every(v)) FROM test_agg GROUP BY udf(k) HAVING every(v) IS NULL
 -- !query 36 schema
-struct<k:int,every(v):boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,CAST(udf(cast(every(v) as string)) AS BOOLEAN):boolean>
 -- !query 36 output
 4	NULL

 -- !query 37
-SELECT k,
-       Every(v) AS every
+SELECT udf(k),
+       udf(Every(v)) AS every
 FROM   test_agg
 WHERE  k = 2
        AND v IN (SELECT Any(v)
                  FROM   test_agg
                  WHERE  k = 1)
-GROUP  BY k
+GROUP  BY udf(k)
 -- !query 37 schema
-struct<k:int,every:boolean>
+struct<CAST(udf(cast(k as string)) AS INT):int,every:boolean>
 -- !query 37 output
 2	true

 -- !query 38
-SELECT k,
+SELECT udf(udf(k)),
        Every(v) AS every
 FROM   test_agg
 WHERE  k = 2
        AND v IN (SELECT Every(v)
                  FROM   test_agg
                  WHERE  k = 1)
-GROUP  BY k
+GROUP  BY udf(udf(k))
 -- !query 38 schema
-struct<k:int,every:boolean>
+struct<CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int,every:boolean>
 -- !query 38 output

 -- !query 39
-SELECT every(1)
+SELECT every(udf(1))
 -- !query 39 schema
 struct<>
 -- !query 39 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7
+cannot resolve 'every(CAST(udf(cast(1 as string)) AS INT))' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7

 -- !query 40
-SELECT some(1S)
+SELECT some(udf(1S))
 -- !query 40 schema
 struct<>
 -- !query 40 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7
+cannot resolve 'some(CAST(udf(cast(1 as string)) AS SMALLINT))' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7

 -- !query 41
-SELECT any(1L)
+SELECT any(udf(1L))
 -- !query 41 schema
 struct<>
 -- !query 41 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7
+cannot resolve 'any(CAST(udf(cast(1 as string)) AS BIGINT))' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7

 -- !query 42
-SELECT every("true")
+SELECT udf(every("true"))
 -- !query 42 schema
 struct<>
 -- !query 42 output
 org.apache.spark.sql.AnalysisException
-cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7
+cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 11

 -- !query 43
 -428,9 +428,9  struct<k:int,v:boolean,every(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST

 -- !query 44
-SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
+SELECT k, udf(udf(v)), some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
 -- !query 44 schema
-struct<k:int,v:boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
+struct<k:int,CAST(udf(cast(cast(udf(cast(v as string)) as boolean) as string)) AS BOOLEAN):boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
 -- !query 44 output
 1	false	false
 1	true	true
 -445,9 +445,9  struct<k:int,v:boolean,some(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST R

 -- !query 45
-SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
+SELECT udf(udf(k)), v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg
 -- !query 45 schema
-struct<k:int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
+struct<CAST(udf(cast(cast(udf(cast(k as string)) as int) as string)) AS INT):int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW):boolean>
 -- !query 45 output
 1	false	false
 1	true	true
 -462,17 +462,17  struct<k:int,v:boolean,any(v) OVER (PARTITION BY k ORDER BY v ASC NULLS FIRST RA

 -- !query 46
-SELECT count(*) FROM test_agg HAVING count(*) > 1L
+SELECT udf(count(*)) FROM test_agg HAVING count(*) > 1L
 -- !query 46 schema
-struct<count(1):bigint>
+struct<CAST(udf(cast(count(1) as string)) AS BIGINT):bigint>
 -- !query 46 output
 10

 -- !query 47
-SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true
+SELECT k, udf(max(v)) FROM test_agg GROUP BY k HAVING max(v) = true
 -- !query 47 schema
-struct<k:int,max(v):boolean>
+struct<k:int,CAST(udf(cast(max(v) as string)) AS BOOLEAN):boolean>
 -- !query 47 output
 1	true
 2	true
 -480,7 +480,7  struct<k:int,max(v):boolean>

 -- !query 48
-SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L
+SELECT * FROM (SELECT udf(COUNT(*)) AS cnt FROM test_agg) WHERE cnt > 1L
 -- !query 48 schema
 struct<cnt:bigint>
 -- !query 48 output
 -488,7 +488,7  struct<cnt:bigint>

 -- !query 49
-SELECT count(*) FROM test_agg WHERE count(*) > 1L
+SELECT udf(count(*)) FROM test_agg WHERE count(*) > 1L
 -- !query 49 schema
 struct<>
 -- !query 49 output
 -500,7 +500,7  Invalid expressions: [count(1)];

 -- !query 50
-SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L
+SELECT udf(count(*)) FROM test_agg WHERE count(*) + 1L > 1L
 -- !query 50 schema
 struct<>
 -- !query 50 output
 -512,7 +512,7  Invalid expressions: [count(1)];

 -- !query 51
-SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1
+SELECT udf(count(*)) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1
 -- !query 51 schema
 struct<>
 -- !query 51 output

```

</p>
</details>

## How was this patch tested?
Tested as instructed in SPARK-27921.

Closes #25360 from skonto/group-by-followup.

Authored-by: Stavros Kontopoulos <st.kontopoulos@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-13 10:06:32 +09:00
Kousuke Saruta 25857c6559 [SPARK-28647][WEBUI] Recover additional metric feature and remove additional-metrics.js
## What changes were proposed in this pull request?

By SPARK-17019, `On Heap Memory` and `Off Heap Memory` are introduced as optional metrics.
But they are not displayed because they are made `display: none` in css and there are no way to appear them.

I know #22595 also try to resolve this issue but that will use `additional-metrics.js`.
Initially, `additional-metrics.js` is created for `StagePage` but `StagePage` currently uses `stagepage.js` for its additional metrics to be toggle because `DataTable (one of jQuery plugins)` was introduced and we needed another mechanism to add/remove columns for additional metrics.

Now that `ExecutorsPage` also uses `DataTable` so it might be better to introduce same mechanism as `StagePage` for additional metrics.

![Screenshot from 2019-08-10 05-37-25](https://user-images.githubusercontent.com/4736016/62807960-c4240f80-bb31-11e9-8e1a-1a44e2f91597.png)

And then, we can remove `additional-metrics.js` which is no longer used from anywhere.

## How was this patch tested?

After this change is applied, I confirmed `ExecutorsPage` and `StagePage` are properly rendered and all checkboxes for additional metrics work.

Closes #25374 from sarutak/remove-additional-metrics.js.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 17:02:28 -07:00
s71955 163f4a45df [SPARK-26969][SQL] Using ODBC client not able to see the query data when column datatype is decimal
## What changes were proposed in this pull request?
While processing the Rowdata in the server side ColumnValue BigDecimal type value processed by server has to converted to the HiveDecmal data type for successful processing of query using Hive ODBC client.As per current logic  corresponding to the Decimal column datatype, the Spark server uses BigDecimal, and the ODBC client uses HiveDecimal. If the data type does not match, the client fail to parse

Since this handing was missing the query executed in Hive ODBC client wont return or provides result to the user even though the decimal type column value data present.

## How was this patch tested?

Manual test report and impact assessment is done using existing test-cases

Before fix
![decimal_odbc](https://user-images.githubusercontent.com/12999161/53440179-e74a7f00-3a29-11e9-93db-83f2ae37ef16.PNG)

After Fix
![hive_odbc](https://user-images.githubusercontent.com/12999161/53679519-70e0a200-3cf3-11e9-9437-9c27d2e5056d.PNG)

Closes #23899 from sujith71955/master_decimalissue.

Authored-by: s71955 <sujithchacko.2010@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 15:47:59 -07:00
Gengliang Wang 48d04f74ca [SPARK-28638][WEBUI] Task summary should only contain successful tasks' metrics
## What changes were proposed in this pull request?

Currently, on requesting summary metrics, cached data are returned if the current number of "SUCCESS" tasks is the same as the value in cached data.
However, the number of "SUCCESS" tasks is wrong when there are running tasks. In `AppStatusStore`, the KVStore is `ElementTrackingStore`, instead of `InMemoryStore`. The value count is always the number of "SUCCESS" tasks + "RUNNING" tasks.
Thus, even when the running tasks are finished, the out-of-update cached data is returned.

This PR is to fix the code in getting the number of "SUCCESS" tasks.

## How was this patch tested?

Test manually, run
```
sc.parallelize(1 to 160, 40).map(i => Thread.sleep(i*100)).collect()
```
and keep refreshing the stage page , we can see the task summary metrics is wrong.

### Before fix:
![image](https://user-images.githubusercontent.com/1097932/62560343-6a141780-b8af-11e9-8942-d88540659a93.png)

### After fix:
![image](https://user-images.githubusercontent.com/1097932/62560355-7009f880-b8af-11e9-8ba8-10c083a48d7b.png)

Closes #25369 from gengliangwang/fixStagePage.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-08-12 11:47:29 -07:00
Maxim Gekk 6964128e25 [SPARK-28017][SPARK-28656][SQL][FOLLOW-UP] Restore comments in date.sql
## What changes were proposed in this pull request?

Restored comments in `date.sql` removed by 924d794a6f and 997d153e54 . The comments was introduced by 51379b731d .

## How was this patch tested?

By re-running `date.sql` via:
```shell
$ build/sbt "sql/test-only *SQLQueryTestSuite -- -z date.sql"
```

Closes #25422 from MaxGekk/sql-comments-followup.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 11:19:19 -07:00
zhengruifeng ae4edd5489 [SPARK-28538][UI] Document SQL page
## What changes were proposed in this pull request?
1, add basic doc for each page;
2, doc SQL page with an exmple;

## How was this patch tested?
locally built

![图片](https://user-images.githubusercontent.com/7322292/62421626-86f5f280-b6d7-11e9-8057-8be3a4afb611.png)

![图片](https://user-images.githubusercontent.com/7322292/62421634-9d9c4980-b6d7-11e9-8e31-1e6ba9b402e8.png)

Closes #25349 from zhengruifeng/doc_ui_sql.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-12 08:36:01 -05:00
WeichenXu 0f2efe6825 [SPARK-28366][CORE][FOLLOW-UP] Refine logging in driver when loading single large unsplittable file
## What changes were proposed in this pull request?

* Add log in `NewHadoopRDD`
* Remove some words in logs which related to specific user API.

## How was this patch tested?

Manual.

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

Closes #25391 from WeichenXu123/log_sf.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-12 19:15:00 +08:00
Yuming Wang e5f4a106db [SPARK-28688][SQL][TEST] Skip VersionsSuite.read hive materialized view test for HMS 3.0+ on JDK9+
## What changes were proposed in this pull request?

This PR makes it skip test `read hive materialized view` since Hive 3.0 in `VersionsSuite.scala` on JDK 11 because [HIVE-19383](https://issues.apache.org/jira/browse/HIVE-19383) added [ArrayList$SubList](ae4df62795/ql/src/java/org/apache/hadoop/hive/ql/exec/SerializationUtilities.java (L383)) which is incompatible with JDK 11:
```java
java.lang.RuntimeException: java.lang.NoSuchFieldException: parentOffset
	at org.apache.hadoop.hive.ql.exec.SerializationUtilities$ArrayListSubListSerializer.<init>(SerializationUtilities.java:389)
	at org.apache.hadoop.hive.ql.exec.SerializationUtilities$1.create(SerializationUtilities.java:235)
...
```
![image](https://issues.apache.org/jira/secure/attachment/12977250/12977250_screenshot-2.png)
![image](https://issues.apache.org/jira/secure/attachment/12977249/12977249_screenshot-1.png)

## How was this patch tested?

manual tests
**Test on JDK 11**:
```
...
[info] - 2.3: sql read hive materialized view (1 second, 253 milliseconds)
...
[info] - 3.0: sql read hive materialized view !!! CANCELED !!! (31 milliseconds)
[info]   "[3.0]" did not equal "[2.3]", and org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (VersionsSuite.scala:624)
...
[info] - 3.1: sql read hive materialized view !!! CANCELED !!! (0 milliseconds)
[info]   "[3.1]" did not equal "[2.3]", and org.apache.commons.lang3.SystemUtils.isJavaVersionAtLeast(JAVA_9) was true (VersionsSuite.scala:624)
...
```

**Test on JDK 1.8**:
```
...
[info] - 2.3: sql read hive materialized view (1 second, 444 milliseconds)
...
[info] - 3.0: sql read hive materialized view (3 seconds, 100 milliseconds)
...
[info] - 3.1: sql read hive materialized view (2 seconds, 941 milliseconds)
...
```

Closes #25414 from wangyum/SPARK-28688.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 03:37:10 -07:00
Yuming Wang 6c06eea411 [SPARK-28686][SQL][TEST] Move udf_radians from HiveCompatibilitySuite to HiveQuerySuite
## What changes were proposed in this pull request?

This PR moves `udf_radians` from `HiveCompatibilitySuite` to `HiveQuerySuite` to make it easy to test with JDK 11 because it returns different value from JDK 9:
```java
public class TestRadians {
  public static void main(String[] args) {
    System.out.println(java.lang.Math.toRadians(57.2958));
  }
}
```
```sh
[rootspark-3267648 ~]# javac TestRadians.java
[rootspark-3267648 ~]# /usr/lib/jdk-9.0.4+11/bin/java TestRadians
1.0000003575641672
[rootspark-3267648 ~]# /usr/lib/jdk-11.0.3/bin/java TestRadians
1.0000003575641672
[rootspark-3267648 ~]# /usr/lib/jdk8u222-b10/bin/java TestRadians
1.000000357564167
```

## How was this patch tested?

manual tests

Closes #25417 from wangyum/SPARK-28686.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-12 02:24:48 -07:00
Liang-Chi Hsieh 37eedf6149 [SPARK-28652][TESTS][K8S] Add python version check for executor
## What changes were proposed in this pull request?

Current two PySpark version tests in PythonTestsSuite, just test against Python version at driver side. Because the test script doesn't run any spark job requiring python worker, it doesn't actually do version check at worker side. This patch adds pieces of code to the test script, to run a simple job to verify Python version.

## How was this patch tested?

Unit test. Locally manual test.

Closes #25411 from viirya/SPARK-28652.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-12 14:43:32 +09:00
Kousuke Saruta 31ef268bae [SPARK-28639][CORE][DOC] Configuration doc for Barrier Execution Mode
## What changes were proposed in this pull request?

SPARK-24817 and SPARK-24819 introduced new 3 non-internal properties for barrier-execution mode but they are not documented.
So I've added a section into configuration.md for barrier-mode execution.

## How was this patch tested?
Built using jekyll and confirm the layout by browser.

Closes #25370 from sarutak/barrier-exec-mode-conf-doc.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-11 08:13:19 -05:00
Yuming Wang 58cc0df59e [SPARK-28685][SQL][TEST] Test HMS 2.0.0+ in VersionsSuite/HiveClientSuites on JDK 11
## What changes were proposed in this pull request?

It seems Datanucleus 3.x can not support JDK 11:
```java
[info]   Cause: org.datanucleus.exceptions.NucleusException: The java type java.lang.Long (jdbc-type="", sql-type="") cant be mapped for this datastore. No mapping is available.
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.getDatastoreMappingClass(RDBMSMappingManager.java:1215)
[info]   at org.datanucleus.store.rdbms.mapping.RDBMSMappingManager.createDatastoreMapping(RDBMSMappingManager.java:1378)
[info]   at org.datanucleus.store.rdbms.table.AbstractClassTable.addDatastoreId(AbstractClassTable.java:392)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.initializePK(ClassTable.java:1087)
[info]   at org.datanucleus.store.rdbms.table.ClassTable.preInitialize(ClassTable.java:247)
```

Hive upgrade Datanucleus to 4.x from Hive 2.0([HIVE-6113](https://issues.apache.org/jira/browse/HIVE-6113)). This PR makes it skip `0.12`, `0.13`, `0.14`, `1.0`, `1.1` and `1.2` when testing with JDK 11.

Note that, this pr will not fix sql read hive materialized view. It's another issue:
```
3.0: sql read hive materialized view *** FAILED *** (1 second, 521 milliseconds)
3.1: sql read hive materialized view *** FAILED *** (1 second, 536 milliseconds)
```

## How was this patch tested?

manual tests:
```shell
export JAVA_HOME="/usr/lib/jdk-11.0.3"
build/sbt "hive/test-only *.VersionsSuite *.HiveClientSuites" -Phive -Phadoop-3.2
```

Closes #25405 from wangyum/SPARK-28685.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-10 17:01:15 -07:00
Yuming Wang 47af8925b6 [SPARK-28675][SQL] Remove maskCredentials and use redactOptions
## What changes were proposed in this pull request?

This PR replaces `CatalogUtils.maskCredentials` with `SQLConf.get.redactOptions` to match other redacts.

## How was this patch tested?

unit test and manual tests:
Before this PR:
```sql
spark-sql> DESC EXTENDED test_spark_28675;
id	int	NULL

# Detailed Table Information
Database	default
Table	test_spark_28675
Owner	root
Created Time	Fri Aug 09 08:23:17 GMT-07:00 2019
Last Access	Wed Dec 31 17:00:00 GMT-07:00 1969
Created By	Spark 3.0.0-SNAPSHOT
Type	MANAGED
Provider	org.apache.spark.sql.jdbc
Location	file:/user/hive/warehouse/test_spark_28675
Serde Library	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat	org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties	[url=###, driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]

spark-sql> SHOW TABLE EXTENDED LIKE 'test_spark_28675';
default	test_spark_28675	false	Database: default
Table: test_spark_28675
Owner: root
Created Time: Fri Aug 09 08:23:17 GMT-07:00 2019
Last Access: Wed Dec 31 17:00:00 GMT-07:00 1969
Created By: Spark 3.0.0-SNAPSHOT
Type: MANAGED
Provider: org.apache.spark.sql.jdbc
Location: file:/user/hive/warehouse/test_spark_28675
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties: [url=###, driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]
Schema: root
 |-- id: integer (nullable = true)

```

After this PR:
```sql
spark-sql> DESC EXTENDED test_spark_28675;
id	int	NULL

# Detailed Table Information
Database	default
Table	test_spark_28675
Owner	root
Created Time	Fri Aug 09 08:19:49 GMT-07:00 2019
Last Access	Wed Dec 31 17:00:00 GMT-07:00 1969
Created By	Spark 3.0.0-SNAPSHOT
Type	MANAGED
Provider	org.apache.spark.sql.jdbc
Location	file:/user/hive/warehouse/test_spark_28675
Serde Library	org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat	org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat	org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties	[url=*********(redacted), driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]

spark-sql> SHOW TABLE EXTENDED LIKE 'test_spark_28675';
default	test_spark_28675	false	Database: default
Table: test_spark_28675
Owner: root
Created Time: Fri Aug 09 08:19:49 GMT-07:00 2019
Last Access: Wed Dec 31 17:00:00 GMT-07:00 1969
Created By: Spark 3.0.0-SNAPSHOT
Type: MANAGED
Provider: org.apache.spark.sql.jdbc
Location: file:/user/hive/warehouse/test_spark_28675
Serde Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
InputFormat: org.apache.hadoop.mapred.SequenceFileInputFormat
OutputFormat: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
Storage Properties: [url=*********(redacted), driver=com.mysql.jdbc.Driver, dbtable=test_spark_28675]
Schema: root
 |-- id: integer (nullable = true)
```

Closes #25395 from wangyum/SPARK-28675.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-10 16:45:59 -07:00
Kousuke Saruta dd5599efaf [SPARK-28677][WEBUI] "Select All" checkbox in StagePage doesn't work properly
## What changes were proposed in this pull request?

In StagePage, only the first optional column (Scheduler Delay, in this case) appears even though "Select All" checkbox is checked.

![Screenshot from 2019-08-09 18-46-05](https://user-images.githubusercontent.com/4736016/62771600-8f379e80-bad8-11e9-9faa-6da8d57739d2.png)

The cause is that wrong method is used to manipulate multiple columns. columns should have been used but column was used.
I've fixed this issue by replacing the `column` with `columns`.

## How was this patch tested?

Confirmed behavior of the check-box.

![Screenshot from 2019-08-09 18-54-33](https://user-images.githubusercontent.com/4736016/62771614-98c10680-bad8-11e9-9cc0-5879ac47d1e1.png)

Closes #25397 from sarutak/fix-stagepage.js.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-10 16:51:12 -05:00
younggyu chun 8535df7261 [MINOR] Fix typos in comments and replace an explicit type with <>
## What changes were proposed in this pull request?
This PR fixed typos in comments and replace the explicit type with '<>' for Java 8+.

## How was this patch tested?
Manually tested.

Closes #25338 from younggyuchun/younggyu.

Authored-by: younggyu chun <younggyuchun@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-10 16:47:11 -05:00
Ajith ef80c32266 [SPARK-28676][CORE] Avoid Excessive logging from ContextCleaner
## What changes were proposed in this pull request?

In high workload environments, ContextCleaner seems to have excessive logging at INFO level which do not give much information. In one Particular case we see that ``INFO ContextCleaner: Cleaned accumulator`` message is 25-30% of the generated logs. We can log this information for cleanup in DEBUG level instead.

## How was this patch tested?

This do not modify any functionality. This is just changing cleanup log levels to DEBUG for  ContextCleaner

Closes #25396 from ajithme/logss.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-09 15:49:20 -07:00
Maxim Gekk 924d794a6f [SPARK-28656][SQL] Support millennium, century and decade at extract()
## What changes were proposed in this pull request?

In the PR, I propose new expressions `Millennium`, `Century` and `Decade`, and support additional parameters of `extract()` for feature parity with PostgreSQL (https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-EXTRACT):

1. `millennium` - the current millennium for given date (or a timestamp implicitly casted to a date). For example, years in the 1900s are in the second millennium. The third millennium started _January 1, 2001_.
2. `century` - the current millennium for given date (or timestamp). The first century starts at 0001-01-01 AD.
3. `decade` - the current decade for given date (or timestamp). Actually, this is the year field divided by 10.

Here are examples:
```sql
spark-sql> SELECT EXTRACT(MILLENNIUM FROM DATE '1981-01-19');
2
spark-sql> SELECT EXTRACT(CENTURY FROM DATE '1981-01-19');
20
spark-sql> SELECT EXTRACT(DECADE FROM DATE '1981-01-19');
198
```

## How was this patch tested?

Added new tests to `DateExpressionsSuite` and uncommented existing tests in `pgSQL/date.sql`.

Closes #25388 from MaxGekk/extract-ext2.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-09 11:18:50 -07:00
zhengruifeng 8b08e14de7 [SPARK-21481][ML][FOLLOWUP] HashingTF Cleanup
## What changes were proposed in this pull request?
some cleanup and tiny optimization
1, since the `transformImpl` method in the .mllib side is no longer used in the .ml side, the scope should be limited;
2, in the `hashUDF`, val `numOfFeatures` is never used;
3, in the udf, it is inefficient to involve param getter (`$(numFeatures)`/`$(binary)`) directly or via method `indexOf` ((`$(numFeatures)`) . instead, the getter should be called outside of the udf;

## How was this patch tested?
existing suites

Closes #25324 from zhengruifeng/hashingtf_cleanup.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-09 10:04:39 -05:00
wuyi cbad616d4c [SPARK-27371][CORE] Support GPU-aware resources scheduling in Standalone
## What changes were proposed in this pull request?

In this PR, we implements a complete process of GPU-aware resources scheduling
in Standalone. The whole process looks like: Worker sets up isolated resources
when it starts up and registers to master along with its resources. And, Master
picks up usable workers according to driver/executor's resource requirements to
launch driver/executor on them. Then, Worker launches the driver/executor after
preparing resources file, which is created under driver/executor's working directory,
with specified resource addresses(told by master). When driver/executor finished,
their resources could be recycled to worker. Finally, if a worker stops, it
should always release its resources firstly.

For the case of Workers and Drivers in **client** mode run on the same host, we introduce
a config option named `spark.resources.coordinate.enable`(default true) to indicate
whether Spark should coordinate resources for user. If `spark.resources.coordinate.enable=false`, user should be responsible for configuring different resources for Workers and Drivers when use resourcesFile or discovery script. If true, Spark would help user to assign different  resources for Workers and Drivers.

The solution for Spark to coordinate resources among Workers and Drivers is:

Generally, use a shared file named *____allocated_resources____.json* to sync allocated
resources info among Workers and Drivers on the same host.

After a Worker or Driver found all resources using the configured resourcesFile and/or
discovery script during launching, it should filter out available resources by excluding resources already allocated in *____allocated_resources____.json* and acquire resources from available resources according to its own requirement. After that, it should write its allocated resources along with its process id (pid) into *____allocated_resources____.json*.  Pid (proposed by tgravescs) here used to check whether the allocated resources are still valid in case of Worker or Driver crashes and doesn't release resources properly. And when a Worker or Driver finished, normally, it would always clean up its own allocated resources in *____allocated_resources____.json*.

Note that we'll always get a file lock before any access to file *____allocated_resources____.json*
and release the lock finally.

Futhermore, we appended resources info in `WorkerSchedulerStateResponse` to work
around master change behaviour in HA mode.

## How was this patch tested?

Added unit tests in WorkerSuite, MasterSuite, SparkContextSuite.

Manually tested with client/cluster mode (e.g. multiple workers) in a single node Standalone.

Closes #25047 from Ngone51/SPARK-27371.

Authored-by: wuyi <ngone_5451@163.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2019-08-09 07:49:03 -05:00
gengjiaan 5159876415 [SPARK-28077][SQL][TEST][FOLLOW-UP] Enable Overlay function tests
## What changes were proposed in this pull request?

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

## How was this patch tested?

Pass the Jenkins with the newly update test files.

Closes #25393 from beliefer/enable-overlay-tests.

Authored-by: gengjiaan <gengjiaan@360.cn>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-08-09 19:05:41 +09:00
WeichenXu f21bc1874a [SPARK-27889][INFRA] Make development scripts under dev/ support Python 3
## What changes were proposed in this pull request?

I made an audit and update all dev scripts to support python3. (except `merge_spark_pr.py` which already updated)

## How was this patch tested?

Manual.

Closes #25289 from WeichenXu123/dev_py3.

Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-09 18:55:48 +09:00
Shixiong Zhu 5bb69945e4 [SPARK-28651][SS] Force the schema of Streaming file source to be nullable
## What changes were proposed in this pull request?

Right now, batch DataFrame always changes the schema to nullable automatically (See this line: 325bc8e9c6/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala (L399)). But streaming file source is missing this.

This PR updates the streaming file source schema to force it be nullable. I also added a flag `spark.sql.streaming.fileSource.schema.forceNullable` to disable this change since some users may rely on the old behavior.

## How was this patch tested?

The new unit test.

Closes #25382 from zsxwing/SPARK-28651.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-09 18:54:55 +09:00
Gabor Somogyi 5663386f4b [SPARK-28163][SS] Use CaseInsensitiveMap for KafkaOffsetReader
## What changes were proposed in this pull request?

There are "unsafe" conversions in the Kafka connector.
`CaseInsensitiveStringMap` comes in which is then converted the following way:
```
...
options.asScala.toMap
...
```
The main problem with this is that such case it looses its case insensitive nature
(case insensitive map is converting the key to lower case when get/contains called).

In this PR I'm using `CaseInsensitiveMap` to solve this problem.

## How was this patch tested?

Existing + additional unit tests.

Closes #24967 from gaborgsomogyi/SPARK-28163.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-09 17:08:11 +08:00
Burak Yavuz 5368eaa2fc [SPARK-28565][SQL] DataFrameWriter saveAsTable support for V2 catalogs
## What changes were proposed in this pull request?

Adds support for V2 catalogs and the V2SessionCatalog for V2 tables for saveAsTable.
If the table can resolve through the V2SessionCatalog, we use SaveMode for datasource v1 for backwards compatibility to select the code path we're going to hit.

Depending on the SaveMode:
 - SaveMode.Append:
     a) If table exists: Use AppendData.byName
     b) If table doesn't exist, use CTAS (ignoreIfExists = false)
 - SaveMode.Overwrite: Use RTAS (orCreate = true)
 - SaveMode.Ignore: Use CTAS (ignoreIfExists = true)
 - SaveMode.ErrorIfExists: Use CTAS (ignoreIfExists = false)

## How was this patch tested?

Unit tests in DataSourceV2DataFrameSuite

Closes #25330 from brkyvz/saveAsTable.

Lead-authored-by: Burak Yavuz <brkyvz@gmail.com>
Co-authored-by: Burak Yavuz <burak@databricks.com>
Signed-off-by: Burak Yavuz <brkyvz@gmail.com>
2019-08-08 22:30:00 -07:00
Maxim Gekk 997d153e54 [SPARK-28017][SQL] Support additional levels of truncations by DATE_TRUNC/TRUNC
## What changes were proposed in this pull request?

I propose new levels of truncations for the `date_trunc()` and `trunc()` functions:
1. `MICROSECOND` and `MILLISECOND` truncate values of the `TIMESTAMP` type to microsecond and millisecond precision.
2. `DECADE`, `CENTURY` and `MILLENNIUM` truncate dates/timestamps to lowest date of current decade/century/millennium.

Also the `WEEK` and `QUARTER` levels have been supported by the `trunc()` function.

The function is implemented similarly to `date_trunc` in PostgreSQL: https://www.postgresql.org/docs/11/functions-datetime.html#FUNCTIONS-DATETIME-TRUNC to maintain feature parity with it.

Here are examples of `TRUNC`:
```sql
spark-sql> SELECT TRUNC('2015-10-27', 'DECADE');
2010-01-01
spark-sql> set spark.sql.datetime.java8API.enabled=true;
spark.sql.datetime.java8API.enabled	true
spark-sql> SELECT TRUNC('1999-10-27', 'millennium');
1001-01-01
```
Examples of `DATE_TRUNC`:
```sql
spark-sql> SELECT DATE_TRUNC('CENTURY', '2015-03-05T09:32:05.123456');
2001-01-01T00:00:00Z
```

## How was this patch tested?

Added new tests to `DateTimeUtilsSuite`, `DateExpressionsSuite` and `DateFunctionsSuite`, and uncommented existing tests in `pgSQL/date.sql`.

Closes #25336 from MaxGekk/date_truct-ext.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-09 12:29:44 +08:00