Commit graph

552 commits

Author SHA1 Message Date
Kent Yao 93529a8536 [SPARK-31957][SQL] Cleanup hive scratch dir for the developer api startWithContext
### What changes were proposed in this pull request?

Comparing to the long-running ThriftServer via start-script, we are more likely to hit the issue https://issues.apache.org/jira/browse/HIVE-10415 / https://issues.apache.org/jira/browse/SPARK-31626 in the developer API `startWithContext`

This PR apply SPARK-31626 to the developer API `startWithContext`

### Why are the changes needed?

Fix the issue described in  SPARK-31626

### Does this PR introduce _any_ user-facing change?

Yes, the hive scratch dir will be deleted if cleanup is enabled for calling `startWithContext`

### How was this patch tested?

new test

Closes #28784 from yaooqinn/SPARK-31957.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-06-19 19:54:46 -07:00
Kent Yao abc8ccc37b [SPARK-31926][SQL][TESTS][FOLLOWUP][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber
### What changes were proposed in this pull request?

This PR brings https://github.com/apache/spark/pull/28751 back

- It once reverted by 4a25200 because of inevitable maven test failure
    - See related updates in this followup a0187cd6b5

- And reverted again because of the flakiness of the added unit tests
   - In this PR, The flakiness reason found is caused by the hive metastore connection that the SparkSQLCLIService trying to create which turns out is unnecessary at all. This metastore client points to a dummy metastore server only.
   - Also, add some cleanups for SharedThriftServer trait in before and after to prevent its configurations being polluted or polluting others

### Why are the changes needed?

fix flaky test

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

passing sbt and maven tests

Closes #28835 from yaooqinn/SPARK-31926-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-19 05:58:54 +00:00
Max Gekk 350aa859fe [SPARK-32006][SQL] Create date/timestamp formatters once before collect in hiveResultString()
### What changes were proposed in this pull request?
1. Add method `getTimeFormatters` to `HiveResult` which creates timestamp and date formatters.
2. Move creation of `dateFormatter` and `timestampFormatter` from the constructor of the `HiveResult` object to `HiveResult. hiveResultString()` via `getTimeFormatters`. This allows to resolve time zone ID from Spark's session time zone `spark.sql.session.timeZone` and create date/timestamp formatters only once before collecting `java.sql.Timestamp`/`java.sql.Date` values.
3. Create date/timestamp formatters once in SparkExecuteStatementOperation.

### Why are the changes needed?
To fix perf regression comparing to Spark 2.4

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
- By existing test suite `HiveResultSuite` and etc.
- Re-generate benchmarks results of `DateTimeBenchmark` in the environment:

| Item | Description |
| ---- | ----|
| Region | us-west-2 (Oregon) |
| Instance | r3.xlarge |
| AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) |
| Java | OpenJDK 64-Bit Server VM 1.8.0_252 and OpenJDK 64-Bit Server VM 11.0.7+10 |

Closes #28842 from MaxGekk/opt-toHiveString-oss-master.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-17 06:28:47 +00:00
yangjie01 d24d27f1bc [SPARK-31997][SQL][TESTS] Drop test_udtf table when SingleSessionSuite test completed
### What changes were proposed in this pull request?
`SingleSessionSuite` not do `DROP TABLE IF EXISTS test_udtf` when test completed, then if we do mvn test `HiveThriftBinaryServerSuite`, the test case named `SPARK-11595 ADD JAR with input path having URL scheme` will FAILED because it want to re-create an exists table test_udtf.

### Why are the changes needed?
test suite shouldn't rely on their execution order

### Does this PR introduce _any_ user-facing change?
no

### How was this patch tested?
Manual test,mvn test SingleSessionSuite and HiveThriftBinaryServerSuite in order

Closes #28838 from LuciferYang/drop-test-table.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-16 19:20:44 +09:00
Dongjoon Hyun 75afd88904 Revert "[SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber"
This reverts commit a0187cd6b5.
2020-06-15 19:04:23 -07:00
Kent Yao a0187cd6b5 [SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber
### What changes were proposed in this pull request?

This PR brings 02f32cfae4 back which reverted by 4a25200cd7 because of maven test failure

diffs newly made:
1. add a missing log4j file to test resources
2. Call `SessionState.detachSession()` to clean the thread local one in `afterAll`.
3. Not use dedicated JVMs for sbt test runner too

### Why are the changes needed?

fix the maven test

### Does this PR introduce _any_ user-facing change?

no

### How was this patch tested?

add new tests

Closes #28797 from yaooqinn/SPARK-31926-NEW.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-15 06:10:24 +00:00
Kousuke Saruta 88a4e55fae [SPARK-31765][WEBUI][TEST-MAVEN] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-11 18:27:53 -05:00
Dongjoon Hyun 4a25200cd7 Revert "[SPARK-31926][SQL][TEST-HIVE1.2] Fix concurrency issue for ThriftCLIService to getPortNumber"
This reverts commit 02f32cfae4.
2020-06-10 17:21:03 -07:00
Kent Yao 02f32cfae4 [SPARK-31926][SQL][TEST-HIVE1.2] Fix concurrency issue for ThriftCLIService to getPortNumber
### What changes were proposed in this pull request?

When` org.apache.spark.sql.hive.thriftserver.HiveThriftServer2#startWithContext` called,
it starts `ThriftCLIService` in the background with a new Thread, at the same time we call `ThriftCLIService.getPortNumber,` we might not get the bound port if it's configured with 0.

This PR moves the  TServer/HttpServer initialization code out of that new Thread.

### Why are the changes needed?

Fix concurrency issue, improve test robustness.

### Does this PR introduce _any_ user-facing change?

NO
### How was this patch tested?

add new tests

Closes #28751 from yaooqinn/SPARK-31926.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-09 16:49:40 +00:00
Max Gekk 2c9988eaf3 [SPARK-31910][SQL] Enable Java 8 time API in Thrift server
### What changes were proposed in this pull request?
Set `spark.sql.datetime.java8API.enabled` to `true` in:
1. `SparkSQLEnv.init()` of Thrift server, and
2. `SparkSQLSessionManager.openSession()`

### Why are the changes needed?
1. Date and timestamp string literals are parsed by using Java 8 time API and Spark's session time zone. Before the changes, date/timestamp values were collected as legacy types `java.sql.Date`/`java.sql.Timestamp`, and the value of such types didn't respect the config `spark.sql.session.timeZone`. To have consistent view, users had to keep JVM time zone and Spark's session time zone in sync.
2. After the changes, formatting of date values doesn't depend on JVM time zone.
3. While returning dates/timestamps of Java 8 type, we can avoid dates/timestamps rebasing from Proleptic Gregorian calendar to the hybrid calendar (Julian + Gregorian), and the issues related to calendar switching.
4. Properly handle negative years (BCE).
5. Consistent conversion of date/timestamp strings to/from internal Catalyst types in both direction to and from Spark.

### Does this PR introduce any user-facing change?
Yes. Before:
```sql
spark-sql> select make_date(-44, 3, 15);
0045-03-15
```
After:
```sql
spark-sql> select make_date(-44, 3, 15);
-0044-03-15
```

### How was this patch tested?
Manually via `bin/spark-sql`.

Closes #28729 from MaxGekk/enable-java8-time-api-in-thrift-server.

Lead-authored-by: Max Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-05 14:18:16 +00:00
Juliusz Sompolski ea010138e9 [SPARK-31859][SPARK-31861][FOLLOWUP] Fix typo in tests
### What changes were proposed in this pull request?

It appears I have unintentionally used nested JDBC statements in the two tests I added.

### Why are the changes needed?

Cleanup a typo. Please merge to master/branch-3.0

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Unit tests.

Closes #28735 from juliuszsompolski/SPARK-31859-fixup.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-06-05 10:01:00 +00:00
HyukjinKwon a8266e44d4 Revert "[SPARK-28624][SQL][TESTS] Run date.sql via Thrift Server"
This reverts commit a4195d28ae.
2020-06-05 15:43:11 +09:00
Max Gekk a4195d28ae [SPARK-28624][SQL][TESTS] Run date.sql via Thrift Server
### What changes were proposed in this pull request?
Enable `date.sql` and run it via Thrift Server in `ThriftServerQueryTestSuite`.

### Why are the changes needed?
To improve test coverage.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
By running the enabled tests via:
```
$ build/sbt -Phive-thriftserver "hive-thriftserver/test-only *ThriftServerQueryTestSuite -- -z date.sql"
```

Closes #28721 from MaxGekk/enable-date.sql-for-thrift.

Authored-by: Max Gekk <max.gekk@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-06-04 10:45:36 +09:00
HyukjinKwon baafd4386c Revert "[SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0"
This reverts commit e5c3463910.
2020-06-03 14:15:30 +09:00
Kousuke Saruta e5c3463910 [SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-06-02 08:29:07 -05:00
Juliusz Sompolski af35691de4 [SPARK-31859][SPARK-31861][SPARK-31863] Fix Thriftserver session timezone issues
### What changes were proposed in this pull request?

Timestamp literals in Spark are interpreted as timestamps in local timezone spark.sql.session.timeZone.

If JDBC client is e.g. in TimeZone UTC-7, and sets spark.sql.session.timeZone to PST, and sends a query "SELECT timestamp '2020-05-20 12:00:00'", and the JVM timezone of the Spark cluster is e.g. UTC+2, then what currently happens is:
* The timestamp literal in the query is interpreted as 12:00:00 UTC-7, i.e. 19:00:00 UTC.
* When it's returned from the query, it is collected as a java.sql.Timestamp object with Dataset.collect(), and put into a Thriftserver RowSet.
* Before sending it over the wire, the Timestamp is converted to String. This happens in explicitly in ColumnValue for RowBasedSet, and implicitly in ColumnBuffer for ColumnBasedSet (all non-primitive types are converted toString() there). The conversion toString uses JVM timezone, which results in a "21:00:00" (UTC+2) string representation.
* The client JDBC application parses gets a "21:00:00" Timestamp back (in it's JVM timezone; if the JDBC application cares about the correct UTC internal value, it should set spark.sql.session.timeZone to be consistent with its JVM timezone)

The problem is caused by the conversion happening in Thriftserver RowSet with the generic toString() function, instead of using HiveResults.toHiveString() that takes care of correct, timezone respecting conversions. This PR fixes it by converting the Timestamp values to String earlier, in SparkExecuteStatementOperation, using that function. This fixes SPARK-31861.

Thriftserver also did not work spark.sql.datetime.java8API.enabled, because the conversions in RowSet expected an Timestamp object instead of Instant object. Using HiveResults.toHiveString() also fixes that. For this reason, we also convert Date values in SparkExecuteStatementOperation as well - so that HiveResults.toHiveString() handles LocalDate as well. This fixes SPARK-31859.

Thriftserver also did not correctly set the active SparkSession. Because of that, configuration obtained using SQLConf.get was not the correct session configuration. This affected getting the correct spark.sql.session.timeZone. It is fixed by extending the use of SparkExecuteStatementOperation.withSchedulerPool to also set the correct active SparkSession. When the correct session is set, we also no longer need to maintain the pool mapping in a sessionToActivePool map. The scheduler pool can be just correctly retrieved from the session config. "withSchedulerPool" is renamed to "withLocalProperties" and moved into a mixin helper trait, because it should be applied with every operation. This fixes SPARK-31863.

I used the opportunity to move some repetitive code from the operations to the mixin helper trait.

Closes #28671 from juliuszsompolski/SPARK-31861.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-30 06:14:32 +00:00
Kent Yao fe1da296da [SPARK-31833][SQL][TEST-HIVE1.2] Set HiveThriftServer2 with actual port while configured 0
### What changes were proposed in this pull request?

When I was developing some stuff based on the `DeveloperAPI ` `org.apache.spark.sql.hive.thriftserver.HiveThriftServer2#startWithContext`, I need to use thrift port randomly to avoid race on ports. But the `org.apache.hive.service.cli.thrift.ThriftCLIService#getPortNumber` do not respond to me with the actual bound port but always 0.

And the server log is not right too, after starting the server, it's hard to form to the right JDBC connection string.

```
INFO ThriftCLIService: Starting ThriftBinaryCLIService on port 0 with 5...500 worker threads
```

Indeed, the `53742` is the right port
```shell
lsof -nP -p `cat ./pid/spark-kentyao-org.apache.spark.sql.hive.thriftserver.HiveThriftServer2-1.pid` | grep LISTEN
java    18990 kentyao  288u    IPv6 0x22858e3e60d6a0a7      0t0                 TCP 10.242.189.214:53723 (LISTEN)
java    18990 kentyao  290u    IPv6 0x22858e3e60d68827      0t0                 TCP *:4040 (LISTEN)
java    18990 kentyao  366u    IPv6 0x22858e3e60d66987      0t0                 TCP 10.242.189.214:53724 (LISTEN)
java    18990 kentyao  438u    IPv6 0x22858e3e60d65d47      0t0                 TCP *:53742 (LISTEN)
```

In the PR, when the port is configured 0, the `portNum` will be set to the real used port during the start process.

Also use 0 in thrift related tests to avoid potential flakiness.

### Why are the changes needed?

1 fix API bug
2 reduce test flakiness

### Does this PR introduce _any_ user-facing change?

yes, `org.apache.hive.service.cli.thrift.ThriftCLIService#getPortNumber` will always give you the actual port when it is configured 0.

### How was this patch tested?

modified unit tests

Closes #28651 from yaooqinn/SPARK-31833.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-29 04:13:23 +00:00
iRakson 765105b6f1 [SPARK-31638][WEBUI] Clean Pagination code for all webUI pages
### What changes were proposed in this pull request?

Pagination code across pages needs to be cleaned.
I have tried to clear out these things :
* Unused methods
* Unused method arguments
* remove redundant `if` expressions
* fix indentation

### Why are the changes needed?
This fix will make code more readable and remove unnecessary methods and variables.

### Does this PR introduce _any_ user-facing change?
No

### How was this patch tested?
Manually

Closes #28448 from iRakson/refactorPagination.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-27 08:59:08 -05:00
Gengliang Wang db5e5fce68 Revert "[SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0"
This reverts commit 92877c4ef2.

Closes #28602 from gengliangwang/revertSPARK-31765.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-21 16:00:58 -07:00
Kousuke Saruta 92877c4ef2 [SPARK-31765][WEBUI] Upgrade HtmlUnit >= 2.37.0
### What changes were proposed in this pull request?

This PR upgrades HtmlUnit.
Selenium and Jetty also upgraded because of dependency.
### Why are the changes needed?

Recently, a security issue which affects HtmlUnit is reported.
https://nvd.nist.gov/vuln/detail/CVE-2020-5529
According to the report, arbitrary code can be run by malicious users.
HtmlUnit is used for test so the impact might not be large but it's better to upgrade it just in case.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Existing testcases.

Closes #28585 from sarutak/upgrade-htmlunit.

Authored-by: Kousuke Saruta <sarutak@oss.nttdata.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-05-21 11:43:25 -07:00
iRakson f1495c5bc0 [SPARK-31688][WEBUI] Refactor Pagination framework
### What changes were proposed in this pull request?
Currently while implementing pagination using the existing pagination framework, a lot of code is being copied as pointed out [here](https://github.com/apache/spark/pull/28485#pullrequestreview-408881656).

I introduced some changes in `PagedTable` which is the main trait for implementing the pagination.
* Added function for getting table parameters.
* Added a function for table header row. This will help in maintaining consistency across the tables. All the header rows across tables will be consistent now.

### Why are the changes needed?

* A lot of code is copied every time pagination is implemented for any table.
* Code readability is not great as lot of HTML is embedded.
* Paginating other tables will be a lot easier now.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Manually. This is mainly refactoring work, no new functionality introduced. Existing test cases should pass.

Closes #28512 from iRakson/refactorPaginationFramework.

Authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-05-21 13:00:00 -05:00
Ali Smesseim d40ecfa3f7
[SPARK-31387][SQL] Handle unknown operation/session ID in HiveThriftServer2Listener
### What changes were proposed in this pull request?

This is a recreation of #28155, which was reverted due to causing test failures.

The update methods in HiveThriftServer2Listener now check if the parameter operation/session ID actually exist in the `sessionList` and `executionList` respectively. This prevents NullPointerExceptions if the operation or session ID is unknown. Instead, a warning is written to the log.

To improve robustness, we also make the following changes in HiveSessionImpl.close():

- Catch any exception thrown by `operationManager.closeOperation`. If for any reason this throws an exception, other operations are not prevented from being closed.
- Handle not being able to access the scratch directory. When closing, all `.pipeout` files are removed from the scratch directory, which would have resulted in an NPE if the directory does not exist.

### Why are the changes needed?

The listener's update methods would throw an exception if the operation or session ID is unknown. In Spark 2, where the listener is called directly, this changes the caller's control flow. In Spark 3, the exception is caught by the ListenerBus but results in an uninformative NullPointerException.

In HiveSessionImpl.close(), if an exception is thrown when closing an operation, all following operations are not closed.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Unit tests

Closes #28544 from alismess-db/hive-thriftserver-listener-update-safer-2.

Authored-by: Ali Smesseim <ali.smesseim@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-05-20 10:30:17 -07:00
Kent Yao 1d66085a93 [SPARK-31289][TEST][TEST-HIVE1.2] Eliminate org.apache.spark.sql.hive.thriftserver.CliSuite flakiness
### What changes were proposed in this pull request?

CliSuite seems to be flaky while using metastoreDir per test.
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120470/testReport/org.apache.spark.sql.hive.thriftserver/CliSuite/

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/120470/testReport/junit/org.apache.spark.sql.hive.thriftserver/CliSuite/history/

According to the error stack trace in the failed test, the test failed to instantiate a hive metastore client because of derby requirements.
```scala
Caused by: ERROR XBM0A:
 The database directory '/home/jenkins/workspace/SparkPullRequestBuilder/target/tmp/spark-9249ce52-0a06-42b6-a3df-e6295e880df0' exists.
However, it does not contain the expected 'service.properties' file.
Perhaps Derby was brought down in the middle of creating this database.
You may want to delete this directory and try creating the database again.
```

The derby requires the metastore dir does not exist, but it does exist probably due to the test case before it failed to clear the metastore dir

In this PR, the metastore is shared across the tests of CliSuite except those explicitly asked a separated metastore env itself

### Why are the changes needed?

CliSuite seems to be flaky while using metastoreDir per test.
 To eliminate test flakiness
### Does this PR introduce any user-facing change?

no
### How was this patch tested?

modified test

Closes #28055 from yaooqinn/clisuite.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-16 09:11:21 +00:00
Kent Yao 503faa24d3 [SPARK-31715][SQL][TEST] Fix flaky SparkSQLEnvSuite that sometimes varies single derby instance standard
### What changes were proposed in this pull request?

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/122622/testReport/junit/org.apache.spark.sql.hive.thriftserver/SparkSQLEnvSuite/SPARK_29604_external_listeners_should_be_initialized_with_Spark_classloader/history/?start=25

According to the test report history of SparkSQLEnvSuite,this test fails frequently which is caused by single derby instance restriction.

```java
Caused by: sbt.ForkMain$ForkError: org.apache.derby.iapi.error.StandardException: Another instance of Derby may have already booted the database /home/jenkins/workspace/SparkPullRequestBuilder/sql/hive-thriftserver/metastore_db.
	at org.apache.derby.iapi.error.StandardException.newException(Unknown Source)
	at org.apache.derby.impl.store.raw.data.BaseDataFileFactory.privGetJBMSLockOnDB(Unknown Source)
	at org.apache.derby.impl.store.raw.data.BaseDataFileFactory.run(Unknown Source)
	at java.security.AccessController.doPrivileged(Native Method)
	at org.apache.derby.impl.store.raw.data.BaseDataFileFactory.getJBMSLockOnDB(Unknown Source)
	at org.apache.derby.impl.store.raw.data.BaseDataFileFactory.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.TopService.bootModule(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.startModule(Unknown Source)
	at org.apache.derby.iapi.services.monitor.Monitor.bootServiceModule(Unknown Source)
	at org.apache.derby.impl.store.raw.RawStore.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.TopService.bootModule(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.startModule(Unknown Source)
	at org.apache.derby.iapi.services.monitor.Monitor.bootServiceModule(Unknown Source)
	at org.apache.derby.impl.store.access.RAMAccessManager.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.TopService.bootModule(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.startModule(Unknown Source)
	at org.apache.derby.iapi.services.monitor.Monitor.bootServiceModule(Unknown Source)
	at org.apache.derby.impl.db.BasicDatabase.bootStore(Unknown Source)
	at org.apache.derby.impl.db.BasicDatabase.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.boot(Unknown Source)
	at org.apache.derby.impl.services.monitor.TopService.bootModule(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.bootService(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.startProviderService(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.findProviderAndStartService(Unknown Source)
	at org.apache.derby.impl.services.monitor.BaseMonitor.startPersistentService(Unknown Source)
	at org.apache.derby.iapi.services.monitor.Monitor.startPersistentService(Unknown Source)
	... 138 more
```

This PR adds a separate directory to locate the metastore_db for this test which runs in a dedicated JVM.

Besides, diable the UI for the potential race on `spark.ui.port` which may also let the test case become flaky.

### Why are the changes needed?
test fix

### Does this PR introduce _any_ user-facing change?
NO

### How was this patch tested?
SparkSQLEnvSuite itself.

Closes #28537 from yaooqinn/SPARK-31715.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-15 06:36:34 +00:00
Dongjoon Hyun bbb62c5405
Revert "[SPARK-31387] Handle unknown operation/session ID in HiveThriftServer2Listener"
This reverts commit 6994c64efd.
2020-05-14 12:01:03 -07:00
Ali Smesseim 6994c64efd [SPARK-31387] Handle unknown operation/session ID in HiveThriftServer2Listener
### What changes were proposed in this pull request?

The update methods in HiveThriftServer2Listener now check if the parameter operation/session ID actually exist in the `sessionList` and `executionList` respectively. This prevents NullPointerExceptions if the operation or session ID is unknown. Instead, a warning is written to the log.

Also, in HiveSessionImpl.close(), we catch any exception thrown by `operationManager.closeOperation`. If for any reason this throws an exception, other operations are not prevented from being closed.

### Why are the changes needed?

The listener's update methods would throw an exception if the operation or session ID is unknown. In Spark 2, where the listener is called directly, this hampers with the caller's control flow. In Spark 3, the exception is caught by the ListenerBus but results in an uninformative NullPointerException.

In HiveSessionImpl.close(), if an exception is thrown when closing an operation, all following operations are not closed.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Unit tests

Closes #28155 from alismess-db/hive-thriftserver-listener-update-safer.

Authored-by: Ali Smesseim <ali.smesseim@databricks.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2020-05-12 09:14:34 -07:00
Javier Fuentes 178ca961fe [SPARK-31102][SQL] Spark-sql fails to parse when contains comment
### What changes were proposed in this pull request?

This PR introduces a change to false for the insideComment flag on a newline. Fixing the issue introduced by SPARK-30049.

### Why are the changes needed?

Previously on SPARK-30049 a comment containing an unclosed quote produced the following issue:
```
spark-sql> SELECT 1 -- someone's comment here
         > ;
Error in query:
extraneous input ';' expecting <EOF>(line 2, pos 0)

== SQL ==
SELECT 1 -- someone's comment here
;
^^^
```

This was caused because there was no flag for comment sections inside the splitSemiColon method to ignore quotes. SPARK-30049 added that flag and fixed the issue, but introduced the follwoing problem:
```
spark-sql> select
         >   1,
         >   -- two
         >   2;
Error in query:
mismatched input '<EOF>' expecting {'(', 'ADD', 'AFTER', 'ALL', 'ALTER', ...}(line 3, pos 2)
== SQL ==
select
  1,
--^^^
```
This issue is generated by a missing turn-off for the insideComment flag with a newline.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

- For previous tests using line-continuity(`\`) it was added a line-continuity rule in the SqlBase.g4 file to add the functionality to the SQL context.
- A new test for inline comments was added.

Closes #27920 from javierivanov/SPARK-31102.

Authored-by: Javier Fuentes <j.fuentes.m@icloud.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-12 13:46:24 +00:00
Kent Yao ce714d8189 [SPARK-31678][SQL] Print error stack trace for Spark SQL CLI when error occurs
### What changes were proposed in this pull request?

When I was finding the root cause for SPARK-31675, I noticed that it was very difficult for me to see what was actually going on, since it output nothing else but only
```sql
Error in query: java.lang.IllegalArgumentException: Wrong FS: blablah/.hive-staging_blahbla, expected: hdfs://cluster1
```

It is really hard for us to find causes through such a simple error message without a certain amount of experience.

In this PR, I propose to print all of the stack traces when AnalysisException occurs if there are underlying root causes, also we can escape this via `-S` option.

### Why are the changes needed?

In SPARK-11188,

>For analysis exceptions in the sql-shell, we should only print the error message to the screen. The stacktrace will never have useful information since this error is used to signify an error with the query.

But nowadays, some `AnalysisException`s do have useful information for us to debug, e.g. the `AnalysisException` below may contain exceptions from hive or Hadoop side.

a28ed86a38/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala (L97-L112)

```scala
	at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649)
	at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:468)
	at org.apache.hadoop.hive.common.FileUtils.isSubDir(FileUtils.java:626)
	at org.apache.hadoop.hive.ql.metadata.Hive.replaceFiles(Hive.java:2850)
	at org.apache.hadoop.hive.ql.metadata.Hive.loadPartition(Hive.java:1398)
	at org.apache.hadoop.hive.ql.metadata.Hive.loadDynamicPartitions(Hive.java:1593)
```

### Does this PR introduce _any_ user-facing change?

Yes, `bin/spark-sql` will print all the stack trace when an AnalysisException which contains root causes occurs, before this fix, only the message will be printed.

#### before

```scala
Error in query: java.lang.IllegalArgumentException: Wrong FS: hdfs:..., expected: hdfs://hz-cluster10;
```

#### After
```scala
Error in query: java.lang.IllegalArgumentException: Wrong FS: hdfs:..., expected: hdfs://hz-cluster10;
org.apache.spark.sql.AnalysisException: java.lang.IllegalArgumentException: Wrong FS: ..., expected: hdfs://hz-cluster10;
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:109)
	at org.apache.spark.sql.hive.HiveExternalCatalog.loadPartition(HiveExternalCatalog.scala:890)
	at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.loadPartition(ExternalCatalogWithListener.scala:179)
	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.processInsert(InsertIntoHiveTable.scala:312)
	at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.run(InsertIntoHiveTable.scala:101)
	at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult$lzycompute(commands.scala:108)
	at org.apache.spark.sql.execution.command.DataWritingCommandExec.sideEffectResult(commands.scala:106)
	at org.apache.spark.sql.execution.command.DataWritingCommandExec.executeCollect(commands.scala:120)
	at org.apache.spark.sql.Dataset.$anonfun$logicalPlan$1(Dataset.scala:229)
	at org.apache.spark.sql.Dataset.$anonfun$withAction$1(Dataset.scala:3644)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$5(SQLExecution.scala:103)
	at org.apache.spark.sql.execution.SQLExecution$.withSQLConfPropagated(SQLExecution.scala:163)
	at org.apache.spark.sql.execution.SQLExecution$.$anonfun$withNewExecutionId$1(SQLExecution.scala:90)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
	at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:64)
	at org.apache.spark.sql.Dataset.withAction(Dataset.scala:3642)
	at org.apache.spark.sql.Dataset.<init>(Dataset.scala:229)
	at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:100)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
	at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:97)
	at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:607)
	at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:764)
	at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:602)
	at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:650)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLDriver.run(SparkSQLDriver.scala:63)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processCmd(SparkSQLCLIDriver.scala:376)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.$anonfun$processLine$1(SparkSQLCLIDriver.scala:486)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at scala.collection.AbstractIterable.foreach(Iterable.scala:56)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.processLine(SparkSQLCLIDriver.scala:480)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver$.main(SparkSQLCLIDriver.scala:282)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver.main(SparkSQLCLIDriver.scala)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.deploy.JavaMainApplication.start(SparkApplication.scala:52)
	at org.apache.spark.deploy.SparkSubmit.org$apache$spark$deploy$SparkSubmit$$runMain(SparkSubmit.scala:934)
	at org.apache.spark.deploy.SparkSubmit$$anon$1.run(SparkSubmit.scala:165)
	at org.apache.spark.deploy.SparkSubmit$$anon$1.run(SparkSubmit.scala:163)
	at java.security.AccessController.doPrivileged(Native Method)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1746)
	at org.apache.spark.deploy.SparkSubmit.doRunMain$1(SparkSubmit.scala:163)
	at org.apache.spark.deploy.SparkSubmit.submit(SparkSubmit.scala:203)
	at org.apache.spark.deploy.SparkSubmit.doSubmit(SparkSubmit.scala:90)
	at org.apache.spark.deploy.SparkSubmit$$anon$2.doSubmit(SparkSubmit.scala:1013)
	at org.apache.spark.deploy.SparkSubmit$.main(SparkSubmit.scala:1022)
	at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala)
