Commit graph

5415 commits

Author SHA1 Message Date
Liang-Chi Hsieh 68c0c460bf [SPARK-13742] [CORE] Add non-iterator interface to RandomSampler
JIRA: https://issues.apache.org/jira/browse/SPARK-13742

## What changes were proposed in this pull request?

`RandomSampler.sample` currently accepts iterator as input and output another iterator. This makes it inappropriate to use in wholestage codegen of `Sampler` operator #11517. This change is to add non-iterator interface to `RandomSampler`.

This change adds a new method `def sample(): Int` to the trait `RandomSampler`. As we don't need to know the actual values of the sampling items, so this new method takes no arguments.

This method will decide whether to sample the next item or not. It returns how many times the next item will be sampled.

For `BernoulliSampler` and `BernoulliCellSampler`, the returned sampling times can only be 0 or 1. It simply means whether to sample the next item or not.

For `PoissonSampler`, the returned value can be more than 1, meaning the next item will be sampled multiple times.

## How was this patch tested?

Tests are added into `RandomSamplerSuite`.

Author: Liang-Chi Hsieh <simonh@tw.ibm.com>
Author: Liang-Chi Hsieh <viirya@appier.com>
Author: Liang-Chi Hsieh <viirya@gmail.com>

Closes #11578 from viirya/random-sampler-no-iterator.
2016-03-28 09:58:47 -07:00
Josh Rosen 20c0bcd972 [SPARK-14135] Add off-heap storage memory bookkeeping support to MemoryManager
This patch extends Spark's `UnifiedMemoryManager` to add bookkeeping support for off-heap storage memory, an requirement for enabling off-heap caching (which will be done by #11805). The `MemoryManager`'s `storageMemoryPool` has been split into separate on- and off-heap pools and the storage and unroll memory allocation methods have been updated to accept a `memoryMode` parameter to specify whether allocations should be performed on- or off-heap.

