Commit graph

417 commits

Author SHA1 Message Date
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
Sean Owen c5d8a51f3b [MINOR][BUILD] Fix about 15 misc build warnings
### What changes were proposed in this pull request?

This addresses about 15 miscellaneous warnings that appear in the current build.

### Why are the changes needed?

No functional changes, it just slightly reduces the amount of extra warning output.

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

No.

### How was this patch tested?

Existing tests, run manually.

Closes #25852 from srowen/BuildWarnings.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-19 11:37:42 -07:00
Dongjoon Hyun 471a3eff51 [SPARK-28932][BUILD][FOLLOWUP] Switch to scala-library compile dependency for JDK11
### What changes were proposed in this pull request?

This is a follow-up of https://github.com/apache/spark/pull/25638 to switch `scala-library` from `test` dependency to `compile` dependency in `network-common` module.

### Why are the changes needed?

Previously, we added `scala-library` as a test dependency to resolve the followings, but it was insufficient to resolve. This PR aims to switch it to compile dependency.
```
$ java -version
openjdk version "11.0.3" 2019-04-16
OpenJDK Runtime Environment AdoptOpenJDK (build 11.0.3+7)
OpenJDK 64-Bit Server VM AdoptOpenJDK (build 11.0.3+7, mixed mode)

$ mvn clean install -pl common/network-common -DskipTests
...
[INFO] --- scala-maven-plugin:4.2.0:doc-jar (attach-scaladocs)  spark-network-common_2.12 ---
error: fatal error: object scala in compiler mirror not found.
one error found
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
```

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

No.

### How was this patch tested?

Manually, run the following on JDK11.
```
$ mvn clean install -pl common/network-common -DskipTests
```

Closes #25800 from dongjoon-hyun/SPARK-28932-2.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-09-16 00:13:07 -07:00
Dongjoon Hyun 1f96ce5443 [SPARK-28932][BUILD] Add scala-library test dependency to network-common module for JDK11
### What changes were proposed in this pull request?

This PR adds `scala-library` test dependency to `network-common` module for JDK11.

### Why are the changes needed?

In JDK11, the following command fails due to scala library.
```
mvn clean install -pl common/network-common -DskipTests
```

**BEFORE**
```
...
error: fatal error: object scala in compiler mirror not found.
one error found
...
[INFO] ------------------------------------------------------------------------
[INFO] BUILD FAILURE
[INFO] ------------------------------------------------------------------------
```

**AFTER**
```
[INFO] ------------------------------------------------------------------------
[INFO] BUILD SUCCESS
[INFO] ------------------------------------------------------------------------
```

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

No.

### How was this patch tested?

Manual. On JDK11, do the following.
```
mvn clean install -pl common/network-common -DskipTests
```

Closes #25638 from dongjoon-hyun/SPARK-28932.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-31 10:59:20 -07:00
younggyu chun 8535df7261 [MINOR] Fix typos in comments and replace an explicit type with <>
## What changes were proposed in this pull request?
This PR fixed typos in comments and replace the explicit type with '<>' for Java 8+.

## How was this patch tested?
Manually tested.

Closes #25338 from younggyuchun/younggyu.

Authored-by: younggyu chun <younggyuchun@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-10 16:47:11 -05:00
Jungtaek Lim (HeartSaVioR) 128ea37bda [SPARK-28601][CORE][SQL] Use StandardCharsets.UTF_8 instead of "UTF-8" string representation, and get rid of UnsupportedEncodingException
## What changes were proposed in this pull request?

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

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

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

## How was this patch tested?

Existing unit tests.

Closes #25335 from HeartSaVioR/SPARK-28601.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-08-05 20:45:54 -07:00
Yuanjian Li db39f45baf [SPARK-28593][CORE] Rename ShuffleClient to BlockStoreClient which more close to its usage
## What changes were proposed in this pull request?

