Commit graph

320 commits

Author SHA1 Message Date
HyukjinKwon 6f4703e22e [SPARK-30690][DOCS][BUILD] Add CalendarInterval into API documentation
### What changes were proposed in this pull request?

We should also expose it in documentation as we marked it as unstable API as of SPARK-30547
Note that, seems Javadoc -> Scaladoc doesn't work but this PR does not target to fix.

### Why are the changes needed?

To show the documentation of API.

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

### How was this patch tested?
Manually built the docs via `jykill serve` under `docs` directory:

![Screen Shot 2020-01-31 at 4 04 15 PM](https://user-images.githubusercontent.com/6477701/73519315-12143300-4444-11ea-9260-070c9f672dde.png)

Closes #27412 from HyukjinKwon/SPARK-30547.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2020-01-31 22:50:01 +09:00
Chandni Singh 6b47ace27d [SPARK-30512] Added a dedicated boss event loop group
### What changes were proposed in this pull request?
Adding a dedicated boss event loop group to the Netty pipeline in the External Shuffle Service to avoid the delay in channel registration.
```
   EventLoopGroup bossGroup = NettyUtils.createEventLoop(ioMode, 1,
      conf.getModuleName() + "-boss");
    EventLoopGroup workerGroup =  NettyUtils.createEventLoop(ioMode, conf.serverThreads(),
    conf.getModuleName() + "-server");

    bootstrap = new ServerBootstrap()
      .group(bossGroup, workerGroup)
      .channel(NettyUtils.getServerChannelClass(ioMode))
      .option(ChannelOption.ALLOCATOR, allocator)
```

### Why are the changes needed?
We have been seeing a large number of SASL authentication (RPC requests) timing out with the external shuffle service.
```
java.lang.RuntimeException: java.util.concurrent.TimeoutException: Timeout waiting for task.
	at org.spark-project.guava.base.Throwables.propagate(Throwables.java:160)
	at org.apache.spark.network.client.TransportClient.sendRpcSync(TransportClient.java:278)
	at org.apache.spark.network.sasl.SaslClientBootstrap.doBootstrap(SaslClientBootstrap.java:80)
	at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:228)
	at org.apache.spark.network.client.TransportClientFactory.createUnmanagedClient(TransportClientFactory.java:181)
	at org.apache.spark.network.shuffle.ExternalShuffleClient.registerWithShuffleServer(ExternalShuffleClient.java:141)
	at org.apache.spark.storage.BlockManager$$anonfun$registerWithExternalShuffleServer$1.apply$mcVI$sp(BlockManager.scala:218)
```
The investigation that we have done is described here:
https://github.com/netty/netty/issues/9890

After adding `LoggingHandler` to the netty pipeline, we saw that the registration of the channel was getting delay which is because the worker threads are busy with the existing channels.

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

### How was this patch tested?
We have tested the patch on our clusters and with a stress testing tool. After this change, we didn't see any SASL requests timing out. Existing unit tests pass.

Closes #27240 from otterc/SPARK-30512.

Authored-by: Chandni Singh <chsingh@linkedin.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2020-01-29 15:02:48 -06:00
Kent Yao af705421db [SPARK-30593][SQL] Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and no round trip
### What changes were proposed in this pull request?

This revert https://github.com/apache/spark/pull/26418, file a new ticket under  https://issues.apache.org/jira/browse/SPARK-30546 for better tracking interval behavior
### Why are the changes needed?

Revert interval ISO/ANSI SQL Standard output since we decide not to follow ANSI and there is no round trip

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

no, not released yet

### How was this patch tested?

existing uts

Closes #27304 from yaooqinn/SPARK-30593.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 20:51:10 +08:00
Kent Yao 730388b369 [SPARK-30547][SQL][FOLLOWUP] Update since anotation for CalendarInterval class
### What changes were proposed in this pull request?
Mark `CalendarInterval` class with `since 3.0.0`.
### Why are the changes needed?

https://www.oracle.com/technetwork/java/javase/documentation/index-137868.html#since

This class is the first time going to the public, the annotation is the first time to add, and we don't want people to get confused and try to use it 2.4.x.

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

no

### How was this patch tested?

no

Closes #27299 from yaooqinn/SPARK-30547-F.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-21 20:35:47 +08:00
Kent Yao 4806cc5bd1 [SPARK-30547][SQL] Add unstable annotation to the CalendarInterval class
### What changes were proposed in this pull request?

`CalendarInterval` is maintained as a private class but might be used in a public way by users
e.g.

```scala
scala> spark.udf.register("getIntervalMonth", (_:org.apache.spark.unsafe.types.CalendarInterval).months)

scala> sql("select interval 2 month 1 day a").selectExpr("getIntervalMonth(a)").show
+-------------------+
|getIntervalMonth(a)|
+-------------------+
|                  2|
+-------------------+
```

And it exists since 1.5.0, now we go to the 3.x era,may be it's time to make it public

### Why are the changes needed?

make the interval more future-proofing

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

doc change

### How was this patch tested?

add ut.

Closes #27258 from yaooqinn/SPARK-30547.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-20 12:17:37 +08:00
Kent Yao 17857f9b8b [SPARK-30551][SQL] Disable comparison for interval type
### What changes were proposed in this pull request?

As we are not going to follow ANSI to implement year-month and day-time interval types, it is weird to compare the year-month part to the day-time part for our current implementation of interval type now.

Additionally, the current ordering logic comes from PostgreSQL where the implementation of the interval is messy. And we are not aiming PostgreSQL compliance at all.

THIS PR will revert https://github.com/apache/spark/pull/26681 and https://github.com/apache/spark/pull/26337

### Why are the changes needed?

make interval type more future-proofing

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

there are new in 3.0, so no

### How was this patch tested?

existing uts shall work

Closes #27262 from yaooqinn/SPARK-30551.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-19 15:27:51 +08:00
Maxim Kolesnikov 830e635e67 [SPARK-27868][CORE][FOLLOWUP] Recover the default value to -1 again
The default value for backLog set back to -1, as any other value may break existing configuration by overriding Netty's default io.netty.util.NetUtil#SOMAXCONN. The documentation accordingly adjusted.
See discussion thread: https://github.com/apache/spark/pull/24732

### What changes were proposed in this pull request?
Partial rollback of https://github.com/apache/spark/pull/24732 (default for backLog set back to -1).

### Why are the changes needed?
Previous change introduces backward incompatibility by overriding default of Netty's `io.netty.util.NetUtil#SOMAXCONN`

Closes #27230 from xCASx/master.

Authored-by: Maxim Kolesnikov <swe.kolesnikov@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-17 10:43:47 -08:00
Henrique Goulart d42cf4566a [SPARK-30246][CORE] OneForOneStreamManager might leak memory in connectionTerminated
### What changes were proposed in this pull request?

Ensure that all StreamStates are removed from OneForOneStreamManager memory map even if there's an error trying to release buffers

### Why are the changes needed?

OneForOneStreamManager may not remove all StreamStates from memory map when a connection is terminated. A RuntimeException might be thrown in StreamState$buffers.next() by one of ExternalShuffleBlockResolver$getBlockData... **breaking the loop through streams.entrySet(), keeping StreamStates in memory forever leaking memory.**
That may happen when an application is terminated abruptly and executors removed before the connection is terminated or if shuffleIndexCache fails to get ShuffleIndexInformation

References:
ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java (L319)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalBlockHandler.java (L357)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java (L195)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java (L208)

ee050ddbc6/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java (L330)

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

### How was this patch tested?
Unit test added

Closes #27064 from hensg/SPARK-30246.

Lead-authored-by: Henrique Goulart <henriquedsg89@gmail.com>
Co-authored-by: Henrique Goulart <henrique.goulart@trivago.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2020-01-15 13:27:15 -08:00
root1 e0efd213eb [SPARK-30292][SQL] Throw Exception when invalid string is cast to numeric type in ANSI mode
### What changes were proposed in this pull request?
If spark.sql.ansi.enabled is set,
throw exception when cast to any numeric type do not follow the ANSI SQL standards.

### Why are the changes needed?
ANSI SQL standards do not allow invalid strings to get casted into numeric types and throw exception for that. Currently spark sql gives NULL in such cases.

Before:
`select cast('str' as decimal)  => NULL`

After :
`select cast('str' as decimal) => invalid input syntax for type numeric: str`

These results are after setting `spark.sql.ansi.enabled=true`

### Does this PR introduce any user-facing change?
Yes. Now when ansi mode is on users will get arithmetic exception for invalid strings.

### How was this patch tested?
Unit Tests Added.

Closes #26933 from iRakson/castDecimalANSI.

Lead-authored-by: root1 <raksonrakesh@gmail.com>
Co-authored-by: iRakson <raksonrakesh@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2020-01-14 17:03:10 +08:00
Ajith 700293207d [SPARK-30406] OneForOneStreamManager ensure that compound operations on shared variables are atomic
Using compound operations as well as increments and decrements on primitive fields are not atomic operations. Here when volatile primitive field is incremented or decremented,  we run into data loss if threads interleave in steps of update.

 Refer: https://wiki.sei.cmu.edu/confluence/display/java/VNA02-J.+Ensure+that+compound+operations+on+shared+variables+are+atomic

### What changes were proposed in this pull request?
Using `AtomicLong` instead of `long`

### Why are the changes needed?
volatile primitive field is incremented or decremented,  we run into data loss if threads interleave in steps of update.

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

### How was this patch tested?
All Existing UT can pass with the Change

Closes #27071 from ajithme/atomic.

Authored-by: Ajith <ajith2489@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2020-01-03 11:41:45 -06:00
Liang-Chi Hsieh 0042ad575a [SPARK-30290][CORE] Count for merged block when fetching continuous blocks in batch
### What changes were proposed in this pull request?

We added shuffle block fetch optimization in SPARK-9853. In ShuffleBlockFetcherIterator, we merge single blocks into batch blocks. During merging, we should count merged blocks for `maxBlocksInFlightPerAddress`, not original single blocks.

### Why are the changes needed?

If `maxBlocksInFlightPerAddress` is specified, like set it to 1, it should mean one batch block, not one original single block. Otherwise, it will conflict with batch shuffle fetch.

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

No

### How was this patch tested?

Unit test.

Closes #26930 from viirya/respect-max-blocks-inflight.

Lead-authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Co-authored-by: Liang-Chi Hsieh <liangchi@uber.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-12-25 18:57:02 +08:00
Sean Owen 7dff3b125d [SPARK-30272][SQL][CORE] Remove usage of Guava that breaks in 27; replace with workalikes
### What changes were proposed in this pull request?

Remove usages of Guava that no longer work in Guava 27, and replace with workalikes. I'll comment on key types of changes below.

### Why are the changes needed?

Hadoop 3.2.1 uses Guava 27, so this helps us avoid problems running on Hadoop 3.2.1+ and generally lowers our exposure to Guava.

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

Should not be, but see notes below on hash codes and toString.

### How was this patch tested?

Existing tests will verify whether these changes break anything for Guava 14.
I manually built with an updated version and it compiles with Guava 27; tests running manually locally now.

Closes #26911 from srowen/SPARK-30272.

Authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2019-12-20 08:55:04 -06: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
Kent Yao d3ec8b1735 [SPARK-30066][SQL] Support columnar execution on interval types
### What changes were proposed in this pull request?

Columnar execution support for interval types

### Why are the changes needed?

support cache tables with interval columns
improve performance too

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

Yes cache table with accept interval columns

### How was this patch tested?

add ut

Closes #26699 from yaooqinn/SPARK-30066.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-14 13:10:46 -08:00
Marcelo Vanzin c5f312a6ac [SPARK-30129][CORE] Set client's id in TransportClient after successful auth
The new auth code was missing this bit, so it was not possible to know which
app a client belonged to when auth was on.

I also refactored the SASL test that checks for this so it also checks the
new protocol (test failed before the fix, passes now).

Closes #26760 from vanzin/SPARK-30129.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-12-04 17:11:50 -08:00
Jungtaek Lim (HeartSaVioR) e04a63437b [SPARK-30075][CORE][TESTS] Fix the hashCode implementation of ArrayKeyIndexType correctly
### What changes were proposed in this pull request?

This patch fixes the bug on ArrayKeyIndexType.hashCode() as it is simply calling Array.hashCode() which in turn calls Object.hashCode(). That should be Arrays.hashCode() to reflect the elements in the array.

### Why are the changes needed?

I've encountered the bug in #25811 while adding test codes for #25811, and I've split the fix into individual PR to speed up reviewing. Without this patch, ArrayKeyIndexType would bring various issues when it's used as type of collections.

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

No.

### How was this patch tested?

I've skipped adding UT as ArrayKeyIndexType is in test and the patch is pretty simple one-liner.

Closes #26709 from HeartSaVioR/SPARK-30075.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan.opensource@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-12-02 09:06:37 -06:00
huangtianhua e842033acc [SPARK-27721][BUILD] Switch to use right leveldbjni according to the platforms
This change adds a profile to switch to use the right leveldbjni package according to the platforms:
aarch64 uses org.openlabtesting.leveldbjni:leveldbjni-all.1.8, and other platforms use the old one org.fusesource.leveldbjni:leveldbjni-all.1.8.
And because some hadoop dependencies packages are also depend on org.fusesource.leveldbjni:leveldbjni-all, but hadoop merge the similar change on trunk, details see
https://issues.apache.org/jira/browse/HADOOP-16614, so exclude the dependency of org.fusesource.leveldbjni for these hadoop packages related.
Then Spark can build/test on aarch64 platform successfully.

Closes #26636 from huangtianhua/add-aarch64-leveldbjni.

Authored-by: huangtianhua <huangtianhua@huawei.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-12-02 09:04:00 -06:00
Liu,Linhong f22177c957 [SPARK-29486][SQL][FOLLOWUP] Document the reason to add days field
### What changes were proposed in this pull request?
Follow up of #26134 to document the reason to add days filed and explain how do we use it

### Why are the changes needed?
only comment

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

### How was this patch tested?
no need test

Closes #26701 from LinhongLiu/spark-29486-followup.

Authored-by: Liu,Linhong <liulinhong@baidu.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-30 08:43:34 -06:00
“attilapiros” fd2bf55aba [SPARK-27651][CORE] Avoid the network when shuffle blocks are fetched from the same host
## What changes were proposed in this pull request?

Before this PR `ShuffleBlockFetcherIterator` was partitioning the block fetches into two distinct sets: local reads and remote fetches. Within this PR (when the feature is enabled by "spark.shuffle.readHostLocalDisk.enabled") a new category is introduced: host-local reads. They are shuffle block fetches where although the block manager is different they are running on the same host along with the requester.

Moreover to get the local directories of the other executors/block managers a new RPC message is introduced `GetLocalDirs` which is sent the the block manager master where it is answered as `BlockManagerLocalDirs`. In `BlockManagerMasterEndpoint` for answering this request the `localDirs` is extracted from the `BlockManagerInfo` and stored separately in a hash map called `executorIdLocalDirs`. Because the earlier used `blockManagerInfo` contains data for the alive block managers (see `org.apache.spark.storage.BlockManagerMasterEndpoint#removeBlockManager`).

Now `executorIdLocalDirs` knows all the local dirs up to the application start (like the external shuffle service does) so in case of an RDD recalculation both host-local shuffle blocks and disk persisted RDD blocks on the same host can be served by reading the files behind the blocks directly.

## How was this patch tested?

### Unit tests

`ExternalShuffleServiceSuite`:
- "SPARK-27651: host local disk reading avoids external shuffle service on the same node"

`ShuffleBlockFetcherIteratorSuite`:
- "successful 3 local reads + 4 host local reads + 2 remote reads"

And with extending existing suites where shuffle metrics was tested.

### Manual tests

Running Spark on YARN in a 4 nodes cluster with 6 executors and having 12 shuffle blocks.

```
$ grep host-local experiment.log
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_2_1, shuffle_0_6_1
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 38 ms
19/07/30 03:57:12 INFO storage.ShuffleBlockFetcherIterator: Getting 12 (1496.8 MB) non-empty blocks including 2 (299.4 MB) local blocks and 2 (299.4 MB) host-local blocks and 8 (1197.4 MB) remote blocks
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Start fetching host-local blocks: shuffle_0_0_0, shuffle_0_8_0
19/07/30 03:57:12 DEBUG storage.ShuffleBlockFetcherIterator: Got host-local blocks in 35 ms
```

Closes #25299 from attilapiros/SPARK-27651.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-26 11:02:25 -08:00
Kent Yao de21f28f8a [SPARK-29986][SQL] casting string to date/timestamp/interval should trim all whitespaces
### What changes were proposed in this pull request?

A java like string trim method trims all whitespaces that less or equal than 0x20. currently, our UTF8String handle the space =0x20 ONLY. This is not suitable for many cases in Spark, like trim for interval strings, date, timestamps, PostgreSQL like cast string to boolean.

### Why are the changes needed?

improve the white spaces handling in UTF8String, also with some bugs fixed

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

yes,
string with `control character` at either end can be convert to date/timestamp and interval now

### How was this patch tested?

add ut

Closes #26626 from yaooqinn/SPARK-29986.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-25 14:37:04 +08:00
Norman Maurer f28eab2de7 [SPARK-29971][CORE] Fix buffer leaks in TransportFrameDecoder/TransportCipher
### What changes were proposed in this pull request?

- Correctly release `ByteBuf` in `TransportCipher` in all cases
- Move closing / releasing logic to `handlerRemoved(...)` so we are guaranteed that is always called.
- Correctly release `frameBuf` it is not null when the handler is removed (and so also when the channel becomes inactive)

### Why are the changes needed?

We need to carefully manage the ownership / lifecycle of `ByteBuf` instances so we don't leak any of these. We did not correctly do this in all cases:
 - when end up in invalid cipher state.
 - when partial data was received and the channel is closed before the full frame is decoded

Fixes https://github.com/netty/netty/issues/9784.

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

No.

### How was this patch tested?

Pass the newly added UTs.

Closes #26609 from normanmaurer/fix_leaks.

Authored-by: Norman Maurer <norman_maurer@apple.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-11-22 15:20:54 -08:00
Kent Yao 2dd6807e42 [SPARK-28023][SQL] Add trim logic in UTF8String's toInt/toLong to make it consistent with other string-numeric casting
### What changes were proposed in this pull request?

Modify `UTF8String.toInt/toLong` to support trim spaces for both sides before converting it to byte/short/int/long.

With this kind of "cheap" trim can help improve performance for casting string to integrals. The idea is from https://github.com/apache/spark/pull/24872#issuecomment-556917834

### Why are the changes needed?

make the behavior consistent.

### Does this PR introduce any user-facing change?
yes, cast string to an integral type, and binary comparison between string and integrals will trim spaces first. their behavior will be consistent with float and double.
### How was this patch tested?
1. add ut.
2. benchmark tests
 the benchmark is modified based on https://github.com/apache/spark/pull/24872#issuecomment-503827016

```scala
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.spark.sql.execution.benchmark

import org.apache.spark.benchmark.Benchmark

/**
 * Benchmark trim the string when casting string type to Boolean/Numeric types.
 * To run this benchmark:
 * {{{
 *   1. without sbt:
 *      bin/spark-submit --class <this class> --jars <spark core test jar> <spark sql test jar>
 *   2. build/sbt "sql/test:runMain <this class>"
 *   3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain <this class>"
 *      Results will be written to "benchmarks/CastBenchmark-results.txt".
 * }}}
 */
object CastBenchmark extends SqlBasedBenchmark {
This conversation was marked as resolved by yaooqinn

  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
    val title = "Cast String to Integral"
    runBenchmark(title) {
      withTempPath { dir =>
        val N = 500L << 14
        val df = spark.range(N)
        val types = Seq("int", "long")
        (1 to 5).by(2).foreach { i =>
          df.selectExpr(s"concat(id, '${" " * i}') as str")
            .write.mode("overwrite").parquet(dir + i.toString)
        }

        val benchmark = new Benchmark(title, N, minNumIters = 5, output = output)
        Seq(true, false).foreach { trim =>
          types.foreach { t =>
            val str = if (trim) "trim(str)" else "str"
            val expr = s"cast($str as $t) as c_$t"
            (1 to 5).by(2).foreach { i =>
              benchmark.addCase(expr + s" - with $i spaces") { _ =>
                spark.read.parquet(dir + i.toString).selectExpr(expr).collect()
              }
            }
          }
        }
        benchmark.run()
      }
    }
  }
}
```
#### benchmark result.
normal trim v.s. trim in toInt/toLong
```java
================================================================================================
Cast String to Integral
================================================================================================

Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.1
Intel(R) Core(TM) i5-5287U CPU  2.90GHz
Cast String to Integral:                  Best Time(ms)   Avg Time(ms)   Stdev(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------------------------------
cast(trim(str) as int) as c_int - with 1 spaces          10220          12994        1337          0.8        1247.5       1.0X
cast(trim(str) as int) as c_int - with 3 spaces           4763           8356         357          1.7         581.4       2.1X
cast(trim(str) as int) as c_int - with 5 spaces           4791           8042         NaN          1.7         584.9       2.1X
cast(trim(str) as long) as c_long - with 1 spaces           4014           6755         NaN          2.0         490.0       2.5X
cast(trim(str) as long) as c_long - with 3 spaces           4737           6938         NaN          1.7         578.2       2.2X
cast(trim(str) as long) as c_long - with 5 spaces           4478           6919        1404          1.8         546.6       2.3X
cast(str as int) as c_int - with 1 spaces           4443           6222         NaN          1.8         542.3       2.3X
cast(str as int) as c_int - with 3 spaces           3659           3842         170          2.2         446.7       2.8X
cast(str as int) as c_int - with 5 spaces           4372           7996         NaN          1.9         533.7       2.3X
cast(str as long) as c_long - with 1 spaces           3866           5838         NaN          2.1         471.9       2.6X
cast(str as long) as c_long - with 3 spaces           3793           5449         NaN          2.2         463.0       2.7X
cast(str as long) as c_long - with 5 spaces           4947           5961        1198          1.7         603.9       2.1X
```

Closes #26622 from yaooqinn/cheapstringtrim.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-22 19:32:27 +08:00
Sean Owen 1febd373ea [MINOR][TESTS] Replace JVM assert with JUnit Assert in tests
### What changes were proposed in this pull request?

Use JUnit assertions in tests uniformly, not JVM assert() statements.

### Why are the changes needed?

assert() statements do not produce as useful errors when they fail, and, if they were somehow disabled, would fail to test anything.

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

No. The assertion logic should be identical.

### How was this patch tested?

Existing tests.

Closes #26581 from srowen/assertToJUnit.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-11-20 14:04:15 -06:00
HyukjinKwon 882f54b0a3 [SPARK-29870][SQL][FOLLOW-UP] Keep CalendarInterval's toString
### What changes were proposed in this pull request?

This is a followup of https://github.com/apache/spark/pull/26418. This PR removed `CalendarInterval`'s `toString` with an unfinished changes.

### Why are the changes needed?

1. Ideally we should make each PR isolated and separate targeting one issue without touching unrelated codes.

2. There are some other places where the string formats were exposed to users. For example:

    ```scala
    scala> sql("select interval 1 days as a").selectExpr("to_csv(struct(a))").show()
    ```
    ```
    +--------------------------+
    |to_csv(named_struct(a, a))|
    +--------------------------+
    |      "CalendarInterval...|
    +--------------------------+
    ```

3.  Such fixes:

    ```diff
     private def writeMapData(
        map: MapData, mapType: MapType, fieldWriter: ValueWriter): Unit = {
      val keyArray = map.keyArray()
    + val keyString = mapType.keyType match {
    +   case CalendarIntervalType =>
    +    (i: Int) => IntervalUtils.toMultiUnitsString(keyArray.getInterval(i))
    +   case _ => (i: Int) => keyArray.get(i, mapType.keyType).toString
    + }
    ```

    can cause performance regression due to type dispatch for each map.

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

Yes, see 2. case above.

### How was this patch tested?

Manually tested.

Closes #26572 from HyukjinKwon/SPARK-29783.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
2019-11-19 09:11:41 +09: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
Kent Yao e026412d9c [SPARK-29679][SQL] Make interval type comparable and orderable
### What changes were proposed in this pull request?

interval type support >, >=, <, <=, =, <=>, order by, min,max..

### Why are the changes needed?

Part of SPARK-27764 Feature Parity between PostgreSQL and Spark
### Does this PR introduce any user-facing change?

yes, we now support compare intervals

### How was this patch tested?

add ut

Closes #26337 from yaooqinn/SPARK-29679.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-08 22:45:11 +08:00
Kent Yao 0a03839366 [SPARK-29787][SQL] Move methods add/subtract/negate from CalendarInterval to IntervalUtils
### What changes were proposed in this pull request?

Move method add/subtract/negate from CalendarInterval to IntervalUtils

### Why are the changes needed?

https://github.com/apache/spark/pull/26410#discussion_r343125468 suggested here
### Does this PR introduce any user-facing change?

no
### How was this patch tested?

add uts and move some

Closes #26423 from yaooqinn/SPARK-29787.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-08 10:28:58 +08:00
Kent Yao 9562b26914 [SPARK-29757][SQL] Move calendar interval constants together
### What changes were proposed in this pull request?
```java
  public static final int YEARS_PER_DECADE = 10;
  public static final int YEARS_PER_CENTURY = 100;
  public static final int YEARS_PER_MILLENNIUM = 1000;

  public static final byte MONTHS_PER_QUARTER = 3;
  public static final int MONTHS_PER_YEAR = 12;

  public static final byte DAYS_PER_WEEK = 7;
  public static final long DAYS_PER_MONTH = 30L;

  public static final long HOURS_PER_DAY = 24L;

  public static final long MINUTES_PER_HOUR = 60L;

  public static final long SECONDS_PER_MINUTE = 60L;
  public static final long SECONDS_PER_HOUR = MINUTES_PER_HOUR * SECONDS_PER_MINUTE;
  public static final long SECONDS_PER_DAY = HOURS_PER_DAY * SECONDS_PER_HOUR;

  public static final long MILLIS_PER_SECOND = 1000L;
  public static final long MILLIS_PER_MINUTE = SECONDS_PER_MINUTE * MILLIS_PER_SECOND;
  public static final long MILLIS_PER_HOUR = MINUTES_PER_HOUR * MILLIS_PER_MINUTE;
  public static final long MILLIS_PER_DAY = HOURS_PER_DAY * MILLIS_PER_HOUR;

  public static final long MICROS_PER_MILLIS = 1000L;
  public static final long MICROS_PER_SECOND = MILLIS_PER_SECOND * MICROS_PER_MILLIS;
  public static final long MICROS_PER_MINUTE = SECONDS_PER_MINUTE * MICROS_PER_SECOND;
  public static final long MICROS_PER_HOUR = MINUTES_PER_HOUR * MICROS_PER_MINUTE;
  public static final long MICROS_PER_DAY = HOURS_PER_DAY * MICROS_PER_HOUR;
  public static final long MICROS_PER_MONTH = DAYS_PER_MONTH * MICROS_PER_DAY;
  /* 365.25 days per year assumes leap year every four years */
  public static final long MICROS_PER_YEAR = (36525L * MICROS_PER_DAY) / 100;

  public static final long NANOS_PER_MICROS = 1000L;
  public static final long NANOS_PER_MILLIS = MICROS_PER_MILLIS * NANOS_PER_MICROS;
  public static final long NANOS_PER_SECOND = MILLIS_PER_SECOND * NANOS_PER_MILLIS;
```
The above parameters are defined in IntervalUtils, DateTimeUtils, and CalendarInterval, some of them are redundant, some of them are cross-referenced.

### Why are the changes needed?
To simplify code, enhance consistency and reduce risks

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

no
### How was this patch tested?

modified uts

Closes #26399 from yaooqinn/SPARK-29757.

Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 19:48:19 +08: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
Maxim Gekk 29dc59ac29 [SPARK-29605][SQL] Optimize string to interval casting
### What changes were proposed in this pull request?
In the PR, I propose new function `stringToInterval()` in `IntervalUtils` for converting `UTF8String` to `CalendarInterval`. The function is used in casting a `STRING` column to an `INTERVAL` column.

### Why are the changes needed?
The proposed implementation is ~10 times faster. For example, parsing 9 interval units on JDK 8:
Before:
```
9 units w/ interval                               14004          14125         116          0.1       14003.6       0.0X
9 units w/o interval                              13785          14056         290          0.1       13784.9       0.0X
```
After:
```
9 units w/ interval                                1343           1344           1          0.7        1343.0       0.3X
9 units w/o interval                               1345           1349           8          0.7        1344.6       0.3X
```

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

### How was this patch tested?
- By new tests for `stringToInterval` in `IntervalUtilsSuite`
- By existing tests

Closes #26256 from MaxGekk/string-to-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-07 12:39:52 +08:00
Maxim Gekk 441d4c953e [SPARK-29723][SQL] Get date and time parts of an interval as java classes
### What changes were proposed in this pull request?
I propose 2 new methods for `CalendarInterval`:
- `extractAsPeriod()` returns the date part of an interval as an instance of `java.time.Period`
- `extractAsDuration()` returns the time part of an interval as an instance of `java.time.Duration`

For example:
```scala
scala> import org.apache.spark.unsafe.types.CalendarInterval
scala> import java.time._
scala> val i = spark.sql("select interval 1 year 3 months 4 days 10 hours 30 seconds").collect()(0).getAs[CalendarInterval](0)
scala> LocalDate.of(2019, 11, 1).plus(i.period())
res8: java.time.LocalDate = 2021-02-05
scala> ZonedDateTime.parse("2019-11-01T12:13:14Z").plus(i.extractAsPeriod()).plus(i.extractAsDuration())
res9: java.time.ZonedDateTime = 2021-02-05T22:13:44Z
```

### Why are the changes needed?
Taking into account that `CalendarInterval` has been already partially exposed to users via the collect operation, and probably it will be fully exposed in the future, it could be convenient for users to get the date and time parts of intervals as java classes:
- to avoid unnecessary dependency from Spark's classes in user code
- to easily use external libraries that accept standard Java classes.

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

### How was this patch tested?
By new test in `CalendarIntervalSuite`.

Closes #26368 from MaxGekk/interval-java-period-duration.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-04 11:07:54 -08:00
Maxim Gekk fb60c2a170 [SPARK-29671][SQL] Simplify string representation of intervals
### What changes were proposed in this pull request?
In the PR, I propose to changed `CalendarInterval.toString`:
- to skip the `week` unit
- to convert `milliseconds` and `microseconds` as the fractional part of the `seconds` unit.

### Why are the changes needed?
To improve readability.

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

### How was this patch tested?
- By `CalendarIntervalSuite` and `IntervalUtilsSuite`
- `literals.sql`, `datetime.sql` and `interval.sql`

Closes #26367 from MaxGekk/interval-to-string-format.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 22:56:59 -08:00
Maxim Gekk 80a89873b2 [SPARK-29733][TESTS] Fix wrong order of parameters passed to assertEquals
### What changes were proposed in this pull request?
The `assertEquals` method of JUnit Assert requires the first parameter to be the expected value. In this PR, I propose to change the order of parameters when the expected value is passed as the second parameter.

### Why are the changes needed?
Wrong order of assert parameters confuses when the assert fails and the parameters have special string representation. For example:
```java
assertEquals(input1.add(input2), new CalendarInterval(5, 5, 367200000000L));
```
```
java.lang.AssertionError:
Expected :interval 5 months 5 days 101 hours
Actual   :interval 5 months 5 days 102 hours
```

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

### How was this patch tested?
By existing tests.

Closes #26377 from MaxGekk/fix-order-in-assert-equals.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-11-03 11:21:28 -08:00
Liu,Linhong a4382f7fe1 [SPARK-29486][SQL] CalendarInterval should have 3 fields: months, days and microseconds
### What changes were proposed in this pull request?
Current CalendarInterval has 2 fields: months and microseconds. This PR try to change it
to 3 fields: months, days and microseconds. This is because one logical day interval may
have different number of microseconds (daylight saving).

### Why are the changes needed?
One logical day interval may have different number of microseconds (daylight saving).
For example, in PST timezone, there will be 25 hours from 2019-11-2 12:00:00 to
2019-11-3 12:00:00

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

### How was this patch tested?
unit test and new added test cases

Closes #26134 from LinhongLiu/calendarinterval.

Authored-by: Liu,Linhong <liulinhong@baidu.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-11-01 18:12:33 +08: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
Maxim Gekk 44c1c03924 [SPARK-29607][SQL] Move static methods from CalendarInterval to IntervalUtils
### What changes were proposed in this pull request?
In the PR, I propose to move all static methods from the `CalendarInterval` class to the `IntervalUtils` object. All those methods are rewritten from Java to Scala.

### Why are the changes needed?
- For consistency with other helper methods. Such methods were placed to the helper object `IntervalUtils`, see https://github.com/apache/spark/pull/26190
- Taking into account that `CalendarInterval` will be fully exposed to users in the future (see https://github.com/apache/spark/pull/25022), it would be nice to clean it up by moving service methods to an internal object.

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

### How was this patch tested?
- By moved tests from `CalendarIntervalSuite` to `IntervalUtilsSuite`
- By existing test suites

Closes #26261 from MaxGekk/refactoring-calendar-interval.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-10-30 01:15:18 +08: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
Wenchen Fan cdea520ff8 [SPARK-29532][SQL] Simplify interval string parsing
### What changes were proposed in this pull request?

Only use antlr4 to parse the interval string, and remove the duplicated parsing logic from `CalendarInterval`.

### Why are the changes needed?

Simplify the code and fix inconsistent behaviors.

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

No

### How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #26190 from cloud-fan/parser.

Lead-authored-by: Wenchen Fan <wenchen@databricks.com>
Co-authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-24 09:15:59 -07: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
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
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
Maxim Gekk b10344956d [SPARK-29342][SQL] Make casting of string values to intervals case insensitive
### What changes were proposed in this pull request?

In the PR, I propose to pass the `Pattern.CASE_INSENSITIVE` flag while compiling interval patterns in `CalendarInterval`. This makes casting string values to intervals case insensitive and tolerant to case of the `interval`, `year(s)`, `month(s)`, `week(s)`, `day(s)`, `hour(s)`, `minute(s)`, `second(s)`, `millisecond(s)` and `microsecond(s)`.

### Why are the changes needed?
There are at least 2 reasons:
- To maintain feature parity with PostgreSQL which is not sensitive to case:
```sql
 # select cast('10 Days' as INTERVAL);
 interval
----------
 10 days
(1 row)
```
- Spark is tolerant to case of interval literals. Case insensitivity in casting should be convenient for Spark users.
```sql
spark-sql> SELECT INTERVAL 1 YEAR 1 WEEK;
interval 1 years 1 weeks
```

### Does this PR introduce any user-facing change?
Yes, current implementation produces `NULL` for `interval`, `year`, ... `microsecond` that are not in lower case.
Before:
```sql
spark-sql> SELECT CAST('INTERVAL 10 DAYS' as INTERVAL);
NULL
```
After:
```sql
spark-sql> SELECT CAST('INTERVAL 10 DAYS' as INTERVAL);
interval 1 weeks 3 days
```

### How was this patch tested?
- by new tests in `CalendarIntervalSuite.java`
- new test in `CastSuite`

Closes #26010 from MaxGekk/interval-case-insensitive.

Authored-by: Maxim Gekk <max.gekk@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-10-07 09:33:01 -07: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
Yuanjian Li f725d472f5 [SPARK-25341][CORE] Support rolling back a shuffle map stage and re-generate the shuffle files
After the newly added shuffle block fetching protocol in #24565, we can keep this work by extending the FetchShuffleBlocks message.

### What changes were proposed in this pull request?
In this patch, we achieve the indeterminate shuffle rerun by reusing the task attempt id(unique id within an application) in shuffle id, so that each shuffle write attempt has a different file name. For the indeterministic stage, when the stage resubmits, we'll clear all existing map status and rerun all partitions.

All changes are summarized as follows:
- Change the mapId to mapTaskAttemptId in shuffle related id.
- Record the mapTaskAttemptId in MapStatus.
- Still keep mapId in ShuffleFetcherIterator for fetch failed scenario.
- Add the determinate flag in Stage and use it in DAGScheduler and the cleaning work for the intermediate stage.

### Why are the changes needed?
This is a follow-up work for #22112's future improvment[1]: `Currently we can't rollback and rerun a shuffle map stage, and just fail.`

Spark will rerun a finished shuffle write stage while meeting fetch failures, currently, the rerun shuffle map stage will only resubmit the task for missing partitions and reuse the output of other partitions. This logic is fine in most scenarios, but for indeterministic operations(like repartition), multiple shuffle write attempts may write different data, only rerun the missing partition will lead a correctness bug. So for the shuffle map stage of indeterministic operations, we need to support rolling back the shuffle map stage and re-generate the shuffle files.

### Does this PR introduce any user-facing change?
Yes, after this PR, the indeterminate stage rerun will be accepted by rerunning the whole stage. The original behavior is aborting the stage and fail the job.

### How was this patch tested?
- UT: Add UT for all changing code and newly added function.
- Manual Test: Also providing a manual test to verify the effect.
```
import scala.sys.process._
import org.apache.spark.TaskContext

val determinateStage0 = sc.parallelize(0 until 1000 * 1000 * 100, 10)
val indeterminateStage1 = determinateStage0.repartition(200)
val indeterminateStage2 = indeterminateStage1.repartition(200)
val indeterminateStage3 = indeterminateStage2.repartition(100)
val indeterminateStage4 = indeterminateStage3.repartition(300)
val fetchFailIndeterminateStage4 = indeterminateStage4.map { x =>
if (TaskContext.get.attemptNumber == 0 && TaskContext.get.partitionId == 190 &&
  TaskContext.get.stageAttemptNumber == 0) {
  throw new Exception("pkill -f -n java".!!)
  }
  x
}
val indeterminateStage5 = fetchFailIndeterminateStage4.repartition(200)
val finalStage6 = indeterminateStage5.repartition(100).collect().distinct.length
```
It's a simple job with multi indeterminate stage, it will get a wrong answer while using old Spark version like 2.2/2.3, and will be killed after #22112. With this fix, the job can retry all indeterminate stage as below screenshot and get the right result.
![image](https://user-images.githubusercontent.com/4833765/63948434-3477de00-caab-11e9-9ed1-75abfe6d16bd.png)

Closes #25620 from xuanyuanking/SPARK-25341-8.27.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-09-23 16:16:52 +08:00
Dongjoon Hyun 76bc9db749 [SPARK-29191][TESTS][SQL] Add tag ExtendedSQLTest for SQLQueryTestSuite
### What changes were proposed in this pull request?

This PR aims to add tag `ExtendedSQLTest` for `SQLQueryTestSuite`.
This doesn't affect our Jenkins test coverage.
Instead, this tag gives us an ability to parallelize them by splitting this test suite and the other suites.

### Why are the changes needed?

`SQLQueryTestSuite` takes 45 mins alone because it has many SQL scripts to run.

<img width="906" alt="time" src="https://user-images.githubusercontent.com/9700541/65353553-4af0f100-dba2-11e9-9f2f-386742d28f92.png">

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

No.

### How was this patch tested?

```
build/sbt "sql/test-only *.SQLQueryTestSuite" -Dtest.exclude.tags=org.apache.spark.tags.ExtendedSQLTest
...
[info] SQLQueryTestSuite:
[info] ScalaTest
[info] Run completed in 3 seconds, 147 milliseconds.
[info] Total number of tests run: 0
[info] Suites: completed 1, aborted 0
[info] Tests: succeeded 0, failed 0, canceled 0, ignored 0, pending 0
[info] No tests were executed.
[info] Passed: Total 0, Failed 0, Errors 0, Passed 0
[success] Total time: 22 s, completed Sep 20, 2019 12:23:13 PM
```

Closes #25872 from dongjoon-hyun/SPARK-29191.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-22 13:53:21 -07:00
colinma 076186e881 [SPARK-19147][CORE] Gracefully handle error in task after executor is stopped
### What changes were proposed in this pull request?

TransportClientFactory.createClient() is called by task and TransportClientFactory.close() is called by executor.
When stop the executor, close() will set workerGroup = null, NPE will occur in createClient which generate many exception in log.
For exception occurs after close(), treated it as an expected Exception
and transform it to InterruptedException which can be processed by Executor.

### Why are the changes needed?

The change can reduce the exception stack trace in log file, and user won't be confused by these excepted exception.

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

N/A

### How was this patch tested?

New tests are added in TransportClientFactorySuite and ExecutorSuite

Closes #25759 from colinmjj/spark-19147.

Authored-by: colinma <colinma@tencent.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-09-21 07:31:39 -05:00