Commit graph

25452 commits

Author SHA1 Message Date
Ivan Gozali 00347a3c78 [SPARK-28762][CORE] Read JAR main class if JAR is not located in local file system
### What changes were proposed in this pull request?

JIRA: https://issues.apache.org/jira/browse/SPARK-28762

TL;DR: Automatically read the `Main-Class` from a JAR's manifest even if the JAR isn't in the local file system (i.e. in S3 or HDFS).

### Why are the changes needed?

When deploying a fat JAR (e.g. using `sbt-assembly`) to S3/HDFS, users might choose to include the main class for the JAR in its manifest. This change allows the user to `spark-submit` the JAR without having to specify the main class again via the `--class` argument.

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

Yes. Previously, if the primary resource is a JAR and isn't located in the local file system, it will fail with the error:
```
$ spark-submit s3a://nonexistent.jar
Exception in thread "main" org.apache.spark.SparkException: Cannot load main class from JAR s3a://nonexistent.jar with URI s3a. Please specify a class through --class.
    ...
```

With this PR, the main class will be read from the manifest, assuming the classpath contains the appropriate JAR to read the file system.

### How was this patch tested?

Added some tests in `core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala`.

Closes #25910 from igozali/SPARK-28762.

Authored-by: Ivan Gozali <gozaliivan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-17 14:36:01 -07:00
igor.calabria 78bdcfade1 [SPARK-27812][K8S] Bump K8S client version to 4.6.1
### What changes were proposed in this pull request?

Updated kubernetes client.

### Why are the changes needed?

https://issues.apache.org/jira/browse/SPARK-27812
https://issues.apache.org/jira/browse/SPARK-27927

We need this fix https://github.com/fabric8io/kubernetes-client/pull/1768 that was released on version 4.6 of the client. The root cause of the problem is better explained in https://github.com/apache/spark/pull/25785

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

Nope, it should be transparent to users

### How was this patch tested?

This patch was tested manually using a simple pyspark job

```python
from pyspark.sql import SparkSession

if __name__ == '__main__':
    spark = SparkSession.builder.getOrCreate()
```

The expected behaviour of this "job" is that both python's and jvm's process exit automatically after the main runs. This is the case for spark versions <= 2.4. On version 2.4.3, the jvm process hangs because there's a non daemon thread running

```
"OkHttp WebSocket https://10.96.0.1/..." #121 prio=5 os_prio=0 tid=0x00007fb27c005800 nid=0x24b waiting on condition [0x00007fb300847000]
"OkHttp WebSocket https://10.96.0.1/..." #117 prio=5 os_prio=0 tid=0x00007fb28c004000 nid=0x247 waiting on condition [0x00007fb300e4b000]
```
This is caused by a bug on `kubernetes-client` library, which is fixed on the version that we are upgrading to.

When the mentioned job is run with this patch applied, the behaviour from spark <= 2.4.3 is restored and both processes terminate successfully

Closes #26093 from igorcalabria/k8s-client-update.

Authored-by: igor.calabria <igor.calabria@ubee.in>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-17 12:23:24 -07:00
Jungtaek Lim (HeartSaVioR) 100fc58da5 [SPARK-28869][CORE] Roll over event log files
### What changes were proposed in this pull request?