After SPARK-27677, the shuffle client not only handles the shuffle block but also responsible for local persist RDD blocks. For better code scalability and precise semantics(as the [discussion](https://github.com/apache/spark/pull/24892#discussion_r300173331)), here we did several changes:

- Rename ShuffleClient to BlockStoreClient.
- Correspondingly rename the ExternalShuffleClient to ExternalBlockStoreClient, also change the server-side class from ExternalShuffleBlockHandler to ExternalBlockHandler.
- Move MesosExternalBlockStoreClient to Mesos package.

Note, we still keep the name of BlockTransferService, because the `Service` contains both client and server, also the name of BlockTransferService is not referencing shuffle client only.

## How was this patch tested?

Existing UT.

Closes #25327 from xuanyuanking/SPARK-28593.

Lead-authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Co-authored-by: Yuanjian Li <yuanjian.li@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-08-05 14:54:45 +08:00
Sean Owen c09675779b [SPARK-28604][ML] Use log1p(x) over log(1+x) and expm1(x) over exp(x)-1 for accuracy
## What changes were proposed in this pull request?

Use `log1p(x)` over `log(1+x)` and `expm1(x)` over `exp(x)-1` for accuracy, where possible. This should improve accuracy a tiny bit in ML-related calculations, and shouldn't hurt in any event.

## How was this patch tested?

Existing tests.

Closes #25337 from srowen/SPARK-28604.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-08-04 17:04:01 -05:00
Zhu, Lipeng d26642dbbc [SPARK-28107][SQL] Support 'DAY TO (HOUR|MINUTE|SECOND)', 'HOUR TO (MINUTE|SECOND)' and 'MINUTE TO SECOND'
## What changes were proposed in this pull request?
The interval conversion behavior is same with the PostgreSQL.

https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/interval.sql#L180-L203

## How was this patch tested?
UT.

Closes #25000 from lipzhu/SPARK-28107.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-10 18:01:42 -07:00
Dongjoon Hyun a5ff9221fc [SPARK-28308][CORE] CalendarInterval sub-second part should be padded before parsing
## What changes were proposed in this pull request?

The sub-second part of the interval should be padded before parsing. Currently, Spark gives a correct value only when there is 9 digits below `.`.
```
spark-sql> select interval '0 0:0:0.123456789' day to second;
interval 123 milliseconds 456 microseconds

spark-sql> select interval '0 0:0:0.12345678' day to second;
interval 12 milliseconds 345 microseconds

spark-sql> select interval '0 0:0:0.1234' day to second;
interval 1 microseconds
```

## How was this patch tested?

Pass the Jenkins with the fixed test cases.

Closes #25079 from dongjoon-hyun/SPARK-28308.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-08 19:40:41 -07:00
Gabor Somogyi e11a55827e [SPARK-28261][CORE] Fix client reuse test
## What changes were proposed in this pull request?

There is the following code in [TransportClientFactory#createClient](https://github.com/apache/spark/blob/master/common/network-common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java#L150)
```
    int clientIndex = rand.nextInt(numConnectionsPerPeer);
    TransportClient cachedClient = clientPool.clients[clientIndex];
```
which choose a client from its pool randomly. If we are unlucky we might not get the max number of connections out, but less than that.

To prove that I've tried out the following test:
```java
  Test
  public void testRandom() {
    Random rand = new Random();
    Set<Integer> clients = Collections.synchronizedSet(new HashSet<>());
    long iterCounter = 0;
    while (true) {
      iterCounter++;
      int maxConnections = 4;
      clients.clear();
      for (int i = 0; i < maxConnections * 10; i++) {
        int clientIndex = rand.nextInt(maxConnections);
        clients.add(clientIndex);
      }
      if (clients.size() != maxConnections) {
        System.err.println("Unexpected clients size (iterCounter=" + iterCounter + "): " + clients.size() + ", maxConnections: " + maxConnections);
      }
      if (iterCounter % 100000 == 0) {
        System.out.println("IterCounter: " + iterCounter);
      }
    }
  }
```

Result:
```
Unexpected clients size (iterCounter=22388): 3, maxConnections: 4
Unexpected clients size (iterCounter=36244): 3, maxConnections: 4
Unexpected clients size (iterCounter=85798): 3, maxConnections: 4
IterCounter: 100000
Unexpected clients size (iterCounter=97108): 3, maxConnections: 4
Unexpected clients size (iterCounter=119121): 3, maxConnections: 4
Unexpected clients size (iterCounter=129948): 3, maxConnections: 4
Unexpected clients size (iterCounter=173736): 3, maxConnections: 4
Unexpected clients size (iterCounter=178138): 3, maxConnections: 4
Unexpected clients size (iterCounter=195108): 3, maxConnections: 4
IterCounter: 200000
Unexpected clients size (iterCounter=209006): 3, maxConnections: 4
Unexpected clients size (iterCounter=217105): 3, maxConnections: 4
Unexpected clients size (iterCounter=222456): 3, maxConnections: 4
Unexpected clients size (iterCounter=226899): 3, maxConnections: 4
Unexpected clients size (iterCounter=229101): 3, maxConnections: 4
Unexpected clients size (iterCounter=253549): 3, maxConnections: 4
Unexpected clients size (iterCounter=277550): 3, maxConnections: 4
Unexpected clients size (iterCounter=289637): 3, maxConnections: 4
...
```

In this PR I've adapted the test code not to have this flakyness.

## How was this patch tested?

Additional (not committed test) + existing unit tests in a loop.

Closes #25075 from gaborgsomogyi/SPARK-28261.

Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-07-08 11:10:03 -07:00
LantaoJin 0e421000e0 [SPARK-28160][CORE] Fix a bug that callback function may hang when unchecked exception missed
## What changes were proposed in this pull request?

This is very like #23590 .

`ByteBuffer.allocate` may throw `OutOfMemoryError` when the response is large but no enough memory is available. However, when this happens, `TransportClient.sendRpcSync` will just hang forever if the timeout set to unlimited.

This PR catches `Throwable` and uses the error to complete `SettableFuture`.

## How was this patch tested?

I tested in my IDE by setting the value of size to -1 to verify the result. Without this patch, it won't be finished until timeout (May hang forever if timeout set to MAX_INT), or the expected `IllegalArgumentException` will be caught.
```java
Override
      public void onSuccess(ByteBuffer response) {
        try {
          int size = response.remaining();
          ByteBuffer copy = ByteBuffer.allocate(size); // set size to -1 in runtime when debug
          copy.put(response);
          // flip "copy" to make it readable
          copy.flip();
          result.set(copy);
        } catch (Throwable t) {
          result.setException(t);
        }
      }
```

Closes #24964 from LantaoJin/SPARK-28160.

Lead-authored-by: LantaoJin <jinlantao@gmail.com>
Co-authored-by: lajin <lajin@ebay.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-30 15:14:41 -05:00
“attilapiros” b71c130fc6 [SPARK-27622][CORE] Avoiding the network when block manager fetches disk persisted RDD blocks from the same host
## What changes were proposed in this pull request?

Before this PR during fetching a disk persisted RDD block the network was always used to get the requested block content even when both the source and fetcher executor was running on the same host.

The idea to access another executor local disk files by directly reading the disk comes from the external shuffle service where the local dirs are stored for each executor (block manager).

To make this possible the following changes are done:
- `RegisterBlockManager` message is extended with the `localDirs` which is stored by the block manager master for each block manager as a new property of the `BlockManagerInfo`
- `GetLocationsAndStatus` is extended with the requester host
- `BlockLocationsAndStatus` (the reply for `GetLocationsAndStatus` message) is extended with the an option of local directories, which is filled with a local directories of a same host executor (if there is any, otherwise None is used). This is where the block content can be read from.

Shuffle blocks are out of scope of this PR: there will be a separate PR opened for that (for another Jira issue).

## How was this patch tested?

With a new unit test in `BlockManagerSuite`. See the the test prefixed by "SPARK-27622: avoid the network when block requested from same host".

Closes #24554 from attilapiros/SPARK-27622.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-06-25 07:35:44 -07:00
ketank-new 1a3858a769 [SPARK-26985][CORE] Fix "access only some column of the all of columns " for big endian architecture BUG
continuation to https://github.com/apache/spark/pull/24788

## What changes were proposed in this pull request?

Changes are related to BIG ENDIAN system
This changes are done to

identify s390x platform.
use byteorder to BIG_ENDIAN for big endian systems
changes for 2 are done in access functions putFloats() and putDouble()

## How was this patch tested?

Changes have been tested to build successfully on s390x as well x86 platform to make sure build is successful.

Closes #24861 from ketank-new/ketan_latest_v2.3.2.

Authored-by: ketank-new <ketan22584@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-25 08:24:10 -05:00
Sean Owen 9df7587eea [MINOR][CORE] Fix line too long in TransportClientFactory
## What changes were proposed in this pull request?

See https://github.com/apache/spark/pull/24702/files#r296765487 -- this just fixes a Java style error. I'm not clear why the PR build didn't catch it.

## How was this patch tested?

N/A

Closes #24951 from srowen/SPARK-27989.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 11:17:54 -05:00
Jose Luis Pedrosa 0671395966 [SPARK-27989][CORE] Added retries on the connection to the driver for k8s
Disabled negative dns caching for docker images

Improved logging on DNS resolution, convenient for slow k8s clusters

## What changes were proposed in this pull request?
Added retries when building the connection to the driver in K8s.
In some scenarios DNS reslution can take more than the timeout.
Also openjdk-8 by default has negative dns caching enabled, which means even retries may not help depending on the times.

## How was this patch tested?
This patch was tested agains an specific k8s cluster with slow response time in DNS to ensure it woks.

Closes #24702 from jlpedrosa/feature/kuberetries.

Authored-by: Jose Luis Pedrosa <jlpedrosa@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-06-24 09:25:43 -05:00
Josh Rosen fc65e0fe2c [SPARK-27839][SQL] Change UTF8String.replace() to operate on UTF8 bytes
## What changes were proposed in this pull request?

This PR significantly improves the performance of `UTF8String.replace()` by performing direct replacement over UTF8 bytes instead of decoding those bytes into Java Strings.

In cases where the search string is not found (i.e. no replacements are performed, a case which I expect to be common) this new implementation performs no object allocation or memory copying.

My implementation is modeled after `commons-lang3`'s `StringUtils.replace()` method. As part of my implementation, I needed a StringBuilder / resizable buffer, so I moved `UTF8StringBuilder` from the `catalyst` package to `unsafe`.

## How was this patch tested?

Copied tests from `StringExpressionSuite` to `UTF8StringSuite` and added a couple of new cases.

To evaluate performance, I did some quick local benchmarking by running the following code in `spark-shell` (with Java 1.8.0_191):

```scala
import org.apache.spark.unsafe.types.UTF8String

def benchmark(text: String, search: String, replace: String) {
  val utf8Text = UTF8String.fromString(text)
  val utf8Search = UTF8String.fromString(search)
  val utf8Replace = UTF8String.fromString(replace)

  val start = System.currentTimeMillis
  var i = 0
  while (i < 1000 * 1000 * 100) {
    utf8Text.replace(utf8Search, utf8Replace)
    i += 1
  }
  val end = System.currentTimeMillis

  println(end - start)
}

benchmark("ABCDEFGH", "DEF", "ZZZZ")  // replacement occurs
benchmark("ABCDEFGH", "Z", "")  // no replacement occurs
```

On my laptop this took ~54 / ~40 seconds seconds before this patch's changes and ~6.5 / ~3.8 seconds afterwards.

Closes #24707 from JoshRosen/faster-string-replace.

Authored-by: Josh Rosen <rosenville@gmail.com>
Signed-off-by: Josh Rosen <rosenville@gmail.com>
2019-06-19 15:21:26 -07:00
Sean Owen 4576dfde19 [SPARK-28066][CORE] Optimize UTF8String.trim() for common case of no whitespace
## What changes were proposed in this pull request?

UTF8String.trim() allocates a new object even if the string has no whitespace, when it can just return itself. A simple check for this case makes the method about 3x faster in the common case.

## How was this patch tested?

Existing tests.

A rough benchmark of 90% strings without whitespace (at ends), and 10% that do have whitespace, suggests the average runtime goes from 20 ns to 6 ns.

Closes #24884 from srowen/SPARK-28066.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-17 08:49:11 -07:00
Zhu, Lipeng 5700c39c89 [SPARK-27578][SQL] Support INTERVAL ... HOUR TO SECOND syntax
## What changes were proposed in this pull request?

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

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

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

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

## How was this patch tested?

Pass the Jenkins with the updated test cases.

Closes #24472 from lipzhu/SPARK-27578.

Lead-authored-by: Zhu, Lipeng <lipzhu@ebay.com>
Co-authored-by: Dongjoon Hyun <dhyun@apple.com>
Co-authored-by: Lipeng Zhu <lipzhu@icloud.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-13 10:12:55 -07:00
Dongjoon Hyun e561e92765 [SPARK-27981][CORE] Remove Illegal reflective access warning for java.nio.Bits.unaligned() in JDK9+
## What changes were proposed in this pull request?

This PR aims to remove the following warnings for `java.nio.Bits.unaligned` at JDK9/10/11/12. Please note that there are more warnings which is beyond of this PR's scope. JDK9+ shows the first warning only if you don't give `--illegal-access=warn`.

**BEFORE (Among 5 warnings, there is `java.nio.Bits.unaligned` warning at the startup)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to method java.nio.Bits.unaligned()
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getInstance()
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/APACHE/spark/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getDefaultRealm()
19/06/08 11:01:19 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560016882712).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
```

**AFTER (Among 4 warnings, there is no `java.nio.Bits.unaligned` warning with `hadoop-2.7` profile)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getInstance()
WARNING: Illegal reflective access by org.apache.hadoop.security.authentication.util.KerberosUtil (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/hadoop-auth-2.7.4.jar) to method sun.security.krb5.Config.getDefaultRealm()
19/06/08 11:08:27 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560017311171).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
```

**AFTER (Among 2 warnings, there is no `java.nio.Bits.unaligned` warning with `hadoop-3.2` profile)**
```
$ bin/spark-shell --driver-java-options=--illegal-access=warn
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to constructor java.nio.DirectByteBuffer(long,int)
WARNING: Illegal reflective access by org.apache.spark.unsafe.Platform (file:/Users/dhyun/PRS/PLATFORM/assembly/target/scala-2.12/jars/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar) to field java.nio.DirectByteBuffer.cleaner
19/06/08 10:52:06 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
Setting default log level to "WARN".
To adjust logging level use sc.setLogLevel(newLevel). For SparkR, use setLogLevel(newLevel).
Spark context Web UI available at http://localhost:4040
Spark context available as 'sc' (master = local[*], app id = local-1560016330287).
Spark session available as 'spark'.
Welcome to
      ____              __
     / __/__  ___ _____/ /__
    _\ \/ _ \/ _ `/ __/  '_/
   /___/ .__/\_,_/_/ /_/\_\   version 3.0.0-SNAPSHOT
      /_/

Using Scala version 2.12.8 (OpenJDK 64-Bit Server VM, Java 11.0.3)
...
```

## How was this patch tested?

Manual. Run Spark command like `spark-shell` with `--driver-java-options=--illegal-access=warn` option in JDK9/10/11/12 environment.

Closes #24825 from dongjoon-hyun/SPARK-27981.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-06-08 16:39:32 -07:00
gatorsmile 2e84181ec3 [SPARK-27773][FOLLOW-UP] Fix Checkstyle failure
## What changes were proposed in this pull request?

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-lint/

```
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[99] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[101] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[103] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java:[105] (sizes) LineLength: Line is longer than 100 characters (found 103).
```

## How was this patch tested?
N/A

Closes #24760 from gatorsmile/updateYarnShuffleServiceMetrics.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-31 09:30:17 -07:00
Steven Rand 568512cc82 [SPARK-27773][SHUFFLE] add metrics for number of exceptions caught in ExternalShuffleBlockHandler
## What changes were proposed in this pull request?

Add a metric for number of exceptions caught in the `ExternalShuffleBlockHandler`, the idea being that spikes in this metric over some time window (or more desirably, the lack thereof) can be used as an indicator of the health of an external shuffle service. (Where "health" refers to its ability to successfully respond to client requests.)

## How was this patch tested?

Deployed a build of this PR to a YARN cluster, and confirmed that the NodeManagers' JMX metrics include `numCaughtExceptions`.

Closes #24645 from sjrand/SPARK-27773.

Authored-by: Steven Rand <srand@palantir.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-30 13:57:15 -07:00
Marcelo Vanzin 09ed64d795 [SPARK-27868][CORE] Better default value and documentation for socket server backlog.
First, there is currently no public documentation for this setting. So it's hard
to even know that it could be a problem if your application starts failing with
weird shuffle errors.

Second, the javadoc attached to the code was incorrect; the default value just uses
the default value from the JRE, which is 50, instead of having an unbounded queue
as the comment implies.

So use a default that is a "rounded" version of the JRE default, and provide
documentation explaining that this value may need to be adjusted. Also added
a log message that was very helpful in debugging an issue caused by this
problem.

Closes #24732 from vanzin/SPARK-27868.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-29 14:56:36 -07:00
Shixiong Zhu 04f142db9c
[SPARK-20547][REPL] Throw RemoteClassLoadedError for transient errors in ExecutorClassLoader
## What changes were proposed in this pull request?

`ExecutorClassLoader`'s `findClass` may fail to fetch a class due to transient exceptions. For example, when a task is interrupted, if `ExecutorClassLoader` is fetching a class, you may see `InterruptedException` or `IOException` wrapped by `ClassNotFoundException`, even if this class can be loaded. Then the result of `findClass` will be cached by JVM, and later when the same class is being loaded in the same executor, it will just throw NoClassDefFoundError even if the class can be loaded.

I found JVM only caches `LinkageError` and `ClassNotFoundException`. Hence in this PR, I changed ExecutorClassLoader to throw `RemoteClassLoadedError` if we cannot get a response from driver.

## How was this patch tested?

New unit tests.

Closes #24683 from zsxwing/SPARK-20547-fix.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2019-05-28 12:56:14 -07:00
Yuanjian Li 8949bc7a3c [SPARK-27665][CORE] Split fetch shuffle blocks protocol from OpenBlocks
## What changes were proposed in this pull request?

As the current approach in OneForOneBlockFetcher, we reuse the OpenBlocks protocol to describe the fetch request for shuffle blocks, and it causes the extension work for shuffle fetching like #19788 and #24110 very awkward.
In this PR, we split the fetch request for shuffle blocks from OpenBlocks which named FetchShuffleBlocks. It's a loose bind with ShuffleBlockId and can easily extend by adding new fields in this protocol.

## How was this patch tested?

Existing and new added UT.

Closes #24565 from xuanyuanking/SPARK-27665.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-05-27 22:19:31 +08:00
“attilapiros” 1e87694f2b [SPARK-27677][CORE] Disable by default fetching of disk persisted RDD blocks via external shuffle service
## What changes were proposed in this pull request?

In the PR the config `spark.shuffle.service.fetch.rdd.enabled` default is changed to **false** to avoid breaking any compatibility with older external shuffle service installations. As external shuffle service is deployed separately and disk persisted RDD block fetching had even introduced new network messages (`RemoveBlocks` and `BlocksRemoved`) and changed the behaviour of the already existing fetching: extended it for RDD blocks.

## How was this patch tested?

With existing unit tests.

Closes #24697 from attilapiros/minor-ext-shuffle-fetch-disabled.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-24 11:58:26 -07:00
“attilapiros” e9f3f62b2c [SPARK-27677][CORE] Serve local disk persisted blocks by the external service after releasing executor by dynamic allocation
# What changes were proposed in this pull request?

## Problem statement

An executor which has persisted blocks does not consider to be idle and this way ready to be released by dynamic allocation after the regular timeout `spark.dynamicAllocation.executorIdleTimeout` but there is separate configuration `spark.dynamicAllocation.cachedExecutorIdleTimeout` which defaults to `Integer.MAX_VALUE`. This is because releasing the executor also means losing the persisted blocks (as the metadata for individual blocks called `BlockInfo` are kept in memory) and when the RDD is referenced latter on this lost blocks will be recomputed.
On the other hand keeping the executors too long without any task to work on is also a waste of resources (as executors are reserved for the application by the resource manager).

## Solution

This PR focuses on the first part of SPARK-25888: it extends the external shuffle service with the capability to serve RDD blocks which are persisted on the local disk store by the executors. Moreover when this feature is enabled by setting the `spark.shuffle.service.fetch.rdd.enabled` config to true and a block is reported to be persisted on to disk the external shuffle service instance running on the same host as the executor is also registered (along with the reporting block manager) as a possible location for fetching it.

## Some implementation detail

Some explanation about the decisions made during the development:
- the location list to fetch a block was randomized but the groups (same host, same rack, others) order was kept. In this PR the order of groups are kept and external shuffle service added to the end of the each group.
- `BlockManagerInfo` is not introduced for external shuffle service but only a lightweight solution is taken. A hash map from `BlockId` to `BlockStatus` is introduced. A type alias would make the source more readable but I know it is discouraged. On the other hand a new class wrapping this hash map would introduce unnecessary indirection.
- when this feature is on the cleanup triggered during removing of executors (which is handled in `ExternalShuffleBlockResolver`) is modified to keep the disk persisted RDD blocks. This cleanup is triggered in standalone mode when the `spark.storage.cleanupFilesAfterExecutorExit` config is set.
- the unpersisting of an RDD is extended to use the external shuffle service for disk persisted RDD blocks when the original executor which created the blocks are already released. New block transport messages are introduced to support this: `RemoveBlocks` and `BlocksRemoved`.

# How was this patch tested?

## Unit tests

### ExternalShuffleServiceSuite

Here the complete use case is tested by the "SPARK-25888: using external shuffle service fetching disk persisted blocks" with a tiny difference: here the executor is killed manually, this way the test is a bit faster than waiting for the idle timeout.

### ExternalShuffleBlockHandlerSuite

Tests the fetching of the RDD blocks via the external shuffle service.

### BlockManagerInfoSuite

This a new suite. As the `BlockManagerInfo` behaviour depends very much on whether the external shuffle service enabled or not all the tests are executed with and without it.

### BlockManagerSuite

Tests the sorting of the block locations.

## Manually on YARN

Spark App was:

~~~scala
package com.mycompany

import org.apache.spark.rdd.RDD
import org.apache.spark.{SparkContext, SparkConf}
import org.apache.spark.storage.StorageLevel

object TestAppDiskOnlyLevel {
  def main(args: Array[String]): Unit = {
    val conf = new SparkConf().setAppName("test-app")

    println("Attila: START")
    val sc = new SparkContext(conf)
    val rdd = sc.parallelize(0 until 100, 10)
      .map { i =>
        println(s"Attila: calculate first rdd i=$i")
        Thread.sleep(1000)
        i
      }

    rdd.persist(StorageLevel.DISK_ONLY)
    rdd.count()

    println("Attila: First RDD is processed, waiting for 60 sec")

    Thread.sleep(60 * 1000)

    println("Attila: Num executors must be 0 as executorIdleTimeout is way over")

    val rdd2 = sc.parallelize(0 until 10, 1)
      .map(i => (i, 1))
      .persist(StorageLevel.DISK_ONLY)

    rdd2.count()

    println("Attila: Second RDD with one partition (only one executors must be alive)")

    // reduce runs as user code to detect the empty seq (empty blocks)
    println("Calling collect on the first RDD: " + rdd.collect().reduce(_ + _))

    println("Attila: STOP")
  }
}
~~~

I have submitted with the following configuration:

~~~bash
spark-submit --master yarn \
  --conf spark.dynamicAllocation.enabled=true \
  --conf spark.dynamicAllocation.executorIdleTimeout=30 \
  --conf spark.dynamicAllocation.cachedExecutorIdleTimeout=90 \
  --class com.mycompany.TestAppDiskOnlyLevel dyn_alloc_demo-core_2.11-0.1.0-SNAPSHOT-jar-with-dependencies.jar
~~~

Checked the result by filtering for the side effect of the task calculations:

~~~bash
[userserver ~]$ yarn logs -applicationId application_1556299359453_0001 | grep "Attila: calculate" | wc -l
WARNING: YARN_OPTS has been replaced by HADOOP_OPTS. Using value of YARN_OPTS.
19/04/26 10:31:59 INFO client.RMProxy: Connecting to ResourceManager at apiros-1.gce.company.com/172.31.115.165:8032
100
~~~

So it is only 100 task execution and not 200 (which would be the case for re-computation).

Moreover from the submit/launcher log we can see executors really stopped in between (see the new total is 0 before the last line):
~~~
[userserver ~]$ grep "Attila: Num executors must be 0" -B 2 spark-submit.log
19/04/26 10:24:27 INFO cluster.YarnScheduler: Executor 9 on apiros-3.gce.company.com killed by driver.
19/04/26 10:24:27 INFO spark.ExecutorAllocationManager: Existing executor 9 has been removed (new total is 0)
Attila: Num executors must be 0 as executorIdleTimeout is way over
~~~

[Full spark submit log](https://github.com/attilapiros/spark/files/3122465/spark-submit.log)

I have done a test also after changing the `DISK_ONLY` storage level to `MEMORY_ONLY` for the first RDD. After this change during the 60sec waiting no executor was removed.

Closes #24499 from attilapiros/SPARK-25888-final.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-23 13:15:39 -07:00
David Navas 9e73be38a5 [SPARK-27726][CORE] Fix performance of ElementTrackingStore deletes when using InMemoryStore under high loads
The details of the PR are explored in-depth in the sub-tasks of the umbrella jira SPARK-27726.
Briefly:
  1. Stop issuing asynchronous requests to cleanup elements in the tracking store when a request is already pending
  2. Fix a couple of thread-safety issues (mutable state and mis-ordered updates)
  3. Move Summary deletion outside of Stage deletion loop like Tasks already are
  4. Reimplement multi-delete in a removeAllKeys call which allows InMemoryStore to implement it in a performant manner.
  5. Some generic typing and exception handling cleanup

We see about five orders of magnitude improvement in the deletion code, which for us is the difference between a server that needs restarting daily, and one that is stable over weeks.

Unit tests for the fire-once asynchronous code and the removeAll calls in both LevelDB and InMemoryStore are supplied.  It was noted that the testing code for the LevelDB and InMemoryStore is highly repetitive, and should probably be merged, but we did not attempt that in this PR.

A version of this code was run in our production 2.3.3 and we were able to sustain higher throughput without going into GC overload (which was happening on a daily basis some weeks ago).

A version of this code was also put under a purpose-built Performance Suite of tests to verify performance under both types of Store implementations for both before and after code streams and for both total and partial delete cases (this code is not included in this PR).

Closes #24616 from davidnavas/PentaBugFix.

Authored-by: David Navas <davidn@clearstorydata.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-21 10:22:21 -07:00
Yuming Wang 93b5a2b686 [SPARK-27610][FOLLOW-UP][YARN] Remove duplicate declaration of plugin maven-antrun-plugin
## What changes were proposed in this pull request?

This pr removes duplicate declaration of plugin `org.apache.maven.plugins:maven-antrun-plugin`:
```
[WARNING] Some problems were encountered while building the effective model for org.apache.spark:spark-network-yarn_2.12🫙3.0.0-SNAPSHOT
[WARNING] 'build.plugins.plugin.(groupId:artifactId)' must be unique but found duplicate declaration of plugin org.apache.maven.plugins:maven-antrun-plugin  line 177, column 15
[WARNING]
[WARNING] It is highly recommended to fix these problems because they threaten the stability of your build.
[WARNING]
[WARNING] For this reason, future Maven versions might no longer support building such malformed projects.
[WARNING]
```
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105523/consoleFull

## How was this patch tested?

Existing test

Closes #24641 from wangyum/SPARK-27610.

Authored-by: Yuming Wang <yumwang@ebay.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-19 20:59:35 -07:00
Shixiong Zhu 6a317c8f01 [SPARK-27735][SS] Parsing interval string should be case-insensitive in SS
## What changes were proposed in this pull request?

Some APIs in Structured Streaming requires the user to specify an interval. Right now these APIs don't accept upper-case strings.

This PR adds a new method `fromCaseInsensitiveString` to `CalendarInterval` to support paring upper-case strings, and fixes all APIs that need to parse an interval string.

## How was this patch tested?

The new unit test.

Closes #24619 from zsxwing/SPARK-27735.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-16 13:58:27 -07:00
Dongjoon Hyun 614a5cc600 [SPARK-27624][CORE] Fix CalenderInterval to show an empty interval correctly
## What changes were proposed in this pull request?

If the interval is `0`, it doesn't show both the value `0` and the unit at all. For example, this happens in the explain plans and Spark Web UI on `EventTimeWatermark` diagram.

**BEFORE**
```scala
scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "1 microsecond").explain
== Physical Plan ==
EventTimeWatermark ts#0: timestamp, interval 1 microseconds
+- StreamingRelation FileSource[/tmp/t], [ts#0]

scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "0 microsecond").explain
== Physical Plan ==
EventTimeWatermark ts#3: timestamp, interval
+- StreamingRelation FileSource[/tmp/t], [ts#3]
```

**AFTER**
```scala
scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "1 microsecond").explain
== Physical Plan ==
EventTimeWatermark ts#0: timestamp, interval 1 microseconds
+- StreamingRelation FileSource[/tmp/t], [ts#0]

scala> spark.readStream.schema("ts timestamp").parquet("/tmp/t").withWatermark("ts", "0 microsecond").explain
== Physical Plan ==
EventTimeWatermark ts#3: timestamp, interval 0 microseconds
+- StreamingRelation FileSource[/tmp/t], [ts#3]
```

## How was this patch tested?

Pass the Jenkins with the updated test case.

Closes #24516 from dongjoon-hyun/SPARK-27624.

Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-05-07 11:08:55 -07:00
Adi Muraru 8ef4da753d [SPARK-27610][YARN] Shade netty native libraries
## What changes were proposed in this pull request?

Fixed the `spark-<version>-yarn-shuffle.jar` artifact packaging to shade the native netty libraries:
- shade the `META-INF/native/libnetty_*` native libraries when packagin
the yarn shuffle service jar. This is required as netty library loader
derives that based on shaded package name.
- updated the `org/spark_project` shade package prefix to `org/sparkproject`
(i.e. removed underscore) as the former breaks the netty native lib loading.

This was causing the yarn external shuffle service to fail
when spark.shuffle.io.mode=EPOLL

## How was this patch tested?
Manual tests

Closes #24502 from amuraru/SPARK-27610_master.

Authored-by: Adi Muraru <amuraru@adobe.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-05-07 10:47:36 -07:00
Sean Owen 4ec7f631aa [SPARK-27404][CORE][SQL][STREAMING][YARN] Fix build warnings for 3.0: postfixOps edition
## What changes were proposed in this pull request?

Fix build warnings -- see some details below.

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

## How was this patch tested?

Existing tests.

Closes #24314 from srowen/SPARK-27404.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-04-11 13:43:44 -05:00
Shixiong Zhu 5624bfbcfe [SPARK-27275][CORE] Fix potential corruption in EncryptedMessage.transferTo
## What changes were proposed in this pull request?

Right now there are several issues in `EncryptedMessage.transferTo`:

- When the underlying buffer has more than `1024 * 32` bytes (this should be rare but it could happen in error messages that send over the wire), it may just send a partial message as `EncryptedMessage.count` becomes less than `transferred`. This will cause the client hang forever (or timeout) as it will wait until receiving expected length of bytes, or weird errors (such as corruption or silent correctness issue) if the channel is reused by other messages.
- When the underlying buffer is full, it's still trying to write out bytes in a busy loop.

This PR fixes  the issues in `EncryptedMessage.transferTo` and also makes it follow the contract of `FileRegion`:

- `count` should be a fixed value which is just the length of the whole message.
- It should be non-blocking. When the underlying socket is not ready to write, it should give up and give control back.
- `transferTo` should return the length of written bytes.

## How was this patch tested?

The new added tests.

Closes #24211 from zsxwing/fix-enc.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-03-26 15:48:29 -07:00
Marcelo Vanzin db801cf3f2 [SPARK-27219][CORE] Treat timeouts as fatal in SASL fallback path.
When a timeout happens we don't know what's the state of the remote end,
so there is no point in doing anything else since it will most probably
fail anyway.

The change also demotes the log message printed when falling back to
SASL, since a warning is too noisy for when the fallback is really
needed (e.g. old shuffle service, or shuffle service with new auth
disabled).

Closes #24160 from vanzin/SPARK-27219.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
2019-03-24 21:49:54 -07:00
weixiuli 8b0aa59218 [SPARK-26288][CORE] add initRegisteredExecutorsDB
## What changes were proposed in this pull request?

As we all know that spark on Yarn uses DB https://github.com/apache/spark/pull/7943 to record RegisteredExecutors information which can be reloaded and used again when the ExternalShuffleService is restarted .

The RegisteredExecutors information can't be recorded both in the mode of spark's standalone and spark on k8s , which will cause the RegisteredExecutors information to be lost ,when the ExternalShuffleService is restarted.

To solve the problem above, a method is proposed and is committed .

## How was this patch tested?
new  unit tests

Closes #23393 from weixiuli/SPARK-26288.

Authored-by: weixiuli <weixiuli@jd.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-03-19 16:16:43 -05:00
sychen 064604aaa7 [SPARK-27073][CORE] Fix a race condition when handling of IdleStateEvent
## What changes were proposed in this pull request?

When TransportChannelHandler processes IdleStateEvent, it first calculates whether the last request time has timed out.
At this time, TransportClient.sendRpc initiates a request.
TransportChannelHandler gets responseHandler.numOutstandingRequests() > 0, causing the normal connection to be closed.

## How was this patch tested?

Closes #23989 from cxzl25/fix_IdleStateEvent_timeout.

Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-11 15:16:16 -07:00
“attilapiros” 5668c42edf [SPARK-27021][CORE] Cleanup of Netty event loop group for shuffle chunk fetch requests
## What changes were proposed in this pull request?

Creating an Netty `EventLoopGroup` leads to creating a new Thread pool for handling the events. For stopping the threads of the pool the event loop group should be shut down which is properly done for transport servers and clients by calling for example the `shutdownGracefully()` method (for details see the `close()` method of `TransportClientFactory` and `TransportServer`). But there is a separate event loop group for shuffle chunk fetch requests which is in pipeline for handling fetch request (shared between the client and server) and owned by the `TransportContext` and this was never shut down.

## How was this patch tested?

With existing unittest.

This leak is in the production system too but its effect is spiking in the unittest.

Checking the core unittest logs before the PR:
```
$ grep "LEAK IN SUITE" unit-tests.log | grep -o shuffle-chunk-fetch-handler | wc -l
381
```

And after the PR without whitelisting in thread audit and with an extra `await` after the
` chunkFetchWorkers.shutdownGracefully()`:
```
$ grep "LEAK IN SUITE" unit-tests.log | grep -o shuffle-chunk-fetch-handler | wc -l
0
```

Closes #23930 from attilapiros/SPARK-27021.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-03-05 12:31:06 -08:00
liupengcheng 52a180f25f [SPARK-26674][CORE] Consolidate CompositeByteBuf when reading large frame
## What changes were proposed in this pull request?

Currently, TransportFrameDecoder will not consolidate the buffers read from network which may cause memory waste. Actually, bytebuf's writtenIndex is far less than it's capacity  in most cases, so we can optimize it by doing consolidation.

This PR will do this optimization.

Related codes:
9a30e23211/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java (L143)

## How was this patch tested?

UT

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

Closes #23602 from liupc/Reduce-memory-consumption-in-TransportFrameDecoder.

Lead-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-02-25 16:40:46 -08:00
schintap 13c56346f4 [SPARK-25692][CORE] Remove static initialization of worker eventLoop handling chunk fetch requests within TransportContext. This fixes ChunkFetchIntegrationSuite as well
## What changes were proposed in this pull request?

How to reproduce
./build/mvn test -Dtest=org.apache.spark.network.RequestTimeoutIntegrationSuite,org.apache.spark.network.ChunkFetchIntegrationSuite -DwildcardSuites=None test
furtherRequestsDelay Test within RequestTimeoutIntegrationSuite was holding onto buffer references within worker threads. The test does close the server context but since the threads are global and there is sleep of 60 secs to fetch a specific chunk within this test, it grabs on it and waits for the client to consume but however the test is testing for a request timeout and it times out after 10 secs, so the workers are just waiting there for the buffer to be consumed by client as per my understanding.

This tends to happen if you dont have enough IO threads available on the specific system and also the order of the tests being run determines its flakyness like if ChunkFetchIntegrationSuite runs first then there is no issue. For example on mac with 8 threads these tests run fine but on my vm with 4 threads it fails. It matches the number of fetch calls in RequestTimeoutIntegrationSuite.

So do we really need it to be static?

I dont think this requires a global declaration as these threads are only required on the shuffle server end and on the client TransportContext initialization i.e the Client don't initialize these threads. The Shuffle Server initializes one TransportContext object. So, I think this is fine to be an instance variable and I see no harm.

## How was this patch tested?
Integration tests, manual tests

Closes #23700 from redsanket/SPARK-25692.

Authored-by: schintap <schintap@oath.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-02-05 10:43:43 -08:00
Nihar Sheth 055bf8ea1f [SPARK-24938][CORE] Prevent Netty from using onheap memory for headers without regard for configuration
## What changes were proposed in this pull request?

In MessageEncoder.java, the header would always be allocated on onheap memory regardless of whether netty was configured to use/prefer onheap or offheap. By default this made netty allocate 16mb of onheap memory for a tiny header message. It would be more practical to use preallocated buffers.

Using a memory monitor tool on a simple spark application, the following services currently allocate 16 mb of onheap memory:
netty-rpc-client
netty-blockTransfer-client
netty-external-shuffle-client

With this change, the memory monitor tool reports all three of these services as using 0 b of onheap memory. The offheap memory allocation does not increase, but more of the already-allocated space is used.

## How was this patch tested?

Manually tested change using spark-memory-tool https://github.com/squito/spark-memory

Closes #22114 from NiharS/nettybuffer.

Lead-authored-by: Nihar Sheth <niharrsheth@gmail.com>
Co-authored-by: Nihar Sheth <nsheth@cloudera.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 08:41:42 -06:00
Liang-Chi Hsieh cf133e6110 [SPARK-26604][CORE] Clean up channel registration for StreamManager
## What changes were proposed in this pull request?

Now in `TransportRequestHandler.processStreamRequest`, when a stream request is processed, the stream id is not registered with the current channel in stream manager. It should do that so in case of that the channel gets terminated we can remove associated streams of stream requests too.

This also cleans up channel registration in `StreamManager`. Since `StreamManager` doesn't register channel but only `OneForOneStreamManager` does it, this removes `registerChannel` from `StreamManager`. When `OneForOneStreamManager` goes to register stream, it will also register channel for the stream.

## How was this patch tested?

Existing tests.

Closes #23521 from viirya/SPARK-26604.

Authored-by: Liang-Chi Hsieh <viirya@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2019-01-16 10:58:07 +08:00
SongYadong a77505d4d3 [CORE][MINOR] Fix some typos about MemoryMode
## What changes were proposed in this pull request?

Fix typos in comments by replacing "in-heap" with "on-heap".

## How was this patch tested?

Existing Tests.

Closes #23533 from SongYadong/typos_inheap_to_onheap.

Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2019-01-15 14:40:00 +08:00
Petar Petrov c01152dd22 [SPARK-23182][CORE] Allow enabling TCP keep alive on the RPC connections
## What changes were proposed in this pull request?

Make it possible for the master to enable TCP keep alive on the RPC connections with clients.

## How was this patch tested?

Manually tested.

Added the following:
```
spark.rpc.io.enableTcpKeepAlive  true
```
to spark-defaults.conf.

Observed the following on the Spark master:
```
$ netstat -town | grep 7077
tcp6       0      0 10.240.3.134:7077       10.240.1.25:42851       ESTABLISHED keepalive (6736.50/0/0)
tcp6       0      0 10.240.3.134:44911      10.240.3.134:7077       ESTABLISHED keepalive (4098.68/0/0)
tcp6       0      0 10.240.3.134:7077       10.240.3.134:44911      ESTABLISHED keepalive (4098.68/0/0)
```

Which proves that the keep alive setting is taking effect.

It's currently possible to enable TCP keep alive on the worker / executor, but is not possible to configure on other RPC connections. It's unclear to me why this could be the case. Keep alive is more important for the master to protect it against suddenly departing workers / executors, thus I think it's very important to have it. Particularly this makes the master resilient in case of using preemptible worker VMs in GCE. GCE has the concept of shutdown scripts, which it doesn't guarantee to execute. So workers often don't get shutdown gracefully and the TCP connections on the master linger as there's nothing to close them. Thus the need of enabling keep alive.

This enables keep-alive on connections besides the master's connections, but that shouldn't cause harm.

Closes #20512 from peshopetrov/master.

Authored-by: Petar Petrov <petar.petrov@leanplum.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-13 13:39:12 -06:00
Dongjoon Hyun 19e17acf2d
[SPARK-25692][TEST] Increase timeout in fetchBothChunks test
## What changes were proposed in this pull request?

`ChunkFetchIntegrationSuite.fetchBothChunks` fails frequently due to timeout in Apache Spark Jenkins environments.

```scala
org.apache.spark.network.ChunkFetchIntegrationSuite
[ERROR] fetchBothChunks(org.apache.spark.network.ChunkFetchIntegrationSuite)
Time elapsed: 5.015 s  <<< FAILURE!
java.lang.AssertionError: Timeout getting response from the server
	at org.apache.spark.network.ChunkFetchIntegrationSuite.fetchChunks(ChunkFetchIntegrationSuite.java:176)
	at org.apache.spark.network.ChunkFetchIntegrationSuite.fetchBothChunks(ChunkFetchIntegrationSuite.java:210)
```

The followings are the recent failures on `amp-jenkins-worker-05`. The timeout seems to be too sensitive in low-end machines. This PR increases the timeout from 5 seconds to 60 seconds in order to be more robust.

- [master 5856](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5856/)
- [master 5837](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5837/testReport)
- [master 5835](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5835/testReport)
- [master 5829](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5829/testReport)
- [master 5828](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5828/testReport)
- [master 5822](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5822/testReport)
- [master 5814](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7/5814/testReport)

- [SparkPullRequestBuilder 100784](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100784/consoleFull)

- [SparkPullRequestBuilder 100785](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100785/consoleFull)

- [SparkPullRequestBuilder 100787](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100787/consoleFull)

- [SparkPullRequestBuilder 100788](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/100788/consoleFull)

## How was this patch tested?

N/A (Monitor the Jenkins on `amp-jenkins-worker-05` machine)

Closes #23522 from dongjoon-hyun/SPARK-25692.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-11 17:22:05 -08:00
“attilapiros” e103c4a5e7 [SPARK-24920][CORE] Allow sharing Netty's memory pool allocators
## What changes were proposed in this pull request?

Introducing shared polled ByteBuf allocators.
This feature can be enabled via the "spark.network.sharedByteBufAllocators.enabled" configuration.

When it is on then only two pooled ByteBuf allocators are created:
- one for transport servers where caching is allowed and
- one for transport clients where caching is disabled

This way the cache allowance remains as before.
Both shareable pools are created with numCores parameter set to 0 (which defaults to the available processors) as conf.serverThreads() and conf.clientThreads() are module dependant and the lazy creation of this allocators would lead to unpredicted behaviour.

When "spark.network.sharedByteBufAllocators.enabled" is false then a new allocator is created for every transport client and server separately as was before this PR.

## How was this patch tested?

Existing unit tests.

Closes #23278 from attilapiros/SPARK-24920.

Authored-by: “attilapiros” <piros.attila.zsolt@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-08 13:11:11 -06:00
Dongjoon Hyun e15a319ccd
[SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4
## What changes were proposed in this pull request?

This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required.

- Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers`
- Replace `anyObject` with `any`
- Replace `getArgumentAt` with `getArgument` and add type annotation.
- Use `isNull` matcher in case of `null` is invoked.
```scala
     saslHandler.channelInactive(null);
-    verify(handler).channelInactive(any(TransportClient.class));
+    verify(handler).channelInactive(isNull());
```

- Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775)
```scala
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23452 from dongjoon-hyun/SPARK-26536.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-04 19:23:38 -08:00
Sean Owen 89cebf4932 [SPARK-24421][CORE][FOLLOWUP] Use normal direct ByteBuffer allocation if Cleaner can't be set
## What changes were proposed in this pull request?

In Java 9+ we can't use sun.misc.Cleaner by default anymore, and this was largely handled in https://github.com/apache/spark/pull/22993 However I think the change there left a significant problem.

If a DirectByteBuffer is allocated using the reflective hack in Platform, now, we by default can't set a Cleaner. But I believe this means the memory isn't freed promptly or possibly at all. If a Cleaner can't be set, I think we need to use normal APIs to allocate the direct ByteBuffer.

According to comments in the code, the downside is simply that the normal APIs will check and impose limits on how much off-heap memory can be allocated. Per the original review on https://github.com/apache/spark/pull/22993 this much seems fine, as either way in this case the user would have to add a JVM setting (increase max, or allow the reflective access).

## How was this patch tested?

Existing tests. This resolved an OutOfMemoryError in Java 11 from TimSort tests without increasing test heap size. (See https://github.com/apache/spark/pull/23419#issuecomment-450772125 ) This suggests there is a problem and that this resolves it.

Closes #23424 from srowen/SPARK-24421.2.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-04 15:37:09 -06:00
Dongjoon Hyun 27e42c1de5
[MINOR][NETWORK][TEST] Fix TransportFrameDecoderSuite to use ByteBuf instead of ByteBuffer
## What changes were proposed in this pull request?

`fireChannelRead` expects `io.netty.buffer.ByteBuf`.I checked that this is the only place which misuse `java.nio.ByteBuffer` in `network` module.

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23442 from dongjoon-hyun/SPARK-NETWORK-COMMON.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-03 20:01:19 -08:00
Kazuaki Ishizaki 79b05481a2 [SPARK-26508][CORE][SQL] Address warning messages in Java reported at lgtm.com
## What changes were proposed in this pull request?

This PR addresses warning messages in Java files reported at [lgtm.com](https://lgtm.com).

[lgtm.com](https://lgtm.com) provides automated code review of Java/Python/JavaScript files for OSS projects. [Here](https://lgtm.com/projects/g/apache/spark/alerts/?mode=list&severity=warning) are warning messages regarding Apache Spark project.

This PR addresses the following warnings:

- Result of multiplication cast to wider type
- Implicit narrowing conversion in compound assignment
- Boxed variable is never null
- Useless null check

NOTE: `Potential input resource leak` looks false positive for now.

## How was this patch tested?

Existing UTs

Closes #23420 from kiszk/SPARK-26508.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-01 22:37:28 -06:00
pgandhi 8dd29fe36b [SPARK-25642][YARN] Adding two new metrics to record the number of registered connections as well as the number of active connections to YARN Shuffle Service
Recently, the ability to expose the metrics for YARN Shuffle Service was added as part of [SPARK-18364](https://github.com/apache/spark/pull/22485). We need to add some metrics to be able to determine the number of active connections as well as open connections to the external shuffle service to benchmark network and connection issues on large cluster environments.

Added two more shuffle server metrics for Spark Yarn shuffle service: numRegisteredConnections which indicate the number of registered connections to the shuffle service and numActiveConnections which indicate the number of active connections to the shuffle service at any given point in time.

If these metrics are outputted to a file, we get something like this:

1533674653489 default.shuffleService: Hostname=server1.abc.com, openBlockRequestLatencyMillis_count=729, openBlockRequestLatencyMillis_rate15=0.7110833548897356, openBlockRequestLatencyMillis_rate5=1.657808981793011, openBlockRequestLatencyMillis_rate1=2.2404486061620474, openBlockRequestLatencyMillis_rateMean=0.9242558551196706,
numRegisteredConnections=35,
blockTransferRateBytes_count=2635880512, blockTransferRateBytes_rate15=2578547.6094160094, blockTransferRateBytes_rate5=6048721.726302424, blockTransferRateBytes_rate1=8548922.518223226, blockTransferRateBytes_rateMean=3341878.633637769, registeredExecutorsSize=5, registerExecutorRequestLatencyMillis_count=5, registerExecutorRequestLatencyMillis_rate15=0.0027973949328659836, registerExecutorRequestLatencyMillis_rate5=0.0021278007987206426, registerExecutorRequestLatencyMillis_rate1=2.8270296777387467E-6, registerExecutorRequestLatencyMillis_rateMean=0.006339206380043053, numActiveConnections=35

Closes #22498 from pgandhi999/SPARK-18364.

Authored-by: pgandhi <pgandhi@oath.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-12-21 11:28:33 -08:00
Wenchen Fan bdf32847b1
[SPARK-26021][SQL][FOLLOWUP] only deal with NaN and -0.0 in UnsafeWriter
## What changes were proposed in this pull request?

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

There are 4 places we need to deal with NaN and -0.0:
1. comparison expressions. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.
2. Join keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.
3. grouping keys. `-0.0` and `0.0` should be assigned to the same group. Different NaNs should be assigned to the same group.
4. window partition keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same.

The case 1 is OK. Our comparison already handles NaN and -0.0, and for struct/array/map, we will recursively compare the fields/elements.

Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different NaNs have different binary representation, and the same thing happens for -0.0 and 0.0.

To fix it, a simple solution is: normalize float/double when building unsafe data (`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`). Then we don't need to worry about it anymore.

Following this direction, this PR moves the handling of NaN and -0.0 from `Platform` to `UnsafeWriter`, so that places like `UnsafeRow.setFloat` will not handle them, which reduces the perf overhead. It's also easier to add comments explaining why we do it in `UnsafeWriter`.

## How was this patch tested?

existing tests

Closes #23239 from cloud-fan/minor.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-12-08 11:18:09 -08:00
Wenchen Fan 09a91d98bd [SPARK-26021][SQL][FOLLOWUP] add test for special floating point values
## What changes were proposed in this pull request?

a followup of https://github.com/apache/spark/pull/23043 . Add a test to show the minor behavior change introduced by #23043 , and add migration guide.

## How was this patch tested?

a new test

Closes #23141 from cloud-fan/follow.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-28 16:21:42 +08:00
Shixiong Zhu 92fc0a8f96 [SPARK-26069][TESTS][FOLLOWUP] Add another possible error message
## What changes were proposed in this pull request?

`org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures` is still flaky and here is error message:

```
sbt.ForkMain$ForkError: java.lang.AssertionError: Got a non-empty set [Failed to send RPC RPC 8249697863992194475 to /172.17.0.2:41177: java.io.IOException: Broken pipe]
	at org.junit.Assert.fail(Assert.java:88)
	at org.junit.Assert.assertTrue(Assert.java:41)
	at org.apache.spark.network.RpcIntegrationSuite.assertErrorAndClosed(RpcIntegrationSuite.java:389)
	at org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures(RpcIntegrationSuite.java:347)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17)
	at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78)
	at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.junit.runners.Suite.runChild(Suite.java:128)
	at org.junit.runners.Suite.runChild(Suite.java:27)
	at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290)
	at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71)
	at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288)
	at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58)
	at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.junit.runner.JUnitCore.run(JUnitCore.java:137)
	at org.junit.runner.JUnitCore.run(JUnitCore.java:115)
	at com.novocode.junit.JUnitRunner$1.execute(JUnitRunner.java:132)
	at sbt.ForkMain$Run$2.call(ForkMain.java:296)
	at sbt.ForkMain$Run$2.call(ForkMain.java:286)
	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 java.lang.Thread.run(Thread.java:748)