Caused by: java.lang.IllegalArgumentException: Wrong FS: ..., expected: hdfs://hz-cluster10
	at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getPathName(DistributedFileSystem.java:194)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getEZForPath(DistributedFileSystem.java:2093)
	at org.apache.hadoop.hdfs.client.HdfsAdmin.getEncryptionZoneForPath(HdfsAdmin.java:289)
	at org.apache.hadoop.hive.shims.Hadoop23Shims$HdfsEncryptionShim.isPathEncrypted(Hadoop23Shims.java:1221)
	at org.apache.hadoop.hive.ql.metadata.Hive.moveFile(Hive.java:2607)
	at org.apache.hadoop.hive.ql.metadata.Hive.replaceFiles(Hive.java:2892)
	at org.apache.hadoop.hive.ql.metadata.Hive.loadPartition(Hive.java:1398)
	at org.apache.hadoop.hive.ql.metadata.Hive.loadPartition(Hive.java:1324)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.spark.sql.hive.client.Shim_v0_14.loadPartition(HiveShim.scala:927)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$loadPartition$1(HiveClientImpl.scala:870)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.sql.hive.client.HiveClientImpl.$anonfun$withHiveState$1(HiveClientImpl.scala:294)
	at org.apache.spark.sql.hive.client.HiveClientImpl.liftedTree1$1(HiveClientImpl.scala:227)
	at org.apache.spark.sql.hive.client.HiveClientImpl.retryLocked(HiveClientImpl.scala:226)
	at org.apache.spark.sql.hive.client.HiveClientImpl.withHiveState(HiveClientImpl.scala:276)
	at org.apache.spark.sql.hive.client.HiveClientImpl.loadPartition(HiveClientImpl.scala:860)
	at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$loadPartition$1(HiveExternalCatalog.scala:911)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
	... 52 more
```

You can use `-S` option to restore old behavior if you find the error is too verbose.

### How was this patch tested?

Existing CliSuite -
`SPARK-11188 Analysis error reporting`

Add new test and verify manually

Closes #28499 from yaooqinn/SPARK-31678.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-12 13:37:13 +00:00
Daoyuan Wang 53a9bf8fec [SPARK-31595][SQL] Spark sql should allow unescaped quote mark in quoted string
### What changes were proposed in this pull request?
`def splitSemiColon` cannot handle unescaped quote mark like "'" or '"' correctly. When there are unmatched quotes in a string, `splitSemiColon` will not drop off semicolon as expected.

### Why are the changes needed?
Some regex expression will use quote mark in string. We should process semicolon correctly.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Added Unit test and also manual test.

Closes #28393 from adrian-wang/unescaped.

Authored-by: Daoyuan Wang <me@daoyuan.wang>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-05-06 04:34:43 +00:00
Yuming Wang 7ef0b69a92 [SPARK-31626][SQL] Port HIVE-10415: hive.start.cleanup.scratchdir configuration is not taking effect
### What changes were proposed in this pull request?
This pr port [HIVE-10415](https://issues.apache.org/jira/browse/HIVE-10415): `hive.start.cleanup.scratchdir` configuration is not taking effect.

### Why are the changes needed?

I encountered this issue:
![image](https://user-images.githubusercontent.com/5399861/80869375-aeafd080-8cd2-11ea-8573-93ec4b422be1.png)
I'd like to make `hive.start.cleanup.scratchdir` effective to reduce this issue.

### Does this PR introduce _any_ user-facing change?
No.

### How was this patch tested?
Unit test

Closes #28436 from wangyum/SPARK-31626.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-05-04 14:59:33 +09:00
angerszhu 6bc8d84130 [SPARK-29492][SQL] Reset HiveSession's SessionState conf's ClassLoader when sync mode
### What changes were proposed in this pull request?
Run sql in spark thrift server, each session 's thrift server about method will be called in one thread, but when running query statement,  we have two mode:
 1. sync
 2. async
 5a482e7209/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala (L205-L238)

In sync mode, we just submit query in current session's corresponding thread and wait Spark to running query and return result,  and the query method will always wait for query return.
In async mode, in SparkExecuteStatementOperation, we will submit query in a backend thread pool, and update operation state,  after submitted to backend thread poll, ExecuteStatement method will return a OperationHandle to client side, and client side will request operation status continuously. after backend thread running sql and return , it will update corresponding  operation status, when client got operation status is final status, it will got error or start fetching result of this operation.

When we use pyhive connect to SparkThriftServer, it will run statement in sync mode.
When we query data of hive table , it will check serde class in HiveTableScanExec#addColumnMetadataToConf

5a482e7209/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala (L123)

```
  public Class<? extends Deserializer> getDeserializerClass() {
    try {
      return Class.forName(this.getSerdeClassName(), true, Utilities.getSessionSpecifiedClassLoader());
    } catch (ClassNotFoundException var2) {
      throw new RuntimeException(var2);
    }
  }

 public static ClassLoader getSessionSpecifiedClassLoader() {
    SessionState state = SessionState.get();
    if (state != null && state.getConf() != null) {
      ClassLoader sessionCL = state.getConf().getClassLoader();
      if (sessionCL != null) {
        if (LOG.isTraceEnabled()) {
          LOG.trace("Use session specified class loader");
        }

        return sessionCL;
      } else {
        if (LOG.isDebugEnabled()) {
          LOG.debug("Session specified class loader not found, use thread based class loader");
        }

        return JavaUtils.getClassLoader();
      }
    } else {
      if (LOG.isDebugEnabled()) {
        LOG.debug("Hive Conf not found or Session not initiated, use thread based class loader instead");
      }

      return JavaUtils.getClassLoader();
    }
  }
```
Since we run statement in sync mode, it will use HiveSession's SessionState,  and use it's conf's classLoader. then error happened.
```
Current operation state RUNNING_STATE,
java.lang.RuntimeException: java.lang.ClassNotFoundException:
xxx.xxx.xxxJsonSerDe
  at org.apache.hadoop.hive.ql.plan.TableDesc.getDeserializerClass(TableDesc.java:74)
  at org.apache.spark.sql.hive.execution.HiveTableScanExec.addColumnMetadataToConf(HiveTableScanExec.scala:123)
  at org.apache.spark.sql.hive.execution.HiveTableScanExec.hadoopConf$lzycompute(HiveTableScanExec.scala:101)
	  at org.apache.spark.sql.hive.execution.HiveTableScanExec.hadoopConf(HiveTableScanExec.scala:98)
	  at org.apache.spark.sql.hive.execution.HiveTableScanExec.org$apache$spark$sql$hive$execution$HiveTableScanExec$$hadoopReader$lzycompute(HiveTableScanExec.scala:110)
  at org.apache.spark.sql.hive.execution.HiveTableScanExec.org$apache$spark$sql$hive$execution$HiveTableScanExec$$hadoopReader(HiveTableScanExec.scala:105)
	  at org.apache.spark.sql.hive.execution.HiveTableScanExec$$anonfun$11.apply(HiveTableScanExec.scala:192)
	  at org.apache.spark.sql.hive.execution.HiveTableScanExec$$anonfun$11.apply(HiveTableScanExec.scala:192)
```
We should reset it when we start run sql in sync mode.
### Why are the changes needed?
Fix bug

### Does this PR introduce any user-facing change?
NO

### How was this patch tested?
UT

Closes #26141 from AngersZhuuuu/add_jar_in_sync_mode.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-29 06:48:46 +00:00
Juliusz Sompolski 560bd5401f [SPARK-31388][SQL][TESTS] org.apache.spark.sql.hive.thriftserver.CliSuite doesn't match results correctly
### What changes were proposed in this pull request?

`CliSuite.runCliWithin` was not matching for expected results correctly. It was matching for expected lines anywhere in stdout or stderr.
On the example of `Single command with --database` test:
In
```
    runCliWithin(2.minute)(
      "CREATE DATABASE hive_db_test;"
        -> "",
      "USE hive_test;"
        -> "",
      "CREATE TABLE hive_test(key INT, val STRING);"
        -> "",
      "SHOW TABLES;"
        -> "hive_test"
    )
