## 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>
## What changes were proposed in this pull request?
It seems that https://bugs.openjdk.java.net/browse/JDK-8068730 makes `InputStreamsSuite` very flaky.
<img width="903" alt="error" src="https://user-images.githubusercontent.com/9700541/59727067-017eb780-91e9-11e9-8bb0-ac5f4c1bc44d.png">
As we can see the Jenkins result, this can be reproduced frequently with JDK9+.
```
$ build/sbt "streaming/testOnly *.InputStreamsSuite"
[info] - Modified files are correctly detected. *** FAILED *** (134 milliseconds)
[info] Set("renamed") did not equal Set() (InputStreamsSuite.scala:312)
[info] org.scalatest.exceptions.TestFailedException:
```
The reason is the `renamed.txt`'s modification time becomes greater than the clock in JDK9+ and Spark ignored it with **not selected** message. In JDK8, the modification time generated by this test case doesn't have `milliseconds` part.
```
Getting new files for time 1560896662000, ignoring files older than 1560896659679
file:/.../streaming/subdir/renamed.txt not selected as mod time 1560896662679 > current time 1560896662000
file:/.../streaming/subdir/existing ignored as mod time 1560896657679 <= ignore time 1560896659679
Finding new files took 0 ms
New files at time 1560896662000 ms:
```
## How was this patch tested?
Pass the Jenkins and manually repeat the following with JDK11 10 times.
```
$ build/sbt "streaming/testOnly *.InputStreamsSuite"
```
Closes#24904 from dongjoon-hyun/SPARK-28101.
Authored-by: Dongjoon Hyun <dhyun@apple.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
This change refactors the portions of the ExecutorAllocationManager class that
track executor state into a new class, to achieve a few goals:
- make the code easier to understand
- better separate concerns (task backlog vs. executor state)
- less synchronization between event and allocation threads
- less coupling between the allocation code and executor state tracking
The executor tracking code was moved to a new class (ExecutorMonitor) that
encapsulates all the logic of tracking what happens to executors and when
they can be timed out. The logic to actually remove the executors remains
in the EAM, since it still requires information that is not tracked by the
new executor monitor code.
In the executor monitor itself, of interest, specifically, is a change in
how cached blocks are tracked; instead of polling the block manager, the
monitor now uses events to track which executors have cached blocks, and
is able to detect also unpersist events and adjust the time when the executor
should be removed accordingly. (That's the bug mentioned in the PR title.)
Because of the refactoring, a few tests in the old EAM test suite were removed,
since they're now covered by the newly added test suite. The EAM suite was
also changed a little bit to not instantiate a SparkContext every time. This
allowed some cleanup, and the tests also run faster.
Tested with new and updated unit tests, and with multiple TPC-DS workloads
running with dynamic allocation on; also some manual tests for the caching
behavior.
Closes#24704 from vanzin/SPARK-20286.
Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
## What changes were proposed in this pull request?
descending sort in HDFSMetadataLog.getLatest instead of two action of ascending sort and reverse
## How was this patch tested?
Jenkins
Closes#24711 from wenxuanguan/bug-fix-hdfsmetadatalog.
Authored-by: wenxuanguan <choose_home@126.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
# 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>
## 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>
## What changes were proposed in this pull request?
I want to get rid of as much use of `scala.language.existentials` as possible for 3.0. It's a complicated language feature that generates warnings unless this value is imported. It might even be on the way out of Scala: https://contributors.scala-lang.org/t/proposal-to-remove-existential-types-from-the-language/2785
For Spark, it comes up mostly where the code plays fast and loose with generic types, not the advanced situations you'll often see referenced where this feature is explained. For example, it comes up in cases where a function returns something like `(String, Class[_])`. Scala doesn't like matching this to any other instance of `(String, Class[_])` because doing so requires inferring the existence of some type that satisfies both. Seems obvious if the generic type is a wildcard, but, not technically something Scala likes to let you get away with.
This is a large PR, and it only gets rid of _most_ instances of `scala.language.existentials`. The change should be all compile-time and shouldn't affect APIs or logic.
Many of the changes simply touch up sloppiness about generic types, making the known correct value explicit in the code.
Some fixes involve being more explicit about the existence of generic types in methods. For instance, `def foo(arg: Class[_])` seems innocent enough but should really be declared `def foo[T](arg: Class[T])` to let Scala select and fix a single type when evaluating calls to `foo`.
For kind of surprising reasons, this comes up in places where code evaluates a tuple of things that involve a generic type, but is OK if the two parts of the tuple are evaluated separately.
One key change was altering `Utils.classForName(...): Class[_]` to the more correct `Utils.classForName[T](...): Class[T]`. This caused a number of small but positive changes to callers that otherwise had to cast the result.
In several tests, `Dataset[_]` was used where `DataFrame` seems to be the clear intent.
Finally, in a few cases in MLlib, the return type `this.type` was used where there are no subclasses of the class that uses it. This really isn't needed and causes issues for Scala reasoning about the return type. These are just changed to be concrete classes as return types.
After this change, we have only a few classes that still import `scala.language.existentials` (because modifying them would require extensive rewrites to fix) and no build warnings.
## How was this patch tested?
Existing tests.
Closes#24431 from srowen/SPARK-27536.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
This patch makes several test flakiness fixes.
## How was this patch tested?
N/A
Closes#24434 from gatorsmile/fixFlakyTest.
Lead-authored-by: gatorsmile <gatorsmile@gmail.com>
Co-authored-by: Hyukjin Kwon <gurwls223@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## 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>
## What changes were proposed in this pull request?
While I was reviewing #24235 I've found a minor addition possibility. Namely `FileSystem.delete` returns a boolean which is not yet checked. In this PR I've added a warning message when it returns false. I've added this as MINOR because no control flow change introduced.
## How was this patch tested?
Existing unit tests.
Closes#24263 from gaborgsomogyi/SPARK-27301-minor.
Authored-by: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Use Single Abstract Method syntax where possible (and minor related cleanup). Comments below. No logic should change here.
## How was this patch tested?
Existing tests.
Closes#24241 from srowen/SPARK-27323.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Dongjoon Hyun <dhyun@apple.com>
## What changes were proposed in this pull request?
The cached FileSystem's token will expire if no tokens explicitly are add into it.
```scala
19/03/28 13:40:16 INFO storage.BlockManager: Removing RDD 83189
19/03/28 13:40:16 INFO rdd.MapPartitionsRDD: Removing RDD 82860 from persistence list
19/03/28 13:40:16 INFO spark.ContextCleaner: Cleaned shuffle 6005
19/03/28 13:40:16 INFO storage.BlockManager: Removing RDD 82860
19/03/28 13:40:16 INFO scheduler.ReceivedBlockTracker: Deleting batches:
19/03/28 13:40:16 INFO scheduler.InputInfoTracker: remove old batch metadata: 1553750250000 ms
19/03/28 13:40:17 WARN security.UserGroupInformation: PriviledgedActionException as:ursHADOOP.HZ.NETEASE.COM (auth:KERBEROS) cause:org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken): token (HDFS_DELEGATION_TOKEN token 53240500 for urs) is expired, current time: 2019-03-28 13:40:17,010+0800 expected renewal time: 2019-03-28 13:39:48,523+0800
19/03/28 13:40:17 WARN ipc.Client: Exception encountered while connecting to the server : org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken): token (HDFS_DELEGATION_TOKEN token 53240500 for urs) is expired, current time: 2019-03-28 13:40:17,010+0800 expected renewal time: 2019-03-28 13:39:48,523+0800
19/03/28 13:40:17 WARN security.UserGroupInformation: PriviledgedActionException as:ursHADOOP.HZ.NETEASE.COM (auth:KERBEROS) cause:org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken): token (HDFS_DELEGATION_TOKEN token 53240500 for urs) is expired, current time: 2019-03-28 13:40:17,010+0800 expected renewal time: 2019-03-28 13:39:48,523+0800
19/03/28 13:40:17 WARN hdfs.LeaseRenewer: Failed to renew lease for [DFSClient_NONMAPREDUCE_-1396157959_1] for 53 seconds. Will retry shortly ...
org.apache.hadoop.ipc.RemoteException(org.apache.hadoop.security.token.SecretManager$InvalidToken): token (HDFS_DELEGATION_TOKEN token 53240500 for urs) is expired, current time: 2019-03-28 13:40:17,010+0800 expected renewal time: 2019-03-28 13:39:48,523+0800
at org.apache.hadoop.ipc.Client.call(Client.java:1468)
at org.apache.hadoop.ipc.Client.call(Client.java:1399)
at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
at com.sun.proxy.$Proxy11.renewLease(Unknown Source)
at org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.renewLease(ClientNamenodeProtocolTranslatorPB.java:571)
at sun.reflect.GeneratedMethodAccessor40.invoke(Unknown Source)
at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
at org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
at com.sun.proxy.$Proxy12.renewLease(Unknown Source)
at org.apache.hadoop.hdfs.DFSClient.renewLease(DFSClient.java:878)
at org.apache.hadoop.hdfs.LeaseRenewer.renew(LeaseRenewer.java:417)
at org.apache.hadoop.hdfs.LeaseRenewer.run(LeaseRenewer.java:442)
at org.apache.hadoop.hdfs.LeaseRenewer.access$700(LeaseRenewer.java:71)
at org.apache.hadoop.hdfs.LeaseRenewer$1.run(LeaseRenewer.java:298)
at java.lang.Thread.run(Thread.java:748)
```
This PR shorten the FileSystem cached life cycle to the cleanup method inner scope in case of token expiry.
## How was this patch tested?
existing ut
Closes#24235 from yaooqinn/SPARK-27301.
Authored-by: Kent Yao <yaooqinn@hotmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Currently, when enabled streaming dynamic allocation for streaming applications, the maxNumExecutorFailures in ApplicationMaster is still computed with `spark.dynamicAllocation.maxExecutors`.
Actually, we should consider `spark.streaming.dynamicAllocation.maxExecutors` instead.
Related codes:
f87153a3ac/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (L101)
## How was this patch tested?
NA
Please review http://spark.apache.org/contributing.html before opening a pull request.
Closes#23845 from liupc/Fix-incorrect-maxNumExecutorFailures-for-streaming.
Lead-authored-by: Liupengcheng <liupengcheng@xiaomi.com>
Co-authored-by: liupengcheng <liupengcheng@xiaomi.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Clarify that text DataSource read/write, and RDD methods that read text, always use UTF-8 as they use Hadoop's implementation underneath. I think these are all the places that this needs a mention in the user-facing docs.
## How was this patch tested?
Doc tests.
Closes#23962 from srowen/SPARK-26016.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Comparing whether Boolean expression is equal to true is redundant
For example:
The datatype of `a` is boolean.
Before:
if (a == true)
After:
if (a)
## How was this patch tested?
N/A
Closes#23884 from 10110346/simplifyboolean.
Authored-by: liuxian <liu.xian3@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to refactor existing code related to date/time conversions, and replace constants like `1000` and `1000000` by `DateTimeUtils` constants and transformation functions from `java.util.concurrent.TimeUnit._`.
## How was this patch tested?
The changes are tested by existing test suites.
Closes#23878 from MaxGekk/magic-time-constants.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to use `System.nanoTime()` instead of `System.currentTimeMillis()` in measurements of time intervals.
`System.currentTimeMillis()` returns current wallclock time and will follow changes to the system clock. Thus, negative wallclock adjustments can cause timeouts to "hang" for a long time (until wallclock time has caught up to its previous value again). This can happen when ntpd does a "step" after the network has been disconnected for some time. The most canonical example is during system bootup when DHCP takes longer than usual. This can lead to failures that are really hard to understand/reproduce. `System.nanoTime()` is guaranteed to be monotonically increasing irrespective of wallclock changes.
## How was this patch tested?
By existing test suites.
Closes#23727 from MaxGekk/system-nanotime.
Lead-authored-by: Maxim Gekk <max.gekk@gmail.com>
Co-authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
SPARK-23991 introduced a bug in `ReceivedBlockTracker#allocateBlocksToBatch`: when a queue with more than a few thousand blocks are in the queue, serializing the queue throws a StackOverflowError. This change just adds `dequeueAll` to the new `clone` operation on the queue so that the fix in 23991 is preserved but the serialized data comes from an ArrayBuffer which doesn't have the serialization problems that mutable.Queue has.
## How was this patch tested?
A unit test was added.
Closes#23716 from rlodge/SPARK-26734.
Authored-by: Ross Lodge <rlodge@concentricsky.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Make .unpersist(), .destroy() non-blocking by default and adjust callers to request blocking only where important.
This also adds an optional blocking argument to Pyspark's RDD.unpersist(), which never had one.
## How was this patch tested?
Existing tests.
Closes#23685 from srowen/SPARK-26771.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Adjust mem settings in UnifiedMemoryManager used in test suites to ha…ve execution memory > 0
Ref: https://github.com/apache/spark/pull/23457#issuecomment-457409976
## How was this patch tested?
Existing tests
Closes#23645 from srowen/SPARK-26725.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
This PR makes hardcoded configs about spark memory and storage to use `ConfigEntry` and put them in the config package.
## How was this patch tested?
Existing unit tests.
Closes#23623 from SongYadong/configEntry_for_mem_storage.
Authored-by: SongYadong <song.yadong1@zte.com.cn>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
The PR makes hardcoded `spark.dynamicAllocation`, `spark.scheduler`, `spark.rpc`, `spark.task`, `spark.speculation`, and `spark.cleaner` configs to use `ConfigEntry`.
## How was this patch tested?
Existing tests
Closes#23416 from kiszk/SPARK-26463.
Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Misc code cleanup from lgtm.com analysis. See comments below for details.
## How was this patch tested?
Existing tests.
Closes#23571 from srowen/SPARK-26640.
Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
The PR makes hardcoded configs below to use `ConfigEntry`.
* spark.ui
* spark.ssl
* spark.authenticate
* spark.master.rest
* spark.master.ui
* spark.metrics
* spark.admin
* spark.modify.acl
This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).
## How was this patch tested?
Existing tests.
Closes#23423 from HeartSaVioR/SPARK-26466.
Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
Remove spark.memory.useLegacyMode and StaticMemoryManager. Update tests that used the StaticMemoryManager to equivalent use of UnifiedMemoryManager.
## How was this patch tested?
Existing tests, with modifications to make them work with a different mem manager.
Closes#23457 from srowen/SPARK-26539.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
Currently there is code scattered in a bunch of places to do different
things related to HTTP security, such as access control, setting
security-related headers, and filtering out bad content. This makes it
really easy to miss these things when writing new UI code.
This change creates a new filter that does all of those things, and
makes sure that all servlet handlers that are attached to the UI get
the new filter and any user-defined filters consistently. The extent
of the actual features should be the same as before.
The new filter is added at the end of the filter chain, because authentication
is done by custom filters and thus needs to happen first. This means that
custom filters see unfiltered HTTP requests - which is actually the current
behavior anyway.
As a side-effect of some of the code refactoring, handlers added after
the initial set also get wrapped with a GzipHandler, which didn't happen
before.
Tested with added unit tests and in a history server with SPNEGO auth
configured.
Closes#23302 from vanzin/SPARK-24522.
Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
## 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>
## What changes were proposed in this pull request?
Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore to clean up SparkCore test cases. thanks.
## How was this patch tested?
N / A
Closes#23151 from heary-cao/withCreateTempDir.
Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
## 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>
## 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>
…due lack of access to tmpDir from $PWD to HDFS
WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode.
## What changes were proposed in this pull request?
This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir
## How was this patch tested?
Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode.
Closes#22867 from gss2002/SPARK-25778.
Authored-by: gss2002 <greg@senia.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
Deprecated in Java 11, replace Class.newInstance with Class.getConstructor.getInstance, and primtive wrapper class constructors with valueOf or equivalent
## How was this patch tested?
Existing tests.
Closes#22988 from srowen/SPARK-25984.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
The write ahead log expects to be able to call hflush, but that is a no-op when writing to a file with hdfs erasure coding. So ensure that file is always written with replication instead, regardless of filesystem defaults.
None yet. I'm posting this mostly to make it visible.
Closes#22882 from squito/SPARK-25871.
Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
'refreshInterval' is not used any where in the headerSparkPage method. So, we don't need to pass the parameter while calling the 'headerSparkPage' method.
## How was this patch tested?
Existing tests
Closes#22864 from shahidki31/unusedCode.
Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Remove JavaSparkContextVarargsWorkaround
## How was this patch tested?
Existing tests.
Closes#22729 from srowen/SPARK-25737.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Fix old oversight in API: Java `flatMapValues` needs a `FlatMapFunction`
## How was this patch tested?
Existing tests.
Closes#22690 from srowen/SPARK-19287.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Removes all vestiges of Flume in the build, for Spark 3.
I don't think this needs Jenkins config changes.
## How was this patch tested?
Existing tests.
Closes#22692 from srowen/SPARK-25598.
Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request?
Original work by Steve Loughran.
Based on #17745.
This is a minimal patch of changes to FileInputDStream to reduce File status requests when querying files. Each call to file status is 3+ http calls to object store. This patch eliminates the need for it, by using FileStatus objects.
This is a minor optimisation when working with filesystems, but significant when working with object stores.
## How was this patch tested?
Tests included. Existing tests pass.
Closes#22339 from ScrapCodes/PR_17745.
Lead-authored-by: Prashant Sharma <prashant@apache.org>
Co-authored-by: Steve Loughran <stevel@hortonworks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
## 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>
## What changes were proposed in this pull request?
SparkSubmit already logs in the user if a keytab is provided, the only issue is that it uses the existing configs which have "yarn" in their name. As such, the configs were changed to:
`spark.kerberos.keytab` and `spark.kerberos.principal`.
## How was this patch tested?
Will be tested with K8S tests, but needs to be tested with Yarn
- [x] K8S Secure HDFS tests
- [x] Yarn Secure HDFS tests vanzin
Closes#22362 from ifilonenko/SPARK-25372.
Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
Several configuration parameters related to Kubernetes need to be
reset, as they are changed with each invokation of spark-submit and
thus prevents recovery of Spark Streaming tasks.
## What changes were proposed in this pull request?
When using the Kubernetes cluster-manager and spawning a Streaming workload, it is important to reset many spark.kubernetes.* properties that are generated by spark-submit but which would get rewritten when restoring a Checkpoint. This is so, because the spark-submit codepath creates Kubernetes resources, such as a ConfigMap, a Secret and other variables, which have an autogenerated name and the previous one will not resolve anymore.
In short, this change enables checkpoint restoration for streaming workloads, and thus enables Spark Streaming workloads in Kubernetes, which were not possible to restore from a checkpoint before if the workload went down.
## How was this patch tested?
This patch needs would benefit from testing in different k8s clusters.
This is similar to the YARN related code for resetting a Spark Streaming workload, but for the Kubernetes scheduler. This PR removes the initcontainers properties that existed before because they are now removed in master.
For a previous discussion, see the non-rebased work at: apache-spark-on-k8s#516
Closes#22392 from ssaavedra/fix-checkpointing-master.
Authored-by: Santiago Saavedra <santiagosaavedra@gmail.com>
Signed-off-by: Yinan Li <ynli@google.com>
## 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>
The problem occurs because stage object is removed from liveStages in
AppStatusListener onStageCompletion. Because of this any onTaskEnd event
received after onStageCompletion event do not update stage metrics.
The fix is to retain stage objects in liveStages until all tasks are complete.
1. Fixed the reproducible example posted in the JIRA
2. Added unit test
Closes#22209 from ankuriitg/ankurgupta/SPARK-24415.
Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
## What changes were proposed in this pull request?
I propose to remove one of `parmap` methods which accepts an execution context as a parameter. The method should be removed to eliminate any deadlocks that can occur if `parmap` is called recursively on thread pools restricted by size.
Closes#22292 from MaxGekk/remove-overloaded-parmap.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
## What changes were proposed in this pull request?
The PR moves the compilation of the regexp for code formatting outside the method which is called for each code block when splitting expressions, in order to avoid recompiling the regexp every time.
Credit should be given to Izek Greenfield.
## How was this patch tested?
existing UTs
Closes#22135 from mgaido91/SPARK-25093.
Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
In the PR, I propose to replace Scala parallel collections by new methods `parmap()`. The methods use futures to transform a sequential collection by applying a lambda function to each element in parallel. The result of `parmap` is another regular (sequential) collection.
The proposed `parmap` method aims to solve the problem of impossibility to interrupt parallel Scala collection. This possibility is needed for reliable task preemption.
## How was this patch tested?
A test was added to `ThreadUtilsSuite`
Closes#21913 from MaxGekk/par-map.
Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
## What changes were proposed in this pull request?
Removes check that `spark.executor.instances` is set to 0 when using Streaming DRA.
## How was this patch tested?
Manual tests
My only concern with this PR is that `spark.executor.instances` (or the actual initial number of executors that the cluster manager gives Spark) can be outside of `spark.streaming.dynamicAllocation.minExecutors` to `spark.streaming.dynamicAllocation.maxExecutors`. I don't see a good way around that, because this code only runs after the SparkContext has been created.
Author: Karthik Palaniappan <karthikpal@google.com>
Closes#19183 from karth295/master.
## What changes were proposed in this pull request?
Updated URL/href links to include a '/' before '?id' to make links consistent and avoid http 302 redirect errors within UI port 4040 tabs.
## How was this patch tested?
Built a runnable distribution and executed jobs. Validated that http 302 redirects are no longer encountered when clicking on links within UI port 4040 tabs.
Author: Steven Kallman <SJKallmangmail.com>
Author: Kallman, Steven <Steven.Kallman@CapitalOne.com>
Closes#21600 from SJKallman/{Spark-24553}{WEB-UI}-redirect-href-fixes.
## What changes were proposed in this pull request?
This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)
The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```
To avoid this problem, this PR performs cast from int to long in right hand side.
## How was this patch tested?
Existing UTs.
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#21481 from kiszk/SPARK-24452.
`WebUI` defines `addStaticHandler` that web UIs don't use (and simply introduce duplication). Let's clean them up and remove duplications.
Local build and waiting for Jenkins
Author: Jacek Laskowski <jacek@japila.pl>
Closes#21510 from jaceklaskowski/SPARK-24490-Use-WebUI.addStaticHandler.
When blocks tried to get allocated to a batch and WAL write fails then the blocks will be removed from the received block queue. This fact simply produces data loss because the next allocation will not find the mentioned blocks in the queue.
In this PR blocks will be removed from the received queue only if WAL write succeded.
Additional unit test.
Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Closes#21430 from gaborgsomogyi/SPARK-23991.
Change-Id: I5ead84f0233f0c95e6d9f2854ac2ff6906f6b341
## What changes were proposed in this pull request?
The PR retrieves the proxyBase automatically from the header `X-Forwarded-Context` (if available). This is the header used by Knox to inform the proxied service about the base path.
This provides 0-configuration support for Knox gateway (instead of having to properly set `spark.ui.proxyBase`) and it allows to access directly SHS when it is proxied by Knox. In the previous scenario, indeed, after setting `spark.ui.proxyBase`, direct access to SHS was not working fine (due to bad link generated).
## How was this patch tested?
added UT + manual tests
Author: Marco Gaido <marcogaido91@gmail.com>
Closes#21268 from mgaido91/SPARK-24209.
Currently, the Spark AM relies on the initial set of tokens created by
the submission client to be able to talk to HDFS and other services that
require delegation tokens. This means that after those tokens expire, a
new AM will fail to start (e.g. when there is an application failure and
re-attempts are enabled).
This PR makes it so that the first thing the AM does when the user provides
a principal and keytab is to create new delegation tokens for use. This
makes sure that the AM can be started irrespective of how old the original
token set is. It also allows all of the token management to be done by the
AM - there is no need for the submission client to set configuration values
to tell the AM when to renew tokens.
Note that even though in this case the AM will not be using the delegation
tokens created by the submission client, those tokens still need to be provided
to YARN, since they are used to do log aggregation.
To be able to re-use the code in the AMCredentialRenewal for the above
purposes, I refactored that class a bit so that it can fetch tokens into
a pre-defined UGI, insted of always logging in.
Another issue with re-attempts is that, after the fix that allows the AM
to restart correctly, new executors would get confused about when to
update credentials, because the credential updater used the update time
initially set up by the submission code. This could make the executor
fail to update credentials in time, since that value would be very out
of date in the situation described in the bug.
To fix that, I changed the YARN code to use the new RPC-based mechanism
for distributing tokens to executors. This allowed the old credential
updater code to be removed, and a lot of code in the renewer to be
simplified.
I also made two currently hardcoded values (the renewal time ratio, and
the retry wait) configurable; while this probably never needs to be set
by anyone in a production environment, it helps with testing; that's also
why they're not documented.
Tested on real cluster with a specially crafted application to test this
functionality: checked proper access to HDFS, Hive and HBase in cluster
mode with token renewal on and AM restarts. Tested things still work in
client mode too.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#20657 from vanzin/SPARK-23361.
A few different things going on:
- Remove unused methods.
- Move JSON methods to the only class that uses them.
- Move test-only methods to TestUtils.
- Make getMaxResultSize() a config constant.
- Reuse functionality from existing libraries (JRE or JavaUtils) where possible.
The change also includes changes to a few tests to call `Utils.createTempFile` correctly,
so that temp dirs are created under the designated top-level temp dir instead of
potentially polluting git index.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#20706 from vanzin/SPARK-23550.
## What changes were proposed in this pull request?
There is a race condition introduced in SPARK-11141 which could cause data loss.
The problem is that ReceivedBlockTracker.insertAllocatedBatch function assumes that all blocks from streamIdToUnallocatedBlockQueues allocated to the batch and clears the queue.
In this PR only the allocated blocks will be removed from the queue which will prevent data loss.
## How was this patch tested?
Additional unit test + manually.
Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>
Closes#20620 from gaborgsomogyi/SPARK-23438.
The code was ignoring the error code from the AddBlock RPC, which
means that a failure to write to the WAL was being ignored by the
receiver, and would lead to the block being acked (in the case of
the Flume receiver) and data potentially lost.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#20161 from vanzin/SPARK-21525.
## What changes were proposed in this pull request?
When using the Kubernetes cluster-manager and spawning a Streaming workload, it is important to reset many spark.kubernetes.* properties that are generated by spark-submit but which would get rewritten when restoring a Checkpoint. This is so, because the spark-submit codepath creates Kubernetes resources, such as a ConfigMap, a Secret and other variables, which have an autogenerated name and the previous one will not resolve anymore.
In short, this change enables checkpoint restoration for streaming workloads, and thus enables Spark Streaming workloads in Kubernetes, which were not possible to restore from a checkpoint before if the workload went down.
## How was this patch tested?
This patch was tested with the twitter-streaming example in AWS, using checkpoints in s3 with the s3a:// protocol, as supported by Hadoop.
This is similar to the YARN related code for resetting a Spark Streaming workload, but for the Kubernetes scheduler. I'm adding the initcontainers properties because even if the discussion is not completely settled on the mailing list, my understanding is that at this moment they are going forward for the moment.
For a previous discussion, see the non-rebased work at: https://github.com/apache-spark-on-k8s/spark/pull/516
Author: Santiago Saavedra <ssaavedra@openshine.com>
Closes#20383 from ssaavedra/fix-k8s-checkpointing.
## What changes were proposed in this pull request?
Increased timeout from 50 ms to 300 ms (50 ms was really too low).
## How was this patch tested?
Multiple rounds of tests.
Author: Tathagata Das <tathagata.das1565@gmail.com>
Closes#20371 from tdas/SPARK-23197.
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.
## How was this patch tested?
N/A
Author: gatorsmile <gatorsmile@gmail.com>
Closes#20222 from gatorsmile/bump24.
Still look at the old one in case any Spark user is setting it
explicitly, though.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#19983 from vanzin/SPARK-22788.
…a-2.12 and JDK9
## What changes were proposed in this pull request?
Some compile error after upgrading to scala-2.12
```javascript
spark_source/core/src/main/scala/org/apache/spark/executor/Executor.scala:455: ambiguous reference to overloaded definition, method limit in class ByteBuffer of type (x$1: Int)java.nio.ByteBuffer
method limit in class Buffer of type ()Int
match expected type ?
val resultSize = serializedDirectResult.limit
error
```
The limit method was moved from ByteBuffer to the superclass Buffer and it can no longer be called without (). The same reason for position method.
```javascript
/home/zly/prj/oss/jdk9_HOS_SOURCE/spark_source/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala:427: ambiguous reference to overloaded definition, [error] both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit [error] and method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit [error] match argument types (java.util.Map[String,String])
[error] props.putAll(outputSerdeProps.toMap.asJava)
[error] ^
```
This is because the key type is Object instead of String which is unsafe.
## How was this patch tested?
running tests
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: kellyzly <kellyzly@126.com>
Closes#19854 from kellyzly/SPARK-22660.
The only remaining use of this class was the SparkStatusTracker, which
was modified to use the new status store. The test code to wait for
executors was moved to TestUtils and now uses the SparkStatusTracker API.
Indirectly, ConsoleProgressBar also uses this data. Because it has
some lower latency requirements, a shortcut to efficiently get the
active stages from the active listener was added to the AppStateStore.
Now that all UI code goes through the status store to get its data,
the FsHistoryProvider can be cleaned up to only replay event logs
when needed - that is, when there is no pre-existing disk store for
the application.
As part of this change I also modified the streaming UI to read the needed
data from the store, which was missed in the previous patch that made
JobProgressListener redundant.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#19750 from vanzin/SPARK-20650.
## What changes were proposed in this pull request?
ScalaTest 3.0 uses an implicit `Signaler`. This PR makes it sure all Spark tests uses `ThreadSignaler` explicitly which has the same default behavior of interrupting a thread on the JVM like ScalaTest 2.2.x. This will reduce potential flakiness.
## How was this patch tested?
This is testsuite-only update. This should passes the Jenkins tests.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#19784 from dongjoon-hyun/use_thread_signaler.
With the new UI store, the API resource classes have a lot less code,
since there's no need for complicated translations between the UI
types and the API types. So the code ended up with a bunch of files
with a single method declared in them.
This change re-structures the API code so that it uses less classes;
mainly, most sub-resources were removed, and the code to deal with
single-attempt and multi-attempt apps was simplified.
The only change was the addition of a method to return a single
attempt's information; that was missing in the old API, so trying
to retrieve "/v1/applications/appId/attemptId" would result in a
404 even if the attempt existed (and URIs under that one would
return valid data).
The streaming API resources also overtook the same treatment, even
though the data is not stored in the new UI store.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#19748 from vanzin/SPARK-20649.
## What changes were proposed in this pull request?
It seems that recovering from a checkpoint can replace the old
driver and executor IP addresses, as the workload can now be taking
place in a different cluster configuration. It follows that the
bindAddress for the master may also have changed. Thus we should not be
keeping the old one, and instead be added to the list of properties to
reset and recreate from the new environment.
## How was this patch tested?
This patch was tested via manual testing on AWS, using the experimental (not yet merged) Kubernetes scheduler, which uses bindAddress to bind to a Kubernetes service (and thus was how I first encountered the bug too), but it is not a code-path related to the scheduler and this may have slipped through when merging SPARK-4563.
Author: Santiago Saavedra <ssaavedra@openshine.com>
Closes#19427 from ssaavedra/fix-checkpointing-master.
## What changes were proposed in this pull request?
Preliminary changes to get ClosureCleaner to work with Scala 2.12. Makes many usages just work, but not all. This does _not_ resolve the JIRA.
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#19675 from srowen/SPARK-14540.0.
## What changes were proposed in this pull request?
SPARK-14930/SPARK-13693 put in a change to set the fs object to null after a failure, however the retry loop does not include initialization. Moved fs initialization inside the retry while loop to aid recoverability.
## How was this patch tested?
Passes all existing unit tests.
Author: Tristan Stevens <tristan@cloudera.com>
Closes#19645 from tmgstevens/SPARK-22429.
## What changes were proposed in this pull request?
Scala test source files like TestHiveSingleton.scala should be in scala source root
## How was this patch tested?
Just move scala file from java directory to scala directory
No new test case in this PR.
```
renamed: mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala -> mllib/src/test/scala/org/apache/spark/ml/util/IdentifiableSuite.scala
renamed: streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala -> streaming/src/test/scala/org/apache/spark/streaming/JavaTestUtils.scala
renamed: streaming/src/test/java/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala -> streaming/src/test/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala
renamed: sql/hive/src/test/java/org/apache/spark/sql/hive/test/TestHiveSingleton.scala sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
```
Author: xubo245 <601450868@qq.com>
Closes#19639 from xubo245/scalaDirectory.
## What changes were proposed in this pull request?
the previous [PR](https://github.com/apache/spark/pull/19469) is deleted by mistake.
the solution is straight forward.
adding "spark.yarn.jars" to propertiesToReload so this property will load from config.
## How was this patch tested?
manual tests
Author: ZouChenjun <zouchenjun@youzan.com>
Closes#19637 from ChenjunZou/checkpoint-yarn-jars.
## What changes were proposed in this pull request?
Scala 2.12's `Future` defines two new methods to implement, `transform` and `transformWith`. These can be implemented naturally in Spark's `FutureAction` extension and subclasses, but, only in terms of the new methods that don't exist in Scala 2.11. To support both at the same time, reflection is used to implement these.
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#19561 from srowen/SPARK-22322.
## What changes were proposed in this pull request?
Enable Scala 2.12 REPL. Fix most remaining issues with 2.12 compilation and warnings, including:
- Selecting Kafka 0.10.1+ for Scala 2.12 and patching over a minor API difference
- Fixing lots of "eta expansion of zero arg method deprecated" warnings
- Resolving the SparkContext.sequenceFile implicits compile problem
- Fixing an odd but valid jetty-server missing dependency in hive-thriftserver
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#19307 from srowen/Scala212.
This change modifies the live listener bus so that all listeners are
added to queues; each queue has its own thread to dispatch events,
making it possible to separate slow listeners from other more
performance-sensitive ones.
The public API has not changed - all listeners added with the existing
"addListener" method, which after this change mostly means all
user-defined listeners, end up in a default queue. Internally, there's
an API allowing listeners to be added to specific queues, and that API
is used to separate the internal Spark listeners into 3 categories:
application status listeners (e.g. UI), executor management (e.g. dynamic
allocation), and the event log.
The queueing logic, while abstracted away in a separate class, is kept
as much as possible hidden away from consumers. Aside from choosing their
queue, there's no code change needed to take advantage of queues.
Test coverage relies on existing tests; a few tests had to be tweaked
because they relied on `LiveListenerBus.postToAll` being synchronous,
and the change makes that method asynchronous. Other tests were simplified
not to use the asynchronous LiveListenerBus.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#19211 from vanzin/SPARK-18838.
Since ScalaTest 3.0.0, `org.scalatest.concurrent.Timeouts` is deprecated.
This PR replaces the deprecated one with `org.scalatest.concurrent.TimeLimits`.
```scala
-import org.scalatest.concurrent.Timeouts._
+import org.scalatest.concurrent.TimeLimits._
```
Pass the existing test suites.
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#19150 from dongjoon-hyun/SPARK-21939.
Change-Id: I1a1b07f1b97e51e2263dfb34b7eaaa099b2ded5e
## What changes were proposed in this pull request?
1.0.0 fixes an issue with import order, explicit type for public methods, line length limitation and comment validation:
```
[error] .../spark/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala:50:16: Are you sure you want to println? If yes, wrap the code block with
[error] // scalastyle:off println
[error] println(...)
[error] // scalastyle:on println
[error] .../spark/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala:49: File line length exceeds 100 characters
[error] .../spark/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala:22:21: Are you sure you want to println? If yes, wrap the code block with
[error] // scalastyle:off println
[error] println(...)
[error] // scalastyle:on println
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:35:6: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:51:6: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:93:15: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:98:15: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:47:2: Insert a space after the start of the comment
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:26:43: JavaDStream should come before JavaDStreamLike.
```
This PR also fixes the workaround added in SPARK-16877 for `org.scalastyle.scalariform.OverrideJavaChecker` feature, added from 0.9.0.
## How was this patch tested?
Manually tested.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#19116 from HyukjinKwon/scalastyle-1.0.0.
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure
## What changes were proposed in this pull request?
This change adds back the infrastructure for a Scala 2.12 build, but does not enable it in the release or Python test scripts.
In order to make that meaningful, it also resolves compile errors that the code hits in 2.12 only, in a way that still works with 2.11.
It also updates dependencies to the earliest minor release of dependencies whose current version does not yet support Scala 2.12. This is in a sense covered by other JIRAs under the main umbrella, but implemented here. The versions below still work with 2.11, and are the _latest_ maintenance release in the _earliest_ viable minor release.
- Scalatest 2.x -> 3.0.3
- Chill 0.8.0 -> 0.8.4
- Clapper 1.0.x -> 1.1.2
- json4s 3.2.x -> 3.4.2
- Jackson 2.6.x -> 2.7.9 (required by json4s)
This change does _not_ fully enable a Scala 2.12 build:
- It will also require dropping support for Kafka before 0.10. Easy enough, just didn't do it yet here
- It will require recreating `SparkILoop` and `Main` for REPL 2.12, which is SPARK-14650. Possible to do here too.
What it does do is make changes that resolve much of the remaining gap without affecting the current 2.11 build.
## How was this patch tested?
Existing tests and build. Manually tested with `./dev/change-scala-version.sh 2.12` to verify it compiles, modulo the exceptions above.
Author: Sean Owen <sowen@cloudera.com>
Closes#18645 from srowen/SPARK-14280.
This version fixes a few issues in the import order checker; it provides
better error messages, and detects more improper ordering (thus the need
to change a lot of files in this patch). The main fix is that it correctly
complains about the order of packages vs. classes.
As part of the above, I moved some "SparkSession" import in ML examples
inside the "$example on$" blocks; that didn't seem consistent across
different source files to start with, and avoids having to add more on/off blocks
around specific imports.
The new scalastyle also seems to have a better header detector, so a few
license headers had to be updated to match the expected indentation.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#18943 from vanzin/SPARK-21731.
## What changes were proposed in this pull request?
```DStreams
class FileInputDStream
[line 162] protected[streaming] override def clearMetadata(time: Time) {
batchTimeToSelectedFiles.synchronized {
val oldFiles = batchTimeToSelectedFiles.filter(_._1 < (time - rememberDuration))
batchTimeToSelectedFiles --= oldFiles.keys
```
The above code does not remove the old generatedRDDs. "super.clearMetadata(time)" was added to the beginning of clearMetadata to remove the old generatedRDDs.
## How was this patch tested?
At the end of clearMetadata, the testing code (print the number of generatedRDDs) was added to check the old RDDS were removed manually.
Author: shaofei007 <1427357147@qq.com>
Author: Fei Shao <1427357147@qq.com>
Closes#18718 from shaofei007/master.
## What changes were proposed in this pull request?
Address scapegoat warnings for:
- BigDecimal double constructor
- Catching NPE
- Finalizer without super
- List.size is O(n)
- Prefer Seq.empty
- Prefer Set.empty
- reverse.map instead of reverseMap
- Type shadowing
- Unnecessary if condition.
- Use .log1p
- Var could be val
In some instances like Seq.empty, I avoided making the change even where valid in test code to keep the scope of the change smaller. Those issues are concerned with performance and it won't matter for tests.
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#18635 from srowen/Scapegoat1.
## What changes were proposed in this pull request?
Reload the `spark.yarn.credentials.file` property when restarting a streaming application from checkpoint.
## How was this patch tested?
Manual tested with 1.6.3 and 2.1.1.
I didn't test this with master because of some compile problems, but I think it will be the same result.
## Notice
This should be merged into maintenance branches too.
jira: [SPARK-21008](https://issues.apache.org/jira/browse/SPARK-21008)
Author: saturday_s <shi.indetail@gmail.com>
Closes#18230 from saturday-shi/SPARK-21008.
## What changes were proposed in this pull request?
This PR proposes to stop `ReceiverTracker` to close `WriteAheadLog` whenever it is and make `WriteAheadLog` and its implementations idempotent.
## How was this patch tested?
Added a test in `WriteAheadLogSuite`. Note that the added test looks passing even if it closes twice (namely even without the changes in `FileBasedWriteAheadLog` and `BatchedWriteAheadLog`. It looks both are already idempotent but this is a rather sanity check.
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#18224 from HyukjinKwon/streaming-closing.
## What changes were proposed in this pull request?
This patch adds Coda Hale metrics for instrumenting the `LiveListenerBus` in order to track the number of events received, dropped, and processed. In addition, it adds per-SparkListener-subclass timers to track message processing time. This is useful for identifying when slow third-party SparkListeners cause performance bottlenecks.
See the new `LiveListenerBusMetrics` for a complete description of the new metrics.
## How was this patch tested?
New tests in SparkListenerSuite, including a test to ensure proper counting of dropped listener events.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#18083 from JoshRosen/listener-bus-metrics.
## What changes were proposed in this pull request?
Include documentation of the fact that the updateFunc is sometimes called with no new values. This is documented in the main documentation here: https://spark.apache.org/docs/latest/streaming-programming-guide.html#updatestatebykey-operation however from the docs included with the code it is not clear that this is the case.
## How was this patch tested?
PR only changes comments. Confirmed code still builds.
Author: Wil Selwood <wil.selwood@sa.catapult.org.uk>
Closes#18088 from wselwood/note-edge-case-in-docs.
## What changes were proposed in this pull request?
Add `ThreadUtils.awaitReady` similar to `ThreadUtils.awaitResult` and disallow `Await.ready`.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17763 from zsxwing/awaitready.
## What changes were proposed in this pull request?
Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up.
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#17949 from srowen/SPARK-20554.
## What changes were proposed in this pull request?
Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these functions at any point that getParameter is called against a HttpServletRequest.
## How was this patch tested?
Unit tests, IBM Security AppScan Standard no longer showing vulnerabilities, manual verification of WebUI pages.
Author: NICHOLAS T. MARION <nmarion@us.ibm.com>
Closes#17686 from n-marion/xss-fix.
## What changes were proposed in this pull request?
This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.
There are several problems with it:
- It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".
- > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.
(see joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))
To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.
There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013
Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.
## How was this patch tested?
Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.
This was tested via manually adding `time.time()` as below:
```diff
profiles_and_goals = build_profiles + sbt_goals
print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
" ".join(profiles_and_goals))
+ import time
+ st = time.time()
exec_sbt(profiles_and_goals)
+ print("Elapsed :[%s]" % str(time.time() - st))
```
produces
```
...
========================================================================
Building Unidoc API Documentation
========================================================================
...
[info] Main Java API documentation successful.
...
Elapsed :[94.8746569157]
...
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17477 from HyukjinKwon/SPARK-18692.
## What changes were proposed in this pull request?
Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").
The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.
## How was this patch tested?
Existing tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#17527 from srowen/SPARK-20156.
…adoc
## What changes were proposed in this pull request?
Use recommended values for row boundaries in Window's scaladoc, i.e. `Window.unboundedPreceding`, `Window.unboundedFollowing`, and `Window.currentRow` (that were introduced in 2.1.0).
## How was this patch tested?
Local build
Author: Jacek Laskowski <jacek@japila.pl>
Closes#17417 from jaceklaskowski/window-expression-scaladoc.
This change modifies the way block data is encrypted to make the more
common cases faster, while penalizing an edge case. As a side effect
of the change, all data that goes through the block manager is now
encrypted only when needed, including the previous path (broadcast
variables) where that did not happen.
The way the change works is by not encrypting data that is stored in
memory; so if a serialized block is in memory, it will only be encrypted
once it is evicted to disk.
The penalty comes when transferring that encrypted data from disk. If the
data ends up in memory again, it is as efficient as before; but if the
evicted block needs to be transferred directly to a remote executor, then
there's now a performance penalty, since the code now uses a custom
FileRegion implementation to decrypt the data before transferring.
This also means that block data transferred between executors now is
not encrypted (and thus relies on the network library encryption support
for secrecy). Shuffle blocks are still transferred in encrypted form,
since they're handled in a slightly different way by the code. This also
keeps compatibility with existing external shuffle services, which transfer
encrypted shuffle blocks, and avoids having to make the external service
aware of encryption at all.
The serialization and deserialization APIs in the SerializerManager now
do not do encryption automatically; callers need to explicitly wrap their
streams with an appropriate crypto stream before using those.
As a result of these changes, some of the workarounds added in SPARK-19520
are removed here.
Testing: a new trait ("EncryptionFunSuite") was added that provides an easy
way to run a test twice, with encryption on and off; broadcast, block manager
and caching tests were modified to use this new trait so that the existing
tests exercise both encrypted and non-encrypted paths. I also ran some
applications with encryption turned on to verify that they still work,
including streaming tests that failed without the fix for SPARK-19520.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#17295 from vanzin/SPARK-19556.
This commit adds a killTaskAttempt method to SparkContext, to allow users to
kill tasks so that they can be re-scheduled elsewhere.
This also refactors the task kill path to allow specifying a reason for the task kill. The reason is propagated opaquely through events, and will show up in the UI automatically as `(N killed: $reason)` and `TaskKilled: $reason`. Without this change, there is no way to provide the user feedback through the UI.
Currently used reasons are "stage cancelled", "another attempt succeeded", and "killed via SparkContext.killTask". The user can also specify a custom reason through `SparkContext.killTask`.
cc rxin
In the stage overview UI the reasons are summarized:
![1](https://cloud.githubusercontent.com/assets/14922/23929209/a83b2862-08e1-11e7-8b3e-ae1967bbe2e5.png)
Within the stage UI you can see individual task kill reasons:
![2](https://cloud.githubusercontent.com/assets/14922/23929200/9a798692-08e1-11e7-8697-72b27ad8a287.png)
Existing tests, tried killing some stages in the UI and verified the messages are as expected.
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekl@google.com>
Closes#17166 from ericl/kill-reason.
## What changes were proposed in this pull request?
- SS python example: `TypeError: 'xxx' object is not callable`
- some other doc issue.
## How was this patch tested?
Jenkins.
Author: uncleGen <hustyugm@gmail.com>
Closes#17257 from uncleGen/docs-ss-python.
## What changes were proposed in this pull request?
https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73800/testReport/
```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException: The code
passed to eventually never returned normally. Attempted 617 times over 10.003740484 seconds.
Last failure message: 8 did not equal 2.
at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:336)
at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
at org.apache.spark.streaming.DStreamCheckpointTester$class.generateOutput(CheckpointSuite
.scala:172)
at org.apache.spark.streaming.CheckpointSuite.generateOutput(CheckpointSuite.scala:211)
```
the check condition is:
```
val checkpointFilesOfLatestTime = Checkpoint.getCheckpointFiles(checkpointDir).filter {
_.toString.contains(clock.getTimeMillis.toString)
}
// Checkpoint files are written twice for every batch interval. So assert that both
// are written to make sure that both of them have been written.
assert(checkpointFilesOfLatestTime.size === 2)
```
the path string may contain the `clock.getTimeMillis.toString`, like `3500` :
```
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500.bk
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500
▲▲▲▲
```
so we should only check the filename, but not the whole path.
## How was this patch tested?
Jenkins.
Author: uncleGen <hustyugm@gmail.com>
Closes#17167 from uncleGen/flaky-CheckpointSuite.
## What changes were proposed in this pull request?
This PR proposes to fix the lint-breaks as below:
```
[ERROR] src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.network.buffer.ManagedBuffer.
[ERROR] src/main/java/org/apache/spark/unsafe/types/UTF8String.java:[156,10] (modifier) ModifierOrder: 'Nonnull' annotation modifier does not precede non-annotation modifiers.
[ERROR] src/main/java/org/apache/spark/SparkFirehoseListener.java:[122] (sizes) LineLength: Line is longer than 100 characters (found 105).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[164,78] (coding) OneStatementPerLine: Only one statement per line allowed.
[ERROR] src/test/java/test/org/apache/spark/JavaAPISuite.java:[1157] (sizes) LineLength: Line is longer than 100 characters (found 121).
[ERROR] src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java:[149] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/test/java/test/org/apache/spark/streaming/Java8APISuite.java:[146] (sizes) LineLength: Line is longer than 100 characters (found 122).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[32,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.Time.
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[611] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[1317] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java:[91] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[113] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[164] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[212] (sizes) LineLength: Line is longer than 100 characters (found 114).
[ERROR] src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java:[36] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java:[26,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[20,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[94] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.api.java.UDF1.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[72] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java:[121] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaRDD.
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaSparkContext.
```
## How was this patch tested?
Manually via
```bash
./dev/lint-java
```
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#17072 from HyukjinKwon/java-lint.