```

This happened when the second RPC message was being sent but the connection was closed at the same time.

## How was this patch tested?

Jenkins

Closes #23109 from zsxwing/SPARK-26069-2.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-23 06:18:44 -06:00
Alon Doron 0ec7b99ea2 [SPARK-26021][SQL] replace minus zero with zero in Platform.putDouble/Float
GROUP BY treats -0.0 and 0.0 as different values which is unlike hive's behavior.
In addition current behavior with codegen is unpredictable (see example in JIRA ticket).

## What changes were proposed in this pull request?

In Platform.putDouble/Float() checking if the value is -0.0, and if so replacing with 0.0.
This is used by UnsafeRow so it won't have -0.0 values.

## How was this patch tested?

Added tests

Closes #23043 from adoron/adoron-spark-26021-replace-minus-zero-with-zero.

Authored-by: Alon Doron <adoron@palantir.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-23 08:55:00 +08:00
Sean Owen 32365f8177 [SPARK-26090][CORE][SQL][ML] Resolve most miscellaneous deprecation and build warnings for Spark 3
## What changes were proposed in this pull request?

The build has a lot of deprecation warnings. Some are new in Scala 2.12 and Java 11. We've fixed some, but I wanted to take a pass at fixing lots of easy miscellaneous ones here.

They're too numerous and small to list here; see the pull request. Some highlights:

- `BeanInfo` is deprecated in 2.12, and BeanInfo classes are pretty ancient in Java. Instead, case classes can explicitly declare getters
- Eta expansion of zero-arg methods; foo() becomes () => foo() in many cases
- Floating-point Range is inexact and deprecated, like 0.0 to 100.0 by 1.0
- finalize() is finally deprecated (just needs to be suppressed)
- StageInfo.attempId was deprecated and easiest to remove here

I'm not now going to touch some chunks of deprecation warnings:

- Parquet deprecations
- Hive deprecations (particularly serde2 classes)
- Deprecations in generated code (mostly Thriftserver CLI)
- ProcessingTime deprecations (we may need to revive this class as internal)
- many MLlib deprecations because they concern methods that may be removed anyway
- a few Kinesis deprecations I couldn't figure out
- Mesos get/setRole, which I don't know well
- Kafka/ZK deprecations (e.g. poll())
- Kinesis
- a few other ones that will probably resolve by deleting a deprecated method

## How was this patch tested?

Existing tests, including manual testing with the 2.11 build and Java 11.

Closes #23065 from srowen/SPARK-26090.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 09:16:42 -06:00
Sean Owen 630e25e355 [SPARK-26026][BUILD] Published Scaladoc jars missing from Maven Central
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

N/A; build runs scaladoc now.

Closes #23069 from srowen/SPARK-26026.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-19 08:06:33 -06:00
Shixiong Zhu 99cbc51b32
[SPARK-26069][TESTS] Fix flaky test: RpcIntegrationSuite.sendRpcWithStreamFailures
## What changes were proposed in this pull request?

The test failure is because `assertErrorAndClosed` misses one possible error message: `java.nio.channels.ClosedChannelException`. This happens when the second `uploadStream` is called after the channel has been closed. This can be reproduced by adding `Thread.sleep(1000)` below this line: 03306a6df3/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java (L217)

This PR fixes the above issue and also improves the test failure messages of `assertErrorAndClosed`.

## How was this patch tested?

Jenkins

Closes #23041 from zsxwing/SPARK-26069.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-11-16 09:51:41 -08:00
Wenchen Fan 9610efc252 [SPARK-26055][CORE] InterfaceStability annotations should be retained at runtime
## What changes were proposed in this pull request?

It's good to have annotations available at runtime, so that tools like MiMa can detect them and deal with then specially. e.g. we don't want to track compatibility for unstable classes.

This PR makes `InterfaceStability` annotations to be retained at runtime, to be consistent with `Experimental` and `DeveloperApi`

## How was this patch tested?
N/A

Closes #23029 from cloud-fan/annotation.

Authored-by: Wenchen Fan <wenchen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-11-15 20:25:27 +08:00
DB Tsai ad853c5678
[SPARK-25956] Make Scala 2.12 as default Scala version in Spark 3.0
## What changes were proposed in this pull request?

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

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

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Yuanjian Li 2977e2312d [SPARK-25986][BUILD] Add rules to ban throw Errors in application code
## What changes were proposed in this pull request?

Add scala and java lint check rules to ban the usage of `throw new xxxErrors` and fix up all exists instance followed by https://github.com/apache/spark/pull/22989#issuecomment-437939830. See more details in https://github.com/apache/spark/pull/22969.

## How was this patch tested?

Local test with lint-scala and lint-java.

Closes #22989 from xuanyuanking/SPARK-25986.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-14 13:05:18 -08:00
Sean Owen 722369ee55 [SPARK-24421][BUILD][CORE] Accessing sun.misc.Cleaner in JDK11
…. Other related changes to get JDK 11 working, to test

## What changes were proposed in this pull request?

- Access `sun.misc.Cleaner` (Java 8) and `jdk.internal.ref.Cleaner` (JDK 9+) by reflection (note: the latter only works if illegal reflective access is allowed)
- Access `sun.misc.Unsafe.invokeCleaner` in Java 9+ instead of `sun.misc.Cleaner` (Java 8)

In order to test anything on JDK 11, I also fixed a few small things, which I include here:

- Fix minor JDK 11 compile issues
- Update scala plugin, Jetty for JDK 11, to facilitate tests too

This doesn't mean JDK 11 tests all pass now, but lots do. Note also that the JDK 9+ solution for the Cleaner has a big caveat.

## How was this patch tested?

Existing tests. Manually tested JDK 11 build and tests, and tests covering this change appear to pass. All Java 8 tests should still pass, but this change alone does not achieve full JDK 11 compatibility.

Closes #22993 from srowen/SPARK-24421.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-14 12:52:54 -08:00
Marcelo Vanzin 3eee9e0246 [SPARK-25535][CORE] Work around bad error handling in commons-crypto.
The commons-crypto library does some questionable error handling internally,
which can lead to JVM crashes if some call into native code fails and cleans
up state it should not.

While the library is not fixed, this change adds some workarounds in Spark code
so that when an error is detected in the commons-crypto side, Spark avoids
calling into the library further.

Tested with existing and added unit tests.

Closes #22557 from vanzin/SPARK-25535.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2018-10-09 09:27:08 -05:00
Sanket Chintapalli 6353425af7 [SPARK-25641] Change the spark.shuffle.server.chunkFetchHandlerThreadsPercent default to 100
## What changes were proposed in this pull request?

We want to change the default percentage to 100 for spark.shuffle.server.chunkFetchHandlerThreadsPercent. The reason being
currently this is set to 0. Which means currently if server.ioThreads > 0, the default number of threads would be 2 * #cores instead of server.io.Threads. We want the default to server.io.Threads in case this is not set at all. Also here a default of 0 would also mean 2 * #cores

## How was this patch tested?
Manual
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Closes #22628 from redsanket/SPARK-25641.

Lead-authored-by: Sanket Chintapalli <schintap@yahoo-inc.com>
Co-authored-by: Sanket Chintapalli <chintapalli.sanketreddy@gmail.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-10-08 13:19:34 -05:00
Fokko Driesprong 1a28625355 [SPARK-25408] Move to more ideomatic Java8
While working on another PR, I noticed that there is quite some legacy Java in there that can be beautified. For example the use of features from Java8, such as:
- Collection libraries
- Try-with-resource blocks

No logic has been changed. I think it is important to have a solid codebase with examples that will inspire next PR's to follow up on the best practices.

What are your thoughts on this?

This makes code easier to read, and using try-with-resource makes is less likely to forget to close something.

## What changes were proposed in this pull request?

No changes in the logic of Spark, but more in the aesthetics of the code.

## How was this patch tested?

Using the existing unit tests. Since no logic is changed, the existing unit tests should pass.

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

Closes #22637 from Fokko/SPARK-25408.

Authored-by: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-08 09:58:52 -05:00
Parker Hegstrom 17781d7530 [SPARK-25202][SQL] Implements split with limit sql function
## What changes were proposed in this pull request?

Adds support for the setting limit in the sql split function

## How was this patch tested?

1. Updated unit tests
2. Tested using Scala spark shell

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

Closes #22227 from phegstrom/master.

Authored-by: Parker Hegstrom <phegstrom@palantir.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-10-06 14:30:43 +08:00
Wenchen Fan 5ae20cf1a9 Revert "[SPARK-25408] Move to mode ideomatic Java8"
This reverts commit 44c1e1ab1c.
2018-10-05 11:03:41 +08:00
Fokko Driesprong 44c1e1ab1c [SPARK-25408] Move to mode ideomatic Java8
While working on another PR, I noticed that there is quite some legacy Java in there that can be beautified. For example the use og features from Java8, such as:
- Collection libraries
- Try-with-resource blocks

No code has been changed

What are your thoughts on this?

This makes code easier to read, and using try-with-resource makes is less likely to forget to close something.

## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Closes #22399 from Fokko/SPARK-25408.

Authored-by: Fokko Driesprong <fokkodriesprong@godatadriven.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-05 02:58:25 +01:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

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

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
marek.simunek a802c69b13 [SPARK-18364][YARN] Expose metrics for YarnShuffleService
## What changes were proposed in this pull request?

This PR is follow-up of closed  https://github.com/apache/spark/pull/17401 which only ended due to of inactivity, but its still nice feature to have.
Given review by jerryshao taken in consideration and edited:
- VisibleForTesting deleted because of dependency conflicts
- removed unnecessary reflection for `MetricsSystemImpl`
- added more available types for gauge

## How was this patch tested?

Manual deploy of new yarn-shuffle jar into a Node Manager and verifying that the metrics appear in the Node Manager-standard location. This is JMX with an query endpoint running on `hostname:port`

Resulting metrics look like this:
```
curl -sk -XGET hostname:port |  grep -v '#' | grep 'shuffleService'
hadoop_nodemanager_openblockrequestlatencymillis_rate15{name="shuffleService",} 0.31428910657834713
hadoop_nodemanager_blocktransferratebytes_rate15{name="shuffleService",} 566144.9983653595
hadoop_nodemanager_blocktransferratebytes_ratemean{name="shuffleService",} 2464409.9678099006
hadoop_nodemanager_openblockrequestlatencymillis_rate1{name="shuffleService",} 1.2893844732240272
hadoop_nodemanager_registeredexecutorssize{name="shuffleService",} 2.0
hadoop_nodemanager_openblockrequestlatencymillis_ratemean{name="shuffleService",} 1.255574678369966
hadoop_nodemanager_openblockrequestlatencymillis_count{name="shuffleService",} 315.0
hadoop_nodemanager_openblockrequestlatencymillis_rate5{name="shuffleService",} 0.7661929192569739
hadoop_nodemanager_registerexecutorrequestlatencymillis_ratemean{name="shuffleService",} 0.0
hadoop_nodemanager_registerexecutorrequestlatencymillis_count{name="shuffleService",} 0.0
hadoop_nodemanager_registerexecutorrequestlatencymillis_rate1{name="shuffleService",} 0.0
hadoop_nodemanager_registerexecutorrequestlatencymillis_rate5{name="shuffleService",} 0.0
hadoop_nodemanager_blocktransferratebytes_count{name="shuffleService",} 6.18271213E8
hadoop_nodemanager_registerexecutorrequestlatencymillis_rate15{name="shuffleService",} 0.0
hadoop_nodemanager_blocktransferratebytes_rate5{name="shuffleService",} 1154114.4881816586
hadoop_nodemanager_blocktransferratebytes_rate1{name="shuffleService",} 574745.0749848988
```

Closes #22485 from mareksimunek/SPARK-18364.

Lead-authored-by: marek.simunek <marek.simunek@firma.seznam.cz>
Co-authored-by: Andrew Ash <andrew@andrewash.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-10-01 11:04:37 -05:00
hyukjinkwon a2f502cf53 [SPARK-25565][BUILD] Add scalastyle rule to check add Locale.ROOT to .toLowerCase and .toUpperCase for internal calls
## What changes were proposed in this pull request?

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

It produces an error as below:

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

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

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

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

## How was this patch tested?

Manually tested, and Jenkins tests.

Closes #22581 from HyukjinKwon/SPARK-25565.

Authored-by: hyukjinkwon <gurwls223@apache.org>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-09-30 14:31:04 +08:00
Sanket Chintapalli ff601cf71d [SPARK-24355] Spark external shuffle server improvement to better handle block fetch requests.
## What changes were proposed in this pull request?

Description:
Right now, the default server side netty handler threads is 2 * # cores, and can be further configured with parameter spark.shuffle.io.serverThreads.
In order to process a client request, it would require one available server netty handler thread.
However, when the server netty handler threads start to process ChunkFetchRequests, they will be blocked on disk I/O, mostly due to disk contentions from the random read operations initiated by all the ChunkFetchRequests received from clients.
As a result, when the shuffle server is serving many concurrent ChunkFetchRequests, the server side netty handler threads could all be blocked on reading shuffle files, thus leaving no handler thread available to process other types of requests which should all be very quick to process.

This issue could potentially be fixed by limiting the number of netty handler threads that could get blocked when processing ChunkFetchRequest. We have a patch to do this by using a separate EventLoopGroup with a dedicated ChannelHandler to process ChunkFetchRequest. This enables shuffle server to reserve netty handler threads for non-ChunkFetchRequest, thus enabling consistent processing time for these requests which are fast to process. After deploying the patch in our infrastructure, we no longer see timeout issues with either executor registration with local shuffle server or shuffle client establishing connection with remote shuffle server.
(Please fill in changes proposed in this fix)

For Original PR please refer here
https://github.com/apache/spark/pull/21402

## How was this patch tested?

Unit tests and stress testing.
(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Closes #22173 from redsanket/SPARK-24335.

Authored-by: Sanket Chintapalli <schintap@yahoo-inc.com>
Signed-off-by: Thomas Graves <tgraves@apache.org>
2018-09-21 09:05:56 -05:00
Imran Rashid a97001d217 [CORE] Updates to remote cache reads
Covered by tests in DistributedSuite
2018-09-17 14:06:09 -05:00
gatorsmile bb2f069cf2 [SPARK-25436] Bump master branch version to 2.5.0-SNAPSHOT
## What changes were proposed in this pull request?
In the dev list, we can still discuss whether the next version is 2.5.0 or 3.0.0. Let us first bump the master branch version to `2.5.0-SNAPSHOT`.

## How was this patch tested?
N/A

Closes #22426 from gatorsmile/bumpVersionMaster.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-15 16:24:02 -07:00
gatorsmile 0b9ccd55c2 Revert [SPARK-10399] [SPARK-23879] [SPARK-23762] [SPARK-25317]
## What changes were proposed in this pull request?

When running TPC-DS benchmarks on 2.4 release, npoggi and winglungngai  saw more than 10% performance regression on the following queries: q67, q24a and q24b. After we applying the PR https://github.com/apache/spark/pull/22338, the performance regression still exists. If we revert the changes in https://github.com/apache/spark/pull/19222, npoggi and winglungngai  found the performance regression was resolved. Thus, this PR is to revert the related changes for unblocking the 2.4 release.

In the future release, we still can continue the investigation and find out the root cause of the regression.

## How was this patch tested?

The existing test cases

Closes #22361 from gatorsmile/revertMemoryBlock.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-09 21:25:19 +08:00
Marco Gaido 64c314e22f [SPARK-25317][CORE] Avoid perf regression in Murmur3 Hash on UTF8String
## What changes were proposed in this pull request?

SPARK-10399 introduced a performance regression on the hash computation for UTF8String.

The regression can be evaluated with the code attached in the JIRA. That code runs in about 120 us per method on my laptop (MacBook Pro 2.5 GHz Intel Core i7, RAM 16 GB 1600 MHz DDR3) while the code from branch 2.3 takes on the same machine about 45 us for me. After the PR, the code takes about 45 us on the master branch too.

## How was this patch tested?

running the perf test from the JIRA

Closes #22338 from mgaido91/SPARK-25317.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-09-06 15:27:59 +08:00
Shixiong Zhu 592e3a42c2
[SPARK-25218][CORE] Fix potential resource leaks in TransportServer and SocketAuthHelper
## What changes were proposed in this pull request?

Make sure TransportServer and SocketAuthHelper close the resources for all types of errors.

## How was this patch tested?

Jenkins

Closes #22210 from zsxwing/SPARK-25218.

Authored-by: Shixiong Zhu <zsxwing@gmail.com>
Signed-off-by: Shixiong Zhu <zsxwing@gmail.com>
2018-08-28 08:36:06 -07:00
Imran Rashid 0ce09ec54e [SPARK-25205][CORE] Fix typo in spark.network.crypto.keyFactoryIterations
Closes #22195 from squito/SPARK-25205.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: hyukjinkwon <gurwls223@apache.org>
2018-08-24 09:31:06 +08:00
Imran Rashid 99d2e4e007 [SPARK-24296][CORE] Replicate large blocks as a stream.
When replicating large cached RDD blocks, it can be helpful to replicate
them as a stream, to avoid using large amounts of memory during the
transfer.  This also allows blocks larger than 2GB to be replicated.

Added unit tests in DistributedSuite.  Also ran tests on a cluster for
blocks > 2gb.

Closes #21451 from squito/clean_replication.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-08-21 11:26:41 -07:00
Norman Maurer 92fd7f321c
[SPARK-25115][CORE] Eliminate extra memory copy done when a ByteBuf is used that is backed by > 1 ByteBuffer.
…d by > 1 ByteBuffer.

## What changes were proposed in this pull request?

Check how many ByteBuffer are used and depending on it do either call nioBuffer(...) or nioBuffers(...) to eliminate extra memory copies.

This is related to netty/netty#8176.

## How was this patch tested?

Unit tests added.

Closes #22105 from normanmaurer/composite_byte_buf_mem_copy.

Authored-by: Norman Maurer <norman_maurer@apple.com>
Signed-off-by: DB Tsai <d_tsai@apple.com>
2018-08-15 00:02:46 +00:00
Kazuhiro Sera 8ec25cd67e Fix typos detected by github.com/client9/misspell
## What changes were proposed in this pull request?

Fixing typos is sometimes very hard. It's not so easy to visually review them. Recently, I discovered a very useful tool for it, [misspell](https://github.com/client9/misspell).

This pull request fixes minor typos detected by [misspell](https://github.com/client9/misspell) except for the false positives. If you would like me to work on other files as well, let me know.

## How was this patch tested?

### before

```
$ misspell . | grep -v '.js'
R/pkg/R/SQLContext.R:354:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:424:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:445:43: "definiton" is a misspelling of "definition"
R/pkg/R/SQLContext.R:495:43: "definiton" is a misspelling of "definition"
NOTICE-binary:454:16: "containd" is a misspelling of "contained"
R/pkg/R/context.R:46:43: "definiton" is a misspelling of "definition"
R/pkg/R/context.R:74:43: "definiton" is a misspelling of "definition"
R/pkg/R/DataFrame.R:591:48: "persistance" is a misspelling of "persistence"
R/pkg/R/streaming.R:166:44: "occured" is a misspelling of "occurred"
R/pkg/inst/worker/worker.R:65:22: "ouput" is a misspelling of "output"
R/pkg/tests/fulltests/test_utils.R:106:25: "environemnt" is a misspelling of "environment"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java:38:39: "existant" is a misspelling of "existent"
common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java:83:39: "existant" is a misspelling of "existent"
common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:243:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:234:19: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:238:63: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:244:46: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:276:39: "transfered" is a misspelling of "transferred"
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala:195:15: "orgin" is a misspelling of "origin"
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:621:39: "gauranteed" is a misspelling of "guaranteed"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/main/scala/org/apache/spark/storage/DiskStore.scala:282:18: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/util/ListenerBus.scala:64:17: "overriden" is a misspelling of "overridden"
core/src/test/scala/org/apache/spark/ShuffleSuite.scala:211:7: "substracted" is a misspelling of "subtracted"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:2468:84: "truely" is a misspelling of "truly"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:25:18: "persistance" is a misspelling of "persistence"
core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala:26:69: "persistance" is a misspelling of "persistence"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
dev/run-pip-tests:55:28: "enviroments" is a misspelling of "environments"
dev/run-pip-tests:91:37: "virutal" is a misspelling of "virtual"
dev/merge_spark_pr.py:377:72: "accross" is a misspelling of "across"
dev/merge_spark_pr.py:378:66: "accross" is a misspelling of "across"
dev/run-pip-tests:126:25: "enviroments" is a misspelling of "environments"
docs/configuration.md:1830:82: "overriden" is a misspelling of "overridden"
docs/structured-streaming-programming-guide.md:525:45: "processs" is a misspelling of "processes"
docs/structured-streaming-programming-guide.md:1165:61: "BETWEN" is a misspelling of "BETWEEN"
docs/sql-programming-guide.md:1891:810: "behaivor" is a misspelling of "behavior"
examples/src/main/python/sql/arrow.py:98:8: "substract" is a misspelling of "subtract"
examples/src/main/python/sql/arrow.py:103:27: "substract" is a misspelling of "subtract"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala:230:24: "inital" is a misspelling of "initial"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala:237:26: "descripiton" is a misspelling of "descriptions"
python/pyspark/find_spark_home.py:30:13: "enviroment" is a misspelling of "environment"
python/pyspark/context.py:937:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:938:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:939:12: "supress" is a misspelling of "suppress"
python/pyspark/context.py:940:12: "supress" is a misspelling of "suppress"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:713:8: "probabilty" is a misspelling of "probability"
python/pyspark/ml/clustering.py:1038:8: "Currenlty" is a misspelling of "Currently"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/ml/regression.py:1378:20: "paramter" is a misspelling of "parameter"
python/pyspark/mllib/stat/_statistics.py:262:8: "probabilty" is a misspelling of "probability"
python/pyspark/rdd.py:1363:32: "paramter" is a misspelling of "parameter"
python/pyspark/streaming/tests.py:825:42: "retuns" is a misspelling of "returns"
python/pyspark/sql/tests.py:768:29: "initalization" is a misspelling of "initialization"
python/pyspark/sql/tests.py:3616:31: "initalize" is a misspelling of "initialize"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala:120:39: "arbitary" is a misspelling of "arbitrary"
resource-managers/mesos/src/test/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcherArgumentsSuite.scala:26:45: "sucessfully" is a misspelling of "successfully"
resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala:358:27: "constaints" is a misspelling of "constraints"
resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala:111:24: "senstive" is a misspelling of "sensitive"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala:1063:5: "overwirte" is a misspelling of "overwrite"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala:1348:17: "compatability" is a misspelling of "compatibility"
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala:77:36: "paramter" is a misspelling of "parameter"
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:1374:22: "precendence" is a misspelling of "precedence"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:238:27: "unnecassary" is a misspelling of "unnecessary"
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala:212:17: "whn" is a misspelling of "when"
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinHelper.scala:147:60: "timestmap" is a misspelling of "timestamp"
sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala:150:45: "precentage" is a misspelling of "percentage"
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala:135:29: "infered" is a misspelling of "inferred"
sql/hive/src/test/resources/golden/udf_instr-1-2e76f819563dbaba4beb51e3a130b922:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_instr-2-32da357fc754badd6e3898dcc8989182:1:52: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-1-6e41693c9c6dceea4d7fab4c02884e4e:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_locate-2-d9b5934457931447874d6bb7c13de478:1:63: "occurance" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:9:79: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/golden/udf_translate-2-f7aa38a33ca0df73b7a1e6b6da4b7fe8:13:110: "occurence" is a misspelling of "occurrence"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/annotate_stats_join.q:46:105: "distint" is a misspelling of "distinct"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q:29:3: "Currenly" is a misspelling of "Currently"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q:72:15: "existant" is a misspelling of "existent"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q:25:3: "substraction" is a misspelling of "subtraction"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q:16:51: "funtion" is a misspelling of "function"
sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q:15:30: "issueing" is a misspelling of "issuing"
sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala:669:52: "wiht" is a misspelling of "with"
sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java:474:9: "Refering" is a misspelling of "Referring"
```

### after

```
$ misspell . | grep -v '.js'
common/network-common/src/main/java/org/apache/spark/network/util/AbstractFileRegion.java:27:20: "transfered" is a misspelling of "transferred"
core/src/main/scala/org/apache/spark/status/storeTypes.scala:113:29: "ect" is a misspelling of "etc"
core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala:1922:49: "agriculteur" is a misspelling of "agriculture"
data/streaming/AFINN-111.txt:1219:0: "humerous" is a misspelling of "humorous"
licenses/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:5:63: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:6:2: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:262:29: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:262:39: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:269:49: "Stichting" is a misspelling of "Stitching"
licenses-binary/LICENSE-heapq.txt:269:59: "Mathematisch" is a misspelling of "Mathematics"
licenses-binary/LICENSE-heapq.txt:274:2: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:274:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
licenses-binary/LICENSE-heapq.txt:276:29: "STICHTING" is a misspelling of "STITCHING"
licenses-binary/LICENSE-heapq.txt:276:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/hungarian.txt:170:0: "teh" is a misspelling of "the"
mllib/src/main/resources/org/apache/spark/ml/feature/stopwords/portuguese.txt:53:0: "eles" is a misspelling of "eels"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:99:20: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala:539:11: "Euclidian" is a misspelling of "Euclidean"
mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAOptimizer.scala:77:36: "Teh" is a misspelling of "The"
mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala:276:9: "Euclidian" is a misspelling of "Euclidean"
python/pyspark/heapq3.py:6:63: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:7:2: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:263:29: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:263:39: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:270:49: "Stichting" is a misspelling of "Stitching"
python/pyspark/heapq3.py:270:59: "Mathematisch" is a misspelling of "Mathematics"
python/pyspark/heapq3.py:275:2: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:275:12: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/heapq3.py:277:29: "STICHTING" is a misspelling of "STITCHING"
python/pyspark/heapq3.py:277:39: "MATHEMATISCH" is a misspelling of "MATHEMATICS"
python/pyspark/ml/stat.py:339:23: "Euclidian" is a misspelling of "Euclidean"
```

Closes #22070 from seratch/fix-typo.

Authored-by: Kazuhiro Sera <seratch@gmail.com>
Signed-off-by: Sean Owen <srowen@gmail.com>
2018-08-11 21:23:36 -05:00
hyukjinkwon 55e3ae6930 [SPARK-25001][BUILD] Fix miscellaneous build warnings
## What changes were proposed in this pull request?

There are many warnings in the current build (for instance see https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/4734/console).

**common**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java:237: warning: [rawtypes] found raw type: LevelDBIterator
[warn]   void closeIterator(LevelDBIterator it) throws IOException {
[warn]                      ^

[warn]   missing type arguments for generic class LevelDBIterator<T>
[warn]   where T is a type-variable:
[warn]     T extends Object declared in class LevelDBIterator
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:151: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:152: warning: [deprecation] group() in AbstractBootstrap has been deprecated
[warn]       bootstrap.group().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:154: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java:155: warning: [deprecation] childGroup() in ServerBootstrap has been deprecated
[warn]       bootstrap.childGroup().shutdownGracefully();
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java:112: warning: [deprecation] PooledByteBufAllocator(boolean,int,int,int,int,int,int,int) in PooledByteBufAllocator has been deprecated
[warn]     return new PooledByteBufAllocator(
[warn]            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java:321: warning: [rawtypes] found raw type: Future
[warn]     public void operationComplete(Future future) throws Exception {
[warn]                                   ^

[warn]   missing type arguments for generic class Future<V>
[warn]   where V is a type-variable:
[warn]     V extends Object declared in interface Future
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]           ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [rawtypes] found raw type: StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                               ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java:215: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]           StreamInterceptor interceptor = new StreamInterceptor(this, resp.streamId, resp.byteCount,
[warn]                                           ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]         ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [rawtypes] found raw type: StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                             ^

[warn]   missing type arguments for generic class StreamInterceptor<T>
[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:255: warning: [unchecked] unchecked call to StreamInterceptor(MessageHandler<T>,String,long,StreamCallback) as a member of the raw type StreamInterceptor
[warn]         StreamInterceptor interceptor = new StreamInterceptor(this, wrappedCallback.getID(),
[warn]                                         ^

[warn]   where T is a type-variable:
[warn]     T extends Message declared in class StreamInterceptor
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java:270: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteRawChannel, region.transfered());
[warn]                                                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslEncryption.java:304: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         region.transferTo(byteChannel, region.transfered());
[warn]                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:119: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]       while (in.transfered() < in.count()) {
[warn]                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/network-common/src/test/java/org/apache/spark/network/ProtocolSuite.java:120: warning: [deprecation] transfered() in FileRegion has been deprecated
[warn]         in.transferTo(channel, in.transfered());
[warn]                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:80: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-300363099, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:84: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-1210324667, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java:88: warning: [static] static method should be qualified by type name, Murmur3_x86_32, instead of by an expression
[warn]     Assert.assertEquals(-634919701, hasher.hashUnsafeWords(bytes, offset, 16, 42));
[warn]                                           ^
```