```
It was looking for lines containing "", "", "" and then "hive_test".
However, the string "hive_test" was contained in "hive_test_db", and hence:
```
2020-04-08 17:53:12,752 INFO  CliSuite - 2020-04-08 17:53:12.752 - stderr> Spark master: local, Application Id: local-1586368384172
2020-04-08 17:53:12,765 INFO  CliSuite - stderr> found expected output line 0: ""
2020-04-08 17:53:12,765 INFO  CliSuite - 2020-04-08 17:53:12.765 - stdout> spark-sql> CREATE DATABASE hive_db_test;
2020-04-08 17:53:12,765 INFO  CliSuite - stdout> found expected output line 1: ""
2020-04-08 17:53:17,688 INFO  CliSuite - 2020-04-08 17:53:17.688 - stderr> chgrp: changing ownership of 'file:///tmp/spark-8811f069-4cba-4c71-a5d6-62dd925fb5ff': chown: changing group of '/tmp/spark-8811f069-4cba-4c71-a5d6-62dd925fb5ff': Operation not permitted
2020-04-08 17:53:12,765 INFO  CliSuite - stderr> found expected output line 2: ""
2020-04-08 17:53:18,069 INFO  CliSuite - 2020-04-08 17:53:18.069 - stderr> Time taken: 5.265 seconds
2020-04-08 17:53:18,087 INFO  CliSuite - 2020-04-08 17:53:18.087 - stdout> spark-sql> USE hive_test;
2020-04-08 17:53:12,765 INFO  CliSuite - stdout> found expected output line 3: "hive_test"
2020-04-08 17:53:21,742 INFO  CliSuite - Found all expected output.
```
Because of that, it could kill the CLI process without really even creating the table. This was not expected. The test could be flaky depending on whether process.destroy() in the finally block managed to kill it before it actually creates the table.

I make the output checking more robust to not match on unexpected output, by making it check the echo of query output on the CLI. Also, wait for the CLI process to finish gracefully (triggered by closing its stdin), instead of killing it forcibly.

### Why are the changes needed?

org.apache.spark.sql.hive.thriftserver.CliSuite was flaky, and didn't test outputs as expected.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests in CLISuite. Tested several times with no flakiness. Was getting flaky results almost on every run before.
```
[info] CliSuite:
[info] - load warehouse dir from hive-site.xml (12 seconds, 568 milliseconds)
[info] - load warehouse dir from --hiveconf (10 seconds, 648 milliseconds)
[info] - load warehouse dir from --conf spark(.hadoop).hive.* (20 seconds, 653 milliseconds)
[info] - load warehouse dir from spark.sql.warehouse.dir (9 seconds, 763 milliseconds)
[info] - Simple commands (16 seconds, 238 milliseconds)
[info] - Single command with -e (9 seconds, 967 milliseconds)
[info] - Single command with --database (21 seconds, 205 milliseconds)
[info] - Commands using SerDe provided in --jars (15 seconds, 51 milliseconds)
[info] - SPARK-29022: Commands using SerDe provided in --hive.aux.jars.path (14 seconds, 625 milliseconds)
[info] - SPARK-11188 Analysis error reporting (7 seconds, 960 milliseconds)
[info] - SPARK-11624 Spark SQL CLI should set sessionState only once (7 seconds, 424 milliseconds)
[info] - list jars (9 seconds, 520 milliseconds)
[info] - list jar <jarfile> (9 seconds, 277 milliseconds)
[info] - list files (9 seconds, 828 milliseconds)
[info] - list file <filepath> (9 seconds, 646 milliseconds)
[info] - apply hiveconf from cli command (9 seconds, 469 milliseconds)
[info] - Support hive.aux.jars.path (10 seconds, 676 milliseconds)
[info] - SPARK-28840 test --jars command (10 seconds, 921 milliseconds)
[info] - SPARK-28840 test --jars and hive.aux.jars.path command (11 seconds, 49 milliseconds)
[info] - SPARK-29022 Commands using SerDe provided in ADD JAR sql (14 seconds, 210 milliseconds)
[info] - SPARK-26321 Should not split semicolon within quoted string literals (12 seconds, 729 milliseconds)
[info] - Pad Decimal numbers with trailing zeros to the scale of the column (10 seconds, 381 milliseconds)
[info] - SPARK-30049 Should not complain for quotes in commented lines (10 seconds, 935 milliseconds)
[info] - SPARK-30049 Should not complain for quotes in commented with multi-lines (20 seconds, 731 milliseconds)
```

Closes #28156 from juliuszsompolski/SPARK-31388.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-27 04:27:55 +00:00
Yuming Wang b11e42663b
[SPARK-31381][SPARK-29245][SQL] Upgrade built-in Hive 2.3.6 to 2.3.7
### What changes were proposed in this pull request?

**Hive 2.3.7** fixed these issues:
- HIVE-21508: ClassCastException when initializing HiveMetaStoreClient on JDK10 or newer
- HIVE-21980:Parsing time can be high in case of deeply nested subqueries
- HIVE-22249: Support Parquet through HCatalog

### Why are the changes needed?
Fix CCE during creating HiveMetaStoreClient in JDK11 environment: [SPARK-29245](https://issues.apache.org/jira/browse/SPARK-29245).

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?

- [x] Test Jenkins with Hadoop 2.7 (https://github.com/apache/spark/pull/28148#issuecomment-616757840)
- [x] Test Jenkins with Hadoop 3.2 on JDK11 (https://github.com/apache/spark/pull/28148#issuecomment-616294353)
- [x] Manual test with remote hive metastore.

Hive side:

```
export JAVA_HOME=/usr/lib/jdk1.8.0_221
export PATH=$JAVA_HOME/bin:$PATH
cd /usr/lib/hive-2.3.6 # Start Hive metastore with Hive 2.3.6
bin/schematool -dbType derby -initSchema --verbose
bin/hive --service metastore
```

Spark side:

```
export JAVA_HOME=/usr/lib/jdk-11.0.3
export PATH=$JAVA_HOME/bin:$PATH
build/sbt clean package -Phive -Phadoop-3.2 -Phive-thriftserver
export SPARK_PREPEND_CLASSES=true
bin/spark-sql --conf spark.hadoop.hive.metastore.uris=thrift://localhost:9083
```

Closes #28148 from wangyum/SPARK-31381.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-20 13:38:24 -07:00
Dongjoon Hyun b4c438a5e0
[SPARK-31291][SQL][TEST][FOLLOWUP] Fix resource loading error in ThriftServerQueryTestSuite
### What changes were proposed in this pull request?

[SPARK-31291](https://github.com/apache/spark/pull/28060) broke `ThriftServerQueryTestSuite` in Maven environment. This PR fixes it by copying the resource file from jars to local temp file.

### Why are the changes needed?

To recover the Jenkins jobs in `master` and `branch-3.0`.
- https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-3.0-test-maven-hadoop-2.7-hive-2.3/211/
```
org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite *** ABORTED ***
...
java.lang.IllegalArgumentException: java.net.URISyntaxException:
Relative path in absolute URI: jar:file:/home/jenkins/workspace/spark-branch-3.0-test-maven-hadoop-2.7-hive-2.3/sql/core/target/
spark-sql_2.12-3.0.1-SNAPSHOT-tests.jar!/test-data/postgresql/agg.data
```

![Screen Shot 2020-04-10 at 9 54 28 PM](https://user-images.githubusercontent.com/9700541/79035702-f03ad900-7b75-11ea-9eee-0c1581a28838.png)

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Pass the Jenkins with SBT and Maven.
- [x] Sbt (`Test build #121117` https://github.com/apache/spark/pull/28186#issuecomment-612329068)
- [x] Maven (`Test build #121118` https://github.com/apache/spark/pull/28186#issuecomment-612414382)

Closes #28186 from dongjoon-hyun/SPARK-31291.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-04-11 08:23:59 -07:00
beliefer 2d3692ed45 [SPARK-31406][SQL][TEST] ThriftServerQueryTestSuite: Sharing test data and test tables among multiple test cases
### What changes were proposed in this pull request?
This PR is related to https://github.com/apache/spark/pull/28060.
`ThriftServerQueryTestSuite` spend 17 minutes time to test.
I checked the code and found `ThriftServerQueryTestSuite` load test data repeatedly.
I've listed all the test cases order by time with desc in the `hive-thriftserver` module below.

Class | Spend time  ↑ | Failure | Skip | Pass | Total test case
-- | -- | -- | -- | -- | --
ThriftServerQueryTestSuite | 17 minutes | 0 | 15 | 140 | 155
CliSuite | 8 minutes 24 seconds | 0 | 0 | 24 | 24
SparkThriftServerProtocolVersionsSuite | 59 seconds | 0 | 0 | 210 | 210
HiveThriftBinaryServerSuite | 36 seconds | 0 | 1 | 21 | 22
SparkMetadataOperationSuite | 19 seconds | 0 | 0 | 7 | 7
HiveCliSessionStateSuite | 16 seconds | 0 | 0 | 2 | 2
SparkSQLEnvSuite | 16 seconds | 0 | 0 | 1 | 1
HiveThriftHttpServerSuite | 15 seconds | 0 | 0 | 3 | 3
SingleSessionSuite | 14 seconds | 0 | 0 | 3 | 3
JdbcConnectionUriSuite | 2.1 seconds | 0 | 0 | 1 | 1
ThriftServerWithSparkContextSuite | 1.4 seconds | 0 | 0 | 1 | 1
SparkExecuteStatementOperationSuite | 63 millseconds | 0 | 0 | 2 | 2
UISeleniumSuite | -1 millseconds | 0 | 1 | 0 | 1

I checked the code of `ThriftServerQueryTestSuite` and found `ThriftServerQueryTestSuite` load test data repeatedly.
This PR will improve the performance of `ThriftServerQueryTestSuite`.
Because https://github.com/apache/spark/pull/28060 provides `createTestTables`(e42a3945ac/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L574)) and `removeTestTables`(e42a3945ac/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L666)), this PR will still uses them.
The total time run `ThriftServerQueryTestSuite` before and after this PR show below.
Before
No | Time
-- | --
1 | 18 minutes, 8 seconds
2 | 22 minutes, 44 seconds
3 | 17 minutes, 48 seconds
4 | 18 minutes, 30 seconds

After
No | Time
-- | --
1 | 16 minutes, 11 seconds
2 | 17 minutes, 19 seconds
3 | 18 minutes, 15 seconds
4 | 17 minutes, 27 seconds

### Why are the changes needed?
Improve the performance of `ThriftServerQueryTestSuite`.

### Does this PR introduce any user-facing change?
'No'.

### How was this patch tested?
Jenkins test

Closes #28180 from beliefer/avoid-load-thrift-test-data-repeatedly.

Authored-by: beliefer <beliefer@163.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-04-10 13:08:19 +00:00
Kent Yao 8be16907c2 [SPARK-31170][SQL] Spark SQL Cli should respect hive-site.xml and spark.sql.warehouse.dir
### What changes were proposed in this pull request?

In Spark CLI, we create a hive `CliSessionState` and it does not load the `hive-site.xml`. So the configurations in `hive-site.xml` will not take effects like other spark-hive integration apps.

Also, the warehouse directory is not correctly picked. If the `default` database does not exist, the `CliSessionState` will create one during the first time it talks to the metastore. The `Location` of the default DB will be neither the value of `spark.sql.warehousr.dir` nor the user-specified value of `hive.metastore.warehourse.dir`, but the default value of `hive.metastore.warehourse.dir `which will always be `/user/hive/warehouse`.

This PR fixes CLiSuite failure with the hive-1.2 profile in https://github.com/apache/spark/pull/27933.

In https://github.com/apache/spark/pull/27933, we fix the issue in JIRA by deciding the warehouse dir using all properties from spark conf and Hadoop conf, but properties from `--hiveconf` is not included,  they will be applied to the `CliSessionState` instance after it initialized. When this command-line option key is `hive.metastore.warehouse.dir`, the actual warehouse dir is overridden. Because of the logic in Hive for creating the non-existing default database changed, that test passed with `Hive 2.3.6` but failed with `1.2`. So in this PR, Hadoop/Hive configurations are ordered by:
` spark.hive.xxx > spark.hadoop.xxx > --hiveconf xxx > hive-site.xml` througth `ShareState.loadHiveConfFile` before sessionState start

### Why are the changes needed?

Bugfix for Spark SQL CLI to pick right confs

### Does this PR introduce any user-facing change?

yes,
1. the non-exists default database will be created in the location specified by the users via `spark.sql.warehouse.dir` or `hive.metastore.warehouse.dir`, or the default value of `spark.sql.warehouse.dir` if none of them specified.

2. configurations from `hive-site.xml` will not override command-line options or the properties defined with `spark.hadoo(hive).` prefix in spark conf.

### How was this patch tested?

add cli ut

Closes #27969 from yaooqinn/SPARK-31170-2.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-27 12:05:45 +08:00
Dongjoon Hyun c6a6d5e006
Revert "[SPARK-31170][SQL] Spark SQL Cli should respect hive-site.xml and spark.sql.warehouse.dir"
This reverts commit 5bc0d76591.

Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-19 16:08:51 -07:00
Kent Yao 5bc0d76591 [SPARK-31170][SQL] Spark SQL Cli should respect hive-site.xml and spark.sql.warehouse.dir
### What changes were proposed in this pull request?

In Spark CLI, we create a hive `CliSessionState` and it does not load the `hive-site.xml`. So the configurations in `hive-site.xml` will not take effects like other spark-hive integration apps.

Also, the warehouse directory is not correctly picked. If the `default` database does not exist, the `CliSessionState` will create one during the first time it talks to the metastore. The `Location` of the default DB will be neither the value of `spark.sql.warehousr.dir` nor the user-specified value of `hive.metastore.warehourse.dir`, but the default value of `hive.metastore.warehourse.dir `which will always be `/user/hive/warehouse`.

### Why are the changes needed?

fix bug for Spark SQL cli to pick right confs

### Does this PR introduce any user-facing change?

yes, the non-exists default database will be created in the location specified by the users via `spark.sql.warehouse.dir` or `hive.metastore.warehouse.dir`, or the default value of `spark.sql.warehouse.dir` if none of them specified.

### How was this patch tested?

add cli ut

Closes #27933 from yaooqinn/SPARK-31170.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-17 23:03:18 +08:00
Shixiong Zhu 1ddf44dfca
[SPARK-31144][SQL] Wrap Error with QueryExecutionException to notify QueryExecutionListener
### What changes were proposed in this pull request?

This PR manually reverts changes in #25292 and then wraps java.lang.Error with `QueryExecutionException` to notify `QueryExecutionListener` to send it to `QueryExecutionListener.onFailure` which only accepts `Exception`.

The bug fix PR for 2.4 is #27904. It needs a separate PR because the touched codes were changed a lot.

### Why are the changes needed?

Avoid API changes and fix a bug.

### Does this PR introduce any user-facing change?

Yes. Reverting an API change happening in 3.0. QueryExecutionListener APIs will be the same as 2.4.

### How was this patch tested?

The new added test.

Closes #27907 from zsxwing/SPARK-31144.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2020-03-13 15:55:29 -07:00
Dale Clarke 2a4fed0443 [SPARK-30654][WEBUI] Bootstrap4 WebUI upgrade
### What changes were proposed in this pull request?
Spark's Web UI is using an older version of Bootstrap (v. 2.3.2) for the portal pages. Bootstrap 2.x was moved to EOL in Aug 2013 and Bootstrap 3.x was moved to EOL in July 2019 (https://github.com/twbs/release). Older versions of Bootstrap are also getting flagged in security scans for various CVEs:

https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-72889
https://snyk.io/vuln/SNYK-JS-BOOTSTRAP-173700
https://snyk.io/vuln/npm:bootstrap:20180529
https://snyk.io/vuln/npm:bootstrap:20160627

I haven't validated each CVE, but it would be nice to resolve any potential issues and get on a supported release.

The bad news is that there have been quite a few changes between Bootstrap 2 and Bootstrap 4. I've tried updating the library, refactoring/tweaking the CSS and JS to maintain a similar appearance and functionality, and testing the UI for functionality and appearance. This is a fairly large change so I'm sure additional testing and fixes will be needed.

### How was this patch tested?
This has been manually tested, but there is a ton of functionality and there are many pages and detail pages so it is very possible bugs introduced from the upgrade were missed. Additional testing and feedback is welcomed. If it appears a whole page was missed let me know and I'll take a pass at addressing that page/section.

Closes #27370 from clarkead/bootstrap4-core-upgrade.

Authored-by: Dale Clarke <a.dale.clarke@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2020-03-13 15:24:48 -07:00
Kent Yao 18f2730874 [SPARK-31066][SQL][TEST-HIVE1.2] Disable useless and uncleaned hive SessionState initialization parts
### What changes were proposed in this pull request?

As a common usage and according to the spark doc, users may often just copy their `hive-site.xml` to Spark directly from hive projects. Sometimes, the config file is not that clean for spark and may cause some side effects.

for example, `hive.session.history.enabled` will create a log for the hive jobs but useless for spark and also it will not be deleted on JVM exit.

this pr
 1) disable `hive.session.history.enabled` explicitly to disable creating `hive_job_log` file, e.g.
```
Hive history file=/var/folders/01/h81cs4sn3dq2dd_k4j6fhrmc0000gn/T//kentyao/hive_job_log_79c63b29-95a4-4935-a9eb-2d89844dfe4f_493861201.txt
```
2) set `hive.execution.engine` to `spark` explicitly in case the config is `tez` and casue uneccesary problem like this:

```
Exception in thread "main" java.lang.NoClassDefFoundError: org/apache/tez/dag/api/SessionNotRunning
	at org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:529)
```

### Why are the changes needed?

reduce overhead of internal complexity and users' hive cognitive load for  running spark

### Does this PR introduce any user-facing change?

yes, `hive_job_log` file will not be created even enabled, and will not try to initialize tez kinds of stuff
### How was this patch tested?

add ut and verify manually

Closes #27827 from yaooqinn/SPARK-31066.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-12 18:13:52 +08:00
Javier 3ff2135686 [SPARK-30049][SQL] SQL fails to parse when comment contains an unmatched quote character
### What changes were proposed in this pull request?

A SQL statement that contains a comment with an unmatched quote character can lead to a parse error:
- Added a insideComment flag in the splitter method to avoid checking single and double quotes within a comment:
```
spark-sql> SELECT 1 -- someone's comment here
         > ;
Error in query:
extraneous input ';' expecting <EOF>(line 2, pos 0)

== SQL ==
SELECT 1 -- someone's comment here
;
^^^
```

### Why are the changes needed?

This misbehaviour was not present on previous spark versions.

### Does this PR introduce any user-facing change?

- No

### How was this patch tested?

- New tests were added.

Closes #27321 from javierivanov/SPARK-30049B.

Lead-authored-by: Javier <jfuentes@hortonworks.com>
Co-authored-by: Javier Fuentes <j.fuentes.m@icloud.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-03-03 09:55:15 -06:00
Kent Yao 1fac06c430 Revert "[SPARK-30808][SQL] Enable Java 8 time API in Thrift server"
This reverts commit afaeb29599.

### What changes were proposed in this pull request?

Based on the result and comment from https://github.com/apache/spark/pull/27552#discussion_r385531744

In the hive module, server-side provides datetime values simply use `value.toSting`, and the client-side regenerates the results back in `HiveBaseResultSet` with `java.sql.Date(Timestamp).valueOf`.
there will be inconsistency between client and server if we use java8 APIs

### Why are the changes needed?

the change is still unclear enough

### Does this PR introduce any user-facing change?

no
### How was this patch tested?

Nah

Closes #27733 from yaooqinn/SPARK-30808.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-03-03 14:21:20 +08:00
gatorsmile 28b8713036 [SPARK-30950][BUILD] Setting version to 3.1.0-SNAPSHOT
### What changes were proposed in this pull request?
This patch is to bump the master branch version to 3.1.0-SNAPSHOT.

### Why are the changes needed?
N/A

### Does this PR introduce any user-facing change?
N/A

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

Closes #27698 from gatorsmile/updateVersion.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-02-25 19:44:31 -08:00
Christian Stuart bcce1b1040 [SPARK-30904][SQL] Thrift RowBasedSet serialization throws NullPointerException on NULL BigDecimal
### What changes were proposed in this pull request?

This PR fixes SPARK-30904 by adding a null check.

### Why are the changes needed?

For HIVE_CLI_SERVICE_PROTOCOL_V5 and below, serialization fails on NULL-containing decimal columns, caused by a call to  `value.toPlainString()`, where `value` might be null. This null check fixes it.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

A test was added for serialization of NULL decimals for all HIVE_CLI_SERVICE_PROTOCOL versions.

Closes #27654 from CJStuart/SPARK-30904.

Authored-by: Christian Stuart <christian.stuart@databricks.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2020-02-21 21:39:35 -07:00
Maxim Gekk afaeb29599 [SPARK-30808][SQL] Enable Java 8 time API in Thrift server
### What changes were proposed in this pull request?
- Set `spark.sql.datetime.java8API.enabled` to `true` in `hiveResultString()`, and restore it back at the end of the call.
- Convert collected `java.time.Instant` & `java.time.LocalDate` to `java.sql.Timestamp` and `java.sql.Date` for correct formatting.

### Why are the changes needed?
Because of textual representation of timestamps/dates before 1582 year is incorrect:
```shell
$ export TZ="America/Los_Angeles"
$ ./bin/spark-sql -S
```
```sql
spark-sql> set spark.sql.session.timeZone=America/Los_Angeles;
spark.sql.session.timeZone	America/Los_Angeles
spark-sql> SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20');
1001-01-01 00:07:02
```
It must be 1001-01-01 00:**00:00**.

### Does this PR introduce any user-facing change?
Yes. After the changes:
```shell
$ export TZ="America/Los_Angeles"
$ ./bin/spark-sql -S
```
```sql
spark-sql> set spark.sql.session.timeZone=America/Los_Angeles;
spark.sql.session.timeZone	America/Los_Angeles
spark-sql> SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20');
1001-01-01 00:00:00
```

### How was this patch tested?
By running hive-thiftserver tests. In particular:
```
./build/sbt -Phadoop-2.7 -Phive-2.3 -Phive-thriftserver "hive-thriftserver/test:testOnly *SparkThriftServerProtocolVersionsSuite"
```

Closes #27552 from MaxGekk/hive-thriftserver-java8-time-api.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-02-18 02:15:44 +08:00
Ali Afroozeh e2d3983de7 [SPARK-30798][SQL] Scope Session.active in QueryExecution
### What changes were proposed in this pull request?

This PR scopes `SparkSession.active` to prevent problems with processing queries with possibly different spark sessions (and different configs). A new method, `withActive` is introduced on `SparkSession` that restores the previous spark session after the block of code is executed.

### Why are the changes needed?
`SparkSession.active` is a thread local variable that points to the current thread's spark session. It is important to note that the `SQLConf.get` method depends on `SparkSession.active`. In the current implementation it is possible that `SparkSession.active` points to a different session which causes various problems. Most of these problems arise because part of the query processing is done using the configurations of a different session. For example, when creating a data frame using a new session, i.e., `session.sql("...")`, part of the data frame is constructed using the currently active spark session, which can be a different session from the one used later for processing the query.

### Does this PR introduce any user-facing change?
The `withActive` method is introduced on `SparkSession`.

### How was this patch tested?
Unit tests (to be added)

Closes #27387 from dbaliafroozeh/UseWithActiveSessionInQueryExecution.

Authored-by: Ali Afroozeh <ali.afroozeh@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2020-02-13 23:58:55 +01:00
uncleGen 7173786153
[SPARK-29543][SS][UI] Structured Streaming Web UI
### What changes were proposed in this pull request?

This PR adds two pages to Web UI for Structured Streaming:
   - "/streamingquery": Streaming Query Page, providing some aggregate information for running/completed streaming queries.
  - "/streamingquery/statistics": Streaming Query Statistics Page, providing detailed information for streaming query, including `Input Rate`, `Process Rate`, `Input Rows`, `Batch Duration` and `Operation Duration`

