## 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>
## 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>
## 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>
## 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?
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>
## 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>
## 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>
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>
## 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>
## 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>
## 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>
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>
## 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>
## 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>
## 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>
## 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>
## 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>
…. 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>
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>
## 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>
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>
## 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>
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>
## 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?
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>
## 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>
## 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>
## 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>
## 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>
## 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>
## 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>
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>
…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>
## 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>
## 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.
## 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.
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.
## 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.
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.
## 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.
## 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.
"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.
## 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.
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.
## 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.
## 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.
## 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.
## 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.
## 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-L65https://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.
## 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.
## 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.
## 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.
## 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.
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.
## 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.
## 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.
There are two main changes to speed up rendering of the tasks list
when rendering the stage page.
The first one makes the code only load the tasks being shown in the
current page of the tasks table, and information related to only
those tasks. One side-effect of this change is that the graph that
shows task-related events now only shows events for the tasks in
the current page, instead of the previously hardcoded limit of "events
for the first 1000 tasks". That ends up helping with readability,
though.
To make sorting efficient when using a disk store, the task wrapper
was extended to include many new indices, one for each of the sortable
columns in the UI, and metrics for which quantiles are calculated.
The second changes the way metric quantiles are calculated for stages.
Instead of using the "Distribution" class to process data for all task
metrics, which requires scanning all tasks of a stage, the code now
uses the KVStore "skip()" functionality to only read tasks that contain
interesting information for the quantiles that are desired.
This is still not cheap; because there are many metrics that the UI
and API track, the code needs to scan the index for each metric to
gather the information. Savings come mainly from skipping deserialization
when using the disk store, but the in-memory code also seems to be
faster than before (most probably because of other changes in this
patch).
To make subsequent calls faster, some quantiles are cached in the
status store. This makes UIs much faster after the first time a stage
has been loaded.
With the above changes, a lot of code in the UI layer could be simplified.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#20013 from vanzin/SPARK-20657.
## What changes were proposed in this pull request?
This patch modifies Spark's `MemoryAllocator` implementations so that `free(MemoryBlock)` mutates the passed block to clear pointers (in the off-heap case) or null out references to backing `long[]` arrays (in the on-heap case). The goal of this change is to add an extra layer of defense against use-after-free bugs because currently it's hard to detect corruption caused by blind writes to freed memory blocks.
## How was this patch tested?
New unit tests in `PlatformSuite`, including new tests for existing functionality because we did not have sufficient mutation coverage of the on-heap memory allocator's pooling logic.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#20191 from JoshRosen/SPARK-22997-add-defenses-against-use-after-free-bugs-in-memory-allocator.
## What changes were proposed in this pull request?
This PR is the second attempt of #18684 , NIO's Files API doesn't override `skip` method for `InputStream`, so it will bring in performance issue (mentioned in #20119). But using `FileInputStream`/`FileOutputStream` will also bring in memory issue (https://dzone.com/articles/fileinputstream-fileoutputstream-considered-harmful), which is severe for long running external shuffle service. So here in this proposal, only fixing the external shuffle service related code.
## How was this patch tested?
Existing tests.
Author: jerryshao <sshao@hortonworks.com>
Closes#20144 from jerryshao/SPARK-21475-v2.
## What changes were proposed in this pull request?
This reverts commit 5fd0294ff8 because of a huge performance regression.
I manually fixed a minor conflict in `OneForOneBlockFetcher.java`.
`Files.newInputStream` returns `sun.nio.ch.ChannelInputStream`. `ChannelInputStream` doesn't override `InputStream.skip`, so it's using the default `InputStream.skip` which just consumes and discards data. This causes a huge performance regression when reading shuffle files.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <zsxwing@gmail.com>
Closes#20119 from zsxwing/revert-SPARK-21475.
## What changes were proposed in this pull request?
This pr modified `concat` to concat binary inputs into a single binary output.
`concat` in the current master always output data as a string. But, in some databases (e.g., PostgreSQL), if all inputs are binary, `concat` also outputs binary.
## How was this patch tested?
Added tests in `SQLQueryTestSuite` and `TypeCoercionSuite`.
Author: Takeshi Yamamuro <yamamuro@apache.org>
Closes#19977 from maropu/SPARK-22771.
## What changes were proposed in this pull request?
Upgrade Spark to Arrow 0.8.0 for Java and Python. Also includes an upgrade of Netty to 4.1.17 to resolve dependency requirements.
The highlights that pertain to Spark for the update from Arrow versoin 0.4.1 to 0.8.0 include:
* Java refactoring for more simple API
* Java reduced heap usage and streamlined hot code paths
* Type support for DecimalType, ArrayType
* Improved type casting support in Python
* Simplified type checking in Python
## How was this patch tested?
Existing tests
Author: Bryan Cutler <cutlerb@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Closes#19884 from BryanCutler/arrow-upgrade-080-SPARK-22324.
## What changes were proposed in this pull request?
`ExternalShuffleClient.close()` should check `clientFactory` null. otherwise it will throw NPE sometimes:
```
17/11/06 20:08:05 ERROR Utils: Uncaught exception in thread main
java.lang.NullPointerException
at org.apache.spark.network.shuffle.ExternalShuffleClient.close(ExternalShuffleClient.java:152)
at org.apache.spark.storage.BlockManager.stop(BlockManager.scala:1407)
at org.apache.spark.SparkEnv.stop(SparkEnv.scala:89)
at org.apache.spark.SparkContext$$anonfun$stop$11.apply$mcV$sp(SparkContext.scala:1849)
```
## How was this patch tested?
manual tests
Author: Yuming Wang <wgyumg@gmail.com>
Closes#19670 from wangyum/SPARK-22454.
The initial listener code is based on the existing JobProgressListener (and others),
and tries to mimic their behavior as much as possible. The change also includes
some minor code movement so that some types and methods from the initial history
server code code can be reused.
The code introduces a few mutable versions of public API types, used internally,
to make it easier to update information without ugly copy methods, and also to
make certain updates cheaper.
Note the code here is not 100% correct. This is meant as a building ground for
the UI integration in the next milestones. As different parts of the UI are
ported, fixes will be made to the different parts of this code to account
for the needed behavior.
I also added annotations to API types so that Jackson is able to correctly
deserialize options, sequences and maps that store primitive types.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#19383 from vanzin/SPARK-20643.
## What changes were proposed in this pull request?
In on-heap mode, when allocating memory from pool,we should fill memory with `MEMORY_DEBUG_FILL_CLEAN_VALUE`
## How was this patch tested?
added unit tests
Author: liuxian <liu.xian3@zte.com.cn>
Closes#19572 from 10110346/MEMORY_DEBUG.
## What changes were proposed in this pull request?
In the current BlockManager's `getRemoteBytes`, it will call `BlockTransferService#fetchBlockSync` to get remote block. In the `fetchBlockSync`, Spark will allocate a temporary `ByteBuffer` to store the whole fetched block. This will potentially lead to OOM if block size is too big or several blocks are fetched simultaneously in this executor.
So here leveraging the idea of shuffle fetch, to spill the large block to local disk before consumed by upstream code. The behavior is controlled by newly added configuration, if block size is smaller than the threshold, then this block will be persisted in memory; otherwise it will first spill to disk, and then read from disk file.
To achieve this feature, what I did is:
1. Rename `TempShuffleFileManager` to `TempFileManager`, since now it is not only used by shuffle.
2. Add a new `TempFileManager` to manage the files of fetched remote blocks, the files are tracked by weak reference, will be deleted when no use at all.
## How was this patch tested?
This was tested by adding UT, also manual verification in local test to perform GC to clean the files.
Author: jerryshao <sshao@hortonworks.com>
Closes#19476 from jerryshao/SPARK-22062.
## What changes were proposed in this pull request?
We should not break the assumption that the length of the allocated byte array is word rounded:
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java#L170
So we want to use `Integer.MAX_VALUE - 15` instead of `Integer.MAX_VALUE - 8` as the upper bound of an allocated byte array.
cc: srowen gatorsmile
## How was this patch tested?
Since the Spark unit test JVM has less than 1GB heap, here we run the test code as a submit job, so it can run on a JVM has 4GB memory.
Please review http://spark.apache.org/contributing.html before opening a pull request.
Author: Feng Liu <fengliu@databricks.com>
Closes#19460 from liufengdb/fix_array_max.
This patch fixes application re-attempts when running spark on yarn using the external shuffle service with security on. Currently executors will fail to launch on any application re-attempt when launched on a nodemanager that had an executor from the first attempt. The reason for this is because we aren't updating the secret key after the first application attempt. The fix here is to just remove the containskey check to see if it already exists. In this way, we always add it and make sure its the most recent secret. Similarly remove the check for containsKey on the remove since its just adding extra check that isn't really needed.
Note this worked before spark 2.2 because the check used to be contains (which was looking for the value) rather then containsKey, so that never matched and it was just always adding the new secret.
Patch was tested on a 10 node cluster as well as added the unit test.
The test ran was a wordcount where the output directory already existed. With the bug present the application attempt failed with max number of executor Failures which were all saslExceptions. With the fix present the application re-attempts fail with directory already exists or when you remove the directory between attempts the re-attemps succeed.
Author: Thomas Graves <tgraves@unharmedunarmed.corp.ne1.yahoo.com>
Closes#19450 from tgravescs/SPARK-22218.
## What changes were proposed in this pull request?
This PR allows us to scan a string including only white space (e.g. `" "`) once while the current implementation scans twice (right to left, and then left to right).
## How was this patch tested?
Existing test suites
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#19355 from kiszk/SPARK-22130.
The application listing is still generated from event logs, but is now stored
in a KVStore instance. By default an in-memory store is used, but a new config
allows setting a local disk path to store the data, in which case a LevelDB
store will be created.
The provider stores things internally using the public REST API types; I believe
this is better going forward since it will make it easier to get rid of the
internal history server API which is mostly redundant at this point.
I also added a finalizer to LevelDBIterator, to make sure that resources are
eventually released. This helps when code iterates but does not exhaust the
iterator, thus not triggering the auto-close code.
HistoryServerSuite was modified to not re-start the history server unnecessarily;
this makes the json validation tests run more quickly.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#18887 from vanzin/SPARK-20642.
## What changes were proposed in this pull request?
Try to avoid allocating an array bigger than Integer.MAX_VALUE - 8, which is the actual max size on some JVMs, in several places
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#19266 from srowen/SPARK-22033.
## What changes were proposed in this pull request?
This is a followup work of SPARK-9104 to expose the Netty memory usage to MetricsSystem. Current the shuffle Netty memory usage of `NettyBlockTransferService` will be exposed, if using external shuffle, then the Netty memory usage of `ExternalShuffleClient` and `ExternalShuffleService` will be exposed instead. Currently I don't expose Netty memory usage of `YarnShuffleService`, because `YarnShuffleService` doesn't have `MetricsSystem` itself, and is better to connect to Hadoop's MetricsSystem.
## How was this patch tested?
Manually verified in local cluster.
Author: jerryshao <sshao@hortonworks.com>
Closes#19160 from jerryshao/SPARK-21934.
## What changes were proposed in this pull request?
Update plugins, including scala-maven-plugin, to latest versions. Update checkstyle to 8.2. Remove bogus checkstyle config and enable it. Fix existing and new Java checkstyle errors.
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#19282 from srowen/SPARK-22066.
#### What changes were proposed in this pull request?
This PR enhances the TRIM function support in Spark SQL by allowing the specification
of trim characters set. Below is the SQL syntax :
``` SQL
<trim function> ::= TRIM <left paren> <trim operands> <right paren>
<trim operands> ::= [ [ <trim specification> ] [ <trim character set> ] FROM ] <trim source>
<trim source> ::= <character value expression>
<trim specification> ::=
LEADING
| TRAILING
| BOTH
<trim character set> ::= <characters value expression>
```
or
``` SQL
LTRIM (source-exp [, trim-exp])
RTRIM (source-exp [, trim-exp])
```
Here are the documentation link of support of this feature by other mainstream databases.
- **Oracle:** [TRIM function](http://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2126.htm#OLADM704)
- **DB2:** [TRIM scalar function](https://www.ibm.com/support/knowledgecenter/en/SSMKHH_10.0.0/com.ibm.etools.mft.doc/ak05270_.htm)
- **MySQL:** [Trim function](http://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim)
- **Oracle:** [ltrim](https://docs.oracle.com/cd/B28359_01/olap.111/b28126/dml_functions_2018.htm#OLADM594)
- **DB2:** [ltrim](https://www.ibm.com/support/knowledgecenter/en/SSEPEK_11.0.0/sqlref/src/tpc/db2z_bif_ltrim.html)
This PR is to implement the above enhancement. In the implementation, the design principle is to keep the changes to the minimum. Also, the exiting trim functions (which handles a special case, i.e., trimming space characters) are kept unchanged for performane reasons.
#### How was this patch tested?
The unit test cases are added in the following files:
- UTF8StringSuite.java
- StringExpressionsSuite.scala
- sql/SQLQuerySuite.scala
- StringFunctionsSuite.scala
Author: Kevin Yu <qyu@us.ibm.com>
Closes#12646 from kevinyu98/spark-14878.
## What changes were proposed in this pull request?
* Using 64 bit unsigned long comparison instead of unsigned int comparison in `org.apache.spark.unsafe.types.UTF8String#compareTo` for better performance.
* Making `IS_LITTLE_ENDIAN` a constant for correctness reasons (shouldn't use a non-constant in `compareTo` implementations and it def. is a constant per JVM)
## How was this patch tested?
Build passes and the functionality is widely covered by existing tests as far as I can see.
Author: Armin <me@obrown.io>
Closes#19180 from original-brownbear/SPARK-21967.
## What changes were proposed in this pull request?
1. Removing all redundant throws declarations from Java codebase.
2. Removing dead code made visible by this from `ShuffleExternalSorter#closeAndGetSpills`
## How was this patch tested?
Build still passes.
Author: Armin <me@obrown.io>
Closes#19182 from original-brownbear/SPARK-21970.
## What changes were proposed in this pull request?
This PR exposes Netty memory usage for Spark's `TransportClientFactory` and `TransportServer`, including the details of each direct arena and heap arena metrics, as well as aggregated metrics. The purpose of adding the Netty metrics is to better know the memory usage of Netty in Spark shuffle, rpc and others network communications, and guide us to better configure the memory size of executors.
This PR doesn't expose these metrics to any sink, to leverage this feature, still requires to connect to either MetricsSystem or collect them back to Driver to display.
## How was this patch tested?
Add Unit test to verify it, also manually verified in real cluster.
Author: jerryshao <sshao@hortonworks.com>
Closes#18935 from jerryshao/SPARK-9104.
In the current code, if NM recovery is not enabled then `YarnShuffleService` will write shuffle metadata to NM local dir-1, if this local dir-1 is on bad disk, then `YarnShuffleService` will be failed to start. So to solve this issue, in Spark side if NM recovery is not enabled, then Spark will not persist data into leveldb, in that case yarn shuffle service can still be served but lose the ability for recovery, (it is fine because the failure of NM will kill the containers as well as applications).
Tested in the local cluster with NM recovery off and on to see if folder is created or not. MiniCluster UT isn't added because in MiniCluster NM will always set port to 0, but NM recovery requires non-ephemeral port.
Author: jerryshao <sshao@hortonworks.com>
Closes#19032 from jerryshao/SPARK-17321.
Change-Id: I8f2fe73d175e2ad2c4e380caede3873e0192d027
## What changes were proposed in this pull request?
Free off -heap memory .
I have checked all the unit tests.
## How was this patch tested?
N/A
Author: liuxian <liu.xian3@zte.com.cn>
Closes#19075 from 10110346/memleak.
## What changes were proposed in this pull request?
Fix build warnings and Java lint errors. This just helps a bit in evaluating (new) warnings in another PR I have open.
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#19051 from srowen/JavaWarnings.
## What changes were proposed in this pull request?
TCP parameters like SO_RCVBUF and SO_SNDBUF can be set in SparkConf, and `org.apache.spark.network.server.TransportServe`r can use those parameters to build server by leveraging netty. But for TransportClientFactory, there is no such way to set those parameters from SparkConf. This could be inconsistent in server and client side when people set parameters in SparkConf. So this PR make RPC client to be enable to use those TCP parameters as well.
## How was this patch tested?
Existing tests.
Author: xu.zhang <xu.zhang@hulu.com>
Closes#18964 from neoremind/add_client_param.
Right now the spark shuffle service has a cache for index files. It is based on a # of files cached (spark.shuffle.service.index.cache.entries). This can cause issues if people have a lot of reducers because the size of each entry can fluctuate based on the # of reducers.
We saw an issues with a job that had 170000 reducers and it caused NM with spark shuffle service to use 700-800MB or memory in NM by itself.
We should change this cache to be memory based and only allow a certain memory size used. When I say memory based I mean the cache should have a limit of say 100MB.
https://issues.apache.org/jira/browse/SPARK-21501
Manual Testing with 170000 reducers has been performed with cache loaded up to max 100MB default limit, with each shuffle index file of size 1.3MB. Eviction takes place as soon as the total cache size reaches the 100MB limit and the objects will be ready for garbage collection there by avoiding NM to crash. No notable difference in runtime has been observed.
Author: Sanket Chintapalli <schintap@yahoo-inc.com>
Closes#18940 from redsanket/SPARK-21501.
This change adds an in-memory implementation of KVStore that can be
used by the live UI.
The implementation is not fully optimized, neither for speed nor
space, but should be fast enough for using in the listener bus.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#18395 from vanzin/SPARK-20655.
## What changes were proposed in this pull request?
**For moudle below:**
common/network-common
streaming
sql/core
sql/catalyst
**tests.jar will install or deploy twice.Like:**
`[DEBUG] Installing org.apache.spark:spark-streaming_2.11/maven-metadata.xml to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/maven-metadata-local.xml
[INFO] Installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar
[DEBUG] Skipped re-installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar, seems unchanged`
**The reason is below:**
`[DEBUG] (f) artifact = org.apache.spark:spark-streaming_2.11🫙2.1.0-mdh2.1.0.1-SNAPSHOT
[DEBUG] (f) attachedArtifacts = [org.apache.spark:spark-streaming_2.11:test-jar:tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11🫙tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark
-streaming_2.11:java-source:sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:java-source:test-sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:javadoc:javadoc:2.1.0
-mdh2.1.0.1-SNAPSHOT]`
when executing 'mvn deploy' to nexus during release.I will fail since release nexus can not be overrided.
## How was this patch tested?
Execute 'mvn clean install -Pyarn -Phadoop-2.6 -Phadoop-provided -DskipTests'
Author: zhoukang <zhoukang199191@gmail.com>
Closes#18745 from caneGuy/zhoukang/fix-installtwice.
`scala-maven-plugin` in `incremental` mode compiles `Scala` and `Java` classes. There is no need to execute `maven-compiler-plugin` goals to compile (in fact recompile) `Java`.
This change reduces compilation time (over 10% on my machine).
Author: Grzegorz Slowikowski <gslowikowski@gmail.com>
Closes#18750 from gslowikowski/remove-redundant-compilation-from-maven.
## What changes were proposed in this pull request?
Java's `FileInputStream` and `FileOutputStream` overrides finalize(), even this file input/output stream is closed correctly and promptly, it will still leave some memory footprints which will only get cleaned in Full GC. This will introduce two side effects:
1. Lots of memory footprints regarding to Finalizer will be kept in memory and this will increase the memory overhead. In our use case of external shuffle service, a busy shuffle service will have bunch of this object and potentially lead to OOM.
2. The Finalizer will only be called in Full GC, and this will increase the overhead of Full GC and lead to long GC pause.
https://bugs.openjdk.java.net/browse/JDK-8080225https://www.cloudbees.com/blog/fileinputstream-fileoutputstream-considered-harmful
So to fix this potential issue, here propose to use NIO's Files#newInput/OutputStream instead in some critical paths like shuffle.
Left unchanged FileInputStream in core which I think is not so critical:
```
./core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:467: val file = new DataInputStream(new FileInputStream(filename))
./core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:942: val in = new FileInputStream(new File(path))
./core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala:76: val fileIn = new FileInputStream(file)
./core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala:248: val fis = new FileInputStream(file)
./core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala:910: input = new FileInputStream(new File(t))
./core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala:20:import java.io.{FileInputStream, InputStream}
./core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala:132: case Some(f) => new FileInputStream(f)
./core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala:20:import java.io.{FileInputStream, InputStream}
./core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala:77: val fis = new FileInputStream(f)
./core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala:27:import org.apache.spark.io.NioBufferedFileInputStream
./core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala:94: new DataInputStream(new NioBufferedFileInputStream(index))
./core/src/main/scala/org/apache/spark/storage/DiskStore.scala:111: val channel = new FileInputStream(file).getChannel()
./core/src/main/scala/org/apache/spark/storage/DiskStore.scala:219: val channel = new FileInputStream(file).getChannel()
./core/src/main/scala/org/apache/spark/TestUtils.scala:20:import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream}
./core/src/main/scala/org/apache/spark/TestUtils.scala:106: val in = new FileInputStream(file)
./core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala:89: inputStream = new FileInputStream(activeFile)
./core/src/main/scala/org/apache/spark/util/Utils.scala:329: if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream]
./core/src/main/scala/org/apache/spark/util/Utils.scala:332: val inChannel = in.asInstanceOf[FileInputStream].getChannel()
./core/src/main/scala/org/apache/spark/util/Utils.scala:1533: gzInputStream = new GZIPInputStream(new FileInputStream(file))
./core/src/main/scala/org/apache/spark/util/Utils.scala:1560: new GZIPInputStream(new FileInputStream(file))
./core/src/main/scala/org/apache/spark/util/Utils.scala:1562: new FileInputStream(file)
./core/src/main/scala/org/apache/spark/util/Utils.scala:2090: val inReader = new InputStreamReader(new FileInputStream(file), StandardCharsets.UTF_8)
```
Left unchanged FileOutputStream in core:
```
./core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:957: val out = new FileOutputStream(file)
./core/src/main/scala/org/apache/spark/api/r/RBackend.scala:20:import java.io.{DataOutputStream, File, FileOutputStream, IOException}
./core/src/main/scala/org/apache/spark/api/r/RBackend.scala:131: val dos = new DataOutputStream(new FileOutputStream(f))
./core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala:62: val fileOut = new FileOutputStream(file)
./core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala:160: val outStream = new FileOutputStream(outPath)
./core/src/main/scala/org/apache/spark/deploy/RPackageUtils.scala:239: val zipOutputStream = new ZipOutputStream(new FileOutputStream(zipFile, false))
./core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala:949: val out = new FileOutputStream(tempFile)
./core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala:20:import java.io.{File, FileOutputStream, InputStream, IOException}
./core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala:106: val out = new FileOutputStream(file, true)
./core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala:109: * Therefore, for local files, use FileOutputStream instead. */
./core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala:112: new FileOutputStream(uri.getPath)
./core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala:20:import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStream}
./core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala:71: private var fos: FileOutputStream = null
./core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala:102: fos = new FileOutputStream(file, true)
./core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala:213: var truncateStream: FileOutputStream = null
./core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala:215: truncateStream = new FileOutputStream(file, true)
./core/src/main/scala/org/apache/spark/storage/DiskStore.scala:153: val out = new FileOutputStream(file).getChannel()
./core/src/main/scala/org/apache/spark/TestUtils.scala:20:import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream}
./core/src/main/scala/org/apache/spark/TestUtils.scala:81: val jarStream = new JarOutputStream(new FileOutputStream(jarFile))
./core/src/main/scala/org/apache/spark/TestUtils.scala:96: val jarFileStream = new FileOutputStream(jarFile)
./core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala:20:import java.io.{File, FileOutputStream, InputStream, IOException}
./core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala:31: volatile private var outputStream: FileOutputStream = null
./core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala:97: outputStream = new FileOutputStream(file, true)
./core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala:90: gzOutputStream = new GZIPOutputStream(new FileOutputStream(gzFile))
./core/src/main/scala/org/apache/spark/util/Utils.scala:329: if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream]
./core/src/main/scala/org/apache/spark/util/Utils.scala:333: val outChannel = out.asInstanceOf[FileOutputStream].getChannel()
./core/src/main/scala/org/apache/spark/util/Utils.scala:527: val out = new FileOutputStream(tempFile)
```
Here in `DiskBlockObjectWriter`, it uses `FileDescriptor` so it is not easy to change to NIO Files API.
For the `FileInputStream` and `FileOutputStream` in common/shuffle* I changed them all.
## How was this patch tested?
Existing tests and manual verification.
Author: jerryshao <sshao@hortonworks.com>
Closes#18684 from jerryshao/SPARK-21475.
## What changes were proposed in this pull request?
Fixes current failures in dev/lint-java
## How was this patch tested?
Existing linter, tests.
Author: Sean Owen <sowen@cloudera.com>
Closes#18757 from srowen/LintJava.
## What changes were proposed in this pull request?
Update the description of `spark.shuffle.maxChunksBeingTransferred` to include that the new coming connections will be closed when the max is hit and client should have retry mechanism.
Author: jinxing <jinxing6042@126.com>
Closes#18735 from jinxing64/SPARK-21530.
There was some code based on the old SASL handler in the new auth client that
was incorrectly using the SASL user as the user to authenticate against the
external shuffle service. This caused the external service to not be able to
find the correct secret to authenticate the connection, failing the connection.
In the course of debugging, I found that some log messages from the YARN shuffle
service were a little noisy, so I silenced some of them, and also added a couple
of new ones that helped find this issue. On top of that, I found that a check
in the code that records app secrets was wrong, causing more log spam and also
using an O(n) operation instead of an O(1) call.
Also added a new integration suite for the YARN shuffle service with auth on,
and verified it failed before, and passes now.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#18706 from vanzin/SPARK-21494.
## What changes were proposed in this pull request?
A shuffle service can serves blocks from multiple apps/tasks. Thus the shuffle service can suffers high memory usage when lots of shuffle-reads happen at the same time. In my cluster, OOM always happens on shuffle service. Analyzing heap dump, memory cost by Netty(ChannelOutboundBufferEntry) can be up to 2~3G. It might make sense to reject "open blocks" request when memory usage is high on shuffle service.
93dd0c518d and 85c6ce6193 tried to alleviate the memory pressure on shuffle service but cannot solve the root cause. This pr proposes to control currency of shuffle read.
## How was this patch tested?
Added unit test.
Author: jinxing <jinxing6042@126.com>
Closes#18388 from jinxing64/SPARK-21175.
## What changes were proposed in this pull request?
Making those two classes will avoid Serialization issues like below:
```
Caused by: java.io.NotSerializableException: org.apache.spark.unsafe.types.UTF8String$IntWrapper
Serialization stack:
- object not serializable (class: org.apache.spark.unsafe.types.UTF8String$IntWrapper, value: org.apache.spark.unsafe.types.UTF8String$IntWrapper326450e)
- field (class: org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToInt$1, name: result$2, type: class org.apache.spark.unsafe.types.UTF8String$IntWrapper)
- object (class org.apache.spark.sql.catalyst.expressions.Cast$$anonfun$castToInt$1, <function1>)
```
## How was this patch tested?
- [x] Manual testing
- [ ] Unit test
Author: Burak Yavuz <brkyvz@gmail.com>
Closes#18660 from brkyvz/serializableutf8.
## What changes were proposed in this pull request?
This PR fixes a wrong comparison for `BinaryType`. This PR enables unsigned comparison and unsigned prefix generation for an array for `BinaryType`. Previous implementations uses signed operations.
## How was this patch tested?
Added a test suite in `OrderingSuite`.
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#18571 from kiszk/SPARK-21344.
## What changes were proposed in this pull request?
Remove all usages of Scala Tuple2 from common/network-* projects. Otherwise, Yarn users cannot use `spark.reducer.maxReqSizeShuffleToMem`.
## How was this patch tested?
Jenkins.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18593 from zsxwing/SPARK-21369.
## What changes were proposed in this pull request?
When `RetryingBlockFetcher` retries fetching blocks. There could be two `DownloadCallback`s download the same content to the same target file. It could cause `ShuffleBlockFetcherIterator` reading a partial result.
This pr proposes to create and delete the tmp files in `OneForOneBlockFetcher`
Author: jinxing <jinxing6042@126.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Closes#18565 from jinxing64/SPARK-21342.
## What changes were proposed in this pull request?
For performance reasons, `UnsafeRow.getString`, `getStruct`, etc. return a "pointer" that points to a memory region of this unsafe row. This makes the unsafe projection a little dangerous, because all of its output rows share one instance.
When we implement SQL operators, we should be careful to not cache the input rows because they may be produced by unsafe projection from child operator and thus its content may change overtime.
However, when we updating values of InternalRow(e.g. in mutable projection and safe projection), we only copy UTF8String, we should also copy InternalRow, ArrayData and MapData. This PR fixes this, and also fixes the copy of vairous InternalRow, ArrayData and MapData implementations.
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#18483 from cloud-fan/fix-copy.
## What changes were proposed in this pull request?
A follow up PR to fix Scala 2.10 build for #18472
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18478 from zsxwing/SPARK-21253-2.
## What changes were proposed in this pull request?
If a network error happens before processing StreamResponse/StreamFailure events, StreamCallback.onFailure won't be called.
This PR fixes `failOutstandingRequests` to also notify outstanding StreamCallbacks.
## How was this patch tested?
The new unit tests.
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#18472 from zsxwing/fix-stream-2.
## What changes were proposed in this pull request?
We are explicitly calling release on the byteBuf's used to encode the string to Base64 to suppress the memory leak error message reported by netty. This is to make it less confusing for the user.
### Changes proposed in this fix
By explicitly invoking release on the byteBuf's we are decrement the internal reference counts for the wrappedByteBuf's. Now, when the GC kicks in, these would be reclaimed as before, just that netty wouldn't report any memory leak error messages as the internal ref. counts are now 0.
## How was this patch tested?
Ran a few spark-applications and examined the logs. The error message no longer appears.
Original PR was opened against branch-2.1 => https://github.com/apache/spark/pull/18392
Author: Dhruve Ashar <dhruveashar@gmail.com>
Closes#18407 from dhruve/master.
## What changes were proposed in this pull request?
Currently the shuffle service registration timeout and retry has been hardcoded. This works well for small workloads but under heavy workload when the shuffle service is busy transferring large amount of data we see significant delay in responding to the registration request, as a result we often see the executors fail to register with the shuffle service, eventually failing the job. We need to make these two parameters configurable.
## How was this patch tested?
* Updated `BlockManagerSuite` to test registration timeout and max attempts configuration actually works.
cc sitalkedia
Author: Li Yichao <lyc@zhihu.com>
Closes#18092 from liyichao/SPARK-20640.
## What changes were proposed in this pull request?
This PR cleans up a few Java linter errors for Apache Spark 2.2 release.
## How was this patch tested?
```bash
$ dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```
We can check the result at Travis CI, [here](https://travis-ci.org/dongjoon-hyun/spark/builds/244297894).
Author: Dongjoon Hyun <dongjoon@apache.org>
Closes#18345 from dongjoon-hyun/fix_lint_java_2.
## What changes were proposed in this pull request?
In current code, blockIds in `OpenBlocks` are stored in the iterator on shuffle service.
There are some redundant characters in blockId(`"shuffle_" + shuffleId + "_" + mapId + "_" + reduceId`). This pr proposes to improve the footprint and alleviate the memory pressure on shuffle service.
Author: jinxing <jinxing6042@126.com>
Closes#18231 from jinxing64/SPARK-20994-v2.
This change adds an abstraction and LevelDB implementation for a key-value
store that will be used to store UI and SHS data.
The interface is described in KVStore.java (see javadoc). Specifics
of the LevelDB implementation are discussed in the javadocs of both
LevelDB.java and LevelDBTypeInfo.java.
Included also are a few small benchmarks just to get some idea of
latency. Because they're too slow for regular unit test runs, they're
disabled by default.
Tested with the included unit tests, and also as part of the overall feature
implementation (including running SHS with hundreds of apps).
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#17902 from vanzin/shs-ng/M1.
## What changes were proposed in this pull request?
This PR adds built-in SQL function `(REPLACE(<string_expression>, <search_string> [, <replacement_string>])`
`REPLACE()` return that string that is replaced all occurrences with given string.
## How was this patch tested?
added new test suites
Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Closes#18047 from kiszk/SPARK-20750.
## What changes were proposed in this pull request?
Currently the whole block is fetched into memory(off heap by default) when shuffle-read. A block is defined by (shuffleId, mapId, reduceId). Thus it can be large when skew situations. If OOM happens during shuffle read, job will be killed and users will be notified to "Consider boosting spark.yarn.executor.memoryOverhead". Adjusting parameter and allocating more memory can resolve the OOM. However the approach is not perfectly suitable for production environment, especially for data warehouse.
Using Spark SQL as data engine in warehouse, users hope to have a unified parameter(e.g. memory) but less resource wasted(resource is allocated but not used). The hope is strong especially when migrating data engine to Spark from another one(e.g. Hive). Tuning the parameter for thousands of SQLs one by one is very time consuming.
It's not always easy to predict skew situations, when happen, it make sense to fetch remote blocks to disk for shuffle-read, rather than kill the job because of OOM.
In this pr, I propose to fetch big blocks to disk(which is also mentioned in SPARK-3019):
1. Track average size and also the outliers(which are larger than 2*avgSize) in MapStatus;
2. Request memory from `MemoryManager` before fetch blocks and release the memory to `MemoryManager` when `ManagedBuffer` is released.
3. Fetch remote blocks to disk when failing acquiring memory from `MemoryManager`, otherwise fetch to memory.
This is an improvement for memory control when shuffle blocks and help to avoid OOM in scenarios like below:
1. Single huge block;
2. Sizes of many blocks are underestimated in `MapStatus` and the actual footprint of blocks is much larger than the estimated.
## How was this patch tested?
Added unit test in `MapStatusSuite` and `ShuffleBlockFetcherIteratorSuite`.
Author: jinxing <jinxing6042@126.com>
Closes#16989 from jinxing64/SPARK-19659.
and contains scala classes
## What changes were proposed in this pull request?
This change ensures that all references to guava from within the yarn shuffle jar pointed to the shaded guava class already provided in the jar.
Also, it explicitly excludes scala classes from being added to the jar.
## How was this patch tested?
Ran unit tests on the module and they passed.
javap now returns the expected result - reference to the shaded guava under `org/spark_project` (previously this was referring to `com.google...`
```
javap -cp common/network-yarn/target/scala-2.11/spark-2.3.0-SNAPSHOT-yarn-shuffle.jar -c org/apache/spark/network/yarn/YarnShuffleService | grep Lists
57: invokestatic #138 // Method org/spark_project/guava/collect/Lists.newArrayList:()Ljava/util/ArrayList;
```
Guava is still shaded in the jar:
```
jar -tf common/network-yarn/target/scala-2.11/spark-2.3.0-SNAPSHOT-yarn-shuffle.jar | grep guava | head
META-INF/maven/com.google.guava/
META-INF/maven/com.google.guava/guava/
META-INF/maven/com.google.guava/guava/pom.properties
META-INF/maven/com.google.guava/guava/pom.xml
org/spark_project/guava/
org/spark_project/guava/annotations/
org/spark_project/guava/annotations/Beta.class
org/spark_project/guava/annotations/GwtCompatible.class
org/spark_project/guava/annotations/GwtIncompatible.class
org/spark_project/guava/annotations/VisibleForTesting.class
```
(not sure if the above META-INF/* is a problem or not)
I took this jar, deployed it on a yarn cluster with shuffle service enabled, and made sure the YARN node managers came up. An application with a shuffle was run and it succeeded.
Author: Mark Grover <mark@apache.org>
Closes#17990 from markgrover/spark-20756.
## What changes were proposed in this pull request?
This PR proposes to fix the lint-breaks as below:
```
[ERROR] src/main/java/org/apache/spark/unsafe/Platform.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[45,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[62,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[78,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[92,25] (naming) MethodName: Method name 'ProcessingTime' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/main/scala/org/apache/spark/sql/streaming/Trigger.java:[102,25] (naming) MethodName: Method name 'Once' must match pattern '^[a-z][a-z0-9][a-zA-Z0-9_]*$'.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisInputDStreamBuilderSuite.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.api.java.JavaDStream.
```
after:
```
dev/lint-java
Checkstyle checks passed.
```
[Test Result](https://travis-ci.org/ConeyLiu/spark/jobs/229666169)
## How was this patch tested?
Travis CI
Author: Xianyang Liu <xianyang.liu@intel.com>
Closes#17890 from ConeyLiu/codestyle.
## What changes were proposed in this pull request?
Fix build warnings primarily related to Breeze 0.13 operator changes, Java style problems
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#17803 from srowen/SPARK-20523.
## What changes were proposed in this pull request?
When application contains large amount of shuffle blocks. NodeManager requires lots of memory to keep metadata(`FileSegmentManagedBuffer`) in `StreamManager`. When the number of shuffle blocks is big enough. NodeManager can run OOM. This pr proposes to do lazy initialization of `FileSegmentManagedBuffer` in shuffle service.
## How was this patch tested?
Manually test.
Author: jinxing <jinxing6042@126.com>
Closes#17744 from jinxing64/SPARK-20426.
…ss NFS directories
## What changes were proposed in this pull request?
Change from using java Files.move to use Hadoop filesystem operations to move the directories. The java Files.move does not work when moving directories across NFS mounts and in fact also says that if the directory has entries you should do a recursive move. We are already using Hadoop filesystem here so just use the local filesystem from there as it handles this properly.
Note that the DB here is actually a directory of files and not just a single file, hence the change in the name of the local var.
## How was this patch tested?
Ran YarnShuffleServiceSuite unit tests. Unfortunately couldn't easily add one here since involves NFS.
Ran manual tests to verify that the DB directories were properly moved across NFS mounted directories. Have been running this internally for weeks.
Author: Tom Graves <tgraves@apache.org>
Closes#17748 from tgravescs/SPARK-19812.
## What changes were proposed in this pull request?
This PR fixs the following failure:
```
sbt.ForkMain$ForkError: java.lang.AssertionError: null
at org.junit.Assert.fail(Assert.java:86)
at org.junit.Assert.assertTrue(Assert.java:41)
at org.junit.Assert.assertTrue(Assert.java:52)
at org.apache.spark.network.RequestTimeoutIntegrationSuite.furtherRequestsDelay(RequestTimeoutIntegrationSuite.java:230)
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:497)
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.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26)
at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
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.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:1142)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```
It happens several times per month on [Jenkins](http://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.network.RequestTimeoutIntegrationSuite&test_name=furtherRequestsDelay). The failure is because `callback1` may not be called before `assertTrue(callback1.failure instanceof IOException);`. It's pretty easy to reproduce this error by adding a sleep before this line: 379b0b0bbd/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java (L267)
The fix is straightforward: just use the latch to wait until `callback1` is called.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#17599 from zsxwing/SPARK-17564.
## 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.
## What changes were proposed in this pull request?
Avoid `NoSuchElementException` every time `ConfigProvider.get(val, default)` falls back to default. This apparently causes non-trivial overhead in at least one path, and can easily be avoided.
See https://github.com/apache/spark/pull/17329
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#17567 from srowen/SPARK-19991.
java.nio.Bits.unaligned() does not return true for the ppc64le arch.
see https://bugs.openjdk.java.net/browse/JDK-8165231
## What changes were proposed in this pull request?
check architecture
## How was this patch tested?
unit test
Author: samelamin <hussam.elamin@gmail.com>
Author: samelamin <sam_elamin@discovery.com>
Closes#17472 from samelamin/SPARK-19999.
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.
## What changes were proposed in this pull request?
During build/sbt publish-local, build breaks due to javadocs errors. This patch fixes those errors.
## How was this patch tested?
Tested by running the sbt build.
Author: Prashant Sharma <prashsh1@in.ibm.com>
Closes#17358 from ScrapCodes/docs-fix.
## What changes were proposed in this pull request?
This is as per suggestion by rxin at : https://github.com/apache/spark/pull/17184#discussion_r104841735
## How was this patch tested?
NA as this is a documentation change
Author: Tejas Patil <tejasp@fb.com>
Closes#17205 from tejasapatil/SPARK-19843_followup.
## What changes were proposed in this pull request?
Jira : https://issues.apache.org/jira/browse/SPARK-19843
Created wrapper classes (`IntWrapper`, `LongWrapper`) to wrap the result of parsing (which are primitive types). In case of problem in parsing, the method would return a boolean.
## How was this patch tested?
- Added new unit tests
- Ran a prod job which had conversion from string -> int and verified the outputs
## Performance
Tiny regression when all strings are valid integers
```
conversion to int: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
--------------------------------------------------------------------------------
trunk 502 / 522 33.4 29.9 1.0X
SPARK-19843 493 / 503 34.0 29.4 1.0X
```
Huge gain when all strings are invalid integers
```
conversion to int: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative
-------------------------------------------------------------------------------
trunk 33913 / 34219 0.5 2021.4 1.0X
SPARK-19843 154 / 162 108.8 9.2 220.0X
```
Author: Tejas Patil <tejasp@fb.com>
Closes#17184 from tejasapatil/SPARK-19843_is_numeric_maybe.
## 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.
## What changes were proposed in this pull request?
This PR adds tests hive-hash by comparing the outputs generated against Hive 1.2.1. Following datatypes are covered by this PR:
- null
- boolean
- byte
- short
- int
- long
- float
- double
- string
- array
- map
- struct
Datatypes that I have _NOT_ covered but I will work on separately are:
- Decimal (handled separately in https://github.com/apache/spark/pull/17056)
- TimestampType
- DateType
- CalendarIntervalType
## How was this patch tested?
NA
Author: Tejas Patil <tejasp@fb.com>
Closes#17049 from tejasapatil/SPARK-17495_remaining_types.
## What changes were proposed in this pull request?
Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code.
## How was this patch tested?
Jenkins tests
Author: Sean Owen <sowen@cloudera.com>
Closes#16964 from srowen/SPARK-19534.
## What changes were proposed in this pull request?
If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory.
Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired.
These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing.
I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one.
## How was this patch tested?
New and existing unit tests. No performance or load tests have been run.
Author: Nathan Howell <nhowell@godaddy.com>
Closes#16386 from NathanHowell/SPARK-18352.
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings
For the future:
- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#16871 from srowen/SPARK-19493.
## What changes were proposed in this pull request?
Netty's `MessageToMessageEncoder` uses [Javassist](91a0bdc17a/common/src/main/java/io/netty/util/internal/JavassistTypeParameterMatcherGenerator.java (L62)) to generate a matcher class and the implementation calls `Class.forName` to check if this class is already generated. If `MessageEncoder` or `MessageDecoder` is created in `ExecutorClassLoader.findClass`, it will cause `ClassCircularityError`. This is because loading this Netty generated class will call `ExecutorClassLoader.findClass` to search this class, and `ExecutorClassLoader` will try to use RPC to load it and cause to load the non-exist matcher class again. JVM will report `ClassCircularityError` to prevent such infinite recursion.
##### Why it only happens in Maven builds
It's because Maven and SBT have different class loader tree. The Maven build will set a URLClassLoader as the current context class loader to run the tests and expose this issue. The class loader tree is as following:
```
bootstrap class loader ------ ... ----- REPL class loader ---- ExecutorClassLoader
|
|
URLClasssLoader
```
The SBT build uses the bootstrap class loader directly and `ReplSuite.test("propagation of local properties")` is the first test in ReplSuite, which happens to load `io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher` into the bootstrap class loader (Note: in maven build, it's loaded into URLClasssLoader so it cannot be found in ExecutorClassLoader). This issue can be reproduced in SBT as well. Here are the produce steps:
- Enable `hadoop.caller.context.enabled`.
- Replace `Class.forName` with `Utils.classForName` in `object CallerContext`.
- Ignore `ReplSuite.test("propagation of local properties")`.
- Run `ReplSuite` using SBT.
This PR just creates a singleton MessageEncoder and MessageDecoder and makes sure they are created before switching to ExecutorClassLoader. TransportContext will be created when creating RpcEnv and that happens before creating ExecutorClassLoader.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16859 from zsxwing/SPARK-17714.
## What changes were proposed in this pull request?
This patch replaces a single `awaitUninterruptibly()` call with a plain `await()` call in Spark's `network-common` library in order to fix a bug which may cause tasks to be uncancellable.
In Spark's Netty RPC layer, `TransportClientFactory.createClient()` calls `awaitUninterruptibly()` on a Netty future while waiting for a connection to be established. This creates problem when a Spark task is interrupted while blocking in this call (which can happen in the event of a slow connection which will eventually time out). This has bad impacts on task cancellation when `interruptOnCancel = true`.
As an example of the impact of this problem, I experienced significant numbers of uncancellable "zombie tasks" on a production cluster where several tasks were blocked trying to connect to a dead shuffle server and then continued running as zombies after I cancelled the associated Spark stage. The zombie tasks ran for several minutes with the following stack:
```
java.lang.Object.wait(Native Method)
java.lang.Object.wait(Object.java:460)
io.netty.util.concurrent.DefaultPromise.await0(DefaultPromise.java:607)
io.netty.util.concurrent.DefaultPromise.awaitUninterruptibly(DefaultPromise.java:301)
org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:224)
org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:179) => holding Monitor(java.lang.Object1849476028})
org.apache.spark.network.shuffle.ExternalShuffleClient$1.createAndStart(ExternalShuffleClient.java:105)
org.apache.spark.network.shuffle.RetryingBlockFetcher.fetchAllOutstanding(RetryingBlockFetcher.java:140)
org.apache.spark.network.shuffle.RetryingBlockFetcher.start(RetryingBlockFetcher.java:120)
org.apache.spark.network.shuffle.ExternalShuffleClient.fetchBlocks(ExternalShuffleClient.java:114)
org.apache.spark.storage.ShuffleBlockFetcherIterator.sendRequest(ShuffleBlockFetcherIterator.scala:169)
org.apache.spark.storage.ShuffleBlockFetcherIterator.fetchUpToMaxBytes(ShuffleBlockFetcherIterator.scala:
350)
org.apache.spark.storage.ShuffleBlockFetcherIterator.initialize(ShuffleBlockFetcherIterator.scala:286)
org.apache.spark.storage.ShuffleBlockFetcherIterator.<init>(ShuffleBlockFetcherIterator.scala:120)
org.apache.spark.shuffle.BlockStoreShuffleReader.read(BlockStoreShuffleReader.scala:45)
org.apache.spark.sql.execution.ShuffledRowRDD.compute(ShuffledRowRDD.scala:169)
org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:323)
org.apache.spark.rdd.RDD.iterator(RDD.scala:287)
[...]
```
As far as I can tell, `awaitUninterruptibly()` might have been used in order to avoid having to declare that methods throw `InterruptedException` (this code is written in Java, hence the need to use checked exceptions). This patch simply replaces this with a regular, interruptible `await()` call,.
This required several interface changes to declare a new checked exception (these are internal interfaces, though, and this change doesn't significantly impact binary compatibility).
An alternative approach would be to wrap `InterruptedException` into `IOException` in order to avoid having to change interfaces. The problem with this approach is that the `network-shuffle` project's `RetryingBlockFetcher` code treats `IOExceptions` as transitive failures when deciding whether to retry fetches, so throwing a wrapped `IOException` might cause an interrupted shuffle fetch to be retried, further prolonging the lifetime of a cancelled zombie task.
Note that there are three other `awaitUninterruptibly()` in the codebase, but those calls have a hard 10 second timeout and are waiting on a `close()` operation which is expected to complete near instantaneously, so the impact of uninterruptibility there is much smaller.
## How was this patch tested?
Manually.
Author: Josh Rosen <joshrosen@databricks.com>
Closes#16866 from JoshRosen/SPARK-19529.
This change introduces a new auth mechanism to the transport library,
to be used when users enable strong encryption. This auth mechanism
has better security than the currently used DIGEST-MD5.
The new protocol uses symmetric key encryption to mutually authenticate
the endpoints, and is very loosely based on ISO/IEC 9798.
The new protocol falls back to SASL when it thinks the remote end is old.
Because SASL does not support asking the server for multiple auth protocols,
which would mean we could re-use the existing SASL code by just adding a
new SASL provider, the protocol is implemented outside of the SASL API
to avoid the boilerplate of adding a new provider.
Details of the auth protocol are discussed in the included README.md
file.
This change partly undos the changes added in SPARK-13331; AES encryption
is now decoupled from SASL authentication. The encryption code itself,
though, has been re-used as part of this change.
## How was this patch tested?
- Unit tests
- Tested Spark 2.2 against Spark 1.6 shuffle service with SASL enabled
- Tested Spark 2.2 against Spark 2.2 shuffle service with SASL fallback disabled
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#16521 from vanzin/SPARK-19139.
## What changes were proposed in this pull request?
When we convert a string to integral, we will convert that string to `decimal(20, 0)` first, so that we can turn a string with decimal format to truncated integral, e.g. `CAST('1.2' AS int)` will return `1`.
However, this brings problems when we convert a string with large numbers to integral, e.g. `CAST('1234567890123' AS int)` will return `1912276171`, while Hive returns null as we expected.
This is a long standing bug(seems it was there the first day Spark SQL was created), this PR fixes this bug by adding the native support to convert `UTF8String` to integral.
## How was this patch tested?
new regression tests
Author: Wenchen Fan <wenchen@databricks.com>
Closes#16550 from cloud-fan/string-to-int.
## What changes were proposed in this pull request?
This adds back a direct dependency on Scala library classes from spark-tags because its Scala annotations need them.
## How was this patch tested?
Existing tests
Author: Sean Owen <sowen@cloudera.com>
Closes#16418 from srowen/SPARK-18993.
fails on big endian. Only change byte order on little endian
## What changes were proposed in this pull request?
Fix test to only change byte order on LE platforms
## How was this patch tested?
Test run on Big Endian and Little Endian platforms
Author: Pete Robbins <robbinspg@gmail.com>
Closes#16375 from robbinspg/SPARK-18963.
## What changes were proposed in this pull request?
Right now the name of threads created by Netty for Spark RPC are `shuffle-client-**` and `shuffle-server-**`. It's pretty confusing.
This PR just uses the module name in TransportConf to set the thread name. In addition, it also includes the following minor fixes:
- TransportChannelHandler.channelActive and channelInactive should call the corresponding super methods.
- Make ShuffleBlockFetcherIterator throw NoSuchElementException if it has no more elements. Otherwise, if the caller calls `next` without `hasNext`, it will just hang.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16380 from zsxwing/SPARK-18972.
Remove spark-tag's compile-scope dependency (and, indirectly, spark-core's compile-scope transitive-dependency) on scalatest by splitting test-oriented tags into spark-tags' test JAR.
Alternative to #16303.
Author: Ryan Williams <ryan.blake.williams@gmail.com>
Closes#16311 from ryan-williams/tt.
## What changes were proposed in this pull request?
This PR proposes to fix lint-check failures and javadoc8 break.
Few errors were introduced as below:
**lint-check failures**
```
[ERROR] src/test/java/org/apache/spark/network/TransportClientFactorySuite.java:[45,1] (imports) RedundantImport: Duplicate import to line 43 - org.apache.spark.network.util.MapConfigProvider.
[ERROR] src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java:[255,10] (modifier) RedundantModifier: Redundant 'final' modifier.
```
**javadoc8**
```
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:19: error: bad use of '>'
[error] * "max" -> "2016-12-05T20:54:20.827Z" // maximum event time seen in this trigger
[error] ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:20: error: bad use of '>'
[error] * "min" -> "2016-12-05T20:54:20.827Z" // minimum event time seen in this trigger
[error] ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:21: error: bad use of '>'
[error] * "avg" -> "2016-12-05T20:54:20.827Z" // average event time seen in this trigger
[error] ^
[error] .../spark/sql/core/target/java/org/apache/spark/sql/streaming/StreamingQueryProgress.java:22: error: bad use of '>'
[error] * "watermark" -> "2016-12-05T20:54:20.827Z" // watermark used in this trigger
[error]
```
## How was this patch tested?
Manually checked as below:
**lint-check failures**
```
./dev/lint-java
Checkstyle checks passed.
```
**javadoc8**
This seems hidden in the API doc but I manually checked after removing access modifier as below:
It looks not rendering properly (scaladoc).
![2016-12-16 3 40 34](https://cloud.githubusercontent.com/assets/6477701/21255175/8df1fe6e-c3ad-11e6-8cda-ce7f76c6677a.png)
After this PR, it renders as below:
- scaladoc
![2016-12-16 3 40 23](https://cloud.githubusercontent.com/assets/6477701/21255135/4a11dab6-c3ad-11e6-8ab2-b091c4f45029.png)
- javadoc
![2016-12-16 3 41 10](https://cloud.githubusercontent.com/assets/6477701/21255137/4bba1d9c-c3ad-11e6-9b88-62f1f697b56a.png)
Author: hyukjinkwon <gurwls223@gmail.com>
Closes#16307 from HyukjinKwon/lint-javadoc8.
This change moves the logic that translates Spark configuration to
commons-crypto configuration to the network-common module. It also
extends TransportConf and ConfigProvider to provide the necessary
interfaces for the translation to work.
As part of the change, I removed SystemPropertyConfigProvider, which
was mostly used as an "empty config" in unit tests, and adjusted the
very few tests that required a specific config.
I also changed the config keys for AES encryption to live under the
"spark.network." namespace, which is more correct than their previous
names under "spark.authenticate.".
Tested via existing unit test.
Author: Marcelo Vanzin <vanzin@cloudera.com>
Closes#16200 from vanzin/SPARK-18773.
## What changes were proposed in this pull request?
We added some codes in https://github.com/apache/spark/pull/14961 because of https://github.com/netty/netty/issues/5833
Now we can remove them as it's fixed in Netty 4.0.42.Final.
## How was this patch tested?
Jenkins
Author: Shixiong Zhu <shixiong@databricks.com>
Closes#16167 from zsxwing/remove-netty-workaround.
## What changes were proposed in this pull request?
This patch bumps master branch version to 2.2.0-SNAPSHOT.
## How was this patch tested?
N/A
Author: Reynold Xin <rxin@databricks.com>
Closes#16126 from rxin/SPARK-18695.
## What changes were proposed in this pull request?
This replaces uses of `TextOutputFormat` with an `OutputStream`, which will either write directly to the filesystem or indirectly via a compressor (if so configured). This avoids intermediate buffering.
The inverse of this (reading directly from a stream) is necessary for streaming large JSON records (when `wholeFile` is enabled) so I wanted to keep the read and write paths symmetric.
## How was this patch tested?
Existing unit tests.
Author: Nathan Howell <nhowell@godaddy.com>
Closes#16089 from NathanHowell/SPARK-18658.
## What changes were proposed in this pull request?
SPARK-18429 introduced count-min sketch aggregate function for SQL, but the implementation and testing is more complicated than needed. This simplifies the test cases and removes support for data types that don't have clear equality semantics:
1. Removed support for floating point and decimal types.
2. Removed the heavy randomized tests. The underlying CountMinSketch implementation already had pretty good test coverage through randomized tests, and the SPARK-18429 implementation is just to add an aggregate function wrapper around CountMinSketch. There is no need for randomized tests at three different levels of the implementations.
## How was this patch tested?
A lot of the change is to simplify test cases.
Author: Reynold Xin <rxin@databricks.com>
Closes#16093 from rxin/SPARK-18663.
## What changes were proposed in this pull request?
This PR implements a new Aggregate to generate count min sketch, which is a wrapper of CountMinSketch.
## How was this patch tested?
add test cases
Author: wangzhenhua <wangzhenhua@huawei.com>
Closes#15877 from wzhfy/cms.
## What changes were proposed in this pull request?
Small fix, fix the errors caused by lint check in Java
- Clear unused objects and `UnusedImports`.
- Add comments around the method `finalize` of `NioBufferedFileInputStream`to turn off checkstyle.
- Cut the line which is longer than 100 characters into two lines.
## How was this patch tested?
Travis CI.
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
```
Before:
```
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[21,8] (imports) UnusedImports: Unused import - org.apache.commons.crypto.cipher.CryptoCipherFactory.
[ERROR] src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java:[516,5] (modifier) RedundantModifier: Redundant 'public' modifier.
[ERROR] src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java:[133] (coding) NoFinalizer: Avoid using finalizer method.
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java:[71] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java:[112] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java:[31,17] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR]src/main/java/org/apache/spark/examples/ml/JavaLogisticRegressionWithElasticNetExample.java:[64] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[22,8] (imports) UnusedImports: Unused import - org.apache.spark.ml.linalg.Vectors.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaInteractionExample.java:[51] (regexp) RegexpSingleline: No trailing whitespace allowed.
```
After:
```
$ build/mvn -T 4 -q -DskipTests -Pyarn -Phadoop-2.3 -Pkinesis-asl -Phive -Phive-thriftserver install
$ dev/lint-java
Using `mvn` from path: /home/travis/build/ConeyLiu/spark/build/apache-maven-3.3.9/bin/mvn
Checkstyle checks passed.
```
Author: Xianyang Liu <xyliu0530@icloud.com>
Closes#15865 from ConeyLiu/master.
This PR adds a new method `withWatermark` to the `Dataset` API, which can be used specify an _event time watermark_. An event time watermark allows the streaming engine to reason about the point in time after which we no longer expect to see late data. This PR also has augmented `StreamExecution` to use this watermark for several purposes:
- To know when a given time window aggregation is finalized and thus results can be emitted when using output modes that do not allow updates (e.g. `Append` mode).
- To minimize the amount of state that we need to keep for on-going aggregations, by evicting state for groups that are no longer expected to change. Although, we do still maintain all state if the query requires (i.e. if the event time is not present in the `groupBy` or when running in `Complete` mode).
An example that emits windowed counts of records, waiting up to 5 minutes for late data to arrive.
```scala
df.withWatermark("eventTime", "5 minutes")
.groupBy(window($"eventTime", "1 minute") as 'window)
.count()
.writeStream
.format("console")
.mode("append") // In append mode, we only output finalized aggregations.
.start()
```
### Calculating the watermark.
The current event time is computed by looking at the `MAX(eventTime)` seen this epoch across all of the partitions in the query minus some user defined _delayThreshold_. An additional constraint is that the watermark must increase monotonically.
Note that since we must coordinate this value across partitions occasionally, the actual watermark used is only guaranteed to be at least `delay` behind the actual event time. In some cases we may still process records that arrive more than delay late.
This mechanism was chosen for the initial implementation over processing time for two reasons:
- it is robust to downtime that could affect processing delay
- it does not require syncing of time or timezones between the producer and the processing engine.
### Other notable implementation details
- A new trigger metric `eventTimeWatermark` outputs the current value of the watermark.
- We mark the event time column in the `Attribute` metadata using the key `spark.watermarkDelay`. This allows downstream operations to know which column holds the event time. Operations like `window` propagate this metadata.
- `explain()` marks the watermark with a suffix of `-T${delayMs}` to ease debugging of how this information is propagated.
- Currently, we don't filter out late records, but instead rely on the state store to avoid emitting records that are both added and filtered in the same epoch.
### Remaining in this PR
- [ ] The test for recovery is currently failing as we don't record the watermark used in the offset log. We will need to do so to ensure determinism, but this is deferred until #15626 is merged.
### Other follow-ups
There are some natural additional features that we should consider for future work:
- Ability to write records that arrive too late to some external store in case any out-of-band remediation is required.
- `Update` mode so you can get partial results before a group is evicted.
- Other mechanisms for calculating the watermark. In particular a watermark based on quantiles would be more robust to outliers.
Author: Michael Armbrust <michael@databricks.com>
Closes#15702 from marmbrus/watermarks.
## What changes were proposed in this pull request?
DIGEST-MD5 mechanism is used for SASL authentication and secure communication. DIGEST-MD5 mechanism supports 3DES, DES, and RC4 ciphers. However, 3DES, DES and RC4 are slow relatively.
AES provide better performance and security by design and is a replacement for 3DES according to NIST. Apache Common Crypto is a cryptographic library optimized with AES-NI, this patch employ Apache Common Crypto as enc/dec backend for SASL authentication and secure channel to improve spark RPC.
## How was this patch tested?
Unit tests and Integration test.
Author: Junjie Chen <junjie.j.chen@intel.com>
Closes#15172 from cjjnjust/shuffle_rpc_encrypt.