This patch is a part of [SPARK-28594](https://issues.apache.org/jira/browse/SPARK-28594) and design doc for SPARK-28594 is linked here: https://docs.google.com/document/d/12bdCC4nA58uveRxpeo8k7kGOI2NRTXmXyBOweSi4YcY/edit?usp=sharing

This patch proposes adding new feature to event logging, rolling event log files via configured file size.

Previously event logging is done with single file and related codebase (`EventLoggingListener`/`FsHistoryProvider`) is tightly coupled with it. This patch adds layer on both reader (`EventLogFileReader`) and writer (`EventLogFileWriter`) to decouple implementation details between "handling events" and "how to read/write events from/to file".

This patch adds two properties, `spark.eventLog.rollLog` and `spark.eventLog.rollLog.maxFileSize` which provides configurable behavior of rolling log. The feature is disabled by default, as we only expect huge event log for huge/long-running application. For other cases single event log file would be sufficient and still simpler.

### Why are the changes needed?

This is a part of SPARK-28594 which addresses event log growing infinitely for long-running application.

This patch itself also provides some option for the situation where event log file gets huge and consume their storage. End users may give up replaying their events and want to delete the event log file, but given application is still running and writing the file, it's not safe to delete the file. End users will be able to delete some of old files after applying rolling over event log.

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

No, as the new feature is turned off by default.

### How was this patch tested?

Added unit tests, as well as basic manual tests.

Basic manual tests - ran SHS, ran structured streaming query with roll event log enabled, verified split files are generated as well as SHS can load these files, with handling app status as incomplete/complete.

Closes #25670 from HeartSaVioR/SPARK-28869.

Lead-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Co-authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-17 11:15:25 -07:00
Marcelo Vanzin 2f0a38cb50 [SPARK-29398][CORE] Support dedicated thread pools for RPC endpoints
The current RPC backend in Spark supports single- and multi-threaded
message delivery to endpoints, but they all share the same underlying
thread pool. So an RPC endpoint that blocks a dispatcher thread can
negatively affect other endpoints.

This can be more pronounced with configurations that limit the number
of RPC dispatch threads based on configuration and / or running
environment. And exposing the RPC layer to other code (for example
with something like SPARK-29396) could make it easy to affect normal
Spark operation with a badly written RPC handler.

This change adds a new RPC endpoint type that tells the RPC env to
create dedicated dispatch threads, so that those effects are minimised.
Other endpoints will still need CPU to process their messages, but
they won't be able to actively block the dispatch thread of these
isolated endpoints.

As part of the change, I've changed the most important Spark endpoints
(the driver, executor and block manager endpoints) to be isolated from
others. This means a couple of extra threads are created on the driver
and executor for these endpoints.

Tested with existing unit tests, which hammer the RPC system extensively,
and also by running applications on a cluster (with a prototype of
SPARK-29396).

Closes #26059 from vanzin/SPARK-29398.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-10-17 13:14:32 -05:00
maruilei f800fa3831 [SPARK-29436][K8S] Support executor for selecting scheduler through scheduler name in the case of k8s multi-scheduler scenario
### What changes were proposed in this pull request?

Support executor for selecting scheduler through scheduler name in the case of k8s multi-scheduler scenario.

### Why are the changes needed?

If there is no such function, spark can not support the case of k8s multi-scheduler scenario.

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

Yes, users can add scheduler name through configuration.

### How was this patch tested?

Manually tested with spark + k8s cluster

Closes #26088 from merrily01/SPARK-29436.

Authored-by: maruilei <maruilei@jd.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-17 07:24:13 -07:00
Jiajia Li dc0bc7a6eb [MINOR][DOCS] Fix some typos
### What changes were proposed in this pull request?

This PR proposes a few typos:
1. Sparks => Spark's
2. parallize => parallelize
3. doesnt => doesn't

Closes #26140 from plusplusjiajia/fix-typos.

Authored-by: Jiajia Li <jiajia.li@intel.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-17 07:22:01 -07:00
Kent Yao 4b902d3b45 [SPARK-29491][SQL] Add bit_count function support
### What changes were proposed in this pull request?

BIT_COUNT(N) - Returns the number of bits that are set in the argument N as an unsigned 64-bit integer, or NULL if the argument is NULL

### Why are the changes needed?

Supported by MySQL,Microsoft SQL Server ,etc.

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

add a built-in function
### How was this patch tested?

add uts

Closes #26139 from yaooqinn/SPARK-29491.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-17 20:22:38 +08:00
Yuanjian Li 239ee3f561 [SPARK-9853][CORE] Optimize shuffle fetch of continuous partition IDs
This PR takes over #19788. After we split the shuffle fetch protocol from `OpenBlock` in #24565, this optimization can be extended in the new shuffle protocol. Credit to yucai, closes #19788.

### What changes were proposed in this pull request?
This PR adds the support for continuous shuffle block fetching in batch:

- Shuffle client changes:
    - Add new feature tag `spark.shuffle.fetchContinuousBlocksInBatch`, implement the decision logic in `BlockStoreShuffleReader`.
    - Merge the continuous shuffle block ids in batch if needed in ShuffleBlockFetcherIterator.
- Shuffle server changes:
    - Add support in `ExternalBlockHandler` for the external shuffle service side.
    - Make `ShuffleBlockResolver.getBlockData` accept getting block data by range.
- Protocol changes:
    - Add new block id type `ShuffleBlockBatchId` represent continuous shuffle block ids.
    - Extend `FetchShuffleBlocks` and `OneForOneBlockFetcher`.
    - After the new shuffle fetch protocol completed in #24565, the backward compatibility for external shuffle service can be controlled by `spark.shuffle.useOldFetchProtocol`.

### Why are the changes needed?
In adaptive execution, one reducer may fetch multiple continuous shuffle blocks from one map output file. However, as the original approach, each reducer needs to fetch those 10 reducer blocks one by one. This way needs many IO and impacts performance. This PR is to support fetching those continuous shuffle blocks in one IO (batch way). See below example:

The shuffle block is stored like below:
![image](https://user-images.githubusercontent.com/2989575/51654634-c37fbd80-1fd3-11e9-935e-5652863676c3.png)
The ShuffleId format is s"shuffle_$shuffleId_$mapId_$reduceId", referring to BlockId.scala.

In adaptive execution, one reducer may want to read output for reducer 5 to 14, whose block Ids are from shuffle_0_x_5 to shuffle_0_x_14.
Before this PR, Spark needs 10 disk IOs + 10 network IOs for each output file.
After this PR, Spark only needs 1 disk IO and 1 network IO. This way can reduce IO dramatically.

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

### How was this patch tested?
Add new UT.
Integrate test with setting `spark.sql.adaptive.enabled=true`.

Closes #26040 from xuanyuanking/SPARK-9853.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: yucai <yyu1@ebay.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-17 14:47:56 +08: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
Fokko Driesprong 8eb8f7478c [SPARK-29483][BUILD] Bump Jackson to 2.10.0
### What changes were proposed in this pull request?

Release blog: https://medium.com/cowtowncoder/jackson-2-10-features-cd880674d8a2

Fixes the following CVE's:
https://www.cvedetails.com/cve/CVE-2019-16942/
https://www.cvedetails.com/cve/CVE-2019-16943/

Looking back, there were 3 major goals for this minor release:

- Resolve the growing problem of “endless CVE patches”, a stream of fixes for reported CVEs related to “Polymorphic Deserialization” problem (described in “On Jackson CVEs… ”) that resulted in security tools forcing Jackson upgrades. 2.10 now includes “Safe Default Typing” that is hoped to resolve this problem.
- Evolve 2.x API towards 3.0, based on changes that were done in master, within limits of 2.x API backwards-compatibility requirements.
- Add JDK support for versions beyond Java 8: specifically add“module-info.class” for JDK9+, defining proper module definitions for Jackson components

Full changelog: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.10

Improved Scala 2.13 support: https://github.com/FasterXML/jackson/wiki/Jackson-Release-2.10#scala

### Why are the changes needed?

Patches CVE's reported by the vulnerability scanner.

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

No

### How was this patch tested?

Ran `mvn clean install -DskipTests` locally.

Closes #26131 from Fokko/SPARK-29483.

Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-16 15:38:54 -07:00
Kent Yao 6d4cc7b855 [SPARK-27880][SQL] Add bool_and for every and bool_or for any as function aliases
### What changes were proposed in this pull request?

bool_or(x) <=> any/some(x) <=> max(x)
bool_and(x) <=> every(x) <=> min(x)
Args:
  x: boolean
### Why are the changes needed?

PostgreSQL, Presto and Vertica, etc also support this feature:
### Does this PR introduce any user-facing change?

add new functions support

### How was this patch tested?

add ut

Closes #26126 from yaooqinn/SPARK-27880.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-16 22:43:47 +08:00
Maxim Gekk d11cbf2e36 [SPARK-29364][SQL] Return an interval from date subtract according to SQL standard
### What changes were proposed in this pull request?
Proposed new expression `SubtractDates` which is used in `date1` - `date2`. It has the `INTERVAL` type, and returns the interval from `date1` (inclusive) and `date2` (exclusive). For example:
```sql
> select date'tomorrow' - date'yesterday';
interval 2 days
```

Closes #26034

### Why are the changes needed?
- To conform the SQL standard which states the result type of `date operand 1` - `date operand 2` must be the interval type. See [4.5.3  Operations involving datetimes and intervals](http://www.contrib.andrew.cmu.edu/~shadow/sql/sql1992.txt).
- Improve Spark SQL UX and allow mixing date and timestamp in subtractions. For example: `select timestamp'now' + (date'2019-10-01' - date'2019-09-15')`

### Does this PR introduce any user-facing change?
Before the query below returns number of days:
```sql
spark-sql> select date'2019-10-05' - date'2018-09-01';
399
```
After it returns an interval:
```sql
spark-sql> select date'2019-10-05' - date'2018-09-01';
interval 1 years 1 months 4 days
```

### How was this patch tested?
- by new tests in `DateExpressionsSuite` and `TypeCoercionSuite`.
- by existing tests in `date.sql`

Closes #26112 from MaxGekk/date-subtract.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-16 06:26:01 -07:00
Jose Torres 5a482e7209 [SPARK-29468][SQL] Change Literal.sql to be correct for floats
### What changes were proposed in this pull request?
Change Literal.sql to output CAST('fpValue' AS FLOAT) instead of CAST(fpValue AS FLOAT) as the SQL for a floating point literal.

### Why are the changes needed?
The old version doesn't work for very small floating point numbers; the value will fail to parse if it doesn't fit in a DECIMAL(38).

This doesn't apply to doubles because they have special literal syntax.

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

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

Closes #26114 from jose-torres/fpliteral.

Authored-by: Jose Torres <joseph.torres@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-16 21:06:13 +08:00
zhengruifeng 9dacdd38b3 [SPARK-23578][ML][PYSPARK] Binarizer support multi-column
### What changes were proposed in this pull request?
Binarizer support multi-column by extending `HasInputCols`/`HasOutputCols`/`HasThreshold`/`HasThresholds`

### Why are the changes needed?
similar algs in `ml.feature` already support multi-column, like `Bucketizer`/`StringIndexer`/`QuantileDiscretizer`

### Does this PR introduce any user-facing change?
yes, add setter/getter of `thresholds`/`inputCols`/`outputCols`

### How was this patch tested?
added suites

Closes #26064 from zhengruifeng/binarizer_multicols.

Authored-by: zhengruifeng <ruifengz@foxmail.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2019-10-16 18:32:07 +08: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
prasha2 57edb42582 [SPARK-27259][CORE] Allow setting -1 as length for FileBlock
### What changes were proposed in this pull request?

This PR aims to update the validation check on `length` from `length >= 0` to `length >= -1` in order to allow set `-1` to keep the default value.

### Why are the changes needed?

At Apache Spark 2.2.0, [SPARK-18702](https://github.com/apache/spark/pull/16133/files#diff-2c5519b1cf4308d77d6f12212971544fR27-R38) adds `class FileBlock` with the default `length` value, `-1`, initially.

There is no way to set `filePath` only while keeping `length` is `-1`.
```scala
  def set(filePath: String, startOffset: Long, length: Long): Unit = {
     require(filePath != null, "filePath cannot be null")
     require(startOffset >= 0, s"startOffset ($startOffset) cannot be negative")
     require(length >= 0, s"length ($length) cannot be negative")
     inputBlock.set(new FileBlock(UTF8String.fromString(filePath), startOffset, length))
   }
```

For compressed files (like GZ), the size of split can be set to -1. This was allowed till Spark 2.1 but regressed starting with spark 2.2.x. Please note that split length of -1 also means the length was unknown - a valid scenario. Thus, split length of -1 should be acceptable like pre Spark 2.2.

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

No

### How was this patch tested?

This is updating the corner case on the requirement check. Manually check the code.

Closes #26123 from praneetsharma/fix-SPARK-27259.

Authored-by: prasha2 <prasha22@mail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-15 22:22:37 -07:00
Liang-Chi Hsieh 93e71e60e6 [SPARK-29469][SHUFFLE] Avoid retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed
### What changes were proposed in this pull request?

When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE. This proposes to skip retries by RetryingBlockFetcher when ExternalBlockStoreClient is closed.

### Why are the changes needed?

When ExternalBlockStoreClient was closed, retries from RetryingBlockFetcher will cause NPE:

```
2019-10-14 20:06:16 ERROR RetryingBlockFetcher:143 - Exception while beginning fetch of 2 outstanding blocks (after 3 retries)
java.lang.NullPointerException
at org.apache.spark.network.shuffle.ExternalShuffleClient.lambda$fetchBlocks$0(ExternalShuffleClient.java:100)
at org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:141)
at org.apache.spark.network.shuffle.RetryingBlockFetcher.lambda$initiateRetry$0(RetryingBlockFetcher.java:169)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138)
```

It was happened after BlockManager and ExternalBlockStoreClient was closed due to previous errors. In this cases, RetryingBlockFetcher does not need to retry. This NPE is harmless for job execution, but is a source of misleading when looking at log. Especially for end-users.

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

No

### How was this patch tested?

Existing tests.

Closes #26115 from viirya/SPARK-29469.

Lead-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Co-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-16 13:11:07 +08:00
Yuming Wang e00344edc1 [SPARK-29423][SS] lazily initialize StreamingQueryManager in SessionState
### What changes were proposed in this pull request?

This PR makes `SessionState` lazily initialize `StreamingQueryManager` to avoid constructing  `StreamingQueryManager` for each session when connecting to ThriftServer.

### Why are the changes needed?

Reduce memory usage.

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

No.

### How was this patch tested?
manual test
1. Start thriftserver:
```
build/sbt clean package -Phive -Phadoop-3.2 -Phive-thriftserver
export SPARK_PREPEND_CLASSES=true
sbin/start-thriftserver.sh
```
2. Open a session:
```
bin/beeline -u jdbc:hive2://localhost:10000
```
3. Check `StreamingQueryManager` instance:
```
jcmd | grep HiveThriftServer2 | awk -F ' ' '{print $1}' | xargs jmap -histo | grep StreamingQueryManager
```

**Before this PR**:
```
[rootspark-3267648 spark]# jcmd | grep HiveThriftServer2 | awk -F ' ' '{print $1}' | xargs jmap -histo | grep StreamingQueryManager
1954:             2             96  org.apache.spark.sql.streaming.StreamingQueryManager
```

**After this PR**:
```
[rootspark-3267648 spark]# jcmd | grep HiveThriftServer2 | awk -F ' ' '{print $1}' | xargs jmap -histo | grep StreamingQueryManager
[rootspark-3267648 spark]#
```

Closes #26089 from wangyum/SPARK-29423.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-15 21:08:15 -07:00
Wenchen Fan 51f10ed90f [SPARK-28560][SQL][FOLLOWUP] code cleanup for local shuffle reader
### What changes were proposed in this pull request?

A followup of https://github.com/apache/spark/pull/25295

This PR proposes a few code cleanups:
1. rename the special `getMapSizesByExecutorId` to `getMapSizesByMapIndex`
2. rename the parameter `mapId` to `mapIndex` as that's really a mapper index.
3. `BlockStoreShuffleReader` should take `blocksByAddress` directly instead of a map id.
4. rename `getMapReader` to `getReaderForOneMapper` to be more clearer.

### Why are the changes needed?

make code easier to understand

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

no

### How was this patch tested?

existing tests

Closes #26128 from cloud-fan/followup.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-16 11:19:16 +08:00
Jeff Evans 95de93b24e [SPARK-24540][SQL] Support for multiple character delimiter in Spark CSV read
Updating univocity-parsers version to 2.8.3, which adds support for multiple character delimiters

Moving univocity-parsers version to spark-parent pom dependencyManagement section

Adding new utility method to build multi-char delimiter string, which delegates to existing one

Adding tests for multiple character delimited CSV

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

Adds support for parsing CSV data using multiple-character delimiters.  Existing logic for converting the input delimiter string to characters was kept and invoked in a loop.  Project dependencies were updated to remove redundant declaration of `univocity-parsers` version, and also to change that version to the latest.

### Why are the changes needed?

It is quite common for people to have delimited data, where the delimiter is not a single character, but rather a sequence of characters.  Currently, it is difficult to handle such data in Spark (typically needs pre-processing).

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

Yes. Specifying the "delimiter" option for the DataFrame read, and providing more than one character, will no longer result in an exception.  Instead, it will be converted as before and passed to the underlying library (Univocity), which has accepted multiple character delimiters since 2.8.0.

### How was this patch tested?

The `CSVSuite` tests were confirmed passing (including new methods), and `sbt` tests for `sql` were executed.

Closes #26027 from jeff303/SPARK-24540.

Authored-by: Jeff Evans <jeffrey.wayne.evans@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-15 15:44:51 -05:00
Kent Yao 02c5b4f763 [SPARK-28947][K8S] Status logging not happens at an interval for liveness
### What changes were proposed in this pull request?

This pr invoke the start method of `LoggingPodStatusWatcherImpl` for status logging at intervals.

### Why are the changes needed?

This pr invoke the start method of `LoggingPodStatusWatcherImpl` is declared but never called

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

no

### How was this patch tested?

manually test

Closes #25648 from yaooqinn/SPARK-28947.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-10-15 12:34:39 -07:00
Dongjoon Hyun 39d53d3e74 [SPARK-29470][BUILD] Update plugins to latest versions
### What changes were proposed in this pull request?

This PR updates plugins to latest versions.

### Why are the changes needed?

This brings bug fixes like the following.
- https://issues.apache.org/jira/projects/MCOMPILER/versions/12343484 (maven-compiler-plugin)
- https://issues.apache.org/jira/projects/MJAVADOC/versions/12345060 (maven-javadoc-plugin)
- https://issues.apache.org/jira/projects/MCHECKSTYLE/versions/12342397 (maven-checkstyle-plugin)
- https://checkstyle.sourceforge.io/releasenotes.html#Release_8.25 (checkstyle)
- https://checkstyle.sourceforge.io/releasenotes.html#Release_8.24 (checkstyle)

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

No.

### How was this patch tested?

Pass the Jenkins building and testing with the existing code.

Closes #26117 from dongjoon-hyun/SPARK-29470.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-15 11:55:52 -07:00
Liang-Chi Hsieh 4ecbdbb6a7 [SPARK-29182][CORE] Cache preferred locations of checkpointed RDD
### What changes were proposed in this pull request?

This proposes to add a Spark config to control the caching behavior of ReliableCheckpointRDD.getPreferredLocations. If it is enabled, getPreferredLocations will only compute preferred locations once and cache it for later usage.

The drawback of caching the preferred locations is that when the cached locations are outdated, and lose data locality. It was documented in config document. To mitigate this, this patch also adds a config to set up expire time (default is 60 mins) for the cache. If time expires, the cache will be invalid and it needs to query updated location info.

This adds a test case. Looks like the most suitable test suite is CheckpointCompressionSuite. So this renames CheckpointCompressionSuite to CheckpointStorageSuite and put the test case into.

### Why are the changes needed?

One Spark job in our cluster fits many ALS models in parallel. The fitting goes well, but in next when we union all factors, the union operation is very slow.

By looking into the driver stack dump, looks like the driver spends a lot of time on computing preferred locations. As we checkpoint training data before fitting ALS, the time is spent on ReliableCheckpointRDD.getPreferredLocations. In this method, it will call DFS interface to query file status and block locations. As we have big number of partitions derived from the checkpointed RDD,  the union will spend a lot of time on querying the same information.

It reduces the time on huge union from few hours to dozens of minutes.

This issue is not limited to ALS so this change is not specified to ALS. Actually it is common usage to checkpoint data in Spark, to increase reliability and cut RDD linage. Spark operations on the checkpointed data, will be beneficial.

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

Yes. This adds a Spark config users can use to control the cache behavior of preferred locations of checkpointed RDD.

### How was this patch tested?

Unit test added and manual test on development cluster.

Closes #25856 from viirya/cache-checkpoint-preferredloc.

Authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Liang-Chi Hsieh <liangchi@uber.com>
2019-10-15 10:45:18 -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
Yifei Huang 2e28622d8a [SPARK-28211][CORE][SHUFFLE] Propose Shuffle Driver Components API
### What changes were proposed in this pull request?

This is the next step of the Spark-25299 work of proposing a new Shuffle storage API. This patch includes the components of the plugin that hook into the driver, including driver shuffle initialization, application cleanup, and shuffle cleanup.

### How was this patch tested?
Existing unit tests, plus an additional test for testing the interactions between the driver and executor initialization.

Closes #25823 from yifeih/yh/upstream/driver-lifecycle.

Lead-authored-by: Yifei Huang <yifeih@palantir.com>
Co-authored-by: mccheah <mcheah@palantir.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-10-15 12:26:49 -05:00
jiake 9ac4b2dbc5 [SPARK-28560][SQL] Optimize shuffle reader to local shuffle reader when smj converted to bhj in adaptive execution
## What changes were proposed in this pull request?
Implement a rule in the new adaptive execution framework introduced in [SPARK-23128](https://issues.apache.org/jira/browse/SPARK-23128). This rule is used to optimize the shuffle reader to local shuffle reader when smj is converted to bhj in adaptive execution.

## How was this patch tested?
Existing tests

Closes #25295 from JkSelf/localShuffleOptimization.

Authored-by: jiake <ke.a.jia@intel.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-15 21:51:15 +08:00
Wenchen Fan 8915966bf4 [SPARK-29473][SQL] move statement logical plans to a new file
### What changes were proposed in this pull request?

move the statement logical plans that were created for v2 commands to a new file `statements.scala`, under the same package of `v2Commands.scala`.

This PR also includes some minor cleanups:
1. remove `private[sql]` from `ParsedStatement` as it's in the private package.
2. remove unnecessary override of `output` and `children`.
3. add missing classdoc.

### Why are the changes needed?

Similar to https://github.com/apache/spark/pull/26111 , this is to better organize the logical plans of data source v2.

It's a bit weird to put the statements in the package `org.apache.spark.sql.catalyst.plans.logical.sql` as `sql` is not a good sub-package name in Spark SQL.

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

no

### How was this patch tested?

existing tests

Closes #26125 from cloud-fan/statement.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-10-15 15:05:49 +02:00
yangjie01 a988aaf3fa [SPARK-29454][SQL] Reduce unsafeProjection times when read Parquet file use non-vectorized mode
### What changes were proposed in this pull request?

There will be 2 times unsafeProjection convert operation When we read a Parquet data file use non-vectorized mode:

1.  `ParquetGroupConverter` call unsafeProjection function to covert `SpecificInternalRow` to `UnsafeRow` every times when read Parquet data file use `ParquetRecordReader`.

2. `ParquetFileFormat` will call unsafeProjection function to covert this `UnsafeRow` to another `UnsafeRow` again when partitionSchema is not empty in DataSourceV1 branch, and `PartitionReaderWithPartitionValues` will  always do this convert operation in DataSourceV2 branch.

In this pr,  remove `unsafeProjection` convert operation in `ParquetGroupConverter` and change `ParquetRecordReader`  to produce `SpecificInternalRow`  instead of `UnsafeRow`.

### Why are the changes needed?
The first time convert in `ParquetGroupConverter` is redundant and `ParquetRecordReader` return a `InternalRow(SpecificInternalRow)` is enough.

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

### How was this patch tested?

Unit Test

Closes #26106 from LuciferYang/spark-parquet-unsafe-projection.

Authored-by: yangjie01 <yangjie01@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-15 12:42:42 +08:00
Marcelo Vanzin 857f109c47 [SPARK-10614][CORE] Add monotonic time to Clock interface
This change adds a new method to the Clock interface that returns
the time from a monotonic time source, so that code that needs that
feature can also mock the Clock in tests.

The original getTimeMillis and waitTillTime methods are unchanged, since
streaming code that uses the Clock interface seems to rely on wall clock
semantics, not monotonic clock. So, in a way, this doesn't directly
address the problem raised in the bug, that waitTillTime can be affected
by drift, but then the places being modified to use the new API don't
really rely on that API.

The dynamic allocation code was modified to use the new time source,
since they shouldn't be basing their decisions on wall clock time.

For a longer discussion on how monotonic clocks work on Linux/x64, the
following blog post (and links within) shed a lot of light on the safety of
`System.nanoTime()`:

http://btorpey.github.io/blog/2014/02/18/clock-sources-in-linux/

Tested with unit test and also running apps with dynamic allocation on.

Closes #26058 from vanzin/SPARK-10614.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-14 19:38:04 -07:00
Wenchen Fan 9407fba037 [SPARK-29412][SQL] refine the document of v2 session catalog config
### What changes were proposed in this pull request?

Refine the document of v2 session catalog config, to clearly explain what it is, when it should be used and how to implement it.

### Why are the changes needed?

Make this config more understandable

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

No

### How was this patch tested?

Pass the Jenkins with the newly updated test cases.

Closes #26071 from cloud-fan/config.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-15 10:18:58 +08:00
herman 1f1443ebb2 [SPARK-29347][SQL] Add JSON serialization for external Rows
### What changes were proposed in this pull request?
This PR adds JSON serialization for Spark external Rows.

### Why are the changes needed?
This is to be used for observable metrics where the `StreamingQueryProgress` contains a map of observed metrics rows which needs to be serialized in some cases.

### Does this PR introduce any user-facing change?
Yes, a user can call `toJson` on rows returned when collecting a DataFrame to the driver.

### How was this patch tested?
Added a new test suite: `RowJsonSuite` that should test this.

Closes #26013 from hvanhovell/SPARK-29347.

Authored-by: herman <herman@databricks.com>
Signed-off-by: herman <herman@databricks.com>
2019-10-15 00:24:17 +02:00
Dongjoon Hyun ff9fcd501c Revert "[SPARK-29107][SQL][TESTS] Port window.sql (Part 1)"
This reverts commit 81915dacc4.
2019-10-14 15:15:32 -07:00
Wenchen Fan bfa09cf049 [SPARK-29463][SQL] move v2 commands to a new file
### What changes were proposed in this pull request?

move the v2 command logical plans from `basicLogicalOperators.scala` to a new file `v2Commands.scala`

### Why are the changes needed?

As we keep adding v2 commands, the `basicLogicalOperators.scala` grows bigger and bigger. It's better to have a separated file for them.

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

no

### How was this patch tested?

not needed

Closes #26111 from cloud-fan/command.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-14 14:09:21 -07:00
Thomas Graves a42d894a40 [SPARK-29417][CORE] Resource Scheduling - add TaskContext.resource java api
### What changes were proposed in this pull request?
We added a TaskContext.resources() api, but I realized this is returning a scala Map which is not ideal for access from Java.  Here I add a resourcesJMap function which returns a java.util.Map to make it easily accessible from Java.

### Why are the changes needed?
Java API access

### Does this PR introduce any user-facing change?
<!--
If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
If no, write 'No'.
-->
Yes, new TaskContext function to access from Java

### How was this patch tested?
<!--
If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
If tests were not added, please describe why they were not added and/or why it was difficult to add.
-->
new unit test

Closes #26083 from tgravescs/SPARK-29417.

Lead-authored-by: Thomas Graves <tgraves@ngvpn01-168-221.dyn.scz.us.nvidia.com>
Co-authored-by: Thomas Graves <tgraves@TGRAVES-MLT.local>
Co-authored-by: Thomas Graves <tgraves@nvidia.com>
Signed-off-by: Xiangrui Meng <meng@databricks.com>
2019-10-14 13:27:34 -07:00
Ilan Filonenko 52186afd84 [SPARK-25152][K8S] Enable SparkR Integration Tests for Kubernetes
## What changes were proposed in this pull request?

Re-introduced SparkR integration tests as part of the SparkR on K8S release. This PR awaits Jenkins availability.

## How was this patch tested?

This patch was tested with unit tests and integration tests.

Closes #22145 from ifilonenko/spark-r-with-tests.

Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: shane knapp <incomplete@gmail.com>
2019-10-14 13:25:54 -07:00
Dongjoon Hyun e696c36e32 [SPARK-29442][SQL] Set default mode should override the existing mode
### What changes were proposed in this pull request?

This PR aims to fix the behavior of `mode("default")` to set `SaveMode.ErrorIfExists`. Also, this PR updates the exception message by adding `default` explicitly.

### Why are the changes needed?

This is reported during `GRAPH API` PR. This builder pattern should work like the documentation.

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

Yes if the app has multiple `mode()` invocation including `mode("default")` and the `mode("default")` is the last invocation. This is really a corner case.
- Previously, the last invocation was handled as `No-Op`.
- After this bug fix, it will work like the documentation.

### How was this patch tested?

Pass the Jenkins with the newly added test case.

Closes #26094 from dongjoon-hyun/SPARK-29442.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-14 13:11:05 -07:00
DylanGuedes 81915dacc4 [SPARK-29107][SQL][TESTS] Port window.sql (Part 1)
### What changes were proposed in this pull request?

This PR ports window.sql from PostgreSQL regression tests https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/window.sql from lines 1~319

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/expected/window.out

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

### Why are the changes needed?
To ensure compatibility with PGSQL

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

### How was this patch tested?
Comparison with PgSQL results.

Closes #25816 from DylanGuedes/spark-29107.

Authored-by: DylanGuedes <djmgguedes@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-14 10:17:16 -07:00
sandeep katta ba04562234 [SPARK-29435][CORE] MapId in Shuffle Block is inconsistent at the writer and reader part when spark.shuffle.useOldFetchProtocol=true
### What changes were proposed in this pull request?
Shuffle Block Construction during Shuffle Write and Read is wrong

Shuffle Map Task  (Shuffle Write)
19/10/11 22:07:32| ERROR| [Executor task launch worker for task 3] org.apache.spark.shuffle.IndexShuffleBlockResolver: ####### For Debug ############ /tmp/hadoop-root1/nm-local-dir/usercache/root1/appcache/application_1570422377362_0008/blockmgr-6d03250d-6e7c-4bc2-bbb7-22b8e3981c35/0d/**shuffle_0_3_0.index**

Result Task (Shuffle Read)
19/10/11 22:07:32| ERROR| [Executor task launch worker for task 6] org.apache.spark.storage.ShuffleBlockFetcherIterator: Error occurred while fetching local blocks
java.nio.file.NoSuchFileException: /tmp/hadoop-root1/nm-local-dir/usercache/root1/appcache/application_1570422377362_0008/blockmgr-6d03250d-6e7c-4bc2-bbb7-22b8e3981c35/30/**shuffle_0_0_0.index**
	at sun.nio.fs.UnixException.translateToIOException(UnixException.java:86)

As per [SPARK-25341](https://issues.apache.org/jira/browse/SPARK-25341) `mapId` of `SortShuffleManager.getWriter `changed to `context.taskAttemptId() ` from `partitionId`

[code]( https://github.com/apache/spark/pull/25620/files#diff-363c53ca5a72cfdc37dac4a723309638R54)

But `MapOutputTracker.convertMapStatuses` returns the wrong ShuffleBlock, if `spark.shuffle.useOldFetchProtocol `enabled, it returns `paritionId` as `mapID` which is wrong . [Code](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/MapOutputTracker.scala#L912)

### Why are the changes needed?
Already MapStatus returned by the ShuffleWriter has the mapId for e.g.[ code here](https://github.com/apache/spark/blob/master/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java#L134). So it's nice to use `status.mapTaskId`

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

### How was this patch tested?
Existing UT  and manually tested with `spark.shuffle.useOldFetchProtocol` as true and false

![image](https://user-images.githubusercontent.com/35216143/66716530-4f4caa80-edec-11e9-833d-7131a9fbd442.png)

Closes #26095 from sandeep-katta/shuffleIssue.

Lead-authored-by: sandeep katta <sandeep.katta2007@gmail.com>
Co-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-15 00:03:13 +08:00
Huaxin Gao cfcaf528cd [SPARK-29381][PYTHON][ML] Add _ before the XXXParams classes
### What changes were proposed in this pull request?
Add _ before XXXParams classes to indicate internal usage

### Why are the changes needed?
Follow the PEP 8 convention to use _single_leading_underscore to indicate internal use

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

### How was this patch tested?
use existing tests

Closes #26103 from huaxingao/spark-29381.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-14 10:52:23 -05:00
Maxim Gekk da576a737c [SPARK-29369][SQL] Support string intervals without the interval prefix
### What changes were proposed in this pull request?
In the PR, I propose to move interval parsing to `CalendarInterval.fromCaseInsensitiveString()` which throws an `IllegalArgumentException` for invalid strings, and reuse it from `CalendarInterval.fromString()`. The former one handles `IllegalArgumentException` only and returns `NULL` for invalid interval strings. This will allow to support interval strings without the `interval` prefix in casting strings to intervals and in interval type constructor because they use `fromString()` for parsing string intervals.

For example:
```sql
spark-sql> select cast('1 year 10 days' as interval);
interval 1 years 1 weeks 3 days
spark-sql> SELECT INTERVAL '1 YEAR 10 DAYS';
interval 1 years 1 weeks 3 days
```

### Why are the changes needed?
To maintain feature parity with PostgreSQL which supports interval strings without prefix:
```sql
# select interval '2 months 1 microsecond';
        interval
------------------------
 2 mons 00:00:00.000001
```
and to improve Spark SQL UX.

### Does this PR introduce any user-facing change?
Yes, previously parsing of interval strings without `interval` gives `NULL`:
```sql
spark-sql> select interval '2 months 1 microsecond';
NULL
```
After:
```sql
spark-sql> select interval '2 months 1 microsecond';
interval 2 months 1 microseconds
```

### How was this patch tested?
- Added new tests to `CalendarIntervalSuite.java`
- A test for casting strings to intervals in `CastSuite`
- Test for interval type constructor from strings in `ExpressionParserSuite`

Closes #26079 from MaxGekk/interval-str-without-prefix.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-14 23:34:18 +08:00
Terry Kim ef6dce29b2 [SPARK-29279][SQL] Merge SHOW NAMESPACES and SHOW DATABASES code path
### What changes were proposed in this pull request?
Currently,  `SHOW NAMESPACES` and `SHOW DATABASES` are separate code paths. This PR merges two implementations.

### Why are the changes needed?
To remove code/behavior duplication

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

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

Closes #26006 from imback82/combine_show.

Authored-by: Terry Kim <yuminkim@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-14 22:35:26 +08:00
Huaxin Gao 67e1360bad [SPARK-29377][PYTHON][ML] Parity between Scala ML tuning and Python ML tuning
### What changes were proposed in this pull request?
Follow Scala ml tuning implementation

- put leading underscore before python ```ValidatorParams``` to indicate private
- add ```_CrossValidatorParams``` and ```_TrainValidationSplitParams```
- separate the getters and setters. Put getters in _XXXParams and setters in the Classes.

### Why are the changes needed?
Keep parity between scala and python

### Does this PR introduce any user-facing change?
add ```CrossValidatorModel.getNumFolds``` and ```TrainValidationSplitModel.getTrainRatio()```

### How was this patch tested?
Add doctest

Closes #26057 from huaxingao/spark-tuning.

Authored-by: Huaxin Gao <huaxing@us.ibm.com>
Signed-off-by: zhengruifeng <ruifengz@foxmail.com>
2019-10-14 14:28:31 +08:00
angerszhu ef81525a1a [SPARK-29308][BUILD] Update deps in dev/deps/spark-deps-hadoop-3.2 for hadoop-3.2
### What changes were proposed in this pull request?

Current dev/deps/spark-deps-hadoop-3.2 have some wrong deps,   it's caused by `dev/test-dependencies.sh ` when build assembly dependencies.
add maven compile parameter `-am` to make it build with all deps, and get right result.

And update NOTICE-binary & NOTICE-binary for updated result.

### Why are the changes needed?
Update dev/deps/spark-hadoop-3.2

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

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

Closes #25984 from AngersZhuuuu/SPARK=29308.

Authored-by: angerszhu <angers.zhu@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-13 12:53:12 -05: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
Dongjoon Hyun abba53e78b [SPARK-27831][FOLLOWUP][SQL][TEST] ADDITIONAL_REMOTE_REPOSITORIES is a comma-delimited string
### What changes were proposed in this pull request?

This PR is a very minor follow-up to become robust because `spark.sql.additionalRemoteRepositories` is a configuration which has a comma-separated value.

### Why are the changes needed?

This makes sure that `getHiveContribJar` will not fail on the configuration changes.

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

No.

### How was this patch tested?

Manual. Change the default value with multiple repositories and run the following.
```
build/sbt -Phive "project hive" "test-only org.apache.spark.sql.hive.HiveSparkSubmitSuite"
```

Closes #26096 from dongjoon-hyun/SPARK-27831.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Yuming Wang <wgyumg@gmail.com>
2019-10-12 18:47:28 -07:00
Maxim Gekk d193248205 [SPARK-29368][SQL][TEST] Port interval.sql
### What changes were proposed in this pull request?
This PR is to port interval.sql from PostgreSQL regression tests: https://raw.githubusercontent.com/postgres/postgres/REL_12_STABLE/src/test/regress/sql/interval.sql

The expected results can be found in the link: https://github.com/postgres/postgres/blob/REL_12_STABLE/src/test/regress/expected/interval.out

When porting the test cases, found PostgreSQL specific features below that do not exist in Spark SQL:
- [SPARK-29369](https://issues.apache.org/jira/browse/SPARK-29369): Accept strings without `interval` prefix in casting to intervals
- [SPARK-29370](https://issues.apache.org/jira/browse/SPARK-29370): Interval strings without explicit unit markings
- [SPARK-29371](https://issues.apache.org/jira/browse/SPARK-29371): Support interval field values with fractional parts
- [SPARK-29382](https://issues.apache.org/jira/browse/SPARK-29382): Support the `INTERVAL` type by Parquet datasource
- [SPARK-29383](https://issues.apache.org/jira/browse/SPARK-29383): Support the optional prefix `` in interval strings
- [SPARK-29384](https://issues.apache.org/jira/browse/SPARK-29384): Support `ago` in interval strings
- [SPARK-29385](https://issues.apache.org/jira/browse/SPARK-29385): Make `INTERVAL` values comparable
- [SPARK-29386](https://issues.apache.org/jira/browse/SPARK-29386): Copy data between a file and a table
- [SPARK-29387](https://issues.apache.org/jira/browse/SPARK-29387): Support `*` and `\` operators for intervals
- [SPARK-29388](https://issues.apache.org/jira/browse/SPARK-29388): Construct intervals from the `millenniums`, `centuries` or `decades` units
- [SPARK-29389](https://issues.apache.org/jira/browse/SPARK-29389): Support synonyms for interval units
- [SPARK-29390](https://issues.apache.org/jira/browse/SPARK-29390): Add the justify_days(), justify_hours() and justify_interval() functions
- [SPARK-29391](https://issues.apache.org/jira/browse/SPARK-29391): Default year-month units
- [SPARK-29393](https://issues.apache.org/jira/browse/SPARK-29393): Add the make_interval() function
- [SPARK-29394](https://issues.apache.org/jira/browse/SPARK-29394): Support ISO 8601 format for intervals
- [SPARK-29395](https://issues.apache.org/jira/browse/SPARK-29395): Precision of the interval type
- [SPARK-29406](https://issues.apache.org/jira/browse/SPARK-29406): Interval output styles
- [SPARK-29407](https://issues.apache.org/jira/browse/SPARK-29407): Support syntax for zero interval
- [SPARK-29408](https://issues.apache.org/jira/browse/SPARK-29408): Support interval literal with negative sign `-`

### Why are the changes needed?
To improve the test coverage, see https://issues.apache.org/jira/browse/SPARK-27763

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

### How was this patch tested?
By manually comparing Spark results with PostgreSQL

Closes #26055 from MaxGekk/port-interval-sql.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-12 17:44:40 -07:00
Pablo 782a94d289 [SPARK-29433][WEBUI] Fix tooltip stages table
### What changes were proposed in this pull request?
In the Web UI, Stages table, the tool tip of Input and output column are not corrrect.

Actual tooltip messages:

Bytes and records read from Hadoop or from Spark storage.
Bytes and records written to Hadoop.
In this column we are only showing bytes, not records

![image](https://user-images.githubusercontent.com/12819544/66608286-85a0e480-ebb6-11e9-812a-9760bea53664.png)
![image](https://user-images.githubusercontent.com/12819544/66608323-96515a80-ebb6-11e9-9e5f-e3f2cc99a3b3.png)
![image](https://user-images.githubusercontent.com/12819544/66608450-de707d00-ebb6-11e9-84e3-0917b5cfe6f6.png)
![image](https://user-images.githubusercontent.com/12819544/66608468-eaf4d580-ebb6-11e9-8c5b-2a9a290bea9c.png)

### Why are the changes needed?
Simple correction of a tooltip

### Does this PR introduce any user-facing change?
Yes, tooltip correction

### How was this patch tested?
Manual testing

Closes #26084 from planga82/feature/SPARK-29433_Tooltip_correction.

Lead-authored-by: Pablo <soypab@gmail.com>
Co-authored-by: Unknown <soypab@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-12 10:34:29 -05:00
Fokko Driesprong b5b1b69f79 [SPARK-29445][CORE] Bump netty-all from 4.1.39.Final to 4.1.42.Final
### What changes were proposed in this pull request?

Minor version bump of Netty to patch reported CVE.

Patches: https://www.cvedetails.com/cve/CVE-2019-16869/

### Why are the changes needed?

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

No

### How was this patch tested?

Compiled locally using `mvn clean install -DskipTests`

Closes #26099 from Fokko/SPARK-29445.

Authored-by: Fokko Driesprong <fokko@apache.org>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-12 09:43:16 -05:00
liucht e94abd7f16 [SPARK-29323][WEBUI] Add tooltip for The Executors Tab's column names in the Spark history server Page
### What changes were proposed in this pull request?

This PR is Adding tooltip for The Executors Tab's column names include RDD Blocks, Disk Used,Cores, Activity Tasks, Failed Tasks , Complete Tasks, Total Tasks in the history server Page.
https://issues.apache.org/jira/browse/SPARK-29323

![image](https://user-images.githubusercontent.com/28332082/66017759-b6c24a80-e50e-11e9-807b-5b076f701d2f.png)

I have modify the following code in executorspage-template.html
Before:
<th>RDD Blocks</th>
<th>Disk Used</th>
<th>Cores</th>
<th>Active Tasks</th>
<th>Failed Tasks</th>
<th>Complete Tasks</th>
<th>Total Tasks</th>

![image](https://user-images.githubusercontent.com/28332082/66018111-4ddbd200-e510-11e9-9cfc-19f3eae81e76.png)

After:

<th><span data-toggle="tooltip" data-placement="top" title="RDD Blocks">RDD Blocks</span></th>
<th><span data-toggle="tooltip" data-placement="top" title="Disk Used">Disk Used</span></th>
<th><span data-toggle="tooltip" data-placement="top" title="Cores">Cores</span></th>
<th><span data-toggle="tooltip" data-placement="top" title="Active Tasks">Active Tasks</span></th>
<th><span data-toggle="tooltip" data-placement="top" title="Failed Tasks">Failed Tasks</span></th>
<th><span data-toggle="tooltip" data-placement="top" title="Complete Tasks">Complete Tasks</span></th>
<th><span data-toggle="tooltip" data-placement="top" title="Total Tasks">Total Tasks</span></th>

![image](https://user-images.githubusercontent.com/28332082/66018157-79f75300-e510-11e9-96ba-6230aa0940c7.png)

### Why are the changes needed?

the spark Executors of history Tab page, the Summary part shows the line in the list of title, but format is irregular.
Some column names have tooltip, such as Storage Memory, Task Time(GC Time), Input, Shuffle Read,
Shuffle Write and Blacklisted, but there are still some list names that have not tooltip. They are RDD Blocks, Disk Used,Cores, Activity Tasks, Failed Tasks , Complete Tasks and Total Tasks. oddly, Executors section below,All the column names Contains the column names above have tooltip .
It's important for open source projects to have consistent style and user-friendly UI, and I'm working on keeping it consistent And more user-friendly.

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

No.

### How was this patch tested?

Manual tests for Chrome, Firefox and Safari

Authored-by: liucht-inspur <liuchtinspur.com>
Signed-off-by: liucht-inspur <liuchtinspur.com>

Closes #25994 from liucht-inspur/master.

Authored-by: liucht <liucht@inspur.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-10-12 09:40:31 -05:00