![Screen Shot 2020-01-29 at 1 38 00 PM](https://user-images.githubusercontent.com/1000778/73399837-cd01cc80-429c-11ea-9d4b-1d200a41b8d5.png)
![Screen Shot 2020-01-29 at 1 39 16 PM](https://user-images.githubusercontent.com/1000778/73399838-cd01cc80-429c-11ea-8185-4e56db6866bd.png)

### Why are the changes needed?

It helps users to better monitor Structured Streaming query.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

- new added and existing UTs
- manual test

Closes #26201 from uncleGen/SPARK-29543.

Lead-authored-by: uncleGen <hustyugm@gmail.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Genmao Yu <hustyugm@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2020-01-29 13:43:51 -08:00
Xiao Li 48f647882a [SPARK-30644][SQL][TEST] Remove query index from the golden files of SQLQueryTestSuite
### What changes were proposed in this pull request?

This PR is to remove query index from the golden files of SQLQueryTestSuite

### Why are the changes needed?

Because the SQLQueryTestSuite's golden files have the query index for each query, removal of any query statement [except the last one] will generate many unneeded difference. This will make code review harder. The number of changed lines is misleading.

### Does this PR introduce any user-facing change?
No

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

Closes #27361 from gatorsmile/removeIndexNum.

Authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-25 23:17:36 -08:00
yi.wu ff39c9271c [SPARK-30252][SQL] Disallow negative scale of Decimal
### What changes were proposed in this pull request?

This PR propose to disallow negative `scale` of `Decimal` in Spark. And this PR brings two behavior changes:

1) for literals like `1.23E4BD` or `1.23E4`(with `spark.sql.legacy.exponentLiteralAsDecimal.enabled`=true, see [SPARK-29956](https://issues.apache.org/jira/browse/SPARK-29956)), we set its `(precision, scale)` to (5, 0) rather than (3, -2);
2) add negative `scale` check inside the decimal method if it exposes to set `scale` explicitly. If check fails, `AnalysisException` throws.

And user could still use `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled` to restore the previous behavior.

### Why are the changes needed?

According to SQL standard,
> 4.4.2 Characteristics of numbers
An exact numeric type has a precision P and a scale S. P is a positive integer that determines the number of significant digits in a particular radix R, where R is either 2 or 10. S is a non-negative integer.

scale of Decimal should always be non-negative. And other mainstream databases, like Presto, PostgreSQL, also don't allow negative scale.

Presto:
```
presto:default> create table t (i decimal(2, -1));
Query 20191213_081238_00017_i448h failed: line 1:30: mismatched input '-'. Expecting: <integer>, <type>
create table t (i decimal(2, -1))
```

PostgrelSQL:
```
postgres=# create table t(i decimal(2, -1));
ERROR:  NUMERIC scale -1 must be between 0 and precision 2
LINE 1: create table t(i decimal(2, -1));
                         ^
```

And, actually, Spark itself already doesn't allow to create table with negative decimal types using SQL:
```
scala> spark.sql("create table t(i decimal(2, -1))");
org.apache.spark.sql.catalyst.parser.ParseException:
no viable alternative at input 'create table t(i decimal(2, -'(line 1, pos 28)

== SQL ==
create table t(i decimal(2, -1))
----------------------------^^^

  at org.apache.spark.sql.catalyst.parser.ParseException.withCommand(ParseDriver.scala:263)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parse(ParseDriver.scala:130)
  at org.apache.spark.sql.execution.SparkSqlParser.parse(SparkSqlParser.scala:48)
  at org.apache.spark.sql.catalyst.parser.AbstractSqlParser.parsePlan(ParseDriver.scala:76)
  at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:605)
  at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
  at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:605)
  ... 35 elided
```

However, it is still possible to create such table or `DatFrame` using Spark SQL programming API:
```
scala> val tb =
 CatalogTable(
  TableIdentifier("test", None),
  CatalogTableType.MANAGED,
  CatalogStorageFormat.empty,
  StructType(StructField("i", DecimalType(2, -1) ) :: Nil))
```
```
scala> spark.sql("SELECT 1.23E4BD")
res2: org.apache.spark.sql.DataFrame = [1.23E+4: decimal(3,-2)]
```
while, these two different behavior could make user confused.

On the other side, even if user creates such table or `DataFrame` with negative scale decimal type, it can't write data out if using format, like `parquet` or `orc`. Because these formats have their own check for negative scale and fail on it.
```
scala> spark.sql("SELECT 1.23E4BD").write.saveAsTable("parquet")
19/12/13 17:37:04 ERROR Executor: Exception in task 0.0 in stage 0.0 (TID 0)
java.lang.IllegalArgumentException: Invalid DECIMAL scale: -2
	at org.apache.parquet.Preconditions.checkArgument(Preconditions.java:53)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.decimalMetadata(Types.java:495)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:403)
	at org.apache.parquet.schema.Types$BasePrimitiveBuilder.build(Types.java:309)
	at org.apache.parquet.schema.Types$Builder.named(Types.java:290)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:428)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convertField(ParquetSchemaConverter.scala:334)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.$anonfun$convert$2(ParquetSchemaConverter.scala:326)
	at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:238)
	at scala.collection.Iterator.foreach(Iterator.scala:941)
	at scala.collection.Iterator.foreach$(Iterator.scala:941)
	at scala.collection.AbstractIterator.foreach(Iterator.scala:1429)
	at scala.collection.IterableLike.foreach(IterableLike.scala:74)
	at scala.collection.IterableLike.foreach$(IterableLike.scala:73)
	at org.apache.spark.sql.types.StructType.foreach(StructType.scala:99)
	at scala.collection.TraversableLike.map(TraversableLike.scala:238)
	at scala.collection.TraversableLike.map$(TraversableLike.scala:231)
	at org.apache.spark.sql.types.StructType.map(StructType.scala:99)
	at org.apache.spark.sql.execution.datasources.parquet.SparkToParquetSchemaConverter.convert(ParquetSchemaConverter.scala:326)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport.init(ParquetWriteSupport.scala:97)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:388)
	at org.apache.parquet.hadoop.ParquetOutputFormat.getRecordWriter(ParquetOutputFormat.java:349)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.<init>(ParquetOutputWriter.scala:37)
	at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anon$1.newInstance(ParquetFileFormat.scala:150)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.newOutputWriter(FileFormatDataWriter.scala:124)
	at org.apache.spark.sql.execution.datasources.SingleDirectoryDataWriter.<init>(FileFormatDataWriter.scala:109)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.executeTask(FileFormatWriter.scala:264)
	at org.apache.spark.sql.execution.datasources.FileFormatWriter$.$anonfun$write$15(FileFormatWriter.scala:205)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:90)
	at org.apache.spark.scheduler.Task.run(Task.scala:127)
	at org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$3(Executor.scala:441)
	at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1377)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:444)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:748)
```

So, I think it would be better to disallow negative scale totally and make behaviors above be consistent.

### Does this PR introduce any user-facing change?

Yes, if `spark.sql.legacy.allowNegativeScaleOfDecimal.enabled=false`, user couldn't create Decimal value with negative scale anymore.

### How was this patch tested?

Added new tests in `ExpressionParserSuite` and `DecimalSuite`;
Updated `SQLQueryTestSuite`.

Closes #26881 from Ngone51/nonnegative-scale.

Authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 21:09:48 +08:00
Sean Owen 789a4abfa9 [MINOR][HIVE] Pick up HIVE-22708 HTTP transport fix
### What changes were proposed in this pull request?

Pick up the HTTP fix from https://issues.apache.org/jira/browse/HIVE-22708

### Why are the changes needed?

This is a small but important fix to digest handling we should pick up from Hive.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing tests

Closes #27273 from srowen/Hive22708.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-18 11:50:59 -08:00
Ajith 2be5286828 [SPARK-30382][SQL] Remove Hive LogUtils usage to prevent ClassNotFoundException
Avoid hive log initialisation as https://github.com/apache/hive/blob/rel/release-2.3.5/common/src/java/org/apache/hadoop/hive/common/LogUtils.java introduces dependency over `org.apache.logging.log4j.core.impl.Log4jContextFactory` which is missing in our spark installer classpath directly. I believe the `LogUtils.initHiveLog4j()` code is here as the HiveServer2 class is copied from Hive.

To make `start-thriftserver.sh --help` command success.

Currently, start-thriftserver.sh --help throws
```
...
Thrift server options:
Exception in thread "main" java.lang.NoClassDefFoundError: org/apache/logging/log4j/spi/LoggerContextFactory
	at org.apache.hive.service.server.HiveServer2.main(HiveServer2.java:167)
	at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2$.main(HiveThriftServer2.scala:82)
	at org.apache.spark.sql.hive.thriftserver.HiveThriftServer2.main(HiveThriftServer2.scala)
Caused by: java.lang.ClassNotFoundException: org.apache.logging.log4j.spi.LoggerContextFactory
	at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
	at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:331)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
	... 3 more
```

No

Checked Manually

Closes #27042 from ajithme/thrifthelp.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2020-01-07 14:26:04 -08:00
Jungtaek Lim (HeartSaVioR) 7adf886792 [SPARK-30345][SQL] Fix intermittent test failure (ConnectException) on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite
### What changes were proposed in this pull request?

This patch fixes the intermittent test failure on ThriftServerQueryTestSuite/ThriftServerWithSparkContextSuite, getting ConnectException when querying to thrift server.
(https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/)

The relevant unit test log messages are following:

```
19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:ThriftBinaryCLIService is started.
19/12/23 13:33:01.875 pool-1-thread-1 INFO AbstractService: Service:HiveServer2 is started.
...
19/12/23 13:33:01.888 pool-1-thread-1 INFO ThriftServerWithSparkContextSuite: HiveThriftServer2 started successfully
...
19/12/23 13:33:01.909 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:

===== TEST OUTPUT FOR o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====

...
19/12/23 13:33:02.017 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Supplied authorities: localhost:15441
19/12/23 13:33:02.018 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO Utils: Resolved authority: localhost:15441
19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
19/12/23 13:33:02.078 HiveServer2-Background-Pool: Thread-213 INFO BaseSessionStateBuilder$$anon$2: Optimization rule 'org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation' is excluded from the optimizer.
19/12/23 13:33:02.121 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite WARN HiveConnection: Failed to connect to localhost:15441
19/12/23 13:33:02.124 pool-1-thread-1-ScalaTest-running-ThriftServerWithSparkContextSuite INFO ThriftServerWithSparkContextSuite:

===== FINISHED o.a.s.sql.hive.thriftserver.ThriftServerWithSparkContextSuite: 'SPARK-29911: Uncache cached tables when session closed' =====

19/12/23 13:33:02.143 Thread-35 INFO ThriftCLIService: Starting ThriftBinaryCLIService on port 15441 with 5...500 worker threads
19/12/23 13:33:02.327 pool-1-thread-1 INFO HiveServer2: Shutting down HiveServer2
19/12/23 13:33:02.328 pool-1-thread-1 INFO ThriftCLIService: Thrift server has stopped
```
(Here the error is logged as `WARN HiveConnection: Failed to connect to localhost:15441` - the actual stack trace can be seen on Jenkins test summary.)

The reason of test failure: Thrift(Binary|Http)CLIService prepare and launch the service asynchronously (in new thread), which suites are not waiting for completion and just start running tests, ends up with race condition.

That can be easily reproduced, via adding artificial sleep in `ThriftBinaryCLIService.run()` here:
ba3f6330dd/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java (L49)

(Note that `sleep` should be added before initializing server socket. E.g. Line 57)

This patch changes the test initialization logic to try executing simple query to wait until the service is available. The patch also refactors the code to apply the change both ThriftServerQueryTestSuite and ThriftServerWithSparkContextSuite easily.

### Why are the changes needed?

This patch fixes the intermittent failure observed here:
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115646/testReport/

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

Artificially made the test fail consistently (by the approach described above), and confirmed the patch fixed the test.

Closes #27001 from HeartSaVioR/SPARK-30345.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-27 15:30:54 +08:00
Yuming Wang 696288f623 [INFRA] Reverts commit 56dcd79 and c216ef1
### What changes were proposed in this pull request?
1. Revert "Preparing development version 3.0.1-SNAPSHOT": 56dcd79

2. Revert "Preparing Spark release v3.0.0-preview2-rc2": c216ef1

### Why are the changes needed?
Shouldn't change master.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
manual test:
https://github.com/apache/spark/compare/5de5e46..wangyum:revert-master

Closes #26915 from wangyum/revert-master.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-12-16 19:57:44 -07:00
Yuming Wang 56dcd79992 Preparing development version 3.0.1-SNAPSHOT 2019-12-17 01:57:27 +00:00
Yuming Wang c216ef1d03 Preparing Spark release v3.0.0-preview2-rc2 2019-12-17 01:57:21 +00:00
Yuanjian Li d9b3069412 [SPARK-30125][SQL] Remove PostgreSQL dialect
### What changes were proposed in this pull request?
Reprocess all PostgreSQL dialect related PRs, listing in order:

- #25158: PostgreSQL integral division support [revert]
- #25170: UT changes for the integral division support [revert]
- #25458: Accept "true", "yes", "1", "false", "no", "0", and unique prefixes as input and trim input for the boolean data type. [revert]
- #25697: Combine below 2 feature tags into "spark.sql.dialect" [revert]
- #26112: Date substraction support [keep the ANSI-compliant part]
- #26444: Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled" [revert]
- #26463: Cast to boolean support for PostgreSQL dialect [revert]
- #26584: Make the behavior of Postgre dialect independent of ansi mode config [keep the ANSI-compliant part]

### Why are the changes needed?
As the discussion in http://apache-spark-developers-list.1001551.n3.nabble.com/DISCUSS-PostgreSQL-dialect-td28417.html, we need to remove PostgreSQL dialect form code base for several reasons:
1. The current approach makes the codebase complicated and hard to maintain.
2. Fully migrating PostgreSQL workloads to Spark SQL is not our focus for now.

### Does this PR introduce any user-facing change?
Yes, the config `spark.sql.dialect` will be removed.

### How was this patch tested?
Existing UT.

Closes #26763 from xuanyuanking/SPARK-30125.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-11 01:22:34 +08:00
Luan 3d98c9f985 [SPARK-30179][SQL][TESTS] Improve test in SingleSessionSuite
### What changes were proposed in this pull request?

improve the temporary functions test in SingleSessionSuite by verifying the result in a query

### Why are the changes needed?

### Does this PR introduce any user-facing change?

### How was this patch tested?

Closes #26812 from leoluan2009/SPARK-30179.

Authored-by: Luan <xuluan@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-10 10:57:32 +09:00
wuyi 58be82ad4b [SPARK-30098][SQL] Use default datasource as provider for CREATE TABLE syntax
### What changes were proposed in this pull request?

In this PR, we propose to use the value of `spark.sql.source.default` as the provider for `CREATE TABLE` syntax instead of `hive` in Spark 3.0.

And to help the migration, we introduce a legacy conf `spark.sql.legacy.respectHiveDefaultProvider.enabled` and set its default to `false`.

### Why are the changes needed?

1. Currently, `CREATE TABLE` syntax use hive provider to create table while `DataFrameWriter.saveAsTable` API using the value of `spark.sql.source.default` as a provider to create table. It would be better to make them consistent.

2. User may gets confused in some cases. For example:

```
CREATE TABLE t1 (c1 INT) USING PARQUET;
CREATE TABLE t2 (c1 INT);
```

In these two DDLs, use may think that `t2` should also use parquet as default provider since Spark always advertise parquet as the default format. However, it's hive in this case.

On the other hand, if we omit the USING clause in a CTAS statement, we do pick parquet by default if `spark.sql.hive.convertCATS=true`:

```
CREATE TABLE t3 USING PARQUET AS SELECT 1 AS VALUE;
CREATE TABLE t4 AS SELECT 1 AS VALUE;
```
And these two cases together can be really confusing.

3. Now, Spark SQL is very independent and popular. We do not need to be fully consistent with Hive's behavior.

### Does this PR introduce any user-facing change?

Yes, before this PR, using `CREATE TABLE` syntax will use hive provider. But now, it use the value of `spark.sql.source.default` as its provider.

### How was this patch tested?

Added tests in `DDLParserSuite` and `HiveDDlSuite`.

Closes #26736 from Ngone51/dev-create-table-using-parquet-by-default.

Lead-authored-by: wuyi <yi.wu@databricks.com>
Co-authored-by: yi.wu <yi.wu@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-07 02:15:25 +08:00
Yuming Wang 708ab57f37 [SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column
## What changes were proposed in this pull request?

[HIVE-12063](https://issues.apache.org/jira/browse/HIVE-12063) improved pad decimal numbers with trailing zeros to the scale of the column. The following description is copied from the description of HIVE-12063.

> HIVE-7373 was to address the problems of trimming tailing zeros by Hive, which caused many problems including treating 0.0, 0.00 and so on as 0, which has different precision/scale. Please refer to HIVE-7373 description. However, HIVE-7373 was reverted by HIVE-8745 while the underlying problems remained. HIVE-11835 was resolved recently to address one of the problems, where 0.0, 0.00, and so on cannot be read into decimal(1,1).
 However, HIVE-11835 didn't address the problem of showing as 0 in query result for any decimal values such as 0.0, 0.00, etc. This causes confusion as 0 and 0.0 have different precision/scale than 0.
The proposal here is to pad zeros for query result to the type's scale. This not only removes the confusion described above, but also aligns with many other DBs. Internal decimal number representation doesn't change, however.

**Spark SQL**:
```sql
// bin/spark-sql
spark-sql> select cast(1 as decimal(38, 18));
1
spark-sql>

// bin/beeline
0: jdbc:hive2://localhost:10000/default> select cast(1 as decimal(38, 18));
+----------------------------+--+
| CAST(1 AS DECIMAL(38,18))  |
+----------------------------+--+
| 1.000000000000000000       |
+----------------------------+--+

// bin/spark-shell
scala> spark.sql("select cast(1 as decimal(38, 18))").show(false)
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|1.000000000000000000     |
+-------------------------+

// bin/pyspark
>>> spark.sql("select cast(1 as decimal(38, 18))").show()
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|     1.000000000000000000|
+-------------------------+

// bin/sparkR
> showDF(sql("SELECT cast(1 as decimal(38, 18))"))
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|     1.000000000000000000|
+-------------------------+
```

**PostgreSQL**:
```sql
postgres=# select cast(1 as decimal(38, 18));
       numeric
----------------------
 1.000000000000000000
(1 row)
```
**Presto**:
```sql
presto> select cast(1 as decimal(38, 18));
        _col0
----------------------
 1.000000000000000000
(1 row)
```

## How was this patch tested?

unit tests and manual test:
```sql
spark-sql> select cast(1 as decimal(38, 18));
1.000000000000000000
```
Spark SQL Upgrading Guide:
![image](https://user-images.githubusercontent.com/5399861/69649620-4405c380-10a8-11ea-84b1-6ee675663b98.png)

Closes #26697 from wangyum/SPARK-28461.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-12-02 09:02:39 +09:00
shahid b182ed83f6 [SPARK-29724][SPARK-29726][WEBUI][SQL] Support JDBC/ODBC tab for HistoryServer WebUI
### What changes were proposed in this pull request?

 Support JDBC/ODBC tab for HistoryServer WebUI. Currently from Historyserver we can't access the JDBC/ODBC tab for thrift server applications. In this PR, I am doing 2 main changes
1. Refactor existing thrift server listener to support kvstore
2. Add history server plugin for thrift server listener and tab.

### Why are the changes needed?
Users can access Thriftserver tab from History server for both running and finished applications,

### Does this PR introduce any user-facing change?
Support for JDBC/ODBC tab  for the WEBUI from History server

### How was this patch tested?
Add UT and Manual tests
1. Start Thriftserver and Historyserver
```
sbin/stop-thriftserver.sh
sbin/stop-historyserver.sh
sbin/start-thriftserver.sh
sbin/start-historyserver.sh
```
2. Launch beeline
`bin/beeline -u jdbc:hive2://localhost:10000`

3. Run queries

Go to the JDBC/ODBC page of the WebUI from History server

![image](https://user-images.githubusercontent.com/23054875/68365501-cf013700-0156-11ea-84b4-fda8008c92c4.png)

Closes #26378 from shahidki31/ThriftKVStore.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Gengliang Wang <gengliang.wang@databricks.com>
2019-11-29 19:44:31 -08:00
Dongjoon Hyun 9cd174a7c9 Revert "[SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column"
This reverts commit 19af1fe3a2.
2019-11-27 11:07:08 -08:00
Yuming Wang 19af1fe3a2 [SPARK-28461][SQL] Pad Decimal numbers with trailing zeros to the scale of the column
## What changes were proposed in this pull request?

[HIVE-12063](https://issues.apache.org/jira/browse/HIVE-12063) improved pad decimal numbers with trailing zeros to the scale of the column. The following description is copied from the description of HIVE-12063.

> HIVE-7373 was to address the problems of trimming tailing zeros by Hive, which caused many problems including treating 0.0, 0.00 and so on as 0, which has different precision/scale. Please refer to HIVE-7373 description. However, HIVE-7373 was reverted by HIVE-8745 while the underlying problems remained. HIVE-11835 was resolved recently to address one of the problems, where 0.0, 0.00, and so on cannot be read into decimal(1,1).
 However, HIVE-11835 didn't address the problem of showing as 0 in query result for any decimal values such as 0.0, 0.00, etc. This causes confusion as 0 and 0.0 have different precision/scale than 0.
The proposal here is to pad zeros for query result to the type's scale. This not only removes the confusion described above, but also aligns with many other DBs. Internal decimal number representation doesn't change, however.

**Spark SQL**:
```sql
// bin/spark-sql
spark-sql> select cast(1 as decimal(38, 18));
1
spark-sql>

// bin/beeline
0: jdbc:hive2://localhost:10000/default> select cast(1 as decimal(38, 18));
+----------------------------+--+
| CAST(1 AS DECIMAL(38,18))  |
+----------------------------+--+
| 1.000000000000000000       |
+----------------------------+--+

// bin/spark-shell
scala> spark.sql("select cast(1 as decimal(38, 18))").show(false)
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|1.000000000000000000     |
+-------------------------+

// bin/pyspark
>>> spark.sql("select cast(1 as decimal(38, 18))").show()
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|     1.000000000000000000|
+-------------------------+

// bin/sparkR
> showDF(sql("SELECT cast(1 as decimal(38, 18))"))
+-------------------------+
|CAST(1 AS DECIMAL(38,18))|
+-------------------------+
|     1.000000000000000000|
+-------------------------+
```

**PostgreSQL**:
```sql
postgres=# select cast(1 as decimal(38, 18));
       numeric
----------------------
 1.000000000000000000
(1 row)
```
**Presto**:
```sql
presto> select cast(1 as decimal(38, 18));
        _col0
----------------------
 1.000000000000000000
(1 row)
```

## How was this patch tested?

unit tests and manual test:
```sql
spark-sql> select cast(1 as decimal(38, 18));
1.000000000000000000
```
Spark SQL Upgrading Guide:
![image](https://user-images.githubusercontent.com/5399861/69649620-4405c380-10a8-11ea-84b1-6ee675663b98.png)

Closes #25214 from wangyum/SPARK-28461.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-27 18:13:33 +09:00
Wenchen Fan bd9ce83063 [SPARK-29975][SQL][FOLLOWUP] document --CONFIG_DIM
### What changes were proposed in this pull request?

add document to address https://github.com/apache/spark/pull/26612#discussion_r349844327

### Why are the changes needed?

help people understand how to use --CONFIG_DIM

### Does this PR introduce any user-facing change?

no

### How was this patch tested?

N/A

Closes #26661 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Takeshi Yamamuro <yamamuro@apache.org>
2019-11-25 20:45:31 +09:00
Dongjoon Hyun 6625b69027 [SPARK-29981][BUILD][FOLLOWUP] Change hive.version.short
### What changes were proposed in this pull request?

This is a follow-up according to liancheng 's advice.
- https://github.com/apache/spark/pull/26619#discussion_r349326090

### Why are the changes needed?

Previously, we chose the full version to be carefully. As of today, it seems that `Apache Hive 2.3` branch seems to become stable.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Pass the compile combination on GitHub Action.
1. hadoop-2.7/hive-1.2/JDK8
2. hadoop-2.7/hive-2.3/JDK8
3. hadoop-3.2/hive-2.3/JDK8
4. hadoop-3.2/hive-2.3/JDK11

Also, pass the Jenkins with `hadoop-2.7` and `hadoop-3.2` for (1) and (4).
(2) and (3) is not ready in Jenkins.

Closes #26645 from dongjoon-hyun/SPARK-RENAME-HIVE-DIRECTORY.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-23 12:50:50 -08:00
LantaoJin 9ec2a4e58c [SPARK-29911][SQL][FOLLOWUP] Move related unit test to ThriftServerWithSparkContextSuite
### What changes were proposed in this pull request?
This is follow up of #26543

See https://github.com/apache/spark/pull/26543#discussion_r348934276

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Exist UT.

Closes #26628 from LantaoJin/SPARK-29911_FOLLOWUP.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-22 18:36:50 +09:00
Wenchen Fan e2f056f4a8 [SPARK-29975][SQL] introduce --CONFIG_DIM directive
### What changes were proposed in this pull request?

allow the sql test files to specify different dimensions of config sets during testing. For example,
```
--CONFIG_DIM1 a=1
--CONFIG_DIM1 b=2,c=3

--CONFIG_DIM2 x=1
--CONFIG_DIM2 y=1,z=2
```

This example defines 2 config dimensions, and each dimension defines 2 config sets. We will run the queries 4 times:
1. a=1, x=1
2. a=1, y=1, z=2
3. b=2, c=3, x=1
4. b=2, c=3, y=1, z=2

### Why are the changes needed?

Currently `SQLQueryTestSuite` takes a long time. This is because we run each test at least 3 times, to check with different codegen modes. This is not necessary for most of the tests, e.g. DESC TABLE. We should only check these codegen modes for certain tests.

With the --CONFIG_DIM directive, we can do things like: test different join operator(broadcast or shuffle join) X different codegen modes.

After reducing testing time, we should be able to run thrifter server SQL tests with config settings.

### Does this PR introduce any user-facing change?

no

### How was this patch tested?

test only

Closes #26612 from cloud-fan/test.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-22 10:56:28 +09:00
LantaoJin 06e203b856 [SPARK-29911][SQL] Uncache cached tables when session closed
### What changes were proposed in this pull request?
The local temporary view is session-scoped. Its lifetime is the lifetime of the session that created it.  But now cache data is cross-session. Its lifetime is the lifetime of the Spark application. That's will cause the memory leak if cache a local temporary view in memory when the session closed.
In this PR, we uncache the cached data of local temporary view when session closed. This PR doesn't impact the cached data of global temp view and persisted view.

How to reproduce:
1. create a local temporary view v1
2. cache it in memory
3. close session without drop table v1.

The application will hold the memory forever. In a long running thrift server scenario. It's worse.
```shell
0: jdbc:hive2://localhost:10000> CACHE TABLE testCacheTable AS SELECT 1;
CACHE TABLE testCacheTable AS SELECT 1;
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (1.498 seconds)
0: jdbc:hive2://localhost:10000> !close
!close
Closing: 0: jdbc:hive2://localhost:10000
0: jdbc:hive2://localhost:10000 (closed)> !connect 'jdbc:hive2://localhost:10000'
!connect 'jdbc:hive2://localhost:10000'
Connecting to jdbc:hive2://localhost:10000
Enter username for jdbc:hive2://localhost:10000:
lajin
Enter password for jdbc:hive2://localhost:10000:
***
Connected to: Spark SQL (version 3.0.0-SNAPSHOT)
Driver: Hive JDBC (version 1.2.1.spark2)
Transaction isolation: TRANSACTION_REPEATABLE_READ
1: jdbc:hive2://localhost:10000> select * from testCacheTable;
select * from testCacheTable;
Error: Error running query: org.apache.spark.sql.AnalysisException: Table or view not found: testCacheTable; line 1 pos 14;
'Project [*]
+- 'UnresolvedRelation [testCacheTable] (state=,code=0)
```
<img width="1047" alt="Screen Shot 2019-11-15 at 2 03 49 PM" src="https://user-images.githubusercontent.com/1853780/68923527-7ca8c180-07b9-11ea-9cc7-74f276c46840.png">

### Why are the changes needed?
Resolve memory leak for thrift server

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Manual test in UI storage tab
And add an UT

Closes #26543 from LantaoJin/SPARK-29911.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 18:19:30 -06:00
Yuming Wang 28a502c6e9 [SPARK-28527][FOLLOW-UP][SQL][TEST] Add guides for ThriftServerQueryTestSuite
### What changes were proposed in this pull request?
This PR add guides for `ThriftServerQueryTestSuite`.

### Why are the changes needed?
Add guides

### Does this PR introduce any user-facing change?
No.

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

Closes #26587 from wangyum/SPARK-28527-FOLLOW-UP.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-18 18:13:11 -08:00
Kent Yao 5cebe587c7 [SPARK-29783][SQL] Support SQL Standard/ISO_8601 output style for interval type
### What changes were proposed in this pull request?

Add 3 interval output types which are named as `SQL_STANDARD`, `ISO_8601`, `MULTI_UNITS`. And we add a new conf `spark.sql.dialect.intervalOutputStyle` for this. The `MULTI_UNITS` style displays the interval values in the former behavior and it is the default. The newly added `SQL_STANDARD`, `ISO_8601` styles can be found in the following table.

Style | conf | Year-Month Interval | Day-Time Interval | Mixed Interval
-- | -- | -- | -- | --
Format With Time Unit Designators | MULTI_UNITS | 1 year 2 mons | 1 days 2 hours 3 minutes 4.123456 seconds | interval 1 days 2 hours 3 minutes 4.123456 seconds
SQL STANDARD  | SQL_STANDARD | 1-2 | 3 4:05:06 | -1-2 3 -4:05:06
ISO8601 Basic Format| ISO_8601| P1Y2M| P3DT4H5M6S|P-1Y-2M3D-4H-5M-6S

### Why are the changes needed?

for ANSI SQL support
### Does this PR introduce any user-facing change?

yes,interval out now has 3 output styles
### How was this patch tested?

add new unit tests

cc cloud-fan maropu MaxGekk HyukjinKwon thanks.

Closes #26418 from yaooqinn/SPARK-29783.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-18 15:42:22 +08:00
Pavithra Ramachandran a9959be2bc [SPARK-29456][WEBUI] Improve tooltip for Session Statistics Table column in JDBC/ODBC Server Tab
What changes were proposed in this pull request?
Some of the columns of JDBC/ODBC tab  Session info in Web UI are hard to understand.

Add tool tip for Start time, finish time , Duration and Total Execution

![Screenshot from 2019-10-16 12-33-17](https://user-images.githubusercontent.com/51401130/66901981-76d68980-f01d-11e9-9686-e20346a38c25.png)

Why are the changes needed?
To improve the understanding of the WebUI

Does this PR introduce any user-facing change?
No

How was this patch tested?
manual test

Closes #26138 from PavithraRamachandran/JDBC_tooltip.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-17 07:04:40 -06:00
Yuanjian Li 40ea4a11d7 [SPARK-29807][SQL] Rename "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"
### What changes were proposed in this pull request?
Rename config "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled"

### Why are the changes needed?
The relation between "spark.sql.ansi.enabled" and "spark.sql.dialect" is confusing, since the "PostgreSQL" dialect should contain the features of "spark.sql.ansi.enabled".

To make things clearer, we can rename the "spark.sql.ansi.enabled" to "spark.sql.dialect.spark.ansi.enabled", thus the option "spark.sql.dialect.spark.ansi.enabled" is only for Spark dialect.

For the casting and arithmetic operations, runtime exceptions should be thrown if "spark.sql.dialect" is "spark" and "spark.sql.dialect.spark.ansi.enabled" is true or "spark.sql.dialect" is PostgresSQL.

### Does this PR introduce any user-facing change?
Yes, the config name changed.

### How was this patch tested?
Existing UT.

Closes #26444 from xuanyuanking/SPARK-29807.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-16 17:46:39 +08:00
Takeshi Yamamuro b5a02d37e6 [SPARK-29873][SQL][TESTS] Support --import directive to load queries from another test case in SQLQueryTestSuite
### What changes were proposed in this pull request?

This pr is to support `--import` directive to load queries from another test case in SQLQueryTestSuite.

This fix comes from the cloud-fan suggestion in https://github.com/apache/spark/pull/26479#discussion_r345086978

### Why are the changes needed?

This functionality might reduce duplicate test code in `SQLQueryTestSuite`.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Run `SQLQueryTestSuite`.

Closes #26497 from maropu/ImportTests.

Authored-by: Takeshi Yamamuro <yamamuro@apache.org>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-14 14:38:27 +08:00
Pavithra Ramachandran e2ca7f396f [SPARK-29601][WEBUI] JDBC ODBC Tab Statement column provide ellipsis for big SQL statement
### What changes were proposed in this pull request?
Provide Ellipses in Statement column , just like description in Jobs page .

### Why are the changes needed?
When a query is executed the whole query statement is displayed no matter how big it is. When bigger queries are executed, it covers a large portion of the page display, when we have multiple queries it is difficult to scroll down to view all.

### Does this PR introduce any user-facing change?
No

Before:
![Screenshot from 2019-11-01 23-15-23](https://user-images.githubusercontent.com/51401130/68064468-ebaa0300-fd41-11e9-8787-c5144c1468d4.png)

After:
![Screenshot from 2019-11-02 07-07-21](https://user-images.githubusercontent.com/51401130/68064471-f19fe400-fd41-11e9-85c6-65f0faa64cc3.png)

### How was this patch tested?
Manual

Closes #26364 from PavithraRamachandran/ellipse_JDBC.

Authored-by: Pavithra Ramachandran <pavi.rams@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-10 13:08:26 -06:00
Wenchen Fan 9b61f90987 [SPARK-29761][SQL] do not output leading 'interval' in CalendarInterval.toString
### What changes were proposed in this pull request?

remove the leading "interval" in `CalendarInterval.toString`.

### Why are the changes needed?

Although it's allowed to have "interval" prefix when casting string to int, it's not recommended.

This is also consistent with pgsql:
```
cloud0fan=# select interval '1' day;
 interval
----------
 1 day
(1 row)
```

### Does this PR introduce any user-facing change?

yes, when display a dataframe with interval type column, the result is different.

### How was this patch tested?

updated tests.

Closes #26401 from cloud-fan/interval.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 15:44:50 +08:00
shahid 90df858a26 [SPARK-29725][SQL][TESTS] Add ThriftServerPageSuite
### What changes were proposed in this pull request?
Added UT for the classes `ThriftServerPage.scala` and `ThriftServerSessionPage.scala`

### Why are the changes needed?

Currently, there are no UTs for testing Thriftserver UI page
### Does this PR introduce any user-facing change?

No

### How was this patch tested?

UT

Closes #26403 from shahidki31/ut.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-06 20:59:45 +09:00
angerszhu e524a3a223 [SPARK-29742][BUILD] Update checkstyle plugin's check dir scope
### What changes were proposed in this pull request?
Current checkstyle checking folder can't cover all folder.
Since for support multi version hive, we have some divided hive folder.
We should check it too.

### Why are the changes needed?
Fix build bug

### Does this PR introduce any user-facing change?
NO

### How was this patch tested?
NO

Closes #26385 from AngersZhuuuu/SPARK-29742.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-04 09:08:47 -08:00
shahid 9023c69db8 [SPARK-29590][WEBUI] JDBC/ODBC tab in the spark UI support hide tables, to make it consistent with other tabs
### What changes were proposed in this pull request?

Currently, JDBC/ODBC tab in the WEBUI doesn't support hiding table. Other tabs in the web ui like, Jobs, stages, SQL etc supports hiding table (refer https://github.com/apache/spark/pull/22592).
In this PR, added the support for hide table in the jdbc/odbc tab also.

### Why are the changes needed?
Spark ui about the contents of the form need to have hidden and show features, when the table records very much. Because sometimes you do not care about the record of the table, you just want to see the contents of the next table, but you have to scroll the scroll bar for a long time to see the contents of the next table.

### Does this PR introduce any user-facing change?
No, except support of hide table

### How was this patch tested?
Manually tested
 ![Screenshot 2019-11-01 at 12 10 05 PM](https://user-images.githubusercontent.com/23054875/68007364-61aa5d80-fca1-11e9-841e-c5a7382871fa.png)
![Screenshot 2019-11-01 at 12 10 43 PM](https://user-images.githubusercontent.com/23054875/68007355-5a834f80-fca1-11e9-844a-f4ba1a333db7.png)

Closes #26353 from shahidki31/hideTable.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-04 09:44:10 -06:00
Xingbo Jiang 8207c835b4 Revert "Prepare Spark release v3.0.0-preview-rc2"
This reverts commit 007c873ae3.
2019-10-30 17:45:44 -07:00
Xingbo Jiang 007c873ae3 Prepare Spark release v3.0.0-preview-rc2
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the sparkR version number check logic to allow jvm version like `3.0.0-preview`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

N/A
2019-10-30 17:42:59 -07:00
Jungtaek Lim (HeartSaVioR) 44a27bdccd [SPARK-29604][SQL] Force initialize SessionState before initializing HiveClient in SparkSQLEnv
### What changes were proposed in this pull request?

This patch fixes the issue that external listeners are not initialized properly when `spark.sql.hive.metastore.jars` is set to either "maven" or custom list of jar.
("builtin" is not a case here - all jars in Spark classloader are also available in separate classloader)

The culprit is lazy initialization (lazy val or passing builder function) & thread context classloader. HiveClient leverages IsolatedClientLoader to properly load Hive and relevant libraries without issue - to not mess up with Spark classpath it uses separate classloader with leveraging thread context classloader.

But there's a messed-up case - SessionState is being initialized while HiveClient changed the thread context classloader from Spark classloader to Hive isolated one, and streaming query listeners are loaded from changed classloader while initializing SessionState.

This patch forces initializing SessionState in SparkSQLEnv to avoid such case.

### Why are the changes needed?

ClassNotFoundException could occur in spark-sql with specific configuration, as explained above.

### Does this PR introduce any user-facing change?

No, as I don't think end users assume the classloader of external listeners is only containing jars for Hive client.

### How was this patch tested?

New UT added which fails on master branch and passes with the patch.

The error message with master branch when running UT:

```
java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':;
org.apache.spark.sql.AnalysisException: java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':;
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:109)
	at org.apache.spark.sql.hive.HiveExternalCatalog.databaseExists(HiveExternalCatalog.scala:221)
	at org.apache.spark.sql.internal.SharedState.externalCatalog$lzycompute(SharedState.scala:147)
	at org.apache.spark.sql.internal.SharedState.externalCatalog(SharedState.scala:137)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLEnv$.init(SparkSQLEnv.scala:59)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.$anonfun$new$2(SparkSQLEnvSuite.scala:44)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.withSystemProperties(SparkSQLEnvSuite.scala:61)
	at org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite.$anonfun$new$1(SparkSQLEnvSuite.scala:43)
	at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
	at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85)
	at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83)
	at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104)
	at org.scalatest.Transformer.apply(Transformer.scala:22)
	at org.scalatest.Transformer.apply(Transformer.scala:20)
	at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:186)
	at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:149)
	at org.scalatest.FunSuiteLike.invokeWithFixture$1(FunSuiteLike.scala:184)
	at org.scalatest.FunSuiteLike.$anonfun$runTest$1(FunSuiteLike.scala:196)
	at org.scalatest.SuperEngine.runTestImpl(Engine.scala:286)
	at org.scalatest.FunSuiteLike.runTest(FunSuiteLike.scala:196)
	at org.scalatest.FunSuiteLike.runTest$(FunSuiteLike.scala:178)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:56)
	at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:221)
	at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:214)
	at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:56)
	at org.scalatest.FunSuiteLike.$anonfun$runTests$1(FunSuiteLike.scala:229)
	at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:393)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:381)
	at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:376)
	at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:458)
	at org.scalatest.FunSuiteLike.runTests(FunSuiteLike.scala:229)
	at org.scalatest.FunSuiteLike.runTests$(FunSuiteLike.scala:228)
	at org.scalatest.FunSuite.runTests(FunSuite.scala:1560)
	at org.scalatest.Suite.run(Suite.scala:1124)
	at org.scalatest.Suite.run$(Suite.scala:1106)
	at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1560)
	at org.scalatest.FunSuiteLike.$anonfun$run$1(FunSuiteLike.scala:233)
	at org.scalatest.SuperEngine.runImpl(Engine.scala:518)
	at org.scalatest.FunSuiteLike.run(FunSuiteLike.scala:233)
	at org.scalatest.FunSuiteLike.run$(FunSuiteLike.scala:232)
	at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:56)
	at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213)
	at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210)
	at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208)
	at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:56)
	at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:45)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1349)
	at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1343)
	at scala.collection.immutable.List.foreach(List.scala:392)
	at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1343)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:1033)
	at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:1011)
	at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1509)
	at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:1011)
	at org.scalatest.tools.Runner$.run(Runner.scala:850)
	at org.scalatest.tools.Runner.run(Runner.scala)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2(ScalaTestRunner.java:133)
	at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:27)
Caused by: java.lang.IllegalArgumentException: Error while instantiating 'org.apache.spark.sql.hive.HiveSessionStateBuilder':
	at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1054)
	at org.apache.spark.sql.SparkSession.$anonfun$sessionState$2(SparkSession.scala:156)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.SparkSession.sessionState$lzycompute(SparkSession.scala:154)
	at org.apache.spark.sql.SparkSession.sessionState(SparkSession.scala:151)
	at org.apache.spark.sql.SparkSession.$anonfun$new$3(SparkSession.scala:105)
	at scala.Option.map(Option.scala:230)
	at org.apache.spark.sql.SparkSession.$anonfun$new$1(SparkSession.scala:105)
	at org.apache.spark.sql.internal.SQLConf$.get(SQLConf.scala:164)
	at org.apache.spark.sql.hive.client.HiveClientImpl.newState(HiveClientImpl.scala:183)
	at org.apache.spark.sql.hive.client.HiveClientImpl.<init>(HiveClientImpl.scala:127)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at org.apache.spark.sql.hive.client.IsolatedClientLoader.createClient(IsolatedClientLoader.scala:300)
	at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:421)
	at org.apache.spark.sql.hive.HiveUtils$.newClientForMetadata(HiveUtils.scala:314)
	at org.apache.spark.sql.hive.HiveExternalCatalog.client$lzycompute(HiveExternalCatalog.scala:68)
	at org.apache.spark.sql.hive.HiveExternalCatalog.client(HiveExternalCatalog.scala:67)
	at org.apache.spark.sql.hive.HiveExternalCatalog.$anonfun$databaseExists$1(HiveExternalCatalog.scala:221)
	at scala.runtime.java8.JFunction0$mcZ$sp.apply(JFunction0$mcZ$sp.java:23)
	at org.apache.spark.sql.hive.HiveExternalCatalog.withClient(HiveExternalCatalog.scala:99)
	... 58 more
Caused by: java.lang.ClassNotFoundException: test.custom.listener.DummyQueryExecutionListener
	at java.net.URLClassLoader.findClass(URLClassLoader.java:382)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
	at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
	at java.lang.Class.forName0(Native Method)
	at java.lang.Class.forName(Class.java:348)
	at org.apache.spark.util.Utils$.classForName(Utils.scala:206)
	at org.apache.spark.util.Utils$.$anonfun$loadExtensions$1(Utils.scala:2746)
	at scala.collection.TraversableLike.$anonfun$flatMap$1(TraversableLike.scala:245)
	at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62)
	at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55)
	at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49)
	at scala.collection.TraversableLike.flatMap(TraversableLike.scala:245)
	at scala.collection.TraversableLike.flatMap$(TraversableLike.scala:242)
	at scala.collection.AbstractTraversable.flatMap(Traversable.scala:108)
	at org.apache.spark.util.Utils$.loadExtensions(Utils.scala:2744)
	at org.apache.spark.sql.util.ExecutionListenerManager.$anonfun$new$1(QueryExecutionListener.scala:83)
	at org.apache.spark.sql.util.ExecutionListenerManager.$anonfun$new$1$adapted(QueryExecutionListener.scala:82)
	at scala.Option.foreach(Option.scala:407)
	at org.apache.spark.sql.util.ExecutionListenerManager.<init>(QueryExecutionListener.scala:82)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.$anonfun$listenerManager$2(BaseSessionStateBuilder.scala:293)
	at scala.Option.getOrElse(Option.scala:189)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.listenerManager(BaseSessionStateBuilder.scala:293)
	at org.apache.spark.sql.internal.BaseSessionStateBuilder.build(BaseSessionStateBuilder.scala:320)
	at org.apache.spark.sql.SparkSession$.org$apache$spark$sql$SparkSession$$instantiateSessionState(SparkSession.scala:1051)
	... 80 more
```

Closes #26258 from HeartSaVioR/SPARK-29604.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-30 01:06:31 -07:00
Xingbo Jiang b33a58c0c6 Revert "Prepare Spark release v3.0.0-preview-rc1"
This reverts commit 5eddbb5f1d.
2019-10-28 22:32:34 -07:00
Xingbo Jiang 5eddbb5f1d Prepare Spark release v3.0.0-preview-rc1
### What changes were proposed in this pull request?

To push the built jars to maven release repository, we need to remove the 'SNAPSHOT' tag from the version name.

Made the following changes in this PR:
* Update all the `3.0.0-SNAPSHOT` version name to `3.0.0-preview`
* Update the PySpark version from `3.0.0.dev0` to `3.0.0`

**Please note those changes were generated by the release script in the past, but this time since we manually add tags on master branch, we need to manually apply those changes too.**

We shall revert the changes after 3.0.0-preview release passed.

### Why are the changes needed?

To make the maven release repository to accept the built jars.

### Does this PR introduce any user-facing change?

No

### How was this patch tested?

N/A

Closes #26243 from jiangxb1987/3.0.0-preview-prepare.

Lead-authored-by: Xingbo Jiang <xingbo.jiang@databricks.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Xingbo Jiang <xingbo.jiang@databricks.com>
2019-10-28 22:31:29 -07:00
angerszhu d6e33dc377 [SPARK-29599][WEBUI] Support pagination for session table in JDBC/ODBC Tab
### What changes were proposed in this pull request?

In this PR, extend the support of pagination to session  table in `JDBC/PDBC` .

### Why are the changes needed?
Some times we may connect a lot client and there a many session info shown in session tab.
make it can be paged for better view.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Manuel verify.
After pr:
<img width="1440" alt="Screen Shot 2019-10-25 at 4 19 27 PM" src="https://user-images.githubusercontent.com/46485123/67555133-50ae9900-f743-11e9-8724-9624a691f232.png">

<img width="1434" alt="Screen Shot 2019-10-25 at 4 19 38 PM" src="https://user-images.githubusercontent.com/46485123/67555165-5906d400-f743-11e9-819e-73f86a333dd3.png">

Closes #26253 from AngersZhuuuu/SPARK-29599.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-28 08:45:21 -05:00
shahid 077fb99a26 [SPARK-29589][WEBUI] Support pagination for sqlstats session table in JDBC/ODBC Session page
### What changes were proposed in this pull request?
In the PR https://github.com/apache/spark/pull/26215, we supported pagination for sqlstats table in JDBC/ODBC server page. In this PR, we are extending the support of pagination to sqlstats session table by making use of existing pagination classes in https://github.com/apache/spark/pull/26215.

### Why are the changes needed?
Support pagination for sqlsessionstats table in JDBC/ODBC server page in the WEBUI. It will easier for user to analyse the table and it may fix the potential issues like oom while loading the page, that may occur similar to the SQL page (refer #22645)

### Does this PR introduce any user-facing change?
There will be no change in the sqlsessionstats table in JDBC/ODBC server page execpt pagination support.

### How was this patch tested?
Manually verified.

Before:

![Screenshot 2019-10-24 at 11 32 27 PM](https://user-images.githubusercontent.com/23054875/67512507-96715000-f6b6-11e9-9f1f-ab1877eb24e6.png)

After:

![Screenshot 2019-10-24 at 10 58 53 PM](https://user-images.githubusercontent.com/23054875/67512314-295dba80-f6b6-11e9-9e3e-dd50c6e62fe9.png)

Closes #26246 from shahidki31/SPARK_29589.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-26 15:46:24 -05:00
shahid 76d4bebb54 [SPARK-29559][WEBUI] Support pagination for JDBC/ODBC Server page
### What changes were proposed in this pull request?
Supports pagination for SQL Statisitcs table in the JDBC/ODBC tab using existing Spark pagination framework.

### Why are the changes needed?
It will easier for user to analyse the table and it may fix the potential issues like oom while loading the page, that may occur similar to the SQL page (refer https://github.com/apache/spark/pull/22645)

### Does this PR introduce any user-facing change?
There will be no change in the `SQLStatistics` table in JDBC/ODBC server page execpt pagination support.

### How was this patch tested?
Manually verified.

Before PR:
![Screenshot 2019-10-22 at 11 37 29 PM](https://user-images.githubusercontent.com/23054875/67316080-73636680-f525-11e9-91bc-ff7e06e3736d.png)

After PR:

![Screenshot 2019-10-22 at 10 33 00 PM](https://user-images.githubusercontent.com/23054875/67316092-778f8400-f525-11e9-93f8-1e2815abd66f.png)

Closes #26215 from shahidki31/jdbcPagination.

Authored-by: shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-24 08:29:05 -05:00
Yuming Wang 3163b6b43b [SPARK-29516][SQL][TEST] Test ThriftServerQueryTestSuite asynchronously
### What changes were proposed in this pull request?
This PR test `ThriftServerQueryTestSuite` in an asynchronous way.

### Why are the changes needed?
The default value of `spark.sql.hive.thriftServer.async` is `true`.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
```
build/sbt "hive-thriftserver/test-only *.ThriftServerQueryTestSuite" -Phive-thriftserver
build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite test -Phive-thriftserver
```

Closes #26172 from wangyum/SPARK-29516.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-22 03:20:49 -07:00
fuwhu 31a5dea48f [SPARK-29531][SQL][TEST] refine ThriftServerQueryTestSuite.blackList to reuse black list in SQLQueryTestSuite
### What changes were proposed in this pull request?
This pr refine the code in ThriftServerQueryTestSuite.blackList to reuse the black list of SQLQueryTestSuite instead of duplicating all test cases from SQLQueryTestSuite.blackList.

### Why are the changes needed?
To reduce code duplication.

### Does this PR introduce any user-facing change?
No

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

Closes #26188 from fuwhu/SPARK-TBD.

Authored-by: fuwhu <bestwwg@163.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-21 05:19:27 -07:00
lajin fda4070ea9 [SPARK-29283][SQL] Error message is hidden when query from JDBC, especially enabled adaptive execution
### What changes were proposed in this pull request?
When adaptive execution is enabled, the Spark users who connected from JDBC always get adaptive execution error whatever the under root cause is. It's very confused. We have to check the driver log to find out why.
```shell
0: jdbc:hive2://localhost:10000> SELECT * FROM testData join testData2 ON key = v;
SELECT * FROM testData join testData2 ON key = v;
Error: Error running query: org.apache.spark.SparkException: Adaptive execution failed due to stage materialization failures. (state=,code=0)
0: jdbc:hive2://localhost:10000>
```

For example, a job queried from JDBC failed due to HDFS missing block. User still get the error message `Adaptive execution failed due to stage materialization failures`.

The easiest way to reproduce is changing the code of `AdaptiveSparkPlanExec`, to let it throws out  an exception when it faces `StageSuccess`.
```scala
  case class AdaptiveSparkPlanExec(
      events.drainTo(rem)
         (Seq(nextMsg) ++ rem.asScala).foreach {
           case StageSuccess(stage, res) =>
//            stage.resultOption = Some(res)
            val ex = new SparkException("Wrapper Exception",
              new IllegalArgumentException("Root cause is IllegalArgumentException for Test"))
            errors.append(
              new SparkException(s"Failed to materialize query stage: ${stage.treeString}", ex))
           case StageFailure(stage, ex) =>
             errors.append(
               new SparkException(s"Failed to materialize query stage: ${stage.treeString}", ex))
```

### Why are the changes needed?
To make the error message more user-friend and more useful for query from JDBC.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Manually test query:
```shell
0: jdbc:hive2://localhost:10000> CREATE TEMPORARY VIEW testData (key, value) AS SELECT explode(array(1, 2, 3, 4)), cast(substring(rand(), 3, 4) as string);
CREATE TEMPORARY VIEW testData (key, value) AS SELECT explode(array(1, 2, 3, 4)), cast(substring(rand(), 3, 4) as string);
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.225 seconds)
0: jdbc:hive2://localhost:10000> CREATE TEMPORARY VIEW testData2 (k, v) AS SELECT explode(array(1, 1, 2, 2)), cast(substring(rand(), 3, 4) as int);
CREATE TEMPORARY VIEW testData2 (k, v) AS SELECT explode(array(1, 1, 2, 2)), cast(substring(rand(), 3, 4) as int);
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.043 seconds)
```
Before:
```shell
0: jdbc:hive2://localhost:10000> SELECT * FROM testData join testData2 ON key = v;
SELECT * FROM testData join testData2 ON key = v;
Error: Error running query: org.apache.spark.SparkException: Adaptive execution failed due to stage materialization failures. (state=,code=0)
0: jdbc:hive2://localhost:10000>
```
After:
```shell
0: jdbc:hive2://localhost:10000> SELECT * FROM testData join testData2 ON key = v;
SELECT * FROM testData join testData2 ON key = v;
Error: Error running query: java.lang.IllegalArgumentException: Root cause is IllegalArgumentException for Test (state=,code=0)
0: jdbc:hive2://localhost:10000>
```

Closes #25960 from LantaoJin/SPARK-29283.

Authored-by: lajin <lajin@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-16 19:51:56 -07:00
Juliusz Sompolski eb8c420edb [SPARK-29349][SQL] Support FETCH_PRIOR in Thriftserver fetch request
### What changes were proposed in this pull request?

Support FETCH_PRIOR fetching in Thriftserver, and report correct fetch start offset it TFetchResultsResp.results.startRowOffset

The semantics of FETCH_PRIOR are as follow: Assuming the previous fetch returned a block of rows from offsets [10, 20)
* calling FETCH_PRIOR(maxRows=5) will scroll back and return rows [5, 10)
* calling FETCH_PRIOR(maxRows=10) again, will scroll back, but can't go earlier than 0. It will nevertheless return 10 rows, returning rows [0, 10) (overlapping with the previous fetch)
* calling FETCH_PRIOR(maxRows=4) again will again return rows starting from offset 0 - [0, 4)
* calling FETCH_NEXT(maxRows=6) after that will move the cursor forward and return rows [4, 10)

##### Client/server backwards/forwards compatibility:

Old driver with new server:
* Drivers that don't support FETCH_PRIOR will not attempt to use it
* Field TFetchResultsResp.results.startRowOffset was not set, old drivers don't depend on it.

New driver with old server
* Using an older thriftserver with FETCH_PRIOR will make the thriftserver return unsupported operation error. The driver can then recognize that it's an old server.
* Older thriftserver will return TFetchResultsResp.results.startRowOffset=0. If the client driver receives 0, it can know that it can not rely on it as correct offset. If the client driver intentionally wants to fetch from 0, it can use FETCH_FIRST.

### Why are the changes needed?

It's intended to be used to recover after connection errors. If a client lost connection during fetching (e.g. of rows [10, 20)), and wants to reconnect and continue, it could not know whether the request  got lost before reaching the server, or on the response back. When it issued another FETCH_NEXT(10) request after reconnecting, because TFetchResultsResp.results.startRowOffset was not set, it could not know if the server will return rows [10,20) (because the previous request didn't reach it) or rows [20, 30) (because it returned data from the previous request but the connection got broken on the way back). Now, with TFetchResultsResp.results.startRowOffset the client can know after reconnecting which rows it is getting, and use FETCH_PRIOR to scroll back if a fetch block was lost in transmission.

Driver should always use FETCH_PRIOR after a broken connection.
* If the Thriftserver returns unsuported operation error, the driver knows that it's an old server that doesn't support it. The driver then must error the query, as it will also not support returning the correct startRowOffset, so the driver cannot reliably guarantee if it hadn't lost any rows on the fetch cursor.
* If the driver gets a response to FETCH_PRIOR, it should also have a correctly set startRowOffset, which the driver can use to position itself back where it left off before the connection broke.
* If FETCH_NEXT was used after a broken connection on the first fetch, and returned with an startRowOffset=0, then the client driver can't know if it's 0 because it's the older server version, or if it's genuinely 0. Better to call FETCH_PRIOR, as scrolling back may anyway be possibly required after a broken connection.

This way it is implemented in a backwards/forwards compatible way, and doesn't require bumping the protocol version. FETCH_ABSOLUTE might have been better, but that would require a bigger protocol change, as there is currently no field to specify the requested absolute offset.

### Does this PR introduce any user-facing change?

ODBC/JDBC drivers connecting to Thriftserver may now implement using the FETCH_PRIOR fetch order to scroll back in query results, and check TFetchResultsResp.results.startRowOffset if their cursor position is consistent after connection errors.

### How was this patch tested?

Added tests to HiveThriftServer2Suites

Closes #26014 from juliuszsompolski/SPARK-29349.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-15 23:22:19 -07:00
Gengliang Wang 322ec0ba9b [SPARK-28885][SQL] Follow ANSI store assignment rules in table insertion by default
### What changes were proposed in this pull request?

When inserting a value into a column with the different data type, Spark performs type coercion. Currently, we support 3 policies for the store assignment rules: ANSI, legacy and strict, which can be set via the option "spark.sql.storeAssignmentPolicy":
1. ANSI: Spark performs the type coercion as per ANSI SQL. In practice, the behavior is mostly the same as PostgreSQL. It disallows certain unreasonable type conversions such as converting `string` to `int` and `double` to `boolean`. It will throw a runtime exception if the value is out-of-range(overflow).
2. Legacy: Spark allows the type coercion as long as it is a valid `Cast`, which is very loose. E.g., converting either `string` to `int` or `double` to `boolean` is allowed. It is the current behavior in Spark 2.x for compatibility with Hive. When inserting an out-of-range value to a integral field, the low-order bits of the value is inserted(the same as Java/Scala numeric type casting). For example, if 257 is inserted to a field of Byte type, the result is 1.
3. Strict: Spark doesn't allow any possible precision loss or data truncation in store assignment, e.g., converting either `double` to `int` or `decimal` to `double` is allowed. The rules are originally for Dataset encoder. As far as I know, no mainstream DBMS is using this policy by default.

Currently, the V1 data source uses "Legacy" policy by default, while V2 uses "Strict". This proposal is to use "ANSI" policy by default for both V1 and V2 in Spark 3.0.

### Why are the changes needed?

Following the ANSI SQL standard is most reasonable among the 3 policies.

### Does this PR introduce any user-facing change?

Yes.
The default store assignment policy is ANSI for both V1 and V2 data sources.

### How was this patch tested?

Unit test

Closes #26107 from gengliangwang/ansiPolicyAsDefault.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-15 10:41:37 -07:00
Yuming Wang 148cd26799 [SPARK-26321][SQL] Port HIVE-15297: Hive should not split semicolon within quoted string literals
## What changes were proposed in this pull request?

This pr port [HIVE-15297](https://issues.apache.org/jira/browse/HIVE-15297) to fix **spark-sql** should not split semicolon within quoted string literals.

## How was this patch tested?
unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/60395592-5666ea00-9b68-11e9-99dc-0e8ea98de32b.png)

Closes #25018 from wangyum/SPARK-26321.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-12 22:21:14 -07:00
Peter Toth 9e12c94c15 [SPARK-29359][SQL][TESTS] Better exception handling in (SQL|ThriftServer)QueryTestSuite
### What changes were proposed in this pull request?
This PR adds 2 changes regarding exception handling in `SQLQueryTestSuite` and `ThriftServerQueryTestSuite`
- fixes an expected output sorting issue in `ThriftServerQueryTestSuite` as if there is an exception then there is no need for sort
- introduces common exception handling in those 2 suites with a new `handleExceptions` method

### Why are the changes needed?

Currently `ThriftServerQueryTestSuite` passes on master, but it fails on one of my PRs (https://github.com/apache/spark/pull/23531) with this error  (https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/111651/testReport/org.apache.spark.sql.hive.thriftserver/ThriftServerQueryTestSuite/sql_3/):
```
org.scalatest.exceptions.TestFailedException: Expected "
[Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit
org.apache.spark.SparkException]
", but got "
[org.apache.spark.SparkException
Recursion level limit 100 reached but query has not exhausted, try increasing spark.sql.cte.recursion.level.limit]
" Result did not match for query #4 WITH RECURSIVE r(level) AS (   VALUES (0)   UNION ALL   SELECT level + 1 FROM r ) SELECT * FROM r
```
The unexpected reversed order of expected output (error message comes first, then the exception class) is due to this line: https://github.com/apache/spark/pull/26028/files#diff-b3ea3021602a88056e52bf83d8782de8L146. It should not sort the expected output if there was an error during execution.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
Existing UTs.

Closes #26028 from peter-toth/SPARK-29359-better-exception-handling.

Authored-by: Peter Toth <peter.toth@gmail.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-12 22:17:37 -07:00
angerszhu 0cf2f48dfe [SPARK-29022][SQL] Fix SparkSQLCLI can not add jars by AddJarCommand
### What changes were proposed in this pull request?
For issue mentioned in [SPARK-29022](https://issues.apache.org/jira/browse/SPARK-29022)
Spark SQL CLI can't use class as serde class in jars add by SQL `ADD JAR`.

When we create table with `serde` class contains by jar added by SQL 'ADD JAR'.
We can create table with `serde` class construct success since we call `HiveClientImpl.createTable` under `withHiveState` method, it will add `clientLoader.classLoader` to `HiveClientImpl.state.getConf.classLoader`.

Jars added by SQL `ADD JAR` will be add to
1. `sparkSession.sharedState.jarClassLoader`.
2. 'HiveClientLoader.clientLoader.classLoader'

In Current spark-sql MODE,  `HiveClientImpl.state`  will use CliSessionState created when initialize
SparkSQLCliDriver,  When we select data from table, it will check `serde` class, when call method `HiveTableScanExec#addColumnMetadataToConf()` to check for table desc serde class.

```
val deserializer = tableDesc.getDeserializerClass.getConstructor().newInstance()
    deserializer.initialize(hiveConf, tableDesc.getProperties)
```

`getDeserializer` will  use CliSessionState's hiveConf's classLoader in `Spark SQL CLI` mode.
But when we call `ADD JAR` in spark, the jar won't be added to `Classloader of  CliSessionState' conf `, then `ClassNotFound` error happen.

So we reset `CliSessionState conf's classLoader ` to `sharedState.jarClassLoader` when `sharedState.jarClassLoader` has added jar passed by `HIVEAUXJARS`
Then when we use  `ADD JAR `  to add jar,  jar path will be added to CliSessionState's conf's ClassLoader

### Why are the changes needed?
Fix bug

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
ADD UT

Closes #25729 from AngersZhuuuu/SPARK-29015.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-01 10:09:29 -05:00
Dongjoon Hyun bd031c2173 [SPARK-29307][BUILD][TESTS] Remove scalatest deprecation warnings
### What changes were proposed in this pull request?

This PR aims to remove `scalatest` deprecation warnings with the following changes.
- `org.scalatest.mockito.MockitoSugar` -> `org.scalatestplus.mockito.MockitoSugar`
- `org.scalatest.selenium.WebBrowser` -> `org.scalatestplus.selenium.WebBrowser`
- `org.scalatest.prop.Checkers` -> `org.scalatestplus.scalacheck.Checkers`
- `org.scalatest.prop.GeneratorDrivenPropertyChecks` -> `org.scalatestplus.scalacheck.ScalaCheckDrivenPropertyChecks`

### Why are the changes needed?

According to the Jenkins logs, there are 118 warnings about this.
```
 grep "is deprecated" ~/consoleText | grep scalatest | wc -l
     118
```

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

After Jenkins passes, we need to check the Jenkins log.

Closes #25982 from dongjoon-hyun/SPARK-29307.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-30 21:00:11 -07:00
Sean Owen e1ea806b30 [SPARK-29291][CORE][SQL][STREAMING][MLLIB] Change procedure-like declaration to function + Unit for 2.13
### What changes were proposed in this pull request?

Scala 2.13 emits a deprecation warning for procedure-like declarations:

```
def foo() {
 ...
```

This is equivalent to the following, so should be changed to avoid a warning:

```
def foo(): Unit = {
  ...
```

### Why are the changes needed?

It will avoid about a thousand compiler warnings when we start to support Scala 2.13. I wanted to make the change in 3.0 as there are less likely to be back-ports from 3.0 to 2.4 than 3.1 to 3.0, for example, minimizing that downside to touching so many files.

Unfortunately, that makes this quite a big change.

### Does this PR introduce any user-facing change?

No behavior change at all.

### How was this patch tested?

Existing tests.

Closes #25968 from srowen/SPARK-29291.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-30 10:03:23 -07:00
Unknown 3ea9d6825b [SPARK-29019][WEBUI] Improve tooltip JDBC/ODBC Server tab
### What changes were proposed in this pull request?
Some of the columns of JDBC/ODBC server tab in Web UI are hard to understand.
We have documented it at SPARK-28373 but I think it is better to have some tooltips in the SQL statistics table to explain the columns
![image](https://user-images.githubusercontent.com/12819544/64489775-38e48980-d257-11e9-868a-5f5f6a0f1e46.png)
The columns with new tooltips are finish time, close time, execution time and duration
![image](https://user-images.githubusercontent.com/12819544/64489858-1141f100-d258-11e9-9e4e-fae3299da465.png)
Improvements in UIUtils can be used in other tables in WebUI to add tooltips

### Why are the changes needed?
It is interesting to improve the undestanding of the WebUI

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Unit tests are added and manual test.

Closes #25723 from planga82/feature/SPARK-29019_tooltipjdbcServer.

Lead-authored-by: Unknown <soypab@gmail.com>
Co-authored-by: Pablo <soypab@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-29 18:34:24 -05:00
Yuming Wang 8167714cab [SPARK-27831][FOLLOW-UP][SQL][TEST] Should not use maven to add Hive test jars
### What changes were proposed in this pull request?

This PR moves Hive test jars(`hive-contrib-*.jar` and `hive-hcatalog-core-*.jar`) from maven dependency to local file.

### Why are the changes needed?
`--jars` can't be tested since `hive-contrib-*.jar` and `hive-hcatalog-core-*.jar` are already in classpath.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?
manual test

Closes #25690 from wangyum/SPARK-27831-revert.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-28 16:55:49 -07:00
Gengliang Wang a1213d5f96 [SPARK-28997][SQL] Add spark.sql.dialect
### What changes were proposed in this pull request?

After https://github.com/apache/spark/pull/25158 and https://github.com/apache/spark/pull/25458, SQL features of PostgreSQL are introduced into Spark. AFAIK, both features are implementation-defined behaviors, which are not specified in ANSI SQL.
In such a case, this proposal is to add a configuration `spark.sql.dialect` for choosing a database dialect.
After this PR, Spark supports two database dialects, `Spark` and `PostgreSQL`. With `PostgreSQL` dialect, Spark will:
1. perform integral division with the / operator if both sides are integral types;
2. accept "true", "yes", "1", "false", "no", "0", and unique prefixes as input and trim input for the boolean data type.

### Why are the changes needed?

Unify the external database dialect with one configuration, instead of small flags.

### Does this PR introduce any user-facing change?

A new configuration `spark.sql.dialect` for choosing a database dialect.

### How was this patch tested?

Existing tests.

Closes #25697 from gengliangwang/dialect.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-09-26 21:00:27 +08:00
Gengliang Wang 66c9dc316a [SPARK-29255][SQL][TESTS] Rename package pgSQL to postgreSQL
### What changes were proposed in this pull request?

Rename the package pgSQL to postgreSQL

### Why are the changes needed?

To address the comment in https://github.com/apache/spark/pull/25697#discussion_r328431070 . The official full name seems more reasonable.

### Does this PR introduce any user-facing change?

No.

### How was this patch tested?

Existing unit tests.

Closes #25936 from gengliangwang/renamePGSQL.

Authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-26 05:36:15 -07:00
Yuming Wang b8b67ae92d [SPARK-28527][SQL][TEST] Enable ThriftServerQueryTestSuite
### What changes were proposed in this pull request?

This PR enable `ThriftServerQueryTestSuite` and fix previously flaky test by:
1. Start thriftserver in `beforeAll()`.
2. Disable `spark.sql.hive.thriftServer.async`.

### Why are the changes needed?

Improve test coverage.

### Does this PR introduce any user-facing change?
No.

### How was this patch tested?

```shell
build/sbt "hive-thriftserver/test-only *.ThriftServerQueryTestSuite "  -Phive-thriftserver
build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.sql.hive.thriftserver.ThriftServerQueryTestSuite test  -Phive-thriftserver
```

Closes #25868 from wangyum/SPARK-28527-enable.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-24 00:44:33 -07:00
angerszhu d22768a6be [SPARK-29036][SQL] SparkThriftServer cancel job after execute() thread interrupted
### What changes were proposed in this pull request?
Discuss in https://github.com/apache/spark/pull/25611

If cancel() and close() is called very quickly after the query is started, then they may both call cleanup() before Spark Jobs are started. Then sqlContext.sparkContext.cancelJobGroup(statementId) does nothing.
But then the execute thread can start the jobs, and only then get interrupted and exit through here. But then it will exit here, and no-one will cancel these jobs and they will keep running even though this execution has exited.

So  when execute() was interrupted by `cancel()`, when get into catch block, we should call canJobGroup again to make sure the job was canceled.

### Why are the changes needed?

### Does this PR introduce any user-facing change?
NO

### How was this patch tested?
MT

Closes #25743 from AngersZhuuuu/SPARK-29036.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-23 05:47:25 -07:00
Yuming Wang 51d3509428 [SPARK-28599][SQL] Fix Execution Time and Duration column sorting for ThriftServerSessionPage
### What changes were proposed in this pull request?

This PR add support sorting `Execution Time` and `Duration` columns for `ThriftServerSessionPage`.

### Why are the changes needed?

Previously, it's not sorted correctly.

### Does this PR introduce any user-facing change?

Yes.

### How was this patch tested?

Manually do the following and test sorting on those columns in the Spark Thrift Server Session Page.
```
$ sbin/start-thriftserver.sh
$ bin/beeline -u jdbc:hive2://localhost:10000
0: jdbc:hive2://localhost:10000> create table t(a int);
+---------+--+
| Result  |
+---------+--+
+---------+--+
No rows selected (0.521 seconds)
0: jdbc:hive2://localhost:10000> select * from t;
+----+--+
| a  |
+----+--+
+----+--+
No rows selected (0.772 seconds)
0: jdbc:hive2://localhost:10000> show databases;
+---------------+--+
| databaseName  |
+---------------+--+
| default       |
+---------------+--+
1 row selected (0.249 seconds)
```

**Sorted by `Execution Time` column**:
![image](https://user-images.githubusercontent.com/5399861/65387476-53038900-dd7a-11e9-885c-fca80287f550.png)

**Sorted by `Duration` column**:
![image](https://user-images.githubusercontent.com/5399861/65387481-6e6e9400-dd7a-11e9-9318-f917247efaa8.png)

Closes #25892 from wangyum/SPARK-28599.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-22 14:12:06 -07:00
aman_omer 93ac4e1b2d [SPARK-29053][WEBUI] Sort does not work on some columns
### What changes were proposed in this pull request?
Setting custom sort key for duration and execution time column.

### Why are the changes needed?
Sorting on duration and execution time columns consider time as a string after converting into readable form which is the reason for wrong sort results as mentioned in [SPARK-29053](https://issues.apache.org/jira/browse/SPARK-29053).

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Test manually. Screenshots are attached.

After patch:
**Duration**
![Duration](https://user-images.githubusercontent.com/40591404/65339861-93cc9800-dbea-11e9-95e6-63b107a5a372.png)
**Execution time**
![Execution Time](https://user-images.githubusercontent.com/40591404/65339870-97601f00-dbea-11e9-9d1d-690c59bc1bde.png)

Closes #25855 from amanomer/SPARK29053.

Authored-by: aman_omer <amanomer1996@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-21 07:34:04 -05:00
Gengliang Wang b917a6593d [SPARK-28989][SQL] Add a SQLConf spark.sql.ansi.enabled
### What changes were proposed in this pull request?
Currently, there are new configurations for compatibility with ANSI SQL:

* `spark.sql.parser.ansi.enabled`
* `spark.sql.decimalOperations.nullOnOverflow`
* `spark.sql.failOnIntegralTypeOverflow`
This PR is to add new configuration `spark.sql.ansi.enabled` and remove the 3 options above. When the configuration is true, Spark tries to conform to the ANSI SQL specification. It will be disabled by default.

### Why are the changes needed?

Make it simple and straightforward.

### Does this PR introduce any user-facing change?

The new features for ANSI compatibility will be set via one configuration `spark.sql.ansi.enabled`.

### How was this patch tested?

Existing unit tests.

Closes #25693 from gengliangwang/ansiEnabled.

Lead-authored-by: Gengliang Wang <gengliang.wang@databricks.com>
Co-authored-by: Xiao Li <gatorsmile@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-09-18 22:30:28 -07:00
Juliusz Sompolski fcf9b41b49 [SPARK-29056] ThriftServerSessionPage displays 1970/01/01 finish and close time when unset
### What changes were proposed in this pull request?

ThriftServerSessionPage displays timestamp 0 (1970/01/01) instead of nothing if query finish time and close time are not set.

![image](https://user-images.githubusercontent.com/25019163/64711118-6d578000-d4b9-11e9-9b11-2e3616319a98.png)

Change it to display nothing, like ThriftServerPage.

### Why are the changes needed?

Obvious bug.

### Does this PR introduce any user-facing change?

Finish time and Close time will be displayed correctly on ThriftServerSessionPage in JDBC/ODBC Spark UI.

### How was this patch tested?

Manual test.

Closes #25762 from juliuszsompolski/SPARK-29056.

Authored-by: Juliusz Sompolski <julek@databricks.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-13 09:13:57 -07:00
sandeep katta 7e6142591f [SPARK-28840][SQL] conf.getClassLoader in SparkSQLCLIDriver should be avoided as it returns the UDFClassLoader which is created by Hive
### What changes were proposed in this pull request?

Spark loads the jars to custom class loader which is returned by `getSubmitClassLoader` .
 [Spark code](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L337)

**In 1.2.1.spark2 version of Hive**

`HiveConf.getClassLoader` returns same the class loader which is set by the spark

**In Hive 2.3.5**
`HiveConf.getClassLoader` returns the UDFClassLoader which is created by Hive. Because of this spark cannot find the jars as class loader got changed
[Hive code](https://github.com/apache/hive/blob/rel/release-2.3.5/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java#L395)

### Why are the changes needed?
Before creating `CliSessionState` object save the current class loader object in some reference.
After SessionState.start() reset back class Loader to the one which saved earlier.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Added Test case and also Manually tested

**Before Fix**
![b4Fix](https://user-images.githubusercontent.com/35216143/63442838-6789f400-c451-11e9-9529-ccf4ea9621b9.png)

**After Fix**
![afterFix](https://user-images.githubusercontent.com/35216143/63442860-707ac580-c451-11e9-8012-2b70934d55f3.png)

Closes #25542 from sandeep-katta/jarIssue.

Lead-authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Co-authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-12 03:47:30 -07:00
angerszhu 54d3f6e7ec [SPARK-28982][SQL] Implementation Spark's own GetTypeInfoOperation
### What changes were proposed in this pull request?
 Current Spark Thrift Server return TypeInfo includes
1.  INTERVAL_YEAR_MONTH
2. INTERVAL_DAY_TIME
3. UNION
4. USER_DEFINED

Spark doesn't support INTERVAL_YEAR_MONTH, INTERVAL_YEAR_MONTH, UNION
and won't return USER)DEFINED type.
This PR overwrite GetTypeInfoOperation with SparkGetTypeInfoOperation to exclude types which we don't need.

In hive-1.2.1 Type class is `org.apache.hive.service.cli.Type`
In hive-2.3.x Type class is `org.apache.hadoop.hive.serde2.thrift.Type`

Use ThrifrserverShimUtils to fit version problem and exclude types we don't need

### Why are the changes needed?

We should return type info of Spark's own type info

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Manuel test & Added UT

Closes #25694 from AngersZhuuuu/SPARK-28982.

Lead-authored-by: angerszhu <angers.zhu@gmail.com>
Co-authored-by: AngersZhuuuu <angers.zhu@gmail.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-10 09:22:50 -07:00
sychen 962e330955 [SPARK-26598][SQL] Fix HiveThriftServer2 cannot be modified hiveconf/hivevar variables
### What changes were proposed in this pull request?
The intent to use the --hiveconf/--hivevar parameter is just an initialization value, so setting it once in ```SparkSQLSessionManager#openSession``` is sufficient, and each time the ```SparkExecuteStatementOperation``` setting causes the variable to not be modified.

### Why are the changes needed?
It is wrong to set the --hivevar/--hiveconf variable in every ```SparkExecuteStatementOperation```, which prevents variable updates.

### Does this PR introduce any user-facing change?
```
cat <<EOF > test.sql
select '\${a}', '\${b}';
set b=bvalue_MOD_VALUE;
set b;
EOF

beeline -u jdbc:hive2://localhost:10000 --hiveconf a=avalue --hivevar b=bvalue -f test.sql
```
current result:
```
+-----------------+-----------------+--+
|     avalue      |     bvalue      |
+-----------------+-----------------+--+
| avalue          | bvalue          |
+-----------------+-----------------+--+
+-----------------+-----------------+--+
|       key       |      value      |
+-----------------+-----------------+--+
| b               | bvalue          |
+-----------------+-----------------+--+
1 row selected (0.022 seconds)
```
after modification:
```
+-----------------+-----------------+--+
|     avalue      |     bvalue      |
+-----------------+-----------------+--+
| avalue          | bvalue          |
+-----------------+-----------------+--+
+-----------------+-----------------+--+
|       key       |      value      |
+-----------------+-----------------+--+
| b               | bvalue_MOD_VALUE|
+-----------------+-----------------+--+
1 row selected (0.022 seconds)
```

### How was this patch tested?
modified the existing unit test

Closes #25722 from cxzl25/fix_SPARK-26598.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-09-09 22:06:19 -07:00
LantaoJin 86fc890d8c [SPARK-28988][SQL][TESTS] Fix invalid tests in CliSuite
### What changes were proposed in this pull request?

1f056eb313/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala (L221) is not strong enough. It will success if class not found.

1f056eb313/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala (L305) is also incorrect. Whatever the right side value is, it always succeeds.

### Why are the changes needed?
Unit tests should failed if the class not found.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Exist UTs

Closes #25724 from LantaoJin/SPARK-28988.

Authored-by: LantaoJin <jinlantao@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-09-10 11:22:06 +09:00
Yuming Wang 4a3a6b66be [SPARK-28637][SQL] Thriftserver support interval type
## What changes were proposed in this pull request?

`bin/spark-shell` support query interval value:
```scala
scala> spark.sql("SELECT interval 3 months 1 hours AS i").show(false)
+-------------------------+
|i                        |
+-------------------------+
|interval 3 months 1 hours|
+-------------------------+
```

But `sbin/start-thriftserver.sh` can't support query interval value:
```sql
0: jdbc:hive2://localhost:10000/default> SELECT interval 3 months 1 hours AS i;
Error: java.lang.IllegalArgumentException: Unrecognized type name: interval (state=,code=0)
```

This PR maps `CalendarIntervalType` to `StringType` for `TableSchema` to make Thriftserver support  query interval value because we do not support `INTERVAL_YEAR_MONTH` type and `INTERVAL_DAY_TIME`:
02c33694c8/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java (L73-L78)
[SPARK-27791](https://issues.apache.org/jira/browse/SPARK-27791): Support SQL year-month INTERVAL type
[SPARK-27793](https://issues.apache.org/jira/browse/SPARK-27793): Support SQL day-time INTERVAL type

## How was this patch tested?

unit tests

Closes #25277 from wangyum/Thriftserver-support-interval-type.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-09-08 23:20:27 -07:00
angerszhu 9f478a6832 [SPARK-28901][SQL] SparkThriftServer's Cancel SQL Operation show it in JDBC Tab UI
### What changes were proposed in this pull request?
Current Spark Thirft Server can't support cancel SQL job,  when we use Hue to query throgh Spark Thrift Server, when we run a sql and then click cancel button to cancel this sql, we will it won't work in backend and in the spark JDBC UI tab, we can see the SQL's status is always COMPILED, then the duration of SQL is always increasing, this may make people confused.

![image](https://user-images.githubusercontent.com/46485123/63869830-60338f00-c9eb-11e9-8776-cee965adcb0a.png)

### Why are the changes needed?

If sql status can't reflect sql's true status, it will make user confused.

### Does this PR introduce any user-facing change?

SparkthriftServer's UI tab will show SQL's status in CANCELED when we cancel a SQL .

### How was this patch tested?
Manuel tested

UI TAB Status
![image](https://user-images.githubusercontent.com/46485123/63915010-80a12f00-ca67-11e9-9342-830dfa9c719f.png)

![image](https://user-images.githubusercontent.com/46485123/63915084-a9292900-ca67-11e9-8e26-375bf8ce0963.png)

backend log
![image](https://user-images.githubusercontent.com/46485123/63914864-1092a900-ca67-11e9-93f2-08690ed9abf4.png)

Closes #25611 from AngersZhuuuu/SPARK-28901.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-09-04 09:20:51 -07:00
Yuming Wang ab1819d38a [SPARK-28527][SQL][TEST][FOLLOW-UP] Ignores Thrift server ThriftServerQueryTestSuite
### What changes were proposed in this pull request?

This PR ignores Thrift server `ThriftServerQueryTestSuite`.

### Why are the changes needed?

This ThriftServerQueryTestSuite test case led to frequent Jenkins build failure.

### Does this PR introduce any user-facing change?

Yes.

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

Closes #25592 from wangyum/SPARK-28527-f1.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-27 15:41:22 +09:00
Yuming Wang 6e12b585a9 [SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server
### What changes were proposed in this pull request?
This PR build a test framework that directly re-run all the tests in `SQLQueryTestSuite` via Thrift Server. But it's a little different from `SQLQueryTestSuite`:
1. Can not support [UDF testing](44e607e921/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L293-L297)).
2. Can not support `DESC` command and `SHOW` command because `SQLQueryTestSuite` [formatted the output](1882912cca/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala (L38-L50).).

When building this framework, found two bug:
[SPARK-28624](https://issues.apache.org/jira/browse/SPARK-28624): `make_date` is inconsistent when reading from table
[SPARK-28611](https://issues.apache.org/jira/browse/SPARK-28611): Histogram's height is different

found two features that ThriftServer can not support:
[SPARK-28636](https://issues.apache.org/jira/browse/SPARK-28636): ThriftServer can not support decimal type with negative scale
[SPARK-28637](https://issues.apache.org/jira/browse/SPARK-28637): ThriftServer can not support interval type

Also, found two inconsistent behavior:
[SPARK-28620](https://issues.apache.org/jira/browse/SPARK-28620): Double type returned for float type in Beeline/JDBC
[SPARK-28619](https://issues.apache.org/jira/browse/SPARK-28619):  The golden result file is different when tested by `bin/spark-sql`

### Why are the changes needed?

Improve the overall test coverage for Thrift Server.

### Does this PR introduce any user-facing change?
No

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

Closes #25567 from wangyum/SPARK-28527.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-26 22:39:57 +09:00
Yuming Wang adb506afd7 [SPARK-28852][SQL] Implement SparkGetCatalogsOperation for Thrift Server
### What changes were proposed in this pull request?
This PR implements `SparkGetCatalogsOperation` for Thrift Server metadata completeness.

### Why are the changes needed?
Thrift Server metadata completeness.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Unit test

Closes #25555 from wangyum/SPARK-28852.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2019-08-25 22:42:50 -07:00
Yuming Wang 02a0cdea13 [SPARK-28723][SQL] Upgrade to Hive 2.3.6 for HiveMetastore Client and Hadoop-3.2 profile
### What changes were proposed in this pull request?

This PR upgrade the built-in Hive to 2.3.6 for `hadoop-3.2`.

Hive 2.3.6 release notes:
- [HIVE-22096](https://issues.apache.org/jira/browse/HIVE-22096): Backport [HIVE-21584](https://issues.apache.org/jira/browse/HIVE-21584) (Java 11 preparation: system class loader is not URLClassLoader)
- [HIVE-21859](https://issues.apache.org/jira/browse/HIVE-21859): Backport [HIVE-17466](https://issues.apache.org/jira/browse/HIVE-17466) (Metastore API to list unique partition-key-value combinations)
- [HIVE-21786](https://issues.apache.org/jira/browse/HIVE-21786): Update repo URLs in poms branch 2.3 version

### Why are the changes needed?
Make Spark support JDK 11.

### Does this PR introduce any user-facing change?
Yes. Please see [SPARK-28684](https://issues.apache.org/jira/browse/SPARK-28684) and [SPARK-24417](https://issues.apache.org/jira/browse/SPARK-24417) for more details.

### How was this patch tested?
Existing unit test and manual test.

Closes #25443 from wangyum/test-on-jenkins.

Lead-authored-by: Yuming Wang <yumwang@ebay.com>
Co-authored-by: HyukjinKwon <gurwls223@apache.org>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-23 21:34:30 -07:00
Dongjoon Hyun f0834d3a7f Revert "[SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server"
This reverts commit efbb035902.
2019-08-18 16:54:24 -07:00
Yuming Wang efbb035902 [SPARK-28527][SQL][TEST] Re-run all the tests in SQLQueryTestSuite via Thrift Server
## What changes were proposed in this pull request?

This PR build a test framework that directly re-run all the tests in `SQLQueryTestSuite` via Thrift Server. But it's a little different from `SQLQueryTestSuite`:
1. Can not support [UDF testing](44e607e921/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala (L293-L297)).
2. Can not support `DESC` command and `SHOW` command because `SQLQueryTestSuite` [formatted the output](1882912cca/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala (L38-L50).).

When building this framework, found two bug:
[SPARK-28624](https://issues.apache.org/jira/browse/SPARK-28624): `make_date` is inconsistent when reading from table
[SPARK-28611](https://issues.apache.org/jira/browse/SPARK-28611): Histogram's height is different

found two features that ThriftServer can not support:
[SPARK-28636](https://issues.apache.org/jira/browse/SPARK-28636): ThriftServer can not support decimal type with negative scale
[SPARK-28637](https://issues.apache.org/jira/browse/SPARK-28637): ThriftServer can not support interval type

Also, found two inconsistent behavior:
[SPARK-28620](https://issues.apache.org/jira/browse/SPARK-28620): Double type returned for float type in Beeline/JDBC
[SPARK-28619](https://issues.apache.org/jira/browse/SPARK-28619):  The golden result file is different when tested by `bin/spark-sql`

## How was this patch tested?

N/A

Closes #25373 from wangyum/SPARK-28527.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-17 19:12:50 -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
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
Yuming Wang 1941d35d1e [SPARK-28644][SQL] Port HIVE-10646: ColumnValue does not handle NULL_TYPE
## What changes were proposed in this pull request?

This PR port [HIVE-10646](https://issues.apache.org/jira/browse/HIVE-10646) to fix Hive 0.12's JDBC client can not handle `NULL_TYPE`:
```sql
Connected to: Hive (version 3.0.0-SNAPSHOT)
Driver: Hive (version 0.12.0)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Beeline version 0.12.0 by Apache Hive
0: jdbc:hive2://localhost:10000> select null;
org.apache.thrift.transport.TTransportException
	at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:132)
	at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
	at org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:346)
	at org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:423)
	at org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:405)
```

Server log:
```
19/08/07 09:34:07 ERROR TThreadPoolServer: Error occurred during processing of message.
java.lang.NullPointerException
	at org.apache.hive.service.cli.thrift.TRow$TRowStandardScheme.write(TRow.java:388)
	at org.apache.hive.service.cli.thrift.TRow$TRowStandardScheme.write(TRow.java:338)
	at org.apache.hive.service.cli.thrift.TRow.write(TRow.java:288)
	at org.apache.hive.service.cli.thrift.TRowSet$TRowSetStandardScheme.write(TRowSet.java:605)
	at org.apache.hive.service.cli.thrift.TRowSet$TRowSetStandardScheme.write(TRowSet.java:525)
	at org.apache.hive.service.cli.thrift.TRowSet.write(TRowSet.java:455)
	at org.apache.hive.service.cli.thrift.TFetchResultsResp$TFetchResultsRespStandardScheme.write(TFetchResultsResp.java:550)
	at org.apache.hive.service.cli.thrift.TFetchResultsResp$TFetchResultsRespStandardScheme.write(TFetchResultsResp.java:486)
	at org.apache.hive.service.cli.thrift.TFetchResultsResp.write(TFetchResultsResp.java:412)
	at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result$FetchResults_resultStandardScheme.write(TCLIService.java:13192)
	at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result$FetchResults_resultStandardScheme.write(TCLIService.java:13156)
	at org.apache.hive.service.cli.thrift.TCLIService$FetchResults_result.write(TCLIService.java:13107)
	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:58)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:819)
```

## How was this patch tested?

unit tests

Closes #25378 from wangyum/SPARK-28644.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-08 17:28:10 +09:00
Yuming Wang c4acfe7761 [SPARK-28474][SQL] Hive 0.12 JDBC client can not handle binary type
## What changes were proposed in this pull request?

This PR fix Hive 0.12 JDBC client can not handle binary type:
```sql
Connected to: Hive (version 3.0.0-SNAPSHOT)
Driver: Hive (version 0.12.0)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Beeline version 0.12.0 by Apache Hive
0: jdbc:hive2://localhost:10000> SELECT cast('ABC' as binary);
Error: java.lang.ClassCastException: [B incompatible with java.lang.String (state=,code=0)
```

Server log:
```
19/08/07 10:10:04 WARN ThriftCLIService: Error fetching results:
java.lang.RuntimeException: java.lang.ClassCastException: [B incompatible with java.lang.String
	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:83)
	at org.apache.hive.service.cli.session.HiveSessionProxy.access$000(HiveSessionProxy.java:36)
	at org.apache.hive.service.cli.session.HiveSessionProxy$1.run(HiveSessionProxy.java:63)
	at java.security.AccessController.doPrivileged(AccessController.java:770)
	at javax.security.auth.Subject.doAs(Subject.java:422)
	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1746)
	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:59)
	at com.sun.proxy.$Proxy26.fetchResults(Unknown Source)
	at org.apache.hive.service.cli.CLIService.fetchResults(CLIService.java:455)
	at org.apache.hive.service.cli.thrift.ThriftCLIService.FetchResults(ThriftCLIService.java:621)
	at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1553)
	at org.apache.hive.service.cli.thrift.TCLIService$Processor$FetchResults.getResult(TCLIService.java:1538)
	at org.apache.thrift.ProcessFunction.process(ProcessFunction.java:38)
	at org.apache.thrift.TBaseProcessor.process(TBaseProcessor.java:39)
	at org.apache.hive.service.auth.TSetIpAddressProcessor.process(TSetIpAddressProcessor.java:53)
	at org.apache.thrift.server.TThreadPoolServer$WorkerProcess.run(TThreadPoolServer.java:310)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
	at java.lang.Thread.run(Thread.java:819)
Caused by: java.lang.ClassCastException: [B incompatible with java.lang.String
	at org.apache.hive.service.cli.ColumnValue.toTColumnValue(ColumnValue.java:198)
	at org.apache.hive.service.cli.RowBasedSet.addRow(RowBasedSet.java:60)
	at org.apache.hive.service.cli.RowBasedSet.addRow(RowBasedSet.java:32)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.$anonfun$getNextRowSet$1(SparkExecuteStatementOperation.scala:151)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation$$Lambda$1923.000000009113BFE0.apply(Unknown Source)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.withSchedulerPool(SparkExecuteStatementOperation.scala:299)
	at org.apache.spark.sql.hive.thriftserver.SparkExecuteStatementOperation.getNextRowSet(SparkExecuteStatementOperation.scala:113)
	at org.apache.hive.service.cli.operation.OperationManager.getOperationNextRowSet(OperationManager.java:220)
	at org.apache.hive.service.cli.session.HiveSessionImpl.fetchResults(HiveSessionImpl.java:785)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.apache.hive.service.cli.session.HiveSessionProxy.invoke(HiveSessionProxy.java:78)
	... 18 more
```

## How was this patch tested?

unit tests

Closes #25379 from wangyum/SPARK-28474.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-08-08 17:01:25 +09:00
Yuming Wang a59fdc4b57 [SPARK-28472][SQL][TEST] Add test for thriftserver protocol versions
## What changes were proposed in this pull request?

This pr adds a test(`SparkThriftServerProtocolVersionsSuite`) to test different versions of the thrift protocol because we use different logic to handle the `RowSet`:
02c33694c8/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java (L28-L40)

When adding this test cases, found three bugs:
[SPARK-26969](https://issues.apache.org/jira/browse/SPARK-26969): Using ODBC not able to see the data in table when datatype is decimal
[SPARK-28463](https://issues.apache.org/jira/browse/SPARK-28463): Thriftserver throws BigDecimal incompatible with HiveDecimal
[SPARK-28474](https://issues.apache.org/jira/browse/SPARK-28474): Lower JDBC client version(Hive 0.12) cannot read binary type

## How was this patch tested?

N/A

Closes #25228 from wangyum/SPARK-28472.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-07 08:51:58 -07:00
Jungtaek Lim (HeartSaVioR) 128ea37bda [SPARK-28601][CORE][SQL] Use StandardCharsets.UTF_8 instead of "UTF-8" string representation, and get rid of UnsupportedEncodingException
## What changes were proposed in this pull request?

This patch tries to keep consistency whenever UTF-8 charset is needed, as using `StandardCharsets.UTF_8` instead of using "UTF-8". If the String type is needed, `StandardCharsets.UTF_8.name()` is used.

This change also brings the benefit of getting rid of `UnsupportedEncodingException`, as we're providing `Charset` instead of `String` whenever possible.

This also changes some private Catalyst helper methods to operate on encodings as `Charset` objects rather than strings.

## How was this patch tested?

Existing unit tests.

Closes #25335 from HeartSaVioR/SPARK-28601.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-05 20:45:54 -07:00
Yuming Wang efd92993f4 [SPARK-28510][SQL] Implement Spark's own GetFunctionsOperation
## What changes were proposed in this pull request?

This PR implements Spark's own GetFunctionsOperation which mitigates the differences between Spark SQL and Hive UDFs. But our implementation is different from Hive's implementation:
- Our implementation always returns results. Hive only returns results when [(null == catalogName || "".equals(catalogName)) && (null == schemaName || "".equals(schemaName))](https://github.com/apache/hive/blob/rel/release-3.1.1/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java#L101-L119).
- Our implementation pads the `REMARKS` field with the function usage - Hive returns an empty string.
- Our implementation does not support `FUNCTION_TYPE`, but Hive does.

## How was this patch tested?

unit tests

Closes #25252 from wangyum/SPARK-28510.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-08-02 08:50:42 -07:00
Dongjoon Hyun a428f40669 [SPARK-28549][BUILD][CORE][SQL] Use text.StringEscapeUtils instead lang3.StringEscapeUtils
## What changes were proposed in this pull request?

`org.apache.commons.lang3.StringEscapeUtils` was deprecated over two years ago at [LANG-1316](https://issues.apache.org/jira/browse/LANG-1316). There is no bug fixes after that.
```java
/**
 * <p>Escapes and unescapes {code String}s for
 * Java, Java Script, HTML and XML.</p>
 *
 * <p>#ThreadSafe#</p>
 * since 2.0
 * deprecated as of 3.6, use commons-text
 * <a href="https://commons.apache.org/proper/commons-text/javadocs/api-release/org/apache/commons/text/StringEscapeUtils.html">
 * StringEscapeUtils</a> instead
 */
Deprecated
public class StringEscapeUtils {
```

This PR aims to use the latest one from `commons-text` module which has more bug fixes like
[TEXT-100](https://issues.apache.org/jira/browse/TEXT-100), [TEXT-118](https://issues.apache.org/jira/browse/TEXT-118) and [TEXT-120](https://issues.apache.org/jira/browse/TEXT-120) by the following replacement.
```scala
-import org.apache.commons.lang3.StringEscapeUtils
+import org.apache.commons.text.StringEscapeUtils
```

This will add a new dependency to `hadoop-2.7` profile distribution. In `hadoop-3.2` profile, we already have it.
```
+commons-text-1.6.jar
```

## How was this patch tested?

Pass the Jenkins with the existing tests.
- [Hadoop 2.7](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108281)
- [Hadoop 3.2](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/108282)

Closes #25281 from dongjoon-hyun/SPARK-28549.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-07-29 11:45:29 +09:00
Yuming Wang 545c7ee00b [SPARK-28463][SQL] Thriftserver throws BigDecimal incompatible with HiveDecimal
## What changes were proposed in this pull request?

How to reproduce this issue:
```shell
build/sbt clean package -Phive -Phive-thriftserver -Phadoop-3.2
export SPARK_PREPEND_CLASSES=true
sbin/start-thriftserver.sh

[rootspark-3267648 spark]# bin/beeline -u jdbc:hive2://localhost:10000/default -e "select cast(1 as decimal(38, 18));"
Connecting to jdbc:hive2://localhost:10000/default
Connected to: Spark SQL (version 3.0.0-SNAPSHOT)
Driver: Hive JDBC (version 2.3.5)
Transaction isolation: TRANSACTION_REPEATABLE_READ
Error: java.lang.ClassCastException: java.math.BigDecimal incompatible with org.apache.hadoop.hive.common.type.HiveDecimal (state=,code=0)
Closing: 0: jdbc:hive2://localhost:10000/default
```

This pr fix this issue.

## How was this patch tested?

unit tests

Closes #25217 from wangyum/SPARK-28463.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 10:30:01 -07:00
Yuming Wang 6807a82047 [SPARK-28524][SQL] Fix ThriftServerTab lost error message
## What changes were proposed in this pull request?

The ThriftServerTab lost the error message since [SPARK-28260](https://issues.apache.org/jira/browse/SPARK-28260):
![image](https://user-images.githubusercontent.com/5399861/61964309-27755400-b000-11e9-8bc4-b5bb01d2b0e6.png)
![image](https://user-images.githubusercontent.com/5399861/61964588-cf8b1d00-b000-11e9-9583-2f14bdb114a2.png)

This pr fix this issue.

## How was this patch tested?

manual tests
![image](https://user-images.githubusercontent.com/5399861/61965964-11699280-b004-11e9-83e8-688e3ef8727f.png)
![image](https://user-images.githubusercontent.com/5399861/61965940-09115780-b004-11e9-9f1c-fe9bfcb38128.png)

Closes #25263 from wangyum/SPARK-28524.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-26 09:49:58 -07:00
Yuming Wang 045191e610 [SPARK-28293][SQL] Implement Spark's own GetTableTypesOperation
## What changes were proposed in this pull request?

The table type is from Hive now. This will have some issues. For example, we don't support `index_table`, different Hive supports different table types:
Build with Hive 1.2.1:
![image](https://user-images.githubusercontent.com/5399861/60792689-be38b880-a198-11e9-82b8-868992a505e3.png)
Build with Hive 2.3.5:
![image](https://user-images.githubusercontent.com/5399861/60792727-d4467900-a198-11e9-952c-210bb7bb3bed.png)

This pr implement Spark's own `GetTableTypesOperation`.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/60793368-2a67ec00-a19a-11e9-9511-c67483dcc370.png)

Closes #25073 from wangyum/SPARK-28293.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-24 11:27:30 -07:00
朱夷 8ecbb67b3b [SPARK-28311][SQL] Fix STS OpenSession failed return wrong origin PROTOCOL_VERSION
## What changes were proposed in this pull request?

For Thrift server, It's downward compatible. Such as if a PROTOCOL_VERSION_V7 client connect to a  PROTOCOL_VERSION_V8 server, when OpenSession, server will change his response's protocol version to min of (client and server).
`TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION,`
 `       req.getClient_protocol());`
then set it to OpenSession's response.

But if OpenSession failed , it won't execute behavior of reset response's protocol_version.
Then it will return server's origin protocol version.
Finally client will get en error as below:
![image](https://user-images.githubusercontent.com/46485123/61023164-54f4b780-a3db-11e9-8c49-60217b36287b.png)
Since we write a wrong database,, OpenSession failed, right protocol version haven't been rest.

## How was this patch tested?

Since I really don't know how to write unit test about this, so I build a jar with this PR,and retry the error above, then it will return a reasonable Error of DB not found :
![image](https://user-images.githubusercontent.com/46485123/61023923-67242500-a3de-11e9-8e98-8f391a038480.png)

Closes #25083 from AngersZhuuuu/SPARK-28311.

Authored-by: 朱夷 <zhuyi01@corp.netease.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-07-15 09:35:56 -05:00
Yuming Wang 687dd4eb55 [SPARK-28260][SQL] Add CLOSED state to ExecutionState
## What changes were proposed in this pull request?

The `ThriftServerTab` displays a FINISHED state when the operation finishes execution, but quite often it still takes a lot of time to fetch the results. OperationState has state CLOSED for when after the iterator is closed. This PR add CLOSED state to ExecutionState, and override the `close()` in SparkExecuteStatementOperation, SparkGetColumnsOperation, SparkGetSchemasOperation and SparkGetTablesOperation.

## How was this patch tested?

manual tests
1. Add `Thread.sleep(10000)` before [SparkExecuteStatementOperation.scala#L112](b2e7677f4d/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala (L112))
2. Switch to `ThriftServerTab`:
![image](https://user-images.githubusercontent.com/5399861/60809590-9dcf2500-a1bd-11e9-826e-33729bb97daf.png)
3. After a while:
![image](https://user-images.githubusercontent.com/5399861/60809719-e850a180-a1bd-11e9-9a6a-546146e626ab.png)

Closes #25062 from wangyum/SPARK-28260.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-12 10:31:28 -07:00
wangguangxin.cn 42b80ae128 [SPARK-28257][SQL] Use ConfigEntry for hardcoded configs in SQL
## What changes were proposed in this pull request?

There are some hardcoded configs, using config entry to replace them.

## How was this patch tested?

Existing UT

Closes #25059 from WangGuangxin/ConfigEntry.

Authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-11 22:36:07 -07:00
Dongjoon Hyun a6506f0c8a [SPARK-28290][CORE][SQL] Use SslContextFactory.Server instead of SslContextFactory
## What changes were proposed in this pull request?

`SslContextFactory` is deprecated at Jetty 9.4 and we are using `9.4.18.v20190429`. This PR aims to replace it with `SslContextFactory.Server`.
- https://www.eclipse.org/jetty/javadoc/9.4.19.v20190610/org/eclipse/jetty/util/ssl/SslContextFactory.html
- https://www.eclipse.org/jetty/javadoc/9.3.24.v20180605/org/eclipse/jetty/util/ssl/SslContextFactory.html

```
[WARNING] /Users/dhyun/APACHE/spark/core/src/main/scala/org/apache/spark/SSLOptions.scala:71:
constructor SslContextFactory in class SslContextFactory is deprecated:
see corresponding Javadoc for more information.
[WARNING]       val sslContextFactory = new SslContextFactory()
[WARNING]                               ^
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #25067 from dongjoon-hyun/SPARK-28290.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 09:05:56 -07:00
Yuming Wang e299f62ec5 [SPARK-28241][SQL] Show metadata operations on ThriftServerTab
## What changes were proposed in this pull request?

This pr add support show metadata operations on ThriftServerTab.

## How was this patch tested?

manual tests:
![image](https://user-images.githubusercontent.com/5399861/60579741-4cd2c180-9db6-11e9-822a-0433be509b67.png)

Closes #25043 from wangyum/SPARK-28241.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: herman <herman@databricks.com>
2019-07-05 12:24:50 +02:00
Yuming Wang ea0303063f [SPARK-28167][SQL] Show global temporary view in database tool
## What changes were proposed in this pull request?

This pr add support show global temporary view and local temporary view in database tool.

TODO: Database tools should support show temporary views because it's schema is null.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/60392266-a5455d00-9b31-11e9-92c8-88a8e6c2aec3.png)

Closes #24972 from wangyum/SPARK-28167.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-07-03 00:01:05 -07:00
Yuming Wang e0e2144ca6 [SPARK-28184][SQL][TEST] Avoid creating new sessions in SparkMetadataOperationSuite
## What changes were proposed in this pull request?

To make the #24972 change smaller. This pr improves `SparkMetadataOperationSuite` to avoid creating new sessions when getSchemas/getTables/getColumns.

## How was this patch tested?

N/A

Closes #24985 from wangyum/SPARK-28184.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-29 18:29:37 -07:00
Yuming Wang 870f972dcc [SPARK-28104][SQL] Implement Spark's own GetColumnsOperation
## What changes were proposed in this pull request?

[SPARK-24196](https://issues.apache.org/jira/browse/SPARK-24196) and [SPARK-24570](https://issues.apache.org/jira/browse/SPARK-24570) implemented Spark's own `GetSchemasOperation` and `GetTablesOperation`. This pr implements Spark's own `GetColumnsOperation`.

## How was this patch tested?

unit tests and manual tests:
![image](https://user-images.githubusercontent.com/5399861/59745367-3a7d6180-92a7-11e9-862d-96bc494c5f00.png)

Closes #24906 from wangyum/SPARK-28104.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-22 09:15:07 -07: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 3f102a8229 [SPARK-27749][SQL] hadoop-3.2 support hive-thriftserver
## What changes were proposed in this pull request?

This PR mainly makes the following changes to make `hadoop-3.2` support `sql/hive-thriftserver`:
1. Upgrade [`TCLIService.thrift`](https://github.com/apache/hive/blob/rel/release-2.3.5/service-rpc/if/TCLIService.thrift) and related code to Hive 2.3.5 because of [HIVE-12442](https://issues.apache.org/jira/browse/HIVE-12442)(Note that we only migrate code without adding features, such as [HIVE-4924](https://issues.apache.org/jira/browse/HIVE-4924) and [HIVE-15473](https://issues.apache.org/jira/browse/HIVE-15473)).
2. Use slf4j as logging facade because of [HIVE-12237](https://issues.apache.org/jira/browse/HIVE-12237).
3. Port [HIVE-13169](https://issues.apache.org/jira/browse/HIVE-13169) to compatible with Hive 2.3.

## How was this patch tested?

Exiting test

Closes #24628 from wangyum/SPARK-27749.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-06-05 08:40:05 -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
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 67582fdbfe [SPARK-27737][SQL][FOLLOW-UP] Move sql/hive-thriftserver/v2.3.4 to sql/hive-thriftserver/v2.3.5
## What changes were proposed in this pull request?

This pr moves `sql/hive-thriftserver/v2.3.4` to `sql/hive-thriftserver/v2.3.5` based on ([comment](https://github.com/apache/spark/pull/24628#issuecomment-496459258)).

## How was this patch tested?

N/A

Closes #24728 from wangyum/SPARK-27737-thriftserver.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-28 16:08:58 -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 02c33694c8 [SPARK-27354][SQL] Move incompatible code from the hive-thriftserver module to sql/hive-thriftserver/v1.2.1
## What changes were proposed in this pull request?

When we upgraded the built-in Hive to 2.3.4, the current `hive-thriftserver` module is not compatible, such as these Hive changes:
1. [HIVE-12442](https://issues.apache.org/jira/browse/HIVE-12442) HiveServer2: Refactor/repackage HiveServer2's Thrift code so that it can be used in the tasks
2. [HIVE-12237](https://issues.apache.org/jira/browse/HIVE-12237) Use slf4j as logging facade
3. [HIVE-13169](https://issues.apache.org/jira/browse/HIVE-13169) HiveServer2: Support delegation token based connection when using http transport

So this PR moves the incompatible code to `sql/hive-thriftserver/v1.2.1` and copies it to `sql/hive-thriftserver/v2.3.4` for the next code review.

## How was this patch tested?

manual tests:
```
diff -urNa sql/hive-thriftserver/v1.2.1 sql/hive-thriftserver/v2.3.4
```

Closes #24282 from wangyum/SPARK-27354.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2019-05-15 14:52:08 -07:00
Yuming Wang 3ea44e52e7 [SPARK-27639][SQL] InMemoryTableScan shows the table name on UI if possible
## What changes were proposed in this pull request?
<img src="https://user-images.githubusercontent.com/5399861/57213799-7bccf100-701a-11e9-9872-d90b4a185dc6.png" width="200">

It only shows `InMemoryTableScan` when scanning InMemoryTable.
When there are many InMemoryTables, it is difficult to distinguish which one is what we are looking for. This PR show the table name when scanning InMemoryTable.

## How was this patch tested?

unit tests and manual tests

After this PR:
<img src="https://user-images.githubusercontent.com/5399861/57269120-d3219e80-70b8-11e9-9e56-1b5d4c071660.png" width="200">

Closes #24534 from wangyum/SPARK-27639.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-07 21:00:13 -07:00
Sean Owen 4ec7f631aa [SPARK-27404][CORE][SQL][STREAMING][YARN] Fix build warnings for 3.0: postfixOps edition
## What changes were proposed in this pull request?

Fix build warnings -- see some details below.

But mostly, remove use of postfix syntax where it causes warnings without the `scala.language.postfixOps` import. This is mostly in expressions like "120000 milliseconds". Which, I'd like to simplify to things like "2.minutes" anyway.

## How was this patch tested?

Existing tests.

Closes #24314 from srowen/SPARK-27404.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-11 13:43:44 -05:00
cxzl25 6450c5948a [SPARK-26992][STS] Fix STS scheduler pool correct delivery
## What changes were proposed in this pull request?

The user sets the value of spark.sql.thriftserver.scheduler.pool.
Spark thrift server saves this value in the LocalProperty of threadlocal type, but does not clean up after running, causing other sessions to run in the previously set pool name.

## How was this patch tested?

manual tests

Closes #23895 from cxzl25/thrift_server_scheduler_pool_pollute.

Lead-authored-by: cxzl25 <cxzl25@users.noreply.github.com>
Co-authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-06 17:14:29 -05:00
Sean Owen d4420b455a [SPARK-27323][CORE][SQL][STREAMING] Use Single-Abstract-Method support in Scala 2.12 to simplify code
## What changes were proposed in this pull request?

Use Single Abstract Method syntax where possible (and minor related cleanup). Comments below. No logic should change here.

## How was this patch tested?

Existing tests.

Closes #24241 from srowen/SPARK-27323.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-04-02 07:37:05 -07:00
zhoukang 43bf4ae641 [SPARK-26914][SQL] Fix scheduler pool may be unpredictable when we only want to use default pool and do not set spark.scheduler.pool for the session
## What changes were proposed in this pull request?

When using fair scheduler mode for thrift server, we may have unpredictable result.
```
val pool = sessionToActivePool.get(parentSession.getSessionHandle)
if (pool != null) {
   sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool)
}
```
The cause is we use thread pool to execute queries for thriftserver, and when we call setLocalProperty we may have unpredictab behavior.

```
/**
   * Set a local property that affects jobs submitted from this thread, such as the Spark fair
   * scheduler pool. User-defined properties may also be set here. These properties are propagated
   * through to worker tasks and can be accessed there via
   * [[org.apache.spark.TaskContext#getLocalProperty]].
   *
   * These properties are inherited by child threads spawned from this thread. This
   * may have unexpected consequences when working with thread pools. The standard java
   * implementation of thread pools have worker threads spawn other worker threads.
   * As a result, local properties may propagate unpredictably.
   */
  def setLocalProperty(key: String, value: String) {
    if (value == null) {
      localProperties.get.remove(key)
    } else {
      localProperties.get.setProperty(key, value)
    }
  }
```

I post an example on https://jira.apache.org/jira/browse/SPARK-26914 .

## How was this patch tested?
UT

Closes #23826 from caneGuy/zhoukang/fix-scheduler-error.

Authored-by: zhoukang <zhoukang199191@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-28 09:24:16 -05:00
zuotingbing 3f9247de1e [SPARK-27010][SQL] Find out the actual port number when hive.server2.thrift.port=0
## What changes were proposed in this pull request?
Currently, if we set hive.server2.thrift.port=0, it hard to find out the actual port number which one we should use when using beeline to connect.

before:
![2019-02-28_170942](https://user-images.githubusercontent.com/24823338/53557240-779ad800-3b80-11e9-9567-175f28aa61da.png)

after:
![2019-02-28_170904](https://user-images.githubusercontent.com/24823338/53557255-7f5a7c80-3b80-11e9-8ba6-9764c03e5407.png)

use beeline to connect success:
![2019-02-28_170844](https://user-images.githubusercontent.com/24823338/53557267-85e8f400-3b80-11e9-90a5-f7f53a51cc32.png)

## How was this patch tested?
 manual tests

Closes #23917 from zuotingbing/SPARK-27010.

Authored-by: zuotingbing <zuo.tingbing9@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-03-12 13:38:41 -05:00