**launcher**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/launcher/src/main/java/org/apache/spark/launcher/AbstractLauncher.java:31: warning: [rawtypes] found raw type: AbstractLauncher
[warn] public abstract class AbstractLauncher<T extends AbstractLauncher> {
[warn]                                                  ^
[warn]   missing type arguments for generic class AbstractLauncher<T>
[warn]   where T is a type-variable:
[warn]     T extends AbstractLauncher declared in class AbstractLauncher
```

**core**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:99: method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.group() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala💯 method group in class AbstractBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.group().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:102: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]     if (bootstrap != null && bootstrap.childGroup() != null) {
[warn]                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/main/scala/org/apache/spark/api/r/RBackend.scala:103: method childGroup in class ServerBootstrap is deprecated: see corresponding Javadoc for more information.
[warn]       bootstrap.childGroup().shutdownGracefully()
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:151: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn] This can be achieved by adding the import clause 'import scala.language.reflectiveCalls'
[warn] or by setting the compiler option -language:reflectiveCalls.
[warn] See the Scaladoc for value scala.language.reflectiveCalls for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member value innerObject2 should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala:175: reflective access of structural type member method getData should be enabled
[warn] by making the implicit value scala.language.reflectiveCalls visible.
[warn]       val rdd = sc.parallelize(1 to 1).map(concreteObject.innerObject2.getData)
[warn]                                                                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/LocalSparkContext.scala:32: constructor Slf4JLoggerFactory in class Slf4JLoggerFactory is deprecated: see corresponding Javadoc for more information.
[warn]     InternalLoggerFactory.setDefaultFactory(new Slf4JLoggerFactory())
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:218: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]         assert(wrapper.stageAttemptId === stages.head.attemptId)
[warn]                                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:261: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:287: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.head.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:471: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       stageAttemptId = stages.last.attemptId))
[warn]                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:966: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskStart(SparkListenerTaskStart(dropped.stageId, dropped.attemptId, task))
[warn]                                                                          ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:972: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]     listener.onTaskEnd(SparkListenerTaskEnd(dropped.stageId, dropped.attemptId,
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:976: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       .taskSummary(dropped.stageId, dropped.attemptId, Array(0.25d, 0.50d, 0.75d))
[warn]                                             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1146: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala:1150: value attemptId in class StageInfo is deprecated: Use attemptNumber instead
[warn]       SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null))
[warn]                                                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:197: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]     while (region.transfered() < region.count()) {
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala:198: method transfered in trait FileRegion is deprecated: see corresponding Javadoc for more information.
[warn]       region.transferTo(byteChannel, region.transfered())
[warn]                                             ^
```

**sql**:

```
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]                                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala:534: abstract type T is unchecked since it is eliminated by erasure
[warn]       assert(partitioning.isInstanceOf[T])
[warn]             ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala:323: inferred existential type Option[Class[_$1]]( forSome { type _$1 }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]       val optClass = Option(collectionCls)
[warn]                            ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:226: warning: [deprecation] ParquetFileReader(Configuration,FileMetaData,Path,List<BlockMetaData>,List<ColumnDescriptor>) in ParquetFileReader has been deprecated
[warn]     this.reader = new ParquetFileReader(
[warn]                   ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:178: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 ||
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:179: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64  &&
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:181: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:182: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE ||
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:183: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:198: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         switch (descriptor.getType()) {
[warn]                           ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:221: warning: [deprecation] getTypeLength() in ColumnDescriptor has been deprecated
[warn]             readFixedLenByteArrayBatch(rowId, num, column, descriptor.getTypeLength());
[warn]                                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:224: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]             throw new IOException("Unsupported type: " + descriptor.getType());
[warn]                                                                    ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:246: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]       descriptor.getType().toString(),
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:258: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]     switch (descriptor.getType()) {
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java:384: warning: [deprecation] getType() in ColumnDescriptor has been deprecated
[warn]         throw new UnsupportedOperationException("Unsupported type: " + descriptor.getType());
[warn]                                                                                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:458: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int index = rowId * accessor.OFFSET_WIDTH;
[warn]                                   ^
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java:460: warning: [static] static variable should be qualified by type name, BaseRepeatedValueVector, instead of by an expression
[warn]       int end = offsets.getInt(index + accessor.OFFSET_WIDTH);
[warn]                                                ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala:57: a pure expression does nothing in statement position; you may be omitting necessary parentheses
[warn]       case s => s
[warn]                 ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:182: inferred existential type org.apache.parquet.column.statistics.Statistics[?0]( forSome { type ?0 <: Comparable[?0] }), which cannot be expressed by wildcards,  should be enabled
[warn] by making the implicit value scala.language.existentials visible.
[warn] This can be achieved by adding the import clause 'import scala.language.existentials'
[warn] or by setting the compiler option -language:existentials.
[warn] See the Scaladoc for value scala.language.existentials for a discussion
[warn] why the feature should be explicitly enabled.
[warn]                 val columnStats = oneBlockColumnMeta.getStatistics
[warn]                                                      ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:146: implicit conversion method conv should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions'
[warn] or by setting the compiler option -language:implicitConversions.
[warn] See the Scaladoc for value scala.language.implicitConversions for a discussion
[warn] why the feature should be explicitly enabled.
[warn]     implicit def conv(x: (Int, Long)): KV = KV(x._1, x._2)
[warn]                  ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala:48: implicit conversion method unsafeRow should be enabled
[warn] by making the implicit value scala.language.implicitConversions visible.
[warn]   private implicit def unsafeRow(value: Int) = {
[warn]                        ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala:178: method getType in class ColumnDescriptor is deprecated: see corresponding Javadoc for more information.
[warn]                 assert(oneFooter.getFileMetaData.getSchema.getColumns.get(0).getType() ===
[warn]                                                                              ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetTest.scala:154: method readAllFootersInParallel in object ParquetFileReader is deprecated: see corresponding Javadoc for more information.
[warn]     ParquetFileReader.readAllFootersInParallel(configuration, fs.getFileStatus(path)).asScala.toSeq
[warn]                       ^

[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/sql/hive/src/test/java/org/apache/spark/sql/hive/test/Complex.java:679: warning: [cast] redundant cast to Complex
[warn]     Complex typedOther = (Complex)other;
[warn]                          ^
```

**mllib**:

```
[warn] Pruning sources from previous analysis, due to incompatible CompileSetup.
[warn] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7/mllib/src/test/scala/org/apache/spark/ml/recommendation/ALSSuite.scala:597: match may not be exhaustive.
[warn] It would fail on the following inputs: None, Some((x: Tuple2[?, ?] forSome x not in (?, ?)))
[warn]     val df = dfs.find {
[warn]                       ^
```

This PR does not target fix all of them since some look pretty tricky to fix and there look too many warnings including false positive (like deprecated API but it's used in its test, etc.)

## How was this patch tested?

Existing tests should cover this.

Author: hyukjinkwon <gurwls223@apache.org>

Closes #21975 from HyukjinKwon/remove-build-warnings.
2018-08-04 11:52:49 -05:00
Misha Dmitriev 094aa59715 [SPARK-24801][CORE] Avoid memory waste by empty byte[] arrays in SaslEncryption$EncryptedMessage
## What changes were proposed in this pull request?

Initialize SaslEncryption$EncryptedMessage.byteChannel lazily,
so that empty, not yet used instances of ByteArrayWritableChannel
referenced by this field don't use up memory.

I analyzed a heap dump from Yarn Node Manager where this code is used, and found that there are over 40,000 of the above objects in memory, each with a big empty byte[] array. The reason they are all there is because of Netty queued up a large number of messages in memory  before transferTo() is called. There is a small number of netty ChannelOutboundBuffer objects, and then collectively , via linked lists starting from their flushedEntry data fields, they end up referencing over 40K ChannelOutboundBuffer$Entry objects, which ultimately reference EncryptedMessage objects.

## How was this patch tested?

Ran all the tests locally.

Author: Misha Dmitriev <misha@cloudera.com>

Closes #21811 from countmdm/misha/spark-24801.
2018-07-26 22:15:12 -05:00
Imran Rashid 16f2c3ea46 [SPARK-6237][NETWORK] Network-layer changes to allow stream upload.
These changes allow an RPCHandler to receive an upload as a stream of
data, without having to buffer the entire message in the FrameDecoder.
The primary use case is for replicating large blocks.  By itself, this change is adding dead-code that is not being used -- it is a step towards SPARK-24296.

Added unit tests for handling streaming data, including successfully sending data, and failures in reading the stream with concurrent requests.

Summary of changes:

* Introduce a new UploadStream RPC which is sent to push a large payload as a stream (in contrast, the pre-existing StreamRequest and StreamResponse RPCs are used for pull-based streaming).
* Generalize RpcHandler.receive() to support requests which contain streams.
* Generalize StreamInterceptor to handle both request and response messages (previously it only handled responses).
* Introduce StdChannelListener to abstract away common logging logic in ChannelFuture listeners.

Author: Imran Rashid <irashid@cloudera.com>

Closes #21346 from squito/upload_stream.
2018-06-26 15:56:58 -07:00
Wenbo Zhao 3f4bda7289 [SPARK-24578][CORE] Cap sub-region's size of returned nio buffer
## What changes were proposed in this pull request?
This PR tries to fix the performance regression introduced by SPARK-21517.

In our production job, we performed many parallel computations, with high possibility, some task could be scheduled to a host-2 where it needs to read the cache block data from host-1. Often, this big transfer makes the cluster suffer time out issue (it will retry 3 times, each with 120s timeout, and then do recompute to put the cache block into the local MemoryStore).

The root cause is that we don't do `consolidateIfNeeded` anymore as we are using
```
Unpooled.wrappedBuffer(chunks.length, getChunks(): _*)
```
in ChunkedByteBuffer. If we have many small chunks, it could cause the `buf.notBuffer(...)` have very bad performance in the case that we have to call `copyByteBuf(...)` many times.

## How was this patch tested?
Existing unit tests and also test in production

Author: Wenbo Zhao <wzhao@twosigma.com>

Closes #21593 from WenboZhao/spark-24578.
2018-06-20 14:26:04 -07:00
Misha Dmitriev de4feae3cd [SPARK-24356][CORE] Duplicate strings in File.path managed by FileSegmentManagedBuffer
This patch eliminates duplicate strings that come from the 'path' field of
java.io.File objects created by FileSegmentManagedBuffer. That is, we want
to avoid the situation when multiple File instances for the same pathname
"foo/bar" are created, each with a separate copy of the "foo/bar" String
instance. In some scenarios such duplicate strings may waste a lot of memory
(~ 10% of the heap). To avoid that, we intern the pathname with
String.intern(), and before that we make sure that it's in a normalized
form (contains no "//", "///" etc.) Otherwise, the code in java.io.File
would normalize it later, creating a new "foo/bar" String copy.
Unfortunately, the normalization code that java.io.File uses internally
is in the package-private class java.io.FileSystem, so we cannot call it
here directly.

## What changes were proposed in this pull request?

Added code to ExternalShuffleBlockResolver.getFile(), that normalizes and then interns the pathname string before passing it to the File() constructor.

## How was this patch tested?

Added unit test

Author: Misha Dmitriev <misha@cloudera.com>

Closes #21456 from countmdm/misha/spark-24356.
2018-06-02 23:07:39 -05:00
Xingbo Jiang 8ef167a5f9 [SPARK-24340][CORE] Clean up non-shuffle disk block manager files following executor exits on a Standalone cluster
## What changes were proposed in this pull request?

Currently we only clean up the local directories on application removed. However, when executors die and restart repeatedly, many temp files are left untouched in the local directories, which is undesired behavior and could cause disk space used up gradually.

We can detect executor death in the Worker, and clean up the non-shuffle files (files not ended with ".index" or ".data") in the local directories, we should not touch the shuffle files since they are expected to be used by the external shuffle service.

Scope of this PR is limited to only implement the cleanup logic on a Standalone cluster, we defer to experts familiar with other cluster managers(YARN/Mesos/K8s) to determine whether it's worth to add similar support.

## How was this patch tested?

Add new test suite to cover.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #21390 from jiangxb1987/cleanupNonshuffleFiles.
2018-06-01 13:46:05 -07:00
Kazuaki Ishizaki 9215ee7a16 [SPARK-23976][CORE] Detect length overflow in UTF8String.concat()/ByteArray.concat()
## What changes were proposed in this pull request?

This PR detects length overflow if total elements in inputs are not acceptable.

For example, when the three inputs has `0x7FFF_FF00`, `0x7FFF_FF00`, and `0xE00`, we should detect length overflow since we cannot allocate such a large structure on `byte[]`.
On the other hand, the current algorithm can allocate the result structure with `0x1000`-byte length due to integer sum overflow.

## How was this patch tested?

Existing UTs.
If we would create UTs, we need large heap (6-8GB). It may make test environment unstable.
If it is necessary to create UTs, I will create them.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21064 from kiszk/SPARK-23976.
2018-05-02 10:41:34 +02:00
Marcelo Vanzin 428b903859 [SPARK-24029][CORE] Follow up: set SO_REUSEADDR on the server socket.
"childOption" is for the remote connections, not for the server socket
that actually listens for incoming connections.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21132 from vanzin/SPARK-24029.2.
2018-04-24 09:10:29 +08:00
Kazuaki Ishizaki c3a86faa53 [SPARK-10399][SPARK-23879][FOLLOWUP][CORE] Free unused off-heap memory in MemoryBlockSuite
## What changes were proposed in this pull request?

As viirya pointed out [here](https://github.com/apache/spark/pull/19222#discussion_r179910484), this PR explicitly frees unused off-heap memory in `MemoryBlockSuite`

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21117 from kiszk/SPARK-10399-free-offheap.
2018-04-23 10:45:25 +08:00
Marcelo Vanzin 32b4bcd6d3 [SPARK-24029][CORE] Set SO_REUSEADDR on listen sockets.
This allows sockets to be bound even if there are sockets
from a previous application that are still pending closure. It
avoids bind issues when, for example, re-starting the SHS.

Don't enable the option on Windows though. The following page
explains some odd behavior that this option can have there:
https://msdn.microsoft.com/en-us/library/windows/desktop/ms740621%28v=vs.85%29.aspx

I intentionally ignored server sockets that always bind to
ephemeral ports, since those don't benefit from this option.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #21110 from vanzin/SPARK-24029.
2018-04-21 23:14:58 +08:00
mn-mikke e6b466084c [SPARK-23736][SQL] Extending the concat function to support array columns
## What changes were proposed in this pull request?
The PR adds a logic for easy concatenation of multiple array columns and covers:
- Concat expression has been extended to support array columns
- A Python wrapper

## How was this patch tested?
New tests added into:
- CollectionExpressionsSuite
- DataFrameFunctionsSuite
- typeCoercion/native/concat.sql

## Codegen examples
### Primitive-type elements
```
val df = Seq(
  (Seq(1 ,2), Seq(3, 4)),
  (Seq(1, 2, 3), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             long project_size = UnsafeArrayData.calculateSizeOfUnderlyingByteArray(
/* 070 */               project_numElements,
/* 071 */               4);
/* 072 */             if (project_size > 2147483632) {
/* 073 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_size +
/* 074 */                 " bytes of data due to exceeding the limit 2147483632 bytes" +
/* 075 */                 " for UnsafeArrayData.");
/* 076 */             }
/* 077 */
/* 078 */             byte[] project_array = new byte[(int)project_size];
/* 079 */             UnsafeArrayData project_arrayData = new UnsafeArrayData();
/* 080 */             Platform.putLong(project_array, 16, project_numElements);
/* 081 */             project_arrayData.pointTo(project_array, 16, (int)project_size);
/* 082 */             int project_counter = 0;
/* 083 */             for (int y = 0; y < 2; y++) {
/* 084 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 085 */                 if (args[y].isNullAt(z)) {
/* 086 */                   project_arrayData.setNullAt(project_counter);
/* 087 */                 } else {
/* 088 */                   project_arrayData.setInt(
/* 089 */                     project_counter,
/* 090 */                     args[y].getInt(z)
/* 091 */                   );
/* 092 */                 }
/* 093 */                 project_counter++;
/* 094 */               }
/* 095 */             }
/* 096 */             return project_arrayData;
/* 097 */           }
/* 098 */         }.concat(project_args);
/* 099 */         boolean project_isNull = project_value == null;
```

### Non-primitive-type elements
```
val df = Seq(
  (Seq("aa" ,"bb"), Seq("ccc", "ddd")),
  (Seq("x", "y"), null)
).toDF("a", "b")
df.filter('a.isNotNull).select(concat('a, 'b)).debugCodegen()
```
Result:
```
/* 033 */         boolean inputadapter_isNull = inputadapter_row.isNullAt(0);
/* 034 */         ArrayData inputadapter_value = inputadapter_isNull ?
/* 035 */         null : (inputadapter_row.getArray(0));
/* 036 */
/* 037 */         if (!(!inputadapter_isNull)) continue;
/* 038 */
/* 039 */         ((org.apache.spark.sql.execution.metric.SQLMetric) references[0] /* numOutputRows */).add(1);
/* 040 */
/* 041 */         ArrayData[] project_args = new ArrayData[2];
/* 042 */
/* 043 */         if (!false) {
/* 044 */           project_args[0] = inputadapter_value;
/* 045 */         }
/* 046 */
/* 047 */         boolean inputadapter_isNull1 = inputadapter_row.isNullAt(1);
/* 048 */         ArrayData inputadapter_value1 = inputadapter_isNull1 ?
/* 049 */         null : (inputadapter_row.getArray(1));
/* 050 */         if (!inputadapter_isNull1) {
/* 051 */           project_args[1] = inputadapter_value1;
/* 052 */         }
/* 053 */
/* 054 */         ArrayData project_value = new Object() {
/* 055 */           public ArrayData concat(ArrayData[] args) {
/* 056 */             for (int z = 0; z < 2; z++) {
/* 057 */               if (args[z] == null) return null;
/* 058 */             }
/* 059 */
/* 060 */             long project_numElements = 0L;
/* 061 */             for (int z = 0; z < 2; z++) {
/* 062 */               project_numElements += args[z].numElements();
/* 063 */             }
/* 064 */             if (project_numElements > 2147483632) {
/* 065 */               throw new RuntimeException("Unsuccessful try to concat arrays with " + project_numElements +
/* 066 */                 " elements due to exceeding the array size limit 2147483632.");
/* 067 */             }
/* 068 */
/* 069 */             Object[] project_arrayObjects = new Object[(int)project_numElements];
/* 070 */             int project_counter = 0;
/* 071 */             for (int y = 0; y < 2; y++) {
/* 072 */               for (int z = 0; z < args[y].numElements(); z++) {
/* 073 */                 project_arrayObjects[project_counter] = args[y].getUTF8String(z);
/* 074 */                 project_counter++;
/* 075 */               }
/* 076 */             }
/* 077 */             return new org.apache.spark.sql.catalyst.util.GenericArrayData(project_arrayObjects);
/* 078 */           }
/* 079 */         }.concat(project_args);
/* 080 */         boolean project_isNull = project_value == null;
```

Author: mn-mikke <mrkAha12346github>

Closes #20858 from mn-mikke/feature/array-api-concat_arrays-to-master.
2018-04-20 14:58:11 +09:00
Kris Mok f94f3624ea [SPARK-23947][SQL] Add hashUTF8String convenience method to hasher classes
## What changes were proposed in this pull request?

Add `hashUTF8String()` to the hasher classes to allow Spark SQL codegen to generate cleaner code for hashing `UTF8String`s. No change in behavior otherwise.

Although with the introduction of SPARK-10399, the code size for hashing `UTF8String` is already smaller, it's still good to extract a separate function in the hasher classes so that the generated code can stay clean.

## How was this patch tested?

Existing tests.

Author: Kris Mok <kris.mok@databricks.com>

Closes #21016 from rednaxelafx/hashutf8.
2018-04-09 21:07:28 -07:00
Kazuaki Ishizaki 710a68cec2 [SPARK-23892][TEST] Improve converge and fix lint error in UTF8String-related tests
## What changes were proposed in this pull request?

This PR improves test coverage in `UTF8StringSuite` and code efficiency in `UTF8StringPropertyCheckSuite`.

This PR also fixes lint-java issue in `UTF8StringSuite` reported at [here](https://github.com/apache/spark/pull/20995#issuecomment-379325527)

```[ERROR] src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.```

## How was this patch tested?

Existing UT

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21000 from kiszk/SPARK-23892.
2018-04-08 20:26:31 +02:00
Kazuaki Ishizaki b6935ffb4d [SPARK-10399][SPARK-23879][HOTFIX] Fix Java lint errors
## What changes were proposed in this pull request?

This PR fixes the following errors in [Java lint](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Compile/job/spark-master-lint/7717/console) after #19222 has been merged. These errors were pointed by ueshin .

```
[ERROR] src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java:[57] (sizes) LineLength: Line is longer than 100 characters (found 106).
[ERROR] src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java:[26,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java:[23,10] (modifier) ModifierOrder: 'public' modifier out of order with the JLS suggestions.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[64,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[69,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[74,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[79,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[84,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[89,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[94,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[99,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[104,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[109,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[114,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[119,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[124,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java:[129,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[60,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[65,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[70,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[75,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[80,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[85,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[90,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[95,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[100,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[105,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[110,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[115,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[120,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java:[125,10] (modifier) RedundantModifier: Redundant 'final' modifier.
[ERROR] src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java:[114,16] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java:[20,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
[ERROR] src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.memory.MemoryBlock.
[ERROR] src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java:[126,15] (naming) MethodName: Method name 'ByteArrayMemoryBlockTest' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java:[143,15] (naming) MethodName: Method name 'OnHeapMemoryBlockTest' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java:[160,15] (naming) MethodName: Method name 'OffHeapArrayMemoryBlockTest' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java:[19,8] (imports) UnusedImports: Unused import - com.google.common.primitives.Ints.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java:[21,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
[ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[20,8] (imports) UnusedImports: Unused import - org.apache.spark.unsafe.Platform.
```

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #20991 from kiszk/SPARK-10399-jlint.
2018-04-06 10:23:26 -07:00
Kazuaki Ishizaki c926acf719 [SPARK-23882][CORE] UTF8StringSuite.writeToOutputStreamUnderflow() is not expected to be supported
## What changes were proposed in this pull request?

This PR excludes an existing UT [`writeToOutputStreamUnderflow()`](https://github.com/apache/spark/blob/master/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java#L519-L532) in `UTF8StringSuite`.

As discussed [here](https://github.com/apache/spark/pull/19222#discussion_r177692142), the behavior of this test looks surprising. This test seems to access metadata area of the JVM object where is reserved by `Platform.BYTE_ARRAY_OFFSET`.

This test is introduced thru #16089 by NathanHowell. More specifically, [the commit](27c102deb1) `Improve test coverage of UTFString.write` introduced this UT. However, I cannot find any discussion about this UT.

I think that it would be good to exclude this UT.

```java
  public void writeToOutputStreamUnderflow() throws IOException {
    // offset underflow is apparently supported?
    final ByteArrayOutputStream outputStream = new ByteArrayOutputStream();
    final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8);

    for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) {
      new UTF8String(
        new ByteArrayMemoryBlock(test, Platform.BYTE_ARRAY_OFFSET - i, test.length + i))
          .writeTo(outputStream);
      final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length);
      assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString());
      outputStream.reset();
    }
  }
```

## How was this patch tested?

Existing UTs

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #20995 from kiszk/SPARK-23882.
2018-04-06 18:42:14 +02:00
Kazuaki Ishizaki 4807d381bb [SPARK-10399][CORE][SQL] Introduce multiple MemoryBlocks to choose several types of memory block
## What changes were proposed in this pull request?

This PR allows us to use one of several types of `MemoryBlock`, such as byte array, int array, long array, or `java.nio.DirectByteBuffer`. To use `java.nio.DirectByteBuffer` allows to have off heap memory which is automatically deallocated by JVM. `MemoryBlock`  class has primitive accessors like `Platform.getInt()`, `Platform.putint()`, or `Platform.copyMemory()`.

This PR uses `MemoryBlock` for `OffHeapColumnVector`, `UTF8String`, and other places. This PR can improve performance of operations involving memory accesses (e.g. `UTF8String.trim`) by 1.8x.

For now, this PR does not use `MemoryBlock` for `BufferHolder` based on cloud-fan's [suggestion](https://github.com/apache/spark/pull/11494#issuecomment-309694290).

Since this PR is a successor of #11494, close #11494. Many codes were ported from #11494. Many efforts were put here. **I think this PR should credit to yzotov.**

This PR can achieve **1.1-1.4x performance improvements** for  operations in `UTF8String` or `Murmur3_x86_32`. Other operations are almost comparable performances.

Without this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 526 /  536          0.0   131399881.5       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       525 /  552       1022.6           1.0       1.0X
substring                                      414 /  423       1298.0           0.8       1.3X
```

With this PR
```
OpenJDK 64-Bit Server VM 1.8.0_121-8u121-b13-0ubuntu1.16.04.2-b13 on Linux 4.4.0-22-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Hash byte arrays with length 268435487:  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Murmur3_x86_32                                 474 /  488          0.0   118552232.0       1.0X

UTF8String benchmark:                    Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
hashCode                                       476 /  480       1127.3           0.9       1.0X
substring                                      287 /  291       1869.9           0.5       1.7X
```

Benchmark program
```
test("benchmark Murmur3_x86_32") {
  val length = 8192 * 32768 + 31
  val seed = 42L
  val iters = 1 << 2
  val random = new Random(seed)
  val arrays = Array.fill[MemoryBlock](numArrays) {
    val bytes = new Array[Byte](length)
    random.nextBytes(bytes)
    new ByteArrayMemoryBlock(bytes, Platform.BYTE_ARRAY_OFFSET, length)
  }

  val benchmark = new Benchmark("Hash byte arrays with length " + length,
    iters * numArrays, minNumIters = 20)
  benchmark.addCase("HiveHasher") { _: Int =>
    var sum = 0L
    for (_ <- 0L until iters) {
      sum += HiveHasher.hashUnsafeBytesBlock(
        arrays(i), Platform.BYTE_ARRAY_OFFSET, length)
    }
  }
  benchmark.run()
}

test("benchmark UTF8String") {
  val N = 512 * 1024 * 1024
  val iters = 2
  val benchmark = new Benchmark("UTF8String benchmark", N, minNumIters = 20)
  val str0 = new java.io.StringWriter() { { for (i <- 0 until N) { write(" ") } } }.toString
  val s0 = UTF8String.fromString(str0)
  benchmark.addCase("hashCode") { _: Int =>
    var h: Int = 0
    for (_ <- 0L until iters) { h += s0.hashCode }
  }
  benchmark.addCase("substring") { _: Int =>
    var s: UTF8String = null
    for (_ <- 0L until iters) { s = s0.substring(N / 2 - 5, N / 2 + 5) }
  }
  benchmark.run()
}
```

I run [this benchmark program](https://gist.github.com/kiszk/94f75b506c93a663bbbc372ffe8f05de) using [the commit](ee5a79861c). I got the following results:

```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Memory access benchmarks:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
ByteArrayMemoryBlock get/putInt()              220 /  221        609.3           1.6       1.0X
Platform get/putInt(byte[])                    220 /  236        610.9           1.6       1.0X
Platform get/putInt(Object)                    492 /  494        272.8           3.7       0.4X
OnHeapMemoryBlock get/putLong()                322 /  323        416.5           2.4       0.7X
long[]                                         221 /  221        608.0           1.6       1.0X
Platform get/putLong(long[])                   321 /  321        418.7           2.4       0.7X
Platform get/putLong(Object)                   561 /  563        239.2           4.2       0.4X
```

I also run [this benchmark program](https://gist.github.com/kiszk/5fdb4e03733a5d110421177e289d1fb5) for comparing performance of `Platform.copyMemory()`.
```
OpenJDK 64-Bit Server VM 1.8.0_151-8u151-b12-0ubuntu0.16.04.2-b12 on Linux 4.4.0-66-generic
Intel(R) Xeon(R) CPU E5-2667 v3  3.20GHz
Platform copyMemory:                     Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
------------------------------------------------------------------------------------------------
Object to Object                              1961 / 1967          8.6         116.9       1.0X
System.arraycopy Object to Object             1917 / 1921          8.8         114.3       1.0X
byte array to byte array                      1961 / 1968          8.6         116.9       1.0X
System.arraycopy byte array to byte array      1909 / 1937          8.8         113.8       1.0X
int array to int array                        1921 / 1990          8.7         114.5       1.0X
double array to double array                  1918 / 1923          8.7         114.3       1.0X
Object to byte array                          1961 / 1967          8.6         116.9       1.0X
Object to short array                         1965 / 1972          8.5         117.1       1.0X
Object to int array                           1910 / 1915          8.8         113.9       1.0X
Object to float array                         1971 / 1978          8.5         117.5       1.0X
Object to double array                        1919 / 1944          8.7         114.4       1.0X
byte array to Object                          1959 / 1967          8.6         116.8       1.0X
int array to Object                           1961 / 1970          8.6         116.9       1.0X
double array to Object                        1917 / 1924          8.8         114.3       1.0X
```

These results show three facts:
1. According to the second/third or sixth/seventh results in the first experiment, if we use `Platform.get/putInt(Object)`, we achieve more than 2x worse performance than `Platform.get/putInt(byte[])` with concrete type (i.e. `byte[]`).
2. According to the second/third or fourth/fifth/sixth results in the first experiment, the fastest way to access an array element on Java heap is `array[]`. **Cons of `array[]` is that it is not possible to support unaligned-8byte access.**
3. According to the first/second/third or fourth/sixth/seventh results in the first experiment, `getInt()/putInt() or getLong()/putLong()` in subclasses of `MemoryBlock` can achieve comparable performance to `Platform.get/putInt()` or `Platform.get/putLong()` with concrete type (second or sixth result). There is no overhead regarding virtual call.
4. According to results in the second experiment, for `Platform.copy()`, to pass `Object` can achieve the same performance as to pass any type of primitive array as source or destination.
5. According to second/fourth results in the second experiment, `Platform.copy()` can achieve the same performance as `System.arrayCopy`. **It would be good to use `Platform.copy()` since `Platform.copy()` can take any types for src and dst.**

We are incrementally replace `Platform.get/putXXX` with `MemoryBlock.get/putXXX`. This is because we have two advantages.
1) Achieve better performance due to having a concrete type for an array.
2) Use simple OO design instead of passing `Object`
It is easy to use `MemoryBlock` in `InternalRow`, `BufferHolder`, `TaskMemoryManager`, and others that are already abstracted. It is not easy to use `MemoryBlock` in utility classes related to hashing or others.

Other candidates are
- UnsafeRow, UnsafeArrayData, UnsafeMapData, SpecificUnsafeRowJoiner
- UTF8StringBuffer
- BufferHolder
- TaskMemoryManager
- OnHeapColumnVector
- BytesToBytesMap
- CachedBatch
- classes for hash
- others.

## How was this patch tested?

Added `UnsafeMemoryAllocator`

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #19222 from kiszk/SPARK-10399.
2018-04-06 10:13:59 +08:00
Maxim Gekk 5e7bc2acef [SPARK-23649][SQL] Skipping chars disallowed in UTF-8
## What changes were proposed in this pull request?

The mapping of UTF-8 char's first byte to char's size doesn't cover whole range 0-255. It is defined only for 0-253:
https://github.com/apache/spark/blob/master/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java#L60-L65
https://github.com/apache/spark/blob/master/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java#L190

If the first byte of a char is 253-255, IndexOutOfBoundsException is thrown. Besides of that values for 244-252 are not correct according to recent unicode standard for UTF-8: http://www.unicode.org/versions/Unicode10.0.0/UnicodeStandard-10.0.pdf

As a consequence of the exception above, the length of input string in UTF-8 encoding cannot be calculated if the string contains chars started from 253 code. It is visible on user's side as for example crashing of schema inferring of csv file which contains such chars but the file can be read if the schema is specified explicitly or if the mode set to multiline.

The proposed changes build correct mapping of first byte of UTF-8 char to its size (now it covers all cases) and skip disallowed chars (counts it as one octet).

## How was this patch tested?

Added a test and a file with a char which is disallowed in UTF-8 - 0xFF.

Author: Maxim Gekk <maxim.gekk@databricks.com>

Closes #20796 from MaxGekk/skip-wrong-utf8-chars.
2018-03-20 10:34:56 -07:00
Shintaro Murakami d5ed2108d3 [SPARK-23381][CORE] Murmur3 hash generates a different value from other implementations
## What changes were proposed in this pull request?
Murmur3 hash generates a different value from the original and other implementations (like Scala standard library and Guava or so) when the length of a bytes array is not multiple of 4.

## How was this patch tested?
Added a unit test.

**Note: When we merge this PR, please give all the credits to Shintaro Murakami.**

Author: Shintaro Murakami <mrkm4ntrgmail.com>

Author: gatorsmile <gatorsmile@gmail.com>
Author: Shintaro Murakami <mrkm4ntr@gmail.com>

Closes #20630 from gatorsmile/pr-20568.
2018-02-16 17:17:55 -08:00
Marco Gaido 0fc26313f8 [SPARK-21860][CORE][FOLLOWUP] fix java style error
## What changes were proposed in this pull request?

#19077 introduced a Java style error (too long line). Quick fix.

## How was this patch tested?

running `./dev/lint-java`

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #20558 from mgaido91/SPARK-21860.
2018-02-09 08:46:27 -06:00
liuxian 76e019d9bd [SPARK-21860][CORE] Improve memory reuse for heap memory in HeapMemoryAllocator
## What changes were proposed in this pull request?
In `HeapMemoryAllocator`, when allocating memory from pool, and the key of pool is memory size.
Actually some size of memory ,such as 1025bytes,1026bytes,......1032bytes, we can think they are the same,because we allocate memory in multiples of 8 bytes.
In this case, we can improve memory reuse.

## How was this patch tested?
Existing tests and added unit tests

Author: liuxian <liu.xian3@zte.com.cn>

Closes #19077 from 10110346/headmemoptimize.
2018-02-08 23:41:30 +08:00
Shixiong Zhu ec63e2d074 [SPARK-23289][CORE] OneForOneBlockFetcher.DownloadCallback.onData should write the buffer fully
## What changes were proposed in this pull request?

`channel.write(buf)` may not write the whole buffer since the underlying channel is a FileChannel, we should retry until the whole buffer is written.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <zsxwing@gmail.com>

Closes #20461 from zsxwing/SPARK-23289.
2018-02-01 21:00:47 +08:00
Marcelo Vanzin aa3a1276f9 [SPARK-23103][CORE] Ensure correct sort order for negative values in LevelDB.
The code was sorting "0" as "less than" negative values, which is a little
wrong. Fix is simple, most of the changes are the added test and related
cleanup.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20284 from vanzin/SPARK-23103.
2018-01-19 13:32:20 -06:00
Dongjoon Hyun 7bd14cfd40 [MINOR][BUILD] Fix Java linter errors
## What changes were proposed in this pull request?

This PR cleans up the java-lint errors (for v2.3.0-rc1 tag). Hopefully, this will be the final one.

```
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java:[85] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/launcher/InProcessAppHandle.java:[20,8] (imports) UnusedImports: Unused import - java.io.IOException.
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java:[41,9] (modifier) ModifierOrder: 'private' modifier out of order with the JLS suggestions.
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java:[464] (sizes) LineLength: Line is longer than 100 characters (found 102).
```

## How was this patch tested?

Manual.

```
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #20242 from dongjoon-hyun/fix_lint_java_2.3_rc1.
2018-01-12 10:18:42 -08:00