Commit graph

64 commits

Author SHA1 Message Date
jerryshao 5fd0294ff8 [SPARK-21475][CORE] Use NIO's Files API to replace FileInputStream/FileOutputStream in some critical paths
## 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-8080225

https://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.
2017-08-01 10:23:45 +01:00
Sean Owen 63d168cbb8 [MINOR][BUILD] Fix current lint-java failures
## 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.
2017-07-28 11:31:40 +01:00
jinxing cfb25b27c0 [SPARK-21530] Update description of spark.shuffle.maxChunksBeingTransferred.
## 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.
2017-07-27 11:55:48 +08:00
Marcelo Vanzin 300807c6e3 [SPARK-21494][NETWORK] Use correct app id when authenticating to external service.
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.
2017-07-25 17:57:26 -07:00
jinxing 799e13161e [SPARK-21175] Reject OpenBlocks when memory shortage on shuffle service.
## 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.
2017-07-25 20:52:07 +08:00
Shixiong Zhu 833eab2c9b [SPARK-21369][CORE] Don't use Scala Tuple2 in common/network-*
## 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.
2017-07-11 11:26:17 +08:00
Shixiong Zhu cfc696f4a4 [SPARK-21253][CORE][HOTFIX] Fix Scala 2.10 build
## 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.
2017-06-29 20:56:37 -07:00
Shixiong Zhu 4996c53949 [SPARK-21253][CORE] Fix a bug that StreamCallback may not be notified if network errors happen
## 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.
2017-06-30 10:56:48 +08:00
Dhruve Ashar 1ebe7ffe07 [SPARK-21181] Release byteBuffers to suppress netty error messages
## 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.
2017-06-23 10:36:29 -07:00
jinxing 3f94e64aa8 [SPARK-19659] Fetch big blocks to disk when shuffle-read.
## 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.
2017-05-25 16:11:30 +08:00
Josh Rosen f44c8a843c [SPARK-20453] Bump master branch version to 2.3.0-SNAPSHOT
This patch bumps the master branch version to `2.3.0-SNAPSHOT`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17753 from JoshRosen/SPARK-20453.
2017-04-24 21:48:04 -07:00
Shixiong Zhu 734dfbfcfe [SPARK-17564][TESTS] Fix flaky RequestTimeoutIntegrationSuite.furtherRequestsDelay
## 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.
2017-04-10 20:41:08 -07:00
Sean Owen a26e3ed5e4 [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
## 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.
2017-04-10 20:11:56 +01:00
Sean Owen 1f0de3c1c8 [SPARK-19991][CORE][YARN] FileSegmentManagedBuffer performance improvement
## 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.
2017-04-09 08:44:02 +01:00
Marcelo Vanzin b56ad2b1ec [SPARK-19556][CORE] Do not encrypt block manager data in memory.
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.
2017-03-29 20:27:41 +08:00
Prashant Sharma 0caade6340 [SPARK-20027][DOCS] Compilation fix in java docs.
## 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.
2017-03-22 13:52:03 +00:00
hyukjinkwon 4ba9c6c453 [MINOR][BUILD] Fix lint-java breaks in Java
## 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.
2017-02-27 08:44:26 +00:00
Sean Owen 1487c9af20
[SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features
## 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.
2017-02-19 09:42:50 -08:00
Sean Owen 0e2405490f
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support
- 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.
2017-02-16 12:32:45 +00:00
Shixiong Zhu 905fdf0c24 [SPARK-17714][CORE][TEST-MAVEN][TEST-HADOOP2.6] Avoid using ExecutorClassLoader to load Netty generated classes
## 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.
2017-02-13 12:03:36 -08:00
Josh Rosen 1c4d10b10c [SPARK-19529] TransportClientFactory.createClient() shouldn't call awaitUninterruptibly()
## 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.
2017-02-13 11:04:27 -08:00
Marcelo Vanzin 8f3f73abc1 [SPARK-19139][CORE] New auth mechanism for transport library.
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.
2017-01-24 10:44:04 -08:00
Shixiong Zhu f252cb5d16 [SPARK-18972][CORE] Fix the netty thread names for RPC
## 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.
2016-12-22 16:22:55 -08:00
Ryan Williams afd9bc1d8a [SPARK-17807][CORE] split test-tags into test-JAR
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.
2016-12-21 16:37:20 -08:00
hyukjinkwon ed84cd0684
[MINOR][BUILD] Fix lint-check failures and javadoc8 break
## 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.
2016-12-16 17:49:43 +00:00
Marcelo Vanzin bc59951bab [SPARK-18773][CORE] Make commons-crypto config translation consistent.
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.
2016-12-12 16:27:04 -08:00
Shixiong Zhu eeed38eaf8 [SPARK-18744][CORE] Remove workaround for Netty memory leak
## 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.
2016-12-06 13:33:05 -08:00
Reynold Xin c7c7265950 [SPARK-18695] Bump master branch version to 2.2.0-SNAPSHOT
## 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.
2016-12-02 21:09:37 -08:00
Xianyang Liu 7569cf6cb8
[SPARK-18420][BUILD] Fix the errors caused by lint check in Java
## 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.
2016-11-16 11:59:00 +00:00
Junjie Chen 4f15d94cfe [SPARK-13331] AES support for over-the-wire encryption
## 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.
2016-11-11 10:37:58 -08:00
Weiqing Yang 1ea49916ac [MINOR][BUILD] Fix CheckStyle Error
## What changes were proposed in this pull request?
This PR is to fix the code style errors before 2.0.1 release.

## How was this patch tested?
Manual.

Before:
```
./dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/network/client/TransportClient.java:[153] (sizes) LineLength: Line is longer than 100 characters (found 107).
[ERROR] src/main/java/org/apache/spark/network/client/TransportClient.java:[196] (sizes) LineLength: Line is longer than 100 characters (found 108).
[ERROR] src/main/java/org/apache/spark/network/client/TransportClient.java:[239] (sizes) LineLength: Line is longer than 100 characters (found 115).
[ERROR] src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:[119] (sizes) LineLength: Line is longer than 100 characters (found 107).
[ERROR] src/main/java/org/apache/spark/network/server/TransportRequestHandler.java:[129] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/network/util/LevelDBProvider.java:[124,11] (modifier) ModifierOrder: 'static' modifier out of order with the JLS suggestions.
[ERROR] src/main/java/org/apache/spark/network/util/TransportConf.java:[26] (regexp) RegexpSingleline: No trailing whitespace allowed.
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java:[33] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java:[38] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java:[43] (sizes) LineLength: Line is longer than 100 characters (found 106).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java:[48] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java:[0] (misc) NewlineAtEndOfFile: File does not end with a newline.
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java:[67] (sizes) LineLength: Line is longer than 100 characters (found 106).
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java:[200] (regexp) RegexpSingleline: No trailing whitespace allowed.
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java:[309] (regexp) RegexpSingleline: No trailing whitespace allowed.
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java:[332] (regexp) RegexpSingleline: No trailing whitespace allowed.
[ERROR] src/main/java/org/apache/spark/network/yarn/YarnShuffleService.java:[348] (regexp) RegexpSingleline: No trailing whitespace allowed.
 ```
After:
```
./dev/lint-java
Using `mvn` from path: /usr/local/bin/mvn
Checkstyle checks passed.
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15170 from Sherry302/fixjavastyle.
2016-09-20 21:48:25 -07:00
Adam Roberts 0ad8eeb4d3 [SPARK-17379][BUILD] Upgrade netty-all to 4.0.41 final for bug fixes
## What changes were proposed in this pull request?
Upgrade netty-all to latest in the 4.0.x line which is 4.0.41, mentions several bug fixes and performance improvements we may find useful, see netty.io/news/2016/08/29/4-0-41-Final-4-1-5-Final.html. Initially tried to use 4.1.5 but noticed it's not backwards compatible.

## How was this patch tested?
Existing unit tests against branch-1.6 and branch-2.0 using IBM Java 8 on Intel, Power and Z architectures

Author: Adam Roberts <aroberts@uk.ibm.com>

Closes #14961 from a-roberts/netty.
2016-09-15 10:40:10 -07:00
Thomas Graves e79962f2f3 [SPARK-16711] YarnShuffleService doesn't re-init properly on YARN rolling upgrade
The Spark Yarn Shuffle Service doesn't re-initialize the application credentials early enough which causes any other spark executors trying to fetch from that node during a rolling upgrade to fail with "java.lang.NullPointerException: Password cannot be null if SASL is enabled".  Right now the spark shuffle service relies on the Yarn nodemanager to re-register the applications, unfortunately this is after we open the port for other executors to connect. If other executors connected before the re-register they get a null pointer exception which isn't a re-tryable exception and cause them to fail pretty quickly. To solve this I added another leveldb file so that it can save and re-initialize all the applications before opening the port for other executors to connect to it.  Adding another leveldb was simpler from the code structure point of view.

Most of the code changes are moving things to common util class.

Patch was tested manually on a Yarn cluster with rolling upgrade was happing while spark job was running. Without the patch I consistently get the NullPointerException, with the patch the job gets a few Connection refused exceptions but the retries kick in and the it succeeds.

Author: Thomas Graves <tgraves@staydecay.corp.gq1.yahoo.com>

Closes #14718 from tgravescs/SPARK-16711.
2016-09-02 10:42:13 -07:00
Sean Owen 5d84c7fd83 [SPARK-17332][CORE] Make Java Loggers static members
## What changes were proposed in this pull request?

Make all Java Loggers static members

## How was this patch tested?

Jenkins

Author: Sean Owen <sowen@cloudera.com>

Closes #14896 from srowen/SPARK-17332.
2016-08-31 11:09:14 -07:00
Shixiong Zhu 02ac379e86 [SPARK-17314][CORE] Use Netty's DefaultThreadFactory to enable its fast ThreadLocal impl
## What changes were proposed in this pull request?

When a thread is a Netty's FastThreadLocalThread, Netty will use its fast ThreadLocal implementation. It has a better performance than JDK's (See the benchmark results in https://github.com/netty/netty/pull/4417, note: it's not a fix to Netty's FastThreadLocal. It just fixed an issue in Netty's benchmark codes)

This PR just changed the ThreadFactory to Netty's DefaultThreadFactory which will use FastThreadLocalThread. There is also a minor change to the thread names. See https://github.com/netty/netty/blob/netty-4.0.22.Final/common/src/main/java/io/netty/util/concurrent/DefaultThreadFactory.java#L94

## How was this patch tested?

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #14879 from zsxwing/netty-thread.
2016-08-30 13:22:21 -07:00
Michael Allman f209310719 [SPARK-17231][CORE] Avoid building debug or trace log messages unless the respective log level is enabled
(This PR addresses https://issues.apache.org/jira/browse/SPARK-17231)

## What changes were proposed in this pull request?

While debugging the performance of a large GraphX connected components computation, we found several places in the `network-common` and `network-shuffle` code bases where trace or debug log messages are constructed even if the respective log level is disabled. According to YourKit, these constructions were creating substantial churn in the eden region. Refactoring the respective code to avoid these unnecessary constructions except where necessary led to a modest but measurable reduction in our job's task time, GC time and the ratio thereof.

## How was this patch tested?

We computed the connected components of a graph with about 2.6 billion vertices and 1.7 billion edges four times. We used four different EC2 clusters each with 8 r3.8xl worker nodes. Two test runs used Spark master. Two used Spark master + this PR. The results from the first test run, master and master+PR:
![master](https://cloud.githubusercontent.com/assets/833693/17951634/7471cbca-6a18-11e6-9c26-78afe9319685.jpg)
![logging_perf_improvements](https://cloud.githubusercontent.com/assets/833693/17951632/7467844e-6a18-11e6-9a0e-053dc7650413.jpg)

The results from the second test run, master and master+PR:
![master 2](https://cloud.githubusercontent.com/assets/833693/17951633/746dd6aa-6a18-11e6-8e27-606680b3f105.jpg)
![logging_perf_improvements 2](https://cloud.githubusercontent.com/assets/833693/17951631/74488710-6a18-11e6-8a32-08692f373386.jpg)

Though modest, I believe these results are significant.

Author: Michael Allman <michael@videoamp.com>

Closes #14798 from mallman/spark-17231-logging_perf_improvements.
2016-08-25 11:57:38 -07:00
Sital Kedia 9c15d079df [SPARK-15074][SHUFFLE] Cache shuffle index file to speedup shuffle fetch
## What changes were proposed in this pull request?

Shuffle fetch on large intermediate dataset is slow because the shuffle service open/close the index file for each shuffle fetch. This change introduces a cache for the index information so that we can avoid accessing the index files for each block fetch

## How was this patch tested?

Tested by running a job on the cluster and the shuffle read time was reduced by 50%.

Author: Sital Kedia <skedia@fb.com>

Closes #12944 from sitalkedia/shuffle_service.
2016-08-04 14:54:38 -07:00
Xin Ren 21a6dd2aef [SPARK-16535][BUILD] In pom.xml, remove groupId which is redundant definition and inherited from the parent
https://issues.apache.org/jira/browse/SPARK-16535

## What changes were proposed in this pull request?

When I scan through the pom.xml of sub projects, I found this warning as below and attached screenshot
```
Definition of groupId is redundant, because it's inherited from the parent
```
![screen shot 2016-07-13 at 3 13 11 pm](https://cloud.githubusercontent.com/assets/3925641/16823121/744f893e-4916-11e6-8a52-042f83b9db4e.png)

I've tried to remove some of the lines with groupId definition, and the build on my local machine is still ok.
```
<groupId>org.apache.spark</groupId>
```
As I just find now `<maven.version>3.3.9</maven.version>` is being used in Spark 2.x, and Maven-3 supports versionless parent elements: Maven 3 will remove the need to specify the parent version in sub modules. THIS is great (in Maven 3.1).

ref: http://stackoverflow.com/questions/3157240/maven-3-worth-it/3166762#3166762

## How was this patch tested?

I've tested by re-building the project, and build succeeded.

Author: Xin Ren <iamshrek@126.com>

Closes #14189 from keypointt/SPARK-16535.
2016-07-19 11:59:46 +01:00
Reynold Xin ffcb6e055a [SPARK-16477] Bump master version to 2.1.0-SNAPSHOT
## What changes were proposed in this pull request?
After SPARK-16476 (committed earlier today as #14128), we can finally bump the version number.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #14130 from rxin/SPARK-16477.
2016-07-11 09:42:56 -07:00
Ryan Blue 67e085ef6d [SPARK-16420] Ensure compression streams are closed.
## What changes were proposed in this pull request?

This uses the try/finally pattern to ensure streams are closed after use. `UnsafeShuffleWriter` wasn't closing compression streams, causing them to leak resources until garbage collected. This was causing a problem with codecs that use off-heap memory.

## How was this patch tested?

Current tests are sufficient. This should not change behavior.

Author: Ryan Blue <blue@apache.org>

Closes #14093 from rdblue/SPARK-16420-unsafe-shuffle-writer-leak.
2016-07-08 12:37:26 -07:00
Sean Owen 4768d037b7 [SPARK-15386][CORE] Master doesn't compile against Java 1.7 / Process.isAlive
## What changes were proposed in this pull request?

Remove call to Process.isAlive -- Java 8 only. Introduced in https://github.com/apache/spark/pull/13042 / SPARK-15263

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #13174 from srowen/SPARK-15386.
2016-05-18 14:22:21 -07:00
Tejas Patil c1fd9cacba [SPARK-15263][CORE] Make shuffle service dir cleanup faster by using rm -rf
## What changes were proposed in this pull request?

Jira: https://issues.apache.org/jira/browse/SPARK-15263

The current logic for directory cleanup is slow because it does directory listing, recurses over child directories, checks for symbolic links, deletes leaf files and finally deletes the dirs when they are empty. There is back-and-forth switching from kernel space to user space while doing this. Since most of the deployment backends would be Unix systems, we could essentially just do `rm -rf` so that entire deletion logic runs in kernel space.

The current Java based impl in Spark seems to be similar to what standard libraries like guava and commons IO do (eg. http://svn.apache.org/viewvc/commons/proper/io/trunk/src/main/java/org/apache/commons/io/FileUtils.java?view=markup#l1540). However, guava removed this method in favour of shelling out to an operating system command (like in this PR). See the `Deprecated` note in older javadocs for guava for details : http://google.github.io/guava/releases/10.0.1/api/docs/com/google/common/io/Files.html#deleteRecursively(java.io.File)

Ideally, Java should be providing such APIs so that users won't have to do such things to get platform specific code. Also, its not just about speed, but also handling race conditions while doing at FS deletions is tricky. I could find this bug for Java in similar context : http://bugs.java.com/bugdatabase/view_bug.do?bug_id=7148952

## How was this patch tested?

I am relying on existing test cases to test the method. If there are suggestions about testing it, welcome to hear about it.

## Performance gains

*Input setup* : Created a nested directory structure of depth 3 and each entry having 50 sub-dirs. The input being cleaned up had total ~125k dirs.

Ran both approaches (in isolation) for 6 times to get average numbers:

Native Java cleanup  | `rm -rf` as a separate process
------------ | -------------
10.04 sec | 4.11 sec

This change made deletion 2.4 times faster for the given test input.

Author: Tejas Patil <tejasp@fb.com>

Closes #13042 from tejasapatil/delete_recursive.
2016-05-18 12:10:32 +01:00
Sean Owen 122302cbf5 [SPARK-15290][BUILD] Move annotations, like @Since / @DeveloperApi, into spark-tags
## What changes were proposed in this pull request?

(See https://github.com/apache/spark/pull/12416 where most of this was already reviewed and committed; this is just the module structure and move part. This change does not move the annotations into test scope, which was the apparently problem last time.)

Rename `spark-test-tags` -> `spark-tags`; move common annotations like `Since` to `spark-tags`

## How was this patch tested?

Jenkins tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #13074 from srowen/SPARK-15290.
2016-05-17 09:55:53 +01:00
Sandeep Singh 6e268b9ee3 [SPARK-15178][CORE] Remove LazyFileRegion instead use netty's DefaultFileRegion
## What changes were proposed in this pull request?
Remove LazyFileRegion instead use netty's DefaultFileRegion, since It was created so that we didn't create a file descriptor before having to send the file.

## How was this patch tested?
Existing tests

Author: Sandeep Singh <sandeep@techaddict.me>

Closes #12977 from techaddict/SPARK-15178.
2016-05-07 12:36:43 -07:00
Yin Huai 9c7c42bc6a Revert "[SPARK-14613][ML] Add @Since into the matrix and vector classes in spark-mllib-local"
This reverts commit dae538a4d7.
2016-04-28 19:57:41 -07:00
Pravin Gadakh dae538a4d7 [SPARK-14613][ML] Add @Since into the matrix and vector classes in spark-mllib-local
## What changes were proposed in this pull request?

This PR adds `since` tag into the matrix and vector classes in spark-mllib-local.

## How was this patch tested?

Scala-style checks passed.

Author: Pravin Gadakh <prgadakh@in.ibm.com>

Closes #12416 from pravingadakh/SPARK-14613.
2016-04-28 15:59:18 -07:00
Reynold Xin c439d88e99 [SPARK-14547] Avoid DNS resolution for reusing connections
## What changes were proposed in this pull request?
This patch changes the connection creation logic in the network client module to avoid DNS resolution when reusing connections.

## How was this patch tested?
Testing in production. This is too difficult to test in isolation (for high fidelity unit tests, we'd need to change the DNS resolution behavior in the JVM).

Author: Reynold Xin <rxin@databricks.com>

Closes #12315 from rxin/SPARK-14547.
2016-04-12 15:28:08 -07:00
Zhang, Liye c4bb02abf2 [SPARK-14290][CORE][NETWORK] avoid significant memory copy in netty's transferTo
## What changes were proposed in this pull request?
When netty transfer data that is not `FileRegion`, data will be in format of `ByteBuf`, If the data is large, there will occur significant performance issue because there is memory copy underlying in `sun.nio.ch.IOUtil.write`, the CPU is 100% used, and network is very low.

In this PR, if data size is large, we will split it into small chunks to call `WritableByteChannel.write()`, so that avoid wasting of memory copy. Because the data can't be written within a single write, and it will call `transferTo` multiple times.

## How was this patch tested?
Spark unit test and manual test.
Manual test:
`sc.parallelize(Array(1,2,3),3).mapPartitions(a=>Array(new Array[Double](1024 * 1024 * 50)).iterator).reduce((a,b)=> a).length`

For more details, please refer to [SPARK-14290](https://issues.apache.org/jira/browse/SPARK-14290)

Author: Zhang, Liye <liye.zhang@intel.com>

Closes #12083 from liyezhang556520/spark-14290.
2016-04-06 16:11:59 -07:00
Dongjoon Hyun 3f749f7ed4 [SPARK-14355][BUILD] Fix typos in Exception/Testcase/Comments and static analysis results
## What changes were proposed in this pull request?

This PR contains the following 5 types of maintenance fix over 59 files (+94 lines, -93 lines).
- Fix typos(exception/log strings, testcase name, comments) in 44 lines.
- Fix lint-java errors (MaxLineLength) in 6 lines. (New codes after SPARK-14011)
- Use diamond operators in 40 lines. (New codes after SPARK-13702)
- Fix redundant semicolon in 5 lines.
- Rename class `InferSchemaSuite` to `CSVInferSchemaSuite` in CSVInferSchemaSuite.scala.

## How was this patch tested?

Manual and pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #12139 from dongjoon-hyun/SPARK-14355.
2016-04-03 18:14:16 -07:00
Zhang, Liye 96941b12f8 [SPARK-14242][CORE][NETWORK] avoid copy in compositeBuffer for frame decoder
## What changes were proposed in this pull request?
In this patch, we set the initial `maxNumComponents` to `Integer.MAX_VALUE` instead of the default size ( which is 16) when allocating `compositeBuffer` in `TransportFrameDecoder` because `compositeBuffer` will introduce too many memory copies underlying if `compositeBuffer` is with default `maxNumComponents` when the frame size is large (which result in many transport messages). For details, please refer to [SPARK-14242](https://issues.apache.org/jira/browse/SPARK-14242).

## How was this patch tested?
spark unit tests and manual tests.
For manual tests, we can reproduce the performance issue with following code:
`sc.parallelize(Array(1,2,3),3).mapPartitions(a=>Array(new Array[Double](1024 * 1024 * 50)).iterator).reduce((a,b)=> a).length`
It's easy to see the performance gain, both from the running time and CPU usage.

Author: Zhang, Liye <liye.zhang@intel.com>

Closes #12038 from liyezhang556520/spark-14242.
2016-03-31 20:17:52 -07:00