In order to reduce the testing surface, the `StaticMemoryManager` does not support off-heap caching (we plan to eventually remove the `StaticMemoryManager`, so this isn't a significant limitation).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11942 from JoshRosen/off-heap-storage-memory-bookkeeping.
2016-03-26 11:03:25 -07:00
Liwei Lin 62a85eb09f [SPARK-14089][CORE][MLLIB] Remove methods that has been deprecated since 1.1, 1.2, 1.3, 1.4, and 1.5
## What changes were proposed in this pull request?

Removed methods that has been deprecated since 1.1, 1.2, 1.3, 1.4, and 1.5.

## How was this patch tested?

- manully checked that no codes in Spark call these methods any more
- existing test suits

Author: Liwei Lin <lwlin7@gmail.com>
Author: proflin <proflin.me@gmail.com>

Closes #11910 from lw-lin/remove-deprecates.
2016-03-26 12:41:34 +00:00
Dongjoon Hyun 1808465855 [MINOR] Fix newly added java-lint errors
## What changes were proposed in this pull request?

This PR fixes some newly added java-lint errors(unused-imports, line-lengsth).

## How was this patch tested?

Pass the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11968 from dongjoon-hyun/SPARK-14167.
2016-03-26 11:55:49 +00:00
Rajesh Balamohan ff7cc45f52 [SPARK-14091][CORE] Improve performance of SparkContext.getCallSite()
Currently SparkContext.getCallSite() makes a call to Utils.getCallSite().

```
 private[spark] def getCallSite(): CallSite = {
    val callSite = Utils.getCallSite()
    CallSite(
      Option(getLocalProperty(CallSite.SHORT_FORM)).getOrElse(callSite.shortForm),
      Option(getLocalProperty(CallSite.LONG_FORM)).getOrElse(callSite.longForm)
    )
  }
```
However, in some places utils.withDummyCallSite(sc) is invoked to avoid expensive threaddumps within getCallSite(). But Utils.getCallSite() is evaluated earlier causing threaddumps to be computed.

This can have severe impact on smaller queries (that finish in 10-20 seconds) having large number of RDDs.

Creating this patch for lazy evaluation of  getCallSite.

No new test cases are added. Following standalone test was tried out manually. Also, built entire spark binary and tried with few SQL queries in TPC-DS  and TPC-H in multi node cluster
```
def run(): Unit = {
    val conf = new SparkConf()
    val sc = new SparkContext("local[1]", "test-context", conf)
    val start: Long = System.currentTimeMillis();
    val confBroadcast = sc.broadcast(new SerializableConfiguration(new Configuration()))
    Utils.withDummyCallSite(sc) {
      //Large tables end up creating 5500 RDDs
      for(i <- 1 to 5000) {
       //ignore nulls in RDD as its mainly for testing callSite
        val testRDD = new HadoopRDD(sc, confBroadcast, None, null,
          classOf[NullWritable], classOf[Writable], 10)
      }
    }
    val end: Long = System.currentTimeMillis();
    println("Time taken : " + (end - start))
  }

def main(args: Array[String]): Unit = {
    run
  }
```

Author: Rajesh Balamohan <rbalamohan@apache.org>

Closes #11911 from rajeshbalamohan/SPARK-14091.
2016-03-25 15:09:52 -07:00
Reynold Xin 70a6f0bb57 [SPARK-14149] Log exceptions in tryOrIOException
## What changes were proposed in this pull request?
We ran into a problem today debugging some class loading problem during deserialization, and JVM was masking the underlying exception which made it very difficult to debug. We can however log the exceptions using try/catch ourselves in serialization/deserialization. The good thing is that all these methods are already using Utils.tryOrIOException, so we can just put the try catch and logging in a single place.

## How was this patch tested?
A logging change with a manual test.

Author: Reynold Xin <rxin@databricks.com>

Closes #11951 from rxin/SPARK-14149.
2016-03-25 01:17:23 -07:00
Josh Rosen fdd460f5f4 [SPARK-13980] Incrementally serialize blocks while unrolling them in MemoryStore
When a block is persisted in the MemoryStore at a serialized storage level, the current MemoryStore.putIterator() code will unroll the entire iterator as Java objects in memory, then will turn around and serialize an iterator obtained from the unrolled array. This is inefficient and doubles our peak memory requirements.

Instead, I think that we should incrementally serialize blocks while unrolling them.

A downside to incremental serialization is the fact that we will need to deserialize the partially-unrolled data in case there is not enough space to unroll the block and the block cannot be dropped to disk. However, I'm hoping that the memory efficiency improvements will outweigh any performance losses as a result of extra serialization in that hopefully-rare case.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11791 from JoshRosen/serialize-incrementally.
2016-03-24 17:33:21 -07:00
Sean Owen 342079dc45 Revert "[SPARK-2208] Fix for local metrics tests can fail on fast machines". The test appears to still be flaky after this change, or more flaky.
This reverts commit 5519760e0f.
2016-03-24 17:27:20 +00:00
Joan 5519760e0f [SPARK-2208] Fix for local metrics tests can fail on fast machines
## What changes were proposed in this pull request?

A fix for local metrics tests that can fail on fast machines.
This is probably what is suggested here #3380 by aarondav?

## How was this patch tested?

CI Tests

Cheers

Author: Joan <joan@goyeau.com>

Closes #11747 from joan38/SPARK-2208-Local-metrics-tests.
2016-03-24 09:47:44 +00:00
Tejas Patil 01849da080 [SPARK-14110][CORE] PipedRDD to print the command ran on non zero exit
## What changes were proposed in this pull request?

In case of failure in subprocess launched in PipedRDD, the failure exception reads “Subprocess exited with status XXX”. Debugging this is not easy for users especially if there are multiple pipe() operations in the Spark application.

Changes done:
- Changed the exception message when non-zero exit code is seen
- If the reader and writer threads see exception, simply logging the command ran. The current model is to propagate the exception "as is" so that upstream Spark logic will take the right action based on what the exception was (eg. for fetch failure, it needs to retry; but for some fatal exception, it will decide to fail the stage / job). So wrapping the exception with a generic exception will not work. Altering the exception message will keep that guarantee but that is ugly (plus not all exceptions might have a constructor for a string message)

## How was this patch tested?

- Added a new test case
- Ran all existing tests for PipedRDD

Author: Tejas Patil <tejasp@fb.com>

Closes #11927 from tejasapatil/SPARK-14110-piperdd-failure.
2016-03-24 00:31:13 -07:00
Liwei Lin de4e48b62b [SPARK-14025][STREAMING][WEBUI] Fix streaming job descriptions on the event timeline
## What changes were proposed in this pull request?

Removed the extra `<a href=...>...</a>` for each streaming job's description on the event timeline.

### [Before]
![before](https://cloud.githubusercontent.com/assets/15843379/13898653/0a6c1838-ee13-11e5-9761-14bb7b114c13.png)

### [After]
![after](https://cloud.githubusercontent.com/assets/15843379/13898650/012b8808-ee13-11e5-92a6-64aff0799c83.png)

## How was this patch tested?

test suits, manual checks (see screenshots above)

Author: Liwei Lin <proflin.me@gmail.com>
Author: proflin <proflin.me@gmail.com>

Closes #11845 from lw-lin/description-event-line.
2016-03-23 15:15:55 -07:00
Ernest 48ee16d801 [SPARK-14055] writeLocksByTask need to be update when removeBlock
## What changes were proposed in this pull request?

https://issues.apache.org/jira/browse/SPARK-14055

## How was this patch tested?

manual tests by running LiveJournalPageRank on a large dataset ( the dataset must larger enough to incure RDD partition eviction).

Author: Ernest <earneyzxl@gmail.com>

Closes #11875 from Earne/issue-14055.
2016-03-23 10:29:36 -07:00
Josh Rosen 3de24ae2ed [SPARK-14075] Refactor MemoryStore to be testable independent of BlockManager
This patch refactors the `MemoryStore` so that it can be tested without needing to construct / mock an entire `BlockManager`.

- The block manager's serialization- and compression-related methods have been moved from `BlockManager` to `SerializerManager`.
- `BlockInfoManager `is now passed directly to classes that need it, rather than being passed via the `BlockManager`.
- The `MemoryStore` now calls `dropFromMemory` via a new `BlockEvictionHandler` interface rather than directly calling the `BlockManager`. This change helps to enforce a narrow interface between the `MemoryStore` and `BlockManager` functionality and makes this interface easier to mock in tests.
- Several of the block unrolling tests have been moved from `BlockManagerSuite` into a new `MemoryStoreSuite`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11899 from JoshRosen/reduce-memorystore-blockmanager-coupling.
2016-03-23 10:15:23 -07:00
Kazuaki Ishizaki 0d51b60443 [SPARK-14072][CORE] Show JVM/OS version information when we run a benchmark program
## What changes were proposed in this pull request?

This PR allows us to identify what JVM is used when someone ran a benchmark program. In some cases, a JVM version may affect performance result. Thus, it would be good to show processor information and JVM version information.

```
model name	: Intel(R) Xeon(R) CPU E5-2697 v2  2.70GHz
JVM information : OpenJDK 64-Bit Server VM, 1.7.0_65-mockbuild_2014_07_14_06_19-b00
Int and String Scan:                Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
SQL Parquet Vectorized                    981 /  994         10.7          93.5       1.0X
SQL Parquet MR                           2518 / 2542          4.2         240.1       0.4X
```

```
model name	: Intel(R) Xeon(R) CPU E5-2697 v2  2.70GHz
JVM information : IBM J9 VM, pxa6480sr2-20151023_01 (SR2)
String Dictionary:                  Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
-------------------------------------------------------------------------------------------
SQL Parquet Vectorized                    693 /  740         15.1          66.1       1.0X
SQL Parquet MR                           2501 / 2562          4.2         238.5       0.3X
```

## How was this patch tested?

Tested by using existing benchmark programs

(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

Closes #11893 from kiszk/SPARK-14072.
2016-03-22 21:01:52 -07:00
Josh Rosen b5f1ab701a [SPARK-13990] Automatically pick serializer when caching RDDs
Building on the `SerializerManager` introduced in SPARK-13926/ #11755, this patch Spark modifies Spark's BlockManager to use RDD's ClassTags in order to select the best serializer to use when caching RDD blocks.

When storing a local block, the BlockManager `put()` methods use implicits to record ClassTags and stores those tags in the blocks' BlockInfo records. When reading a local block, the stored ClassTag is used to pick the appropriate serializer. When a block is stored with replication, the class tag is written into the block transfer metadata and will also be stored in the remote BlockManager.

There are two or three places where we don't properly pass ClassTags, including TorrentBroadcast and BlockRDD. I think this happens to work because the missing ClassTag always happens to be `ClassTag.Any`, but it might be worth looking more carefully at those places to see whether we should be more explicit.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11801 from JoshRosen/pick-best-serializer-for-caching.
2016-03-21 17:19:39 -07:00
Davies Liu 9b4e15ba13 [SPARK-14007] [SQL] Manage the memory used by hash map in shuffled hash join
## What changes were proposed in this pull request?

This PR try acquire the memory for hash map in shuffled hash join, fail the task if there is no enough memory (otherwise it could OOM the executor).

It also removed unused HashedRelation.

## How was this patch tested?

Existing unit tests. Manual tests with TPCDS Q78.

Author: Davies Liu <davies@databricks.com>

Closes #11826 from davies/cleanup_hash2.
2016-03-21 11:21:39 -07:00
Dongjoon Hyun df61fbd978 [SPARK-13986][CORE][MLLIB] Remove DeveloperApi-annotations for non-publics
## What changes were proposed in this pull request?

Spark uses `DeveloperApi` annotation, but sometimes it seems to conflict with visibility. This PR tries to fix those conflict by removing annotations for non-publics. The following is the example.

**JobResult.scala**
```scala
DeveloperApi
sealed trait JobResult

DeveloperApi
case object JobSucceeded extends JobResult

-DeveloperApi
private[spark] case class JobFailed(exception: Exception) extends JobResult
```

## How was this patch tested?

Pass the existing Jenkins test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11797 from dongjoon-hyun/SPARK-13986.
2016-03-21 14:57:52 +00:00
Dongjoon Hyun 761c2d1b6e [MINOR][DOCS] Add proper periods and spaces for CLI help messages and config doc.
## What changes were proposed in this pull request?

This PR adds some proper periods and spaces to Spark CLI help messages and SQL/YARN conf docs for consistency.

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11848 from dongjoon-hyun/add_proper_period_and_space.
2016-03-21 08:00:09 +00:00
Dongjoon Hyun 20fd254101 [SPARK-14011][CORE][SQL] Enable LineLength Java checkstyle rule
## What changes were proposed in this pull request?

[Spark Coding Style Guide](https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide) has 100-character limit on lines, but it's disabled for Java since 11/09/15. This PR enables **LineLength** checkstyle again. To help that, this also introduces **RedundantImport** and **RedundantModifier**, too. The following is the diff on `checkstyle.xml`.

```xml
-        <!-- TODO: 11/09/15 disabled - the lengths are currently > 100 in many places -->
-        <!--
         <module name="LineLength">
             <property name="max" value="100"/>
             <property name="ignorePattern" value="^package.*|^import.*|a href|href|http://|https://|ftp://"/>
         </module>
-        -->
         <module name="NoLineWrap"/>
         <module name="EmptyBlock">
             <property name="option" value="TEXT"/>
 -167,5 +164,7
         </module>
         <module name="CommentsIndentation"/>
         <module name="UnusedImports"/>
+        <module name="RedundantImport"/>
+        <module name="RedundantModifier"/>
```

## How was this patch tested?

Currently, `lint-java` is disabled in Jenkins. It needs a manual test.
After passing the Jenkins tests, `dev/lint-java` should passes locally.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11831 from dongjoon-hyun/SPARK-14011.
2016-03-21 07:58:57 +00:00
Sital Kedia 2e0c5284fd [SPARK-13958] Executor OOM due to unbounded growth of pointer array in…
## What changes were proposed in this pull request?

This change fixes the executor OOM which was recently introduced in PR apache/spark#11095
(Please fill in changes proposed in this fix)

## How was this patch tested?
Tested by running a spark job on the cluster.
(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)

… Sorter

Author: Sital Kedia <skedia@fb.com>

Closes #11794 from sitalkedia/SPARK-13958.
2016-03-18 12:56:06 -07:00
jerryshao 3537782168 [SPARK-13885][YARN] Fix attempt id regression for Spark running on Yarn
## What changes were proposed in this pull request?

This regression is introduced in #9182, previously attempt id is simply as counter "1" or "2". With the change of #9182, it is changed to full name as "appattemtp-xxx-00001", this will affect all the parts which uses this attempt id, like event log file name, history server app url link. So here change it back to the counter to keep consistent with previous code.

Also revert back this patch #11518, this patch fix the url link of history log according to the new way of attempt id, since here we change back to the previous way, so this patch is not necessary, here to revert it.

Also clean "spark.yarn.app.id" and "spark.yarn.app.attemptId", since it is useless now.

## How was this patch tested?

Test it with unit test and manually test different scenario:

1. application running in yarn-client mode.
2. application running in yarn-cluster mode.
3. application running in yarn-cluster mode with multiple attempts.

Checked both the event log file name and url link.

CC vanzin tgravescs , please help to review, thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #11721 from jerryshao/SPARK-13885.
2016-03-18 12:39:49 -07:00
Josh Rosen 6c2d894a2f [SPARK-13921] Store serialized blocks as multiple chunks in MemoryStore
This patch modifies the BlockManager, MemoryStore, and several other storage components so that serialized cached blocks are stored as multiple small chunks rather than as a single contiguous ByteBuffer.

This change will help to improve the efficiency of memory allocation and the accuracy of memory accounting when serializing blocks. Our current serialization code uses a ByteBufferOutputStream, which doubles and re-allocates its backing byte array; this increases the peak memory requirements during serialization (since we need to hold extra memory while expanding the array). In addition, we currently don't account for the extra wasted space at the end of the ByteBuffer's backing array, so a 129 megabyte serialized block may actually consume 256 megabytes of memory. After switching to storing blocks in multiple chunks, we'll be able to efficiently trim the backing buffers so that no space is wasted.

This change is also a prerequisite to being able to cache blocks which are larger than 2GB (although full support for that depends on several other changes which have not bee implemented yet).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11748 from JoshRosen/chunked-block-serialization.
2016-03-17 20:00:56 -07:00
Shixiong Zhu 65b75e66e8 [SPARK-13776][WEBUI] Limit the max number of acceptors and selectors for Jetty
## What changes were proposed in this pull request?

As each acceptor/selector in Jetty will use one thread, the number of threads should at least be the number of acceptors and selectors plus 1. Otherwise, the thread pool of Jetty server may be exhausted by acceptors/selectors and not be able to response any request.

To avoid wasting threads, the PR limits the max number of acceptors and selectors and also updates the max thread number if necessary.

## How was this patch tested?

Just make sure we don't break any existing tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11615 from zsxwing/SPARK-13776.
2016-03-17 13:05:29 +00:00
Wenchen Fan 8ef3399aff [SPARK-13928] Move org.apache.spark.Logging into org.apache.spark.internal.Logging
## What changes were proposed in this pull request?

Logging was made private in Spark 2.0. If we move it, then users would be able to create a Logging trait themselves to avoid changing their own code.

## How was this patch tested?

existing tests.

Author: Wenchen Fan <wenchen@databricks.com>

Closes #11764 from cloud-fan/logger.
2016-03-17 19:23:38 +08:00
trueyao ea9ca6f04c [SPARK-13901][CORE] correct the logDebug information when jump to the next locality level
JIRA Issue:https://issues.apache.org/jira/browse/SPARK-13901
In getAllowedLocalityLevel method of TaskSetManager,we get wrong logDebug information when jump to the next locality level.So we should fix it.

Author: trueyao <501663994@qq.com>

Closes #11719 from trueyao/logDebug-localityWait.
2016-03-17 09:45:06 +00:00
Josh Rosen de1a84e56e [SPARK-13926] Automatically use Kryo serializer when shuffling RDDs with simple types
Because ClassTags are available when constructing ShuffledRDD we can use them to automatically use Kryo for shuffle serialization when the RDD's types are known to be compatible with Kryo.

This patch introduces `SerializerManager`, a component which picks the "best" serializer for a shuffle given the elements' ClassTags. It will automatically pick a Kryo serializer for ShuffledRDDs whose key, value, and/or combiner types are primitives, arrays of primitives, or strings. In the future we can use this class as a narrow extension point to integrate specialized serializers for other types, such as ByteBuffers.

In a planned followup patch, I will extend the BlockManager APIs so that we're able to use similar automatic serializer selection when caching RDDs (this is a little trickier because the ClassTags need to be threaded through many more places).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11755 from JoshRosen/automatically-pick-best-serializer.
2016-03-16 22:52:55 -07:00
Wesley Tang 5f6bdf97c5 [SPARK-13281][CORE] Switch broadcast of RDD to exception from warning
## What changes were proposed in this pull request?

In SparkContext, throw Illegalargumentexception when trying to broadcast rdd directly, instead of logging the warning.

## How was this patch tested?

mvn clean install
Add UT in BroadcastSuite

Author: Wesley Tang <tangmingjun@mininglamp.com>

Closes #11735 from breakdawn/master.
2016-03-16 16:12:17 +00:00
Tejas Patil 1d95fb6785 [SPARK-13793][CORE] PipedRDD doesn't propagate exceptions while reading parent RDD
## What changes were proposed in this pull request?

PipedRDD creates a child thread to read output of the parent stage and feed it to the pipe process. Used a variable to save the exception thrown in the child thread and then propagating the exception in the main thread if the variable was set.

## How was this patch tested?

- Added a unit test
- Ran all the existing tests in PipedRDDSuite and they all pass with the change
- Tested the patch with a real pipe() job, bounced the executor node which ran the parent stage to simulate a fetch failure and observed that the parent stage was re-ran.

Author: Tejas Patil <tejasp@fb.com>

Closes #11628 from tejasapatil/pipe_rdd.
2016-03-16 09:58:53 +00:00
GayathriMurali 56d88247f1 [SPARK-13396] Stop using our internal deprecated .metrics on Exceptio…
JIRA: https://issues.apache.org/jira/browse/SPARK-13396

Stop using our internal deprecated .metrics on ExceptionFailure instead use accumUpdates

Author: GayathriMurali <gayathri.m.softie@gmail.com>

Closes #11544 from GayathriMurali/SPARK-13396.
2016-03-16 09:39:41 +00:00
Sean Owen 3b461d9ecd [SPARK-13823][SPARK-13397][SPARK-13395][CORE] More warnings, StandardCharset follow up
## What changes were proposed in this pull request?

Follow up to https://github.com/apache/spark/pull/11657

- Also update `String.getBytes("UTF-8")` to use `StandardCharsets.UTF_8`
- And fix one last new Coverity warning that turned up (use of unguarded `wait()` replaced by simpler/more robust `java.util.concurrent` classes in tests)
- And while we're here cleaning up Coverity warnings, just fix about 15 more build warnings

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #11725 from srowen/SPARK-13823.2.
2016-03-16 09:36:34 +00:00
Yonathan Randolph 05ab2948ab [SPARK-13906] Ensure that there are at least 2 dispatcher threads.
## What changes were proposed in this pull request?

Force at least two dispatcher-event-loop threads. Since SparkDeploySchedulerBackend (in AppClient) calls askWithRetry to CoarseGrainedScheduler in the same process, there the driver needs at least two dispatcher threads to prevent the dispatcher thread from hanging.

## How was this patch tested?

Manual.

Author: Yonathan Randolph <yonathangmail.com>

Author: Yonathan Randolph <yonathan@liftigniter.com>

Closes #11728 from yonran/SPARK-13906.
2016-03-16 09:34:04 +00:00
Marcelo Vanzin 41eaabf593 [SPARK-13626][CORE] Revert change to SparkConf's constructor.
It shouldn't be private.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #11734 from vanzin/SPARK-13626-api.
2016-03-15 14:51:25 -07:00
CodingCat dddf2f2d87 [MINOR] a minor fix for the comments of a method in RPC Dispatcher
## What changes were proposed in this pull request?

a minor fix for the comments of a method in RPC Dispatcher

## How was this patch tested?

existing unit tests

Author: CodingCat <zhunansjtu@gmail.com>

Closes #11738 from CodingCat/minor_rpc.
2016-03-15 14:46:21 -07:00
CodingCat bd5365bbe9 [SPARK-13803] restore the changes in SPARK-3411
## What changes were proposed in this pull request?

This patch contains the functionality to balance the load of the cluster-mode drivers among workers

This patch restores the changes in https://github.com/apache/spark/pull/1106 which was erased due to the merging of https://github.com/apache/spark/pull/731

## How was this patch tested?

test with existing test cases

Author: CodingCat <zhunansjtu@gmail.com>

Closes #11702 from CodingCat/SPARK-13803.
2016-03-15 10:10:23 +00:00
Marcelo Vanzin 8301fadd8d [SPARK-13626][CORE] Avoid duplicate config deprecation warnings.
Three different things were needed to get rid of spurious warnings:
- silence deprecation warnings when cloning configuration
- change the way SparkHadoopUtil instantiates SparkConf to silence
  warnings
- avoid creating new SparkConf instances where it's not needed.

On top of that, I changed the way that Logging.scala detects the repl;
now it uses a method that is overridden in the repl's Main class, and
the hack in Utils.scala is not needed anymore. This makes the 2.11 repl
behave like the 2.10 one and set the default log level to WARN, which
is a lot better. Previously, this wasn't working because the 2.11 repl
triggers log initialization earlier than the 2.10 one.

I also removed and simplified some other code in the 2.11 repl's Main
to avoid replicating logic that already exists elsewhere in Spark.

Tested the 2.11 repl in local and yarn modes.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #11510 from vanzin/SPARK-13626.
2016-03-14 14:27:33 -07:00
Josh Rosen 38529d8f23 [SPARK-10907][SPARK-6157] Remove pendingUnrollMemory from MemoryStore
This patch refactors the MemoryStore to remove the concept of `pendingUnrollMemory`. It also fixes fixes SPARK-6157: "Unrolling with MEMORY_AND_DISK should always release memory".

Key changes:

- Inline `MemoryStore.tryToPut` at its three call sites in the `MemoryStore`.
- Inline `Memory.unrollSafely` at its only call site (in `MemoryStore.putIterator`).
- Inline `MemoryManager.acquireStorageMemory` at its call sites.
- Simplify the code as a result of this inlining (some parameters have fixed values after inlining, so lots of branches can be removed).
- Remove the `pendingUnrollMemory` map by returning the amount of unrollMemory allocated when returning an iterator after a failed `putIterator` call.
- Change `putIterator` to return an instance of `PartiallyUnrolledIterator`, a special iterator subclass which will automatically free the unroll memory of its partially-unrolled elements when the iterator is consumed. To handle cases where the iterator is not consumed (e.g. when a MEMORY_ONLY put fails), `PartiallyUnrolledIterator` exposes a `close()` method which may be called to discard the unrolled values and free their memory.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11613 from JoshRosen/cleanup-unroll-memory.
2016-03-14 14:26:39 -07:00
Thomas Graves 23385e853e [SPARK-13054] Always post TaskEnd event for tasks
I am using dynamic container allocation and speculation and am seeing issues with the active task accounting. The Executor UI still shows active tasks on the an executor but the job/stage is all completed. I think its also affecting the dynamic allocation being able to release containers because it thinks there are still tasks.
There are multiple issues with this:
-  If the task end for tasks (in this case probably because of speculation) comes in after the stage is finished, then the DAGScheduler.handleTaskCompletion will skip the task completion event

Author: Thomas Graves <tgraves@prevailsail.corp.gq1.yahoo.com>
Author: Thomas Graves <tgraves@staydecay.corp.gq1.yahoo.com>
Author: Tom Graves <tgraves@yahoo-inc.com>

Closes #10951 from tgravescs/SPARK-11701.
2016-03-14 12:31:46 -07:00
Bertrand Bossy 310981d49a [SPARK-12583][MESOS] Mesos shuffle service: Don't delete shuffle files before application has stopped
## Problem description:

Mesos shuffle service is completely unusable since Spark 1.6.0 . The problem seems to occur since the move from akka to netty in the networking layer. Until now, a connection from the driver to each shuffle service was used as a signal for the shuffle service to determine, whether the driver is still running. Since 1.6.0, this connection is closed after spark.shuffle.io.connectionTimeout (or spark.network.timeout if the former is not set) due to it being idle. The shuffle service interprets this as a signal that the driver has stopped, despite the driver still being alive. Thus, shuffle files are deleted before the application has stopped.

### Context and analysis:

spark shuffle fails with mesos after 2mins: https://issues.apache.org/jira/browse/SPARK-12583
External shuffle service broken w/ Mesos: https://issues.apache.org/jira/browse/SPARK-13159

This is a follow up on #11207 .

## What changes were proposed in this pull request?

This PR adds a heartbeat signal from the Driver (in MesosExternalShuffleClient) to all registered external mesos shuffle service instances. In MesosExternalShuffleBlockHandler, a thread periodically checks whether a driver has timed out and cleans an application's shuffle files if this is the case.

## How was the this patch tested?

This patch has been tested on a small mesos test cluster using the spark-shell. Log output from mesos shuffle service:
```
16/02/19 15:13:45 INFO mesos.MesosExternalShuffleBlockHandler: Received registration request from app 294def07-3249-4e0f-8d71-bf8c83c58a50-0018 (remote address /xxx.xxx.xxx.xxx:52391, heartbeat timeout 120000 ms).
16/02/19 15:13:47 INFO shuffle.ExternalShuffleBlockResolver: Registered executor AppExecId{appId=294def07-3249-4e0f-8d71-bf8c83c58a50-0018, execId=3} with ExecutorShuffleInfo{localDirs=[/foo/blockmgr-c84c0697-a3f9-4f61-9c64-4d3ee227c047], subDirsPerLocalDir=64, shuffleManager=sort}
16/02/19 15:13:47 INFO shuffle.ExternalShuffleBlockResolver: Registered executor AppExecId{appId=294def07-3249-4e0f-8d71-bf8c83c58a50-0018, execId=7} with ExecutorShuffleInfo{localDirs=[/foo/blockmgr-bf46497a-de80-47b9-88f9-563123b59e03], subDirsPerLocalDir=64, shuffleManager=sort}
16/02/19 15:16:02 INFO mesos.MesosExternalShuffleBlockHandler: Application 294def07-3249-4e0f-8d71-bf8c83c58a50-0018 timed out. Removing shuffle files.
16/02/19 15:16:02 INFO shuffle.ExternalShuffleBlockResolver: Application 294def07-3249-4e0f-8d71-bf8c83c58a50-0018 removed, cleanupLocalDirs = true
16/02/19 15:16:02 INFO shuffle.ExternalShuffleBlockResolver: Cleaning up executor AppExecId{appId=294def07-3249-4e0f-8d71-bf8c83c58a50-0018, execId=3}'s 1 local dirs
16/02/19 15:16:02 INFO shuffle.ExternalShuffleBlockResolver: Cleaning up executor AppExecId{appId=294def07-3249-4e0f-8d71-bf8c83c58a50-0018, execId=7}'s 1 local dirs
```
Note: there are 2 executors running on this slave.

Author: Bertrand Bossy <bertrand.bossy@teralytics.net>

Closes #11272 from bbossy/SPARK-12583-mesos-shuffle-service-heartbeat.
2016-03-14 12:22:57 -07:00
Josh Rosen 07cb323e7a [SPARK-13848][SPARK-5185] Update to Py4J 0.9.2 in order to fix classloading issue
This patch upgrades Py4J from 0.9.1 to 0.9.2 in order to include a patch which modifies Py4J to use the current thread's ContextClassLoader when performing reflection / class loading. This is necessary in order to fix [SPARK-5185](https://issues.apache.org/jira/browse/SPARK-5185), a longstanding issue affecting the use of `--jars` and `--packages` in PySpark.

In order to demonstrate that the fix works, I removed the workarounds which were added as part of [SPARK-6027](https://issues.apache.org/jira/browse/SPARK-6027) / #4779 and other patches.

Py4J diff: https://github.com/bartdag/py4j/compare/0.9.1...0.9.2

/cc zsxwing tdas davies brkyvz

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11687 from JoshRosen/py4j-0.9.2.
2016-03-14 12:22:02 -07:00
Josh Rosen 9a87afd7d1 [SPARK-13833] Guard against race condition when re-caching disk blocks in memory
When reading data from the DiskStore and attempting to cache it back into the memory store, we should guard against race conditions where multiple readers are attempting to re-cache the same block in memory.

This patch accomplishes this by synchronizing on the block's `BlockInfo` object while trying to re-cache a block.

(Will file JIRA as soon as ASF JIRA stops being down / laggy).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #11660 from JoshRosen/concurrent-recaching-fixes.
2016-03-14 10:48:24 -07:00
Wilson Wu 31d069d4c2 [SPARK-13746][TESTS] stop using deprecated SynchronizedSet
trait SynchronizedSet in package mutable is deprecated

Author: Wilson Wu <wilson888888888@gmail.com>

Closes #11580 from wilson888888888/spark-synchronizedset.
2016-03-14 09:13:29 +00:00
Dongjoon Hyun acdf219703 [MINOR][DOCS] Fix more typos in comments/strings.
## What changes were proposed in this pull request?

This PR fixes 135 typos over 107 files:
* 121 typos in comments
* 11 typos in testcase name
* 3 typos in log messages

## How was this patch tested?

Manual.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11689 from dongjoon-hyun/fix_more_typos.
2016-03-14 09:07:39 +00:00
Sean Owen 1840852841 [SPARK-13823][CORE][STREAMING][SQL] Always specify Charset in String <-> byte[] conversions (and remaining Coverity items)
## What changes were proposed in this pull request?

- Fixes calls to `new String(byte[])` or `String.getBytes()` that rely on platform default encoding, to use UTF-8
- Same for `InputStreamReader` and `OutputStreamWriter` constructors
- Standardizes on UTF-8 everywhere
- Standardizes specifying the encoding with `StandardCharsets.UTF-8`, not the Guava constant or "UTF-8" (which means handling `UnuspportedEncodingException`)
- (also addresses the other remaining Coverity scan issues, which are pretty trivial; these are separated into commit 1deecd8d9c )

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #11657 from srowen/SPARK-13823.
2016-03-13 21:03:49 -07:00
Bjorn Jonsson 515e4afbc7 [SPARK-13810][CORE] Add Port Configuration Suggestions on Bind Exceptions
## What changes were proposed in this pull request?
Currently, when a java.net.BindException is thrown, it displays the following message:

java.net.BindException: Address already in use: Service '$serviceName' failed after 16 retries!

This change adds port configuration suggestions to the BindException, for example, for the UI, it now displays

java.net.BindException: Address already in use: Service 'SparkUI' failed after 16 retries! Consider explicitly setting the appropriate port for 'SparkUI' (for example spark.ui.port for SparkUI) to an available port or increasing spark.port.maxRetries.

## How was this patch tested?
Manual tests

Author: Bjorn Jonsson <bjornjon@gmail.com>

Closes #11644 from bjornjon/master.
2016-03-13 10:18:24 +00:00
Davies Liu 2ef4c5963b [SPARK-13830] prefer block manager than direct result for large result
## What changes were proposed in this pull request?

The current RPC can't handle large blocks very well, it's very slow to fetch 100M block (about 1 minute). Once switch to block manager to fetch that, it took about 10 seconds (still could be improved).

## How was this patch tested?

existing unit tests.

Author: Davies Liu <davies@databricks.com>

Closes #11659 from davies/direct_result.
2016-03-11 15:39:21 -08:00
Nezih Yigitbasi ff776b2fc1 [SPARK-13328][CORE] Poor read performance for broadcast variables with dynamic resource allocation
When dynamic resource allocation is enabled fetching broadcast variables from removed executors were causing job failures and SPARK-9591 fixed this problem by trying all locations of a block before giving up. However, the locations of a block is retrieved only once from the driver in this process and the locations in this list can be stale due to dynamic resource allocation. This situation gets worse when running on a large cluster as the size of this location list can be in the order of several hundreds out of which there may be tens of stale entries. What we have observed is with the default settings of 3 max retries and 5s between retries (that's 15s per location) the time it takes to read a broadcast variable can be as high as ~17m (70 failed attempts * 15s/attempt)

Author: Nezih Yigitbasi <nyigitbasi@netflix.com>

Closes #11241 from nezihyigitbasi/SPARK-13328.
2016-03-11 11:11:53 -08:00
Marcelo Vanzin 07f1c54477 [SPARK-13577][YARN] Allow Spark jar to be multiple jars, archive.
In preparation for the demise of assemblies, this change allows the
YARN backend to use multiple jars and globs as the "Spark jar". The
config option has been renamed to "spark.yarn.jars" to reflect that.

A second option "spark.yarn.archive" was also added; if set, this
takes precedence and uploads an archive expected to contain the jar
files with the Spark code and its dependencies.

Existing deployments should keep working, mostly. This change drops
support for the "SPARK_JAR" environment variable, and also does not
fall back to using "jarOfClass" if no configuration is set, falling
back to finding files under SPARK_HOME instead. This should be fine
since "jarOfClass" probably wouldn't work unless you were using
spark-submit anyway.

Tested with the unit tests, and trying the different config options
on a YARN cluster.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #11500 from vanzin/SPARK-13577.
2016-03-11 07:54:57 -06:00
Marcelo Vanzin e33bc67c8f [MINOR][CORE] Fix a duplicate "and" in a log message.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #11642 from vanzin/spark-conf-typo.
2016-03-10 22:15:30 -08:00
Shixiong Zhu 27fe6bacc5 [SPARK-13604][CORE] Sync worker's state after registering with master
## What changes were proposed in this pull request?

Here lists all cases that Master cannot talk with Worker for a while and then network is back.

1. Master doesn't know the network issue (not yet timeout)

  a. Worker doesn't know the network issue (onDisconnected is not called)
    - Worker keeps sending Heartbeat. Both Worker and Master don't know the network issue. Nothing to do. (Finally, Master will notice the heartbeat timeout if network is not recovered)

  b. Worker knows the network issue (onDisconnected is called)
    - Worker stops sending Heartbeat and sends `RegisterWorker` to master. Master will reply `RegisterWorkerFailed("Duplicate worker ID")`. Worker calls "System.exit(1)" (Finally, Master will notice the heartbeat timeout if network is not recovered) (May leak driver processes. See [SPARK-13602](https://issues.apache.org/jira/browse/SPARK-13602))

2. Worker timeout (Master knows the network issue). In such case,  master removes Worker and its executors and drivers.

  a. Worker doesn't know the network issue (onDisconnected is not called)
    - Worker keeps sending Heartbeat.
    - If the network is back, say Master receives Heartbeat, Master sends `ReconnectWorker` to Worker
    - Worker send `RegisterWorker` to master.
    - Master accepts `RegisterWorker` but doesn't know executors and drivers in Worker. (may leak executors)

  b. Worker knows the network issue (onDisconnected is called)
    - Worker stop sending `Heartbeat`. Worker will send "RegisterWorker" to master.
    - Master accepts `RegisterWorker` but doesn't know executors and drivers in Worker. (may leak executors)

This PR fixes executors and drivers leak in 2.a and 2.b when Worker reregisters with Master. The approach is making Worker send `WorkerLatestState` to sync the state after registering with master successfully. Then Master will ask Worker to kill unknown executors and drivers.

Note:  Worker cannot just kill executors after registering with master because in the worker, `LaunchExecutor` and `RegisteredWorker` are processed in two threads. If `LaunchExecutor` happens before `RegisteredWorker`, Worker's executor list will contain new executors after Master accepts `RegisterWorker`. We should not kill these executors. So sending the list to Master and let Master tell Worker which executors should be killed.

## How was this patch tested?

test("SPARK-13604: Master should ask Worker kill unknown executors and drivers")

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #11455 from zsxwing/orphan-executors.
2016-03-10 16:59:14 -08:00
Dongjoon Hyun 91fed8e9c5 [SPARK-3854][BUILD] Scala style: require spaces before {.
## What changes were proposed in this pull request?

Since the opening curly brace, '{', has many usages as discussed in [SPARK-3854](https://issues.apache.org/jira/browse/SPARK-3854), this PR adds a ScalaStyle rule to prevent '){' pattern  for the following majority pattern and fixes the code accordingly. If we enforce this in ScalaStyle from now, it will improve the Scala code quality and reduce review time.
```
// Correct:
if (true) {
  println("Wow!")
}

// Incorrect:
if (true){
   println("Wow!")
}
```
IntelliJ also shows new warnings based on this.

## How was this patch tested?

Pass the Jenkins ScalaStyle test.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #11637 from dongjoon-hyun/SPARK-3854.
2016-03-10 15:57:22 -08:00