Commit graph

5199 commits

Author SHA1 Message Date
Richard W. Eggert II 765a488494 [SPARK-9026][SPARK-4514] Modifications to JobWaiter, FutureAction, and AsyncRDDActions to support non-blocking operation
These changes rework the implementations of `SimpleFutureAction`, `ComplexFutureAction`, `JobWaiter`, and `AsyncRDDActions` such that asynchronous callbacks on the generated `Futures` NEVER block waiting for a job to complete. A small amount of mutex synchronization is necessary to protect the internal fields that manage cancellation, but these locks are only held very briefly and in practice should almost never cause any blocking to occur. The existing blocking APIs of these classes are retained, but they simply delegate to the underlying non-blocking API and `Await` the results with indefinite timeouts.

Associated JIRA ticket: https://issues.apache.org/jira/browse/SPARK-9026
Also fixes: https://issues.apache.org/jira/browse/SPARK-4514

This pull request contains all my own original work, which I release to the Spark project under its open source license.

Author: Richard W. Eggert II <richard.eggert@gmail.com>

Closes #9264 from reggert/fix-futureaction.
2015-12-15 18:22:58 -08:00
CodingCat a63d9edcfb [SPARK-9516][UI] Improvement of Thread Dump Page
https://issues.apache.org/jira/browse/SPARK-9516

- [x] new look of Thread Dump Page

- [x] click column title to sort

- [x] grep

- [x] search as you type

squito JoshRosen It's ready for the review now

Author: CodingCat <zhunansjtu@gmail.com>

Closes #7910 from CodingCat/SPARK-9516.
2015-12-15 18:21:00 -08:00
Lianhui Wang 369127f032 [SPARK-12130] Replace shuffleManagerClass with shortShuffleMgrNames in ExternalShuffleBlockResolver
Replace shuffleManagerClassName with shortShuffleMgrName is  to reduce time of string's comparison. and put sort's comparison on the front. cc JoshRosen andrewor14

Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #10131 from lianhuiwang/spark-12130.
2015-12-15 18:17:48 -08:00
Holden Karau c59df8c516 [SPARK-12332][TRIVIAL][TEST] Fix minor typo in ResetSystemProperties
Fix a minor typo (unbalanced bracket) in ResetSystemProperties.

Author: Holden Karau <holden@us.ibm.com>

Closes #10303 from holdenk/SPARK-12332-trivial-typo-in-ResetSystemProperties-comment.
2015-12-15 11:38:57 +00:00
Shixiong Zhu 2aecda284e [SPARK-12281][CORE] Fix a race condition when reporting ExecutorState in the shutdown hook
1. Make sure workers and masters exit so that no worker or master will still be running when triggering the shutdown hook.
2. Set ExecutorState to FAILED if it's still RUNNING when executing the shutdown hook.

This should fix the potential exceptions when exiting a local cluster
```
java.lang.AssertionError: assertion failed: executor 4 state transfer from RUNNING to RUNNING is illegal
	at scala.Predef$.assert(Predef.scala:179)
	at org.apache.spark.deploy.master.Master$$anonfun$receive$1.applyOrElse(Master.scala:260)
	at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:116)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:204)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)
	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)

java.lang.IllegalStateException: Shutdown hooks cannot be modified during shutdown.
	at org.apache.spark.util.SparkShutdownHookManager.add(ShutdownHookManager.scala:246)
	at org.apache.spark.util.ShutdownHookManager$.addShutdownHook(ShutdownHookManager.scala:191)
	at org.apache.spark.util.ShutdownHookManager$.addShutdownHook(ShutdownHookManager.scala:180)
	at org.apache.spark.deploy.worker.ExecutorRunner.start(ExecutorRunner.scala:73)
	at org.apache.spark.deploy.worker.Worker$$anonfun$receive$1.applyOrElse(Worker.scala:474)
	at org.apache.spark.rpc.netty.Inbox$$anonfun$process$1.apply$mcV$sp(Inbox.scala:116)
	at org.apache.spark.rpc.netty.Inbox.safelyCall(Inbox.scala:204)
	at org.apache.spark.rpc.netty.Inbox.process(Inbox.scala:100)
	at org.apache.spark.rpc.netty.Dispatcher$MessageLoop.run(Dispatcher.scala:215)
	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)
```

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10269 from zsxwing/executor-state.
2015-12-13 22:06:39 -08:00
Shixiong Zhu 8af2f8c61a [SPARK-12267][CORE] Store the remote RpcEnv address to send the correct disconnetion message
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10261 from zsxwing/SPARK-12267.
2015-12-12 21:58:55 -08:00
Andrew Or 5030923ea8 [SPARK-12155][SPARK-12253] Fix executor OOM in unified memory management
**Problem.** In unified memory management, acquiring execution memory may lead to eviction of storage memory. However, the space freed from evicting cached blocks is distributed among all active tasks. Thus, an incorrect upper bound on the execution memory per task can cause the acquisition to fail, leading to OOM's and premature spills.

**Example.** Suppose total memory is 1000B, cached blocks occupy 900B, `spark.memory.storageFraction` is 0.4, and there are two active tasks. In this case, the cap on task execution memory is 100B / 2 = 50B. If task A tries to acquire 200B, it will evict 100B of storage but can only acquire 50B because of the incorrect cap. For another example, see this [regression test](https://github.com/andrewor14/spark/blob/fix-oom/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala#L233) that I stole from JoshRosen.

**Solution.** Fix the cap on task execution memory. It should take into account the space that could have been freed by storage in addition to the current amount of memory available to execution. In the example above, the correct cap should have been 600B / 2 = 300B.

This patch also guards against the race condition (SPARK-12253):
(1) Existing tasks collectively occupy all execution memory
(2) New task comes in and blocks while existing tasks spill
(3) After tasks finish spilling, another task jumps in and puts in a large block, stealing the freed memory
(4) New task still cannot acquire memory and goes back to sleep

Author: Andrew Or <andrew@databricks.com>

Closes #10240 from andrewor14/fix-oom.
2015-12-10 15:30:08 -08:00
Josh Rosen 23a9e62bad [SPARK-12251] Document and improve off-heap memory configurations
This patch adds documentation for Spark configurations that affect off-heap memory and makes some naming and validation improvements for those configs.

- Change `spark.memory.offHeapSize` to `spark.memory.offHeap.size`. This is fine because this configuration has not shipped in any Spark release yet (it's new in Spark 1.6).
- Deprecated `spark.unsafe.offHeap` in favor of a new `spark.memory.offHeap.enabled` configuration. The motivation behind this change is to gather all memory-related configurations under the same prefix.
- Add a check which prevents users from setting `spark.memory.offHeap.enabled=true` when `spark.memory.offHeap.size == 0`. After SPARK-11389 (#9344), which was committed in Spark 1.6, Spark enforces a hard limit on the amount of off-heap memory that it will allocate to tasks. As a result, enabling off-heap execution memory without setting `spark.memory.offHeap.size` will lead to immediate OOMs. The new configuration validation makes this scenario easier to diagnose, helping to avoid user confusion.
- Document these configurations on the configuration page.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10237 from JoshRosen/SPARK-12251.
2015-12-10 15:29:04 -08:00
Marcelo Vanzin 4a46b8859d [SPARK-11563][CORE][REPL] Use RpcEnv to transfer REPL-generated classes.
This avoids bringing up yet another HTTP server on the driver, and
instead reuses the file server already managed by the driver's
RpcEnv. As a bonus, the repl now inherits the security features of
the network library.

There's also a small change to create the directory for storing classes
under the root temp dir for the application (instead of directly
under java.io.tmpdir).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9923 from vanzin/SPARK-11563.
2015-12-10 13:26:30 -08:00
Andrew Or 8770bd1213 [SPARK-12165][ADDENDUM] Fix outdated comments on unroll test
JoshRosen

Author: Andrew Or <andrew@databricks.com>

Closes #10229 from andrewor14/unroll-test-comments.
2015-12-09 17:24:04 -08:00
Sean Owen 1eb7c22ce7 [SPARK-11824][WEBUI] WebUI does not render descriptions with 'bad' HTML, throws console error
Don't warn when description isn't valid HTML since it may properly be like "SELECT ... where foo <= 1"

The tests for this code indicate that it's normal to handle strings like this that don't contain HTML as a string rather than markup. Hence logging every such instance as a warning is too noisy since it's not a problem. this is an issue for stages whose name contain SQL like the above

CC tdas as author of this bit of code

Author: Sean Owen <sowen@cloudera.com>

Closes #10159 from srowen/SPARK-11824.
2015-12-09 19:47:38 +00:00
Josh Rosen aec5ea000e [SPARK-12165][SPARK-12189] Fix bugs in eviction of storage memory by execution
This patch fixes a bug in the eviction of storage memory by execution.

## The bug:

In general, execution should be able to evict storage memory when the total storage memory usage is greater than `maxMemory * spark.memory.storageFraction`. Due to a bug, however, Spark might wind up evicting no storage memory in certain cases where the storage memory usage was between `maxMemory * spark.memory.storageFraction` and `maxMemory`. For example, here is a regression test which illustrates the bug:

```scala
    val maxMemory = 1000L
    val taskAttemptId = 0L
    val (mm, ms) = makeThings(maxMemory)
    // Since we used the default storage fraction (0.5), we should be able to allocate 500 bytes
    // of storage memory which are immune to eviction by execution memory pressure.

    // Acquire enough storage memory to exceed the storage region size
    assert(mm.acquireStorageMemory(dummyBlock, 750L, evictedBlocks))
    assertEvictBlocksToFreeSpaceNotCalled(ms)
    assert(mm.executionMemoryUsed === 0L)
    assert(mm.storageMemoryUsed === 750L)

    // At this point, storage is using 250 more bytes of memory than it is guaranteed, so execution
    // should be able to reclaim up to 250 bytes of storage memory.
    // Therefore, execution should now be able to require up to 500 bytes of memory:
    assert(mm.acquireExecutionMemory(500L, taskAttemptId, MemoryMode.ON_HEAP) === 500L) // <--- fails by only returning 250L
    assert(mm.storageMemoryUsed === 500L)
    assert(mm.executionMemoryUsed === 500L)
    assertEvictBlocksToFreeSpaceCalled(ms, 250L)
```

The problem relates to the control flow / interaction between `StorageMemoryPool.shrinkPoolToReclaimSpace()` and `MemoryStore.ensureFreeSpace()`. While trying to allocate the 500 bytes of execution memory, the `UnifiedMemoryManager` discovers that it will need to reclaim 250 bytes of memory from storage, so it calls `StorageMemoryPool.shrinkPoolToReclaimSpace(250L)`. This method, in turn, calls `MemoryStore.ensureFreeSpace(250L)`. However, `ensureFreeSpace()` first checks whether the requested space is less than `maxStorageMemory - storageMemoryUsed`, which will be true if there is any free execution memory because it turns out that `MemoryStore.maxStorageMemory = (maxMemory - onHeapExecutionMemoryPool.memoryUsed)` when the `UnifiedMemoryManager` is used.

The control flow here is somewhat confusing (it grew to be messy / confusing over time / as a result of the merging / refactoring of several components). In the pre-Spark 1.6 code, `ensureFreeSpace` was called directly by the `MemoryStore` itself, whereas in 1.6 it's involved in a confusing control flow where `MemoryStore` calls `MemoryManager.acquireStorageMemory`, which then calls back into `MemoryStore.ensureFreeSpace`, which, in turn, calls `MemoryManager.freeStorageMemory`.

## The solution:

The solution implemented in this patch is to remove the confusing circular control flow between `MemoryManager` and `MemoryStore`, making the storage memory acquisition process much more linear / straightforward. The key changes:

- Remove a layer of inheritance which made the memory manager code harder to understand (53841174760a24a0df3eb1562af1f33dbe340eb9).
- Move some bounds checks earlier in the call chain (13ba7ada77f87ef1ec362aec35c89a924e6987cb).
- Refactor `ensureFreeSpace()` so that the part which evicts blocks can be called independently from the part which checks whether there is enough free space to avoid eviction (7c68ca09cb1b12f157400866983f753ac863380e).
- Realize that this lets us remove a layer of overloads from `ensureFreeSpace` (eec4f6c87423d5e482b710e098486b3bbc4daf06).
- Realize that `ensureFreeSpace()` can simply be replaced with an `evictBlocksToFreeSpace()` method which is called [after we've already figured out](2dc842aea8/core/src/main/scala/org/apache/spark/memory/StorageMemoryPool.scala (L88)) how much memory needs to be reclaimed via eviction; (2dc842aea82c8895125d46a00aa43dfb0d121de9).

Along the way, I fixed some problems with the mocks in `MemoryManagerSuite`: the old mocks would [unconditionally](80a824d36e/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala (L84)) report that a block had been evicted even if there was enough space in the storage pool such that eviction would be avoided.

I also fixed a problem where `StorageMemoryPool._memoryUsed` might become negative due to freed memory being double-counted when excution evicts storage. The problem was that `StorageMemoryPoolshrinkPoolToFreeSpace` would [decrement `_memoryUsed`](7c68ca09cb (diff-935c68a9803be144ed7bafdd2f756a0fL133)) even though `StorageMemoryPool.freeMemory` had already decremented it as each evicted block was freed. See SPARK-12189 for details.

Author: Josh Rosen <joshrosen@databricks.com>
Author: Andrew Or <andrew@databricks.com>

Closes #10170 from JoshRosen/SPARK-12165.
2015-12-09 11:39:59 -08:00
jerryshao 6900f01737 [SPARK-10582][YARN][CORE] Fix AM failure situation for dynamic allocation
Because of AM failure, the target executor number between driver and AM will be different, which will lead to unexpected behavior in dynamic allocation. So when AM is re-registered with driver, state in `ExecutorAllocationManager` and `CoarseGrainedSchedulerBacked` should be reset.

This issue is originally addressed in #8737 , here re-opened again. Thanks a lot KaiXinXiaoLei for finding this issue.

andrewor14 and vanzin would you please help to review this, thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #9963 from jerryshao/SPARK-10582.
2015-12-09 09:52:03 -08:00
uncleGen a113216865 [SPARK-12031][CORE][BUG] Integer overflow when do sampling
Author: uncleGen <hustyugm@gmail.com>

Closes #10023 from uncleGen/1.6-bugfix.
2015-12-09 15:09:40 +00:00
Fei Wang 3934562d34 [SPARK-12222] [CORE] Deserialize RoaringBitmap using Kryo serializer throw Buffer underflow exception
Jira: https://issues.apache.org/jira/browse/SPARK-12222

Deserialize RoaringBitmap using Kryo serializer throw Buffer underflow exception:
```
com.esotericsoftware.kryo.KryoException: Buffer underflow.
	at com.esotericsoftware.kryo.io.Input.require(Input.java:156)
	at com.esotericsoftware.kryo.io.Input.skip(Input.java:131)
	at com.esotericsoftware.kryo.io.Input.skip(Input.java:264)
```

This is caused by a bug of kryo's `Input.skip(long count)`(https://github.com/EsotericSoftware/kryo/issues/119) and we call this method in `KryoInputDataInputBridge`.

Instead of upgrade kryo's version, this pr bypass the  kryo's `Input.skip(long count)` by directly call another `skip` method in kryo's Input.java(https://github.com/EsotericSoftware/kryo/blob/kryo-2.21/src/com/esotericsoftware/kryo/io/Input.java#L124), i.e. write the bug-fixed version of `Input.skip(long count)` in KryoInputDataInputBridge's `skipBytes` method.

more detail link to https://github.com/apache/spark/pull/9748#issuecomment-162860246

Author: Fei Wang <wangfei1@huawei.com>

Closes #10213 from scwf/patch-1.
2015-12-08 21:32:31 -08:00
Andrew Or 9494521695 [SPARK-12187] *MemoryPool classes should not be fully public
This patch tightens them to `private[memory]`.

Author: Andrew Or <andrew@databricks.com>

Closes #10182 from andrewor14/memory-visibility.
2015-12-08 14:34:15 -08:00
tedyu 75c60bf4ba [SPARK-12074] Avoid memory copy involving ByteBuffer.wrap(ByteArrayOutputStream.toByteArray)
SPARK-12060 fixed JavaSerializerInstance.serialize
This PR applies the same technique on two other classes.

zsxwing

Author: tedyu <yuzhihong@gmail.com>

Closes #10177 from tedyu/master.
2015-12-08 10:01:44 -08:00
Xin Ren 6cb06e8711 [SPARK-11155][WEB UI] Stage summary json should include stage duration
The json endpoint for stages doesn't include information on the stage duration that is present in the UI. This looks like a simple oversight, they should be included. eg., the metrics should be included at api/v1/applications/<appId>/stages.

Metrics I've added are: submissionTime, firstTaskLaunchedTime and completionTime

Author: Xin Ren <iamshrek@126.com>

Closes #10107 from keypointt/SPARK-11155.
2015-12-08 11:46:46 -06:00
Shixiong Zhu 3f4efb5c23 [SPARK-12060][CORE] Avoid memory copy in JavaSerializerInstance.serialize
Merged #10051 again since #10083 is resolved.

This reverts commit 328b757d5d.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10167 from zsxwing/merge-SPARK-12060.
2015-12-07 12:01:09 -08:00
Shixiong Zhu 3af53e61fd [SPARK-12084][CORE] Fix codes that uses ByteBuffer.array incorrectly
`ByteBuffer` doesn't guarantee all contents in `ByteBuffer.array` are valid. E.g, a ByteBuffer returned by `ByteBuffer.slice`. We should not use the whole content of `ByteBuffer` unless we know that's correct.

This patch fixed all places that use `ByteBuffer.array` incorrectly.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10083 from zsxwing/bytebuffer-array.
2015-12-04 17:02:04 -08:00
rotems f30373f5ee [SPARK-12080][CORE] Kryo - Support multiple user registrators
Author: rotems <roter>

Closes #10078 from Botnaim/KryoMultipleCustomRegistrators.
2015-12-04 16:58:34 -08:00
meiyoula bbfc16ec9d [SPARK-12142][CORE]Reply false when container allocator is not ready and reset target
Using Dynamic Allocation function, when a new AM is starting, and ExecutorAllocationManager send RequestExecutor message to AM. If the container allocator is not ready, the whole app will hang on

Author: meiyoula <1039320815@qq.com>

Closes #10138 from XuTingjun/patch-1.
2015-12-04 16:50:43 -08:00
Josh Rosen b7204e1d41 [SPARK-12112][BUILD] Upgrade to SBT 0.13.9
We should upgrade to SBT 0.13.9, since this is a requirement in order to use SBT's new Maven-style resolution features (which will be done in a separate patch, because it's blocked by some binary compatibility issues in the POM reader plugin).

I also upgraded Scalastyle to version 0.8.0, which was necessary in order to fix a Scala 2.10.5 compatibility issue (see https://github.com/scalastyle/scalastyle/issues/156). The newer Scalastyle is slightly stricter about whitespace surrounding tokens, so I fixed the new style violations.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10112 from JoshRosen/upgrade-to-sbt-0.13.9.
2015-12-05 08:15:30 +08:00
Dmitry Erastov d0d8222778 [SPARK-6990][BUILD] Add Java linting script; fix minor warnings
This replaces https://github.com/apache/spark/pull/9696

Invoke Checkstyle and print any errors to the console, failing the step.
Use Google's style rules modified according to
https://cwiki.apache.org/confluence/display/SPARK/Spark+Code+Style+Guide
Some important checks are disabled (see TODOs in `checkstyle.xml`) due to
multiple violations being present in the codebase.

Suggest fixing those TODOs in a separate PR(s).

More on Checkstyle can be found on the [official website](http://checkstyle.sourceforge.net/).

Sample output (from [build 46345](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/46345/consoleFull)) (duplicated because I run the build twice with different profiles):

> Checkstyle checks failed at following occurrences:
[ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/UnsafeRowParquetRecordReader.java:[217,7] (coding) MissingSwitchDefault: switch without "default" clause.
> [ERROR] src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:[198,10] (modifier) ModifierOrder: 'protected' modifier out of order with the JLS suggestions.
> [error] running /home/jenkins/workspace/SparkPullRequestBuilder2/dev/lint-java ; received return code 1

Also fix some of the minor violations that didn't require sweeping changes.

Apologies for the previous botched PRs - I finally figured out the issue.

cr: JoshRosen, pwendell

> I state that the contribution is my original work, and I license the work to the project under the project's open source license.

Author: Dmitry Erastov <derastov@gmail.com>

Closes #9867 from dskrvk/master.
2015-12-04 12:03:45 -08:00
Nong 95296d9b1a [SPARK-12089] [SQL] Fix memory corrupt due to freeing a page being referenced
When the spillable sort iterator was spilled, it was mistakenly keeping
the last page in memory rather than the current page. This causes the
current record to get corrupted.

Author: Nong <nong@cloudera.com>

Closes #10142 from nongli/spark-12089.
2015-12-04 10:01:20 -08:00
Carson Wang b6e9963ee4 [SPARK-11206] Support SQL UI on the history server (resubmit)
Resubmit #9297 and #9991
On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.

To support SQL UI on the history server:
1. I added an onOtherEvent method to the SparkListener trait and post all SQL related events to the same event bus.
2. Two SQL events SparkListenerSQLExecutionStart and SparkListenerSQLExecutionEnd are defined in the sql module.
3. The new SQL events are written to event log using Jackson.
4. A new trait SparkHistoryListenerFactory is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using java.util.ServiceLoader.

Author: Carson Wang <carson.wang@intel.com>

Closes #10061 from carsonwang/SqlHistoryUI.
2015-12-03 16:39:12 -08:00
Anderson de Andrade f434f36d50 [SPARK-12056][CORE] Create a TaskAttemptContext only after calling setConf.
TaskAttemptContext's constructor will clone the configuration instead of referencing it. Calling setConf after creating TaskAttemptContext makes any changes to the configuration made inside setConf unperceived by RecordReader instances.

As an example, Titan's InputFormat will change conf when calling setConf. They wrap their InputFormat around Cassandra's ColumnFamilyInputFormat, and append Cassandra's configuration. This change fixes the following error when using Titan's CassandraInputFormat with Spark:

*java.lang.RuntimeException: org.apache.thrift.protocol.TProtocolException: Required field 'keyspace' was not present! Struct: set_key space_args(keyspace:null)*

There's a discussion of this error here: https://groups.google.com/forum/#!topic/aureliusgraphs/4zpwyrYbGAE

Author: Anderson de Andrade <adeandrade@verticalscope.com>

Closes #10046 from adeandrade/newhadooprdd-fix.
2015-12-03 16:37:00 -08:00
Andrew Or 688e521c28 [SPARK-12108] Make event logs smaller
**Problem.** Event logs in 1.6 were much bigger than 1.5. I ran page rank and the event log size in 1.6 was almost 5x that in 1.5. I did a bisect to find that the RDD callsite added in #9398 is largely responsible for this.

**Solution.** This patch removes the long form of the callsite (which is not used!) from the event log. This reduces the size of the event log significantly.

*Note on compatibility*: if this patch is to be merged into 1.6.0, then it won't break any compatibility. Otherwise, if it is merged into 1.6.1, then we might need to add more backward compatibility handling logic (currently does not exist yet).

Author: Andrew Or <andrew@databricks.com>

Closes #10115 from andrewor14/smaller-event-logs.
2015-12-03 11:09:29 -08:00
Shixiong Zhu 649be4fa45 [SPARK-12101][CORE] Fix thread pools that cannot cache tasks in Worker and AppClient
`SynchronousQueue` cannot cache any task. This issue is similar to #9978. It's an easy fix. Just use the fixed `ThreadUtils.newDaemonCachedThreadPool`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10108 from zsxwing/fix-threadpool.
2015-12-03 11:06:25 -08:00
jerryshao 7bc9e1db2c [SPARK-12059][CORE] Avoid assertion error when unexpected state transition met in Master
Downgrade to warning log for unexpected state transition.

andrewor14 please review, thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #10091 from jerryshao/SPARK-12059.
2015-12-03 11:05:12 -08:00
Steve Loughran 8fa3e474a8 [SPARK-11314][YARN] add service API and test service for Yarn Cluster schedulers
This is purely the yarn/src/main and yarn/src/test bits of the YARN ATS integration: the extension model to load and run implementations of `SchedulerExtensionService` in the yarn cluster scheduler process —and to stop them afterwards.

There's duplication between the two schedulers, yarn-client and yarn-cluster, at least in terms of setting everything up, because the common superclass, `YarnSchedulerBackend` is in spark-core, and the extension services need the YARN app/attempt IDs.

If you look at how the the extension services are loaded, the case class `SchedulerExtensionServiceBinding` is used to pass in config info -currently just the spark context and the yarn IDs, of which one, the attemptID, will be null when running client-side. I'm passing in a case class to ensure that it would be possible in future to add extra arguments to the binding class, yet, as the method signature will not have changed, still be able to load existing services.

There's no functional extension service here, just one for testing. The real tests come in the bigger pull requests. At the same time, there's no restriction of this extension service purely to the ATS history publisher. Anything else that wants to listen to the spark context and publish events could use this, and I'd also consider writing one for the YARN-913 registry service, so that the URLs of the web UI would be locatable through that (low priority; would make more sense if integrated with a REST client).

There's no minicluster test. Given the test execution overhead of setting up minicluster tests, it'd  probably be better to add an extension service into one of the existing tests.

Author: Steve Loughran <stevel@hortonworks.com>

Closes #9182 from steveloughran/stevel/feature/SPARK-1537-service.
2015-12-03 10:33:06 -08:00
Josh Rosen ae40253373 [SPARK-12082][FLAKY-TEST] Increase timeouts in NettyBlockTransferSecuritySuite
We should try increasing a timeout in NettyBlockTransferSecuritySuite in order to reduce that suite's flakiness in Jenkins.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10113 from JoshRosen/SPARK-12082.
2015-12-03 11:12:02 +08:00
Jeroen Schot 128c29035b [SPARK-3580][CORE] Add Consistent Method To Get Number of RDD Partitions Across Different Languages
I have tried to address all the comments in pull request https://github.com/apache/spark/pull/2447.

Note that the second commit (using the new method in all internal code of all components) is quite intrusive and could be omitted.

Author: Jeroen Schot <jeroen.schot@surfsara.nl>

Closes #9767 from schot/master.
2015-12-02 09:40:07 +00:00
Andrew Or d96f8c997b [SPARK-12081] Make unified memory manager work with small heaps
The existing `spark.memory.fraction` (default 0.75) gives the system 25% of the space to work with. For small heaps, this is not enough: e.g. default 1GB leaves only 250MB system memory. This is especially a problem in local mode, where the driver and executor are crammed in the same JVM. Members of the community have reported driver OOM's in such cases.

**New proposal.** We now reserve 300MB before taking the 75%. For 1GB JVMs, this leaves `(1024 - 300) * 0.75 = 543MB` for execution and storage. This is proposal (1) listed in the [JIRA](https://issues.apache.org/jira/browse/SPARK-12081).

Author: Andrew Or <andrew@databricks.com>

Closes #10081 from andrewor14/unified-memory-small-heaps.
2015-12-01 19:51:12 -08:00
Andrew Or 1ce4adf55b [SPARK-8414] Ensure context cleaner periodic cleanups
Garbage collection triggers cleanups. If the driver JVM is huge and there is little memory pressure, we may never clean up shuffle files on executors. This is a problem for long-running applications (e.g. streaming).

Author: Andrew Or <andrew@databricks.com>

Closes #10070 from andrewor14/periodic-gc.
2015-12-01 19:36:34 -08:00
Shixiong Zhu 328b757d5d Revert "[SPARK-12060][CORE] Avoid memory copy in JavaSerializerInstance.serialize"
This reverts commit 1401166576.
2015-12-01 15:13:10 -08:00
Tathagata Das 60b541ee1b [SPARK-12004] Preserve the RDD partitioner through RDD checkpointing
The solution is the save the RDD partitioner in a separate file in the RDD checkpoint directory. That is, `<checkpoint dir>/_partitioner`.  In most cases, whether the RDD partitioner was recovered or not, does not affect the correctness, only reduces performance. So this solution makes a best-effort attempt to save and recover the partitioner. If either fails, the checkpointing is not affected. This makes this patch safe and backward compatible.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #9983 from tdas/SPARK-12004.
2015-12-01 14:08:36 -08:00
Josh Rosen 34e7093c11 [SPARK-12065] Upgrade Tachyon from 0.8.1 to 0.8.2
This commit upgrades the Tachyon dependency from 0.8.1 to 0.8.2.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #10054 from JoshRosen/upgrade-to-tachyon-0.8.2.
2015-12-01 11:49:20 -08:00
woj-i 6a8cf80cc8 [SPARK-11821] Propagate Kerberos keytab for all environments
andrewor14 the same PR as in branch 1.5
harishreedharan

Author: woj-i <wojciechindyk@gmail.com>

Closes #9859 from woj-i/master.
2015-12-01 11:05:45 -08:00
Cheng Lian 69dbe6b40d [SPARK-12046][DOC] Fixes various ScalaDoc/JavaDoc issues
This PR backports PR #10039 to master

Author: Cheng Lian <lian@databricks.com>

Closes #10063 from liancheng/spark-12046.doc-fix.master.
2015-12-01 10:21:31 -08:00
Shixiong Zhu 1401166576 [SPARK-12060][CORE] Avoid memory copy in JavaSerializerInstance.serialize
`JavaSerializerInstance.serialize` uses `ByteArrayOutputStream.toByteArray` to get the serialized data. `ByteArrayOutputStream.toByteArray` needs to copy the content in the internal array to a new array. However, since the array will be converted to `ByteBuffer` at once, we can avoid the memory copy.

This PR added `ByteBufferOutputStream` to access the protected `buf` and convert it to a `ByteBuffer` directly.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #10051 from zsxwing/SPARK-12060.
2015-12-01 09:45:55 -08:00
Sean Owen 96bf468c78 [SPARK-12049][CORE] User JVM shutdown hook can cause deadlock at shutdown
Avoid potential deadlock with a user app's shutdown hook thread by more narrowly synchronizing access to 'hooks'

Author: Sean Owen <sowen@cloudera.com>

Closes #10042 from srowen/SPARK-12049.
2015-11-30 17:33:09 -08:00
Marcelo Vanzin 9bf2120672 [SPARK-12007][NETWORK] Avoid copies in the network lib's RPC layer.
This change seems large, but most of it is just replacing `byte[]`
with `ByteBuffer` and `new byte[]` with `ByteBuffer.allocate()`,
since it changes the network library's API.

The following are parts of the code that actually have meaningful
changes:

- The Message implementations were changed to inherit from a new
  AbstractMessage that can optionally hold a reference to a body
  (in the form of a ManagedBuffer); this is similar to how
  ResponseWithBody worked before, except now it's not restricted
  to just responses.

- The TransportFrameDecoder was pretty much rewritten to avoid
  copies as much as possible; it doesn't rely on CompositeByteBuf
  to accumulate incoming data anymore, since CompositeByteBuf
  has issues when slices are retained. The code now is able to
  create frames without having to resort to copying bytes except
  for a few bytes (containing the frame length) in very rare cases.

- Some minor changes in the SASL layer to convert things back to
  `byte[]` since the JDK SASL API operates on those.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9987 from vanzin/SPARK-12007.
2015-11-30 17:22:05 -08:00
CodingCat 0a46e43772 [SPARK-12037][CORE] initialize heartbeatReceiverRef before calling startDriverHeartbeat
https://issues.apache.org/jira/browse/SPARK-12037

a simple fix by changing the order of the statements

Author: CodingCat <zhunansjtu@gmail.com>

Closes #10032 from CodingCat/SPARK-12037.
2015-11-30 17:19:26 -08:00
Teng Qiu a8ceec5e8c [SPARK-12053][CORE] EventLoggingListener.getLogPath needs 4 parameters
```EventLoggingListener.getLogPath``` needs 4 input arguments:
https://github.com/apache/spark/blob/v1.6.0-preview2/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala#L276-L280

the 3rd parameter should be appAttemptId, 4th parameter is codec...

Author: Teng Qiu <teng.qiu@gmail.com>

Closes #10044 from chutium/SPARK-12053.
2015-12-01 07:27:32 +09:00
Josh Rosen 2c5dee0fb8 Revert "[SPARK-11206] Support SQL UI on the history server"
This reverts commit cc243a079b / PR #9297

I'm reverting this because it broke SQLListenerMemoryLeakSuite in the master Maven builds.

See #9991 for a discussion of why this broke the tests.
2015-11-30 13:42:35 -08:00
Davies Liu 8df584b020 [SPARK-11982] [SQL] improve performance of cartesian product
This PR improve the performance of CartesianProduct by caching the result of right plan.

After this patch, the query time of TPC-DS Q65 go down to 4 seconds from 28 minutes (420X faster).

cc nongli

Author: Davies Liu <davies@databricks.com>

Closes #9969 from davies/improve_cartesian.
2015-11-30 11:54:18 -08:00
Wieland Hoffmann 26c3581f17 [DOC] Explicitly state that top maintains the order of elements
Top is implemented in terms of takeOrdered, which already maintains the
order, so top should, too.

Author: Wieland Hoffmann <themineo@gmail.com>

Closes #10013 from mineo/top-order.
2015-11-30 09:32:48 +00:00
toddwan e074944205 [SPARK-11859][MESOS] SparkContext accepts invalid Master URLs in the form zk://host:port for a multi-master Mesos cluster using ZooKeeper
* According to below doc and validation logic in [SparkSubmit.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L231), master URL for a mesos cluster should always start with `mesos://`

http://spark.apache.org/docs/latest/running-on-mesos.html
`The Master URLs for Mesos are in the form mesos://host:5050 for a single-master Mesos cluster, or mesos://zk://host:2181 for a multi-master Mesos cluster using ZooKeeper.`

* However, [SparkContext.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L2749) fails the validation and can receive master URL in the form `zk://host:port`

* For the master URLs in the form `zk:host:port`, the valid form should be `mesos://zk://host:port`

* This PR restrict the validation in `SparkContext.scala`, and now only mesos master URLs prefixed with `mesos://` can be accepted.

* This PR also updated corresponding unit test.

Author: toddwan <tawan0109@outlook.com>

Closes #9886 from toddwan/S11859.
2015-11-30 09:26:29 +00:00
Shixiong Zhu 0c1e72e7f7 [SPARK-11996][CORE] Make the executor thread dump work again
In the previous implementation, the driver needs to know the executor listening address to send the thread dump request. However, in Netty RPC, the executor doesn't listen to any port, so the executor thread dump feature is broken.

This patch makes the driver use the endpointRef stored in BlockManagerMasterEndpoint to send the thread dump request to fix it.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9976 from zsxwing/executor-thread-dump.
2015-11-26 18:56:22 -08:00
Shixiong Zhu d3ef693325 [SPARK-11999][CORE] Fix the issue that ThreadUtils.newDaemonCachedThreadPool doesn't cache any task
In the previous codes, `newDaemonCachedThreadPool` uses `SynchronousQueue`, which is wrong. `SynchronousQueue` is an empty queue that cannot cache any task. This patch uses `LinkedBlockingQueue` to fix it along with other fixes to make sure `newDaemonCachedThreadPool` can use at most `maxThreadNumber` threads, and after that, cache tasks to `LinkedBlockingQueue`.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9978 from zsxwing/cached-threadpool.
2015-11-25 23:31:21 -08:00
Carson Wang cc243a079b [SPARK-11206] Support SQL UI on the history server
On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution.

To support SQL UI on the history server:
1. I added an `onOtherEvent` method to the `SparkListener` trait and post all SQL related events to the same event bus.
2. Two SQL events `SparkListenerSQLExecutionStart` and `SparkListenerSQLExecutionEnd` are defined in the sql module.
3. The new SQL events are written to event log using Jackson.
4.  A new trait `SparkHistoryListenerFactory` is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using `java.util.ServiceLoader`.

Author: Carson Wang <carson.wang@intel.com>

Closes #9297 from carsonwang/SqlHistoryUI.
2015-11-25 15:13:13 -08:00
Marcelo Vanzin 4e81783e92 [SPARK-11866][NETWORK][CORE] Make sure timed out RPCs are cleaned up.
This change does a couple of different things to make sure that the RpcEnv-level
code and the network library agree about the status of outstanding RPCs.

For RPCs that do not expect a reply ("RpcEnv.send"), support for one way
messages (hello CORBA!) was added to the network layer. This is a
"fire and forget" message that does not require any state to be kept
by the TransportClient; as a result, the RpcEnv 'Ack' message is not needed
anymore.

For RPCs that do expect a reply ("RpcEnv.ask"), the network library now
returns the internal RPC id; if the RpcEnv layer decides to time out the
RPC before the network layer does, it now asks the TransportClient to
forget about the RPC, so that if the network-level timeout occurs, the
client is not killed.

As part of implementing the above, I cleaned up some of the code in the
netty rpc backend, removing types that were not necessary and factoring
out some common code. Of interest is a slight change in the exceptions
when posting messages to a stopped RpcEnv; that's mostly to avoid nasty
error messages from the local-cluster backend when shutting down, which
pollutes the terminal output.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9917 from vanzin/SPARK-11866.
2015-11-25 12:58:18 -08:00
jerryshao 88875d9413 [SPARK-10558][CORE] Fix wrong executor state in Master
`ExecutorAdded` can only be sent to `AppClient` when worker report back the executor state as `LOADING`, otherwise because of concurrency issue, `AppClient` will possibly receive `ExectuorAdded` at first, then `ExecutorStateUpdated` with `LOADING` state.

Also Master will change the executor state from `LAUNCHING` to `RUNNING` (`AppClient` report back the state as `RUNNING`), then to `LOADING` (worker report back to state as `LOADING`), it should be `LAUNCHING` -> `LOADING` -> `RUNNING`.

Also it is wrongly shown in master UI, the state of executor should be `RUNNING` rather than `LOADING`:

![screen shot 2015-09-11 at 2 30 28 pm](https://cloud.githubusercontent.com/assets/850797/9809254/3155d840-5899-11e5-8cdf-ad06fef75762.png)

Author: jerryshao <sshao@hortonworks.com>

Closes #8714 from jerryshao/SPARK-10558.
2015-11-25 11:42:53 -08:00
Alex Bozarth 83653ac5e7 [SPARK-10864][WEB UI] app name is hidden if window is resized
Currently the Web UI navbar has a minimum width of 1200px; so if a window is resized smaller than that the app name goes off screen. The 1200px width seems to have been chosen since it fits the longest example app name without wrapping.

To work with smaller window widths I made the tabs wrap since it looked better than wrapping the app name. This is a distinct change in how the navbar looks and I'm not sure if it's what we actually want to do.

Other notes:
- min-width set to 600px to keep the tabs from wrapping individually (will need to be adjusted if tabs are added)
- app name will also wrap (making three levels) if a really really long app name is used

Author: Alex Bozarth <ajbozart@us.ibm.com>

Closes #9874 from ajbozarth/spark10864.
2015-11-25 11:39:00 -08:00
Zhongshuai Pei 6b781576a1 [SPARK-11974][CORE] Not all the temp dirs had been deleted when the JVM exits
deleting the temp dir like that

```

scala> import scala.collection.mutable
import scala.collection.mutable

scala> val a = mutable.Set(1,2,3,4,7,0,8,98,9)
a: scala.collection.mutable.Set[Int] = Set(0, 9, 1, 2, 3, 7, 4, 8, 98)

scala> a.foreach(x => {a.remove(x) })

scala> a.foreach(println(_))
98
```

You may not modify a collection while traversing or iterating over it.This can not delete all element of the collection

Author: Zhongshuai Pei <peizhongshuai@huawei.com>

Closes #9951 from DoingDone9/Bug_RemainDir.
2015-11-25 10:37:34 -08:00
Marcelo Vanzin c1f85fc71e [SPARK-11956][CORE] Fix a few bugs in network lib-based file transfer.
- NettyRpcEnv::openStream() now correctly propagates errors to
  the read side of the pipe.
- NettyStreamManager now throws if the file being transferred does
  not exist.
- The network library now correctly handles zero-sized streams.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9941 from vanzin/SPARK-11956.
2015-11-25 09:47:20 -08:00
Mark Hamstra 0a5aef753e [SPARK-10666][SPARK-6880][CORE] Use properties from ActiveJob associated with a Stage
This issue was addressed in https://github.com/apache/spark/pull/5494, but the fix in that PR, while safe in the sense that it will prevent the SparkContext from shutting down, misses the actual bug.  The intent of `submitMissingTasks` should be understood as "submit the Tasks that are missing for the Stage, and run them as part of the ActiveJob identified by jobId".  Because of a long-standing bug, the `jobId` parameter was never being used.  Instead, we were trying to use the jobId with which the Stage was created -- which may no longer exist as an ActiveJob, hence the crash reported in SPARK-6880.

The correct fix is to use the ActiveJob specified by the supplied jobId parameter, which is guaranteed to exist at the call sites of submitMissingTasks.

This fix should be applied to all maintenance branches, since it has existed since 1.0.

kayousterhout pankajarora12

Author: Mark Hamstra <markhamstra@gmail.com>
Author: Imran Rashid <irashid@cloudera.com>

Closes #6291 from markhamstra/SPARK-6880.
2015-11-25 09:34:34 -06:00
Ashwin Swaroop 6385002657 [SPARK-11686][CORE] Issue WARN when dynamic allocation is disabled due to spark.dynamicAllocation.enabled and spark.executor.instances both set
Changed the log type to a 'warning' instead of 'info' as required.

Author: Ashwin Swaroop <Ashwin Swaroop>

Closes #9926 from ashwinswaroop/master.
2015-11-25 13:41:14 +00:00
Davies Liu 58d9b26055 [SPARK-11805] free the array in UnsafeExternalSorter during spilling
After calling spill() on SortedIterator, the array inside InMemorySorter is not needed, it should be freed during spilling, this could help to join multiple tables with limited memory.

Author: Davies Liu <davies@databricks.com>

Closes #9793 from davies/free_array.
2015-11-24 14:33:28 -08:00
Marcelo Vanzin e6dd237463 [SPARK-11929][CORE] Make the repl log4j configuration override the root logger.
In the default Spark distribution, there are currently two separate
log4j config files, with different default values for the root logger,
so that when running the shell you have a different default log level.
This makes the shell more usable, since the logs don't overwhelm the
output.

But if you install a custom log4j.properties, you lose that, because
then it's going to be used no matter whether you're running a regular
app or the shell.

With this change, the overriding of the log level is done differently;
the log level repl's main class (org.apache.spark.repl.Main) is used
to define the root logger's level when running the shell, defaulting
to WARN if it's not set explicitly.

On a somewhat related change, the shell output about the "sc" variable
was changed a bit to contain a little more useful information about
the application, since when the root logger's log level is WARN, that
information is never shown to the user.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9816 from vanzin/shell-logging.
2015-11-24 15:08:02 -06:00
Reynold Xin f315272279 [SPARK-11946][SQL] Audit pivot API for 1.6.
Currently pivot's signature looks like

```scala
scala.annotation.varargs
def pivot(pivotColumn: Column, values: Column*): GroupedData

scala.annotation.varargs
def pivot(pivotColumn: String, values: Any*): GroupedData
```

I think we can remove the one that takes "Column" types, since callers should always be passing in literals. It'd also be more clear if the values are not varargs, but rather Seq or java.util.List.

I also made similar changes for Python.

Author: Reynold Xin <rxin@databricks.com>

Closes #9929 from rxin/SPARK-11946.
2015-11-24 12:54:37 -08:00
tedyu 81012546ee [SPARK-11872] Prevent the call to SparkContext#stop() in the listener bus's thread
This is continuation of SPARK-11761

Andrew suggested adding this protection. See tail of https://github.com/apache/spark/pull/9741

Author: tedyu <yuzhihong@gmail.com>

Closes #9852 from tedyu/master.
2015-11-24 12:22:33 -08:00
Forest Fang 800bd799ac [SPARK-11906][WEB UI] Speculation Tasks Cause ProgressBar UI Overflow
When there are speculative tasks in the stage, running progress bar could overflow and goes hidden on a new line:
![image](https://cloud.githubusercontent.com/assets/4317392/11326841/5fd3482e-9142-11e5-8ca5-cb2f0c0c8964.png)
3 completed / 2 running (including 1 speculative) out of 4 total tasks

This is a simple fix by capping the started tasks at `total - completed` tasks
![image](https://cloud.githubusercontent.com/assets/4317392/11326842/6bb67260-9142-11e5-90f0-37f9174878ec.png)

I should note my preferred way to fix it is via css style
```css
.progress { display: flex; }
```
which shifts the correction burden from driver to web browser. However I couldn't get selenium test to measure the position/dimension of the progress bar correctly to get this unit tested.

It also has the side effect that the width will be calibrated so the running occupies 2 / 5 instead of 1 / 4.
![image](https://cloud.githubusercontent.com/assets/4317392/11326848/7b03e9f0-9142-11e5-89ad-bd99cb0647cf.png)

All in all, since this cosmetic bug is minor enough, I suppose the original simple fix should be good enough.

Author: Forest Fang <forest.fang@outlook.com>

Closes #9896 from saurfang/progressbar.
2015-11-24 09:03:32 +00:00
Reynold Xin 8d57524662 [SPARK-11933][SQL] Rename mapGroup -> mapGroups and flatMapGroup -> flatMapGroups.
Based on feedback from Matei, this is more consistent with mapPartitions in Spark.

Also addresses some of the cleanups from a previous commit that renames the type variables.

Author: Reynold Xin <rxin@databricks.com>

Closes #9919 from rxin/SPARK-11933.
2015-11-23 22:22:15 -08:00
Marcelo Vanzin c2467dadae [SPARK-11140][CORE] Transfer files using network lib when using NettyRpcEnv.
This change abstracts the code that serves jars / files to executors so that
each RpcEnv can have its own implementation; the akka version uses the existing
HTTP-based file serving mechanism, while the netty versions uses the new
stream support added to the network lib, which makes file transfers benefit
from the easier security configuration of the network library, and should also
reduce overhead overall.

The change includes a small fix to TransportChannelHandler so that it propagates
user events to downstream handlers.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9530 from vanzin/SPARK-11140.
2015-11-23 13:54:19 -08:00
Reynold Xin ff442bbcff [SPARK-11899][SQL] API audit for GroupedDataset.
1. Renamed map to mapGroup, flatMap to flatMapGroup.
2. Renamed asKey -> keyAs.
3. Added more documentation.
4. Changed type parameter T to V on GroupedDataset.
5. Added since versions for all functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #9880 from rxin/SPARK-11899.
2015-11-21 15:00:37 -08:00
Nong Li 58b4e4f88a [SPARK-11787][SPARK-11883][SQL][FOLLOW-UP] Cleanup for this patch.
This mainly moves SqlNewHadoopRDD to the sql package. There is some state that is
shared between core and I've left that in core. This allows some other associated
minor cleanup.

Author: Nong Li <nong@databricks.com>

Closes #9845 from nongli/spark-11787.
2015-11-20 15:30:53 -08:00
Josh Rosen 89fd9bd061 [SPARK-11887] Close PersistenceEngine at the end of PersistenceEngineSuite tests
In PersistenceEngineSuite, we do not call `close()` on the PersistenceEngine at the end of the test. For the ZooKeeperPersistenceEngine, this causes us to leak a ZooKeeper client, causing the logs of unrelated tests to be periodically spammed with connection error messages from that client:

```
15/11/20 05:13:35.789 pool-1-thread-1-ScalaTest-running-PersistenceEngineSuite-SendThread(localhost:15741) INFO ClientCnxn: Opening socket connection to server localhost/127.0.0.1:15741. Will not attempt to authenticate using SASL (unknown error)
15/11/20 05:13:35.790 pool-1-thread-1-ScalaTest-running-PersistenceEngineSuite-SendThread(localhost:15741) WARN ClientCnxn: Session 0x15124ff48dd0000 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:739)
	at org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:350)
	at org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1068)
```

This patch fixes this by using a `finally` block.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9864 from JoshRosen/close-zookeeper-client-in-tests.
2015-11-20 14:31:26 -08:00
Josh Rosen 652def318e [SPARK-11650] Reduce RPC timeouts to speed up slow AkkaUtilsSuite test
This patch reduces some RPC timeouts in order to speed up the slow "AkkaUtilsSuite.remote fetch ssl on - untrusted server", which used to take two minutes to run.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9869 from JoshRosen/SPARK-11650.
2015-11-20 13:17:35 -08:00
Tathagata Das b2cecb80ec [SPARK-11845][STREAMING][TEST] Added unit test to verify TrackStateRDD is correctly checkpointed
To make sure that all lineage is correctly truncated for TrackStateRDD when checkpointed.

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #9831 from tdas/SPARK-11845.
2015-11-19 16:50:08 -08:00
Marcelo Vanzin 880128f37e [SPARK-4134][CORE] Lower severity of some executor loss logs.
Don't log ERROR messages when executors are explicitly killed or when
the exit reason is not yet known.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9780 from vanzin/SPARK-11789.
2015-11-19 16:49:18 -08:00
hushan 01403aa97b [SPARK-11746][CORE] Use cache-aware method dependencies
a small change

Author: hushan <hushan@xiaomi.com>

Closes #9691 from suyanNone/unify-getDependency.
2015-11-19 14:56:00 -08:00
Marcelo Vanzin f7135ed719 [SPARK-11828][CORE] Register DAGScheduler metrics source after app id is known.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9820 from vanzin/SPARK-11828.
2015-11-19 14:53:58 -08:00
Srinivasa Reddy Vundela 3bd77b213a [SPARK-11799][CORE] Make it explicit in executor logs that uncaught e…
…xceptions are thrown during executor shutdown

This commit will make sure that when uncaught exceptions are prepended with [Container in shutdown] when JVM is shutting down.

Author: Srinivasa Reddy Vundela <vsr@cloudera.com>

Closes #9809 from vundela/master_11799.
2015-11-19 14:51:40 -08:00
Shixiong Zhu 90d384dcbc [SPARK-11831][CORE][TESTS] Use port 0 to avoid port conflicts in tests
Use port 0 to fix port-contention-related flakiness

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9841 from zsxwing/SPARK-11831.
2015-11-19 14:49:25 -08:00
zsxwing 72d150c271 [SPARK-11830][CORE] Make NettyRpcEnv bind to the specified host
This PR includes the following change:

1. Bind NettyRpcEnv to the specified host
2. Fix the port information in the log for NettyRpcEnv.
3. Fix the service name of NettyRpcEnv.

Author: zsxwing <zsxwing@gmail.com>
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9821 from zsxwing/SPARK-11830.
2015-11-19 11:57:50 -08:00
Nong Li 6d0848b53b [SPARK-11787][SQL] Improve Parquet scan performance when using flat schemas.
This patch adds an alternate to the Parquet RecordReader from the parquet-mr project
that is much faster for flat schemas. Instead of using the general converter mechanism
from parquet-mr, this directly uses the lower level APIs from parquet-columnar and a
customer RecordReader that directly assembles into UnsafeRows.

This is optionally disabled and only used for supported schemas.

Using the tpcds store sales table and doing a sum of increasingly more columns, the results
are:

For 1 Column:
  Before: 11.3M rows/second
  After: 18.2M rows/second

For 2 Columns:
  Before: 7.2M rows/second
  After: 11.2M rows/second

For 5 Columns:
  Before: 2.9M rows/second
  After: 4.5M rows/second

Author: Nong Li <nong@databricks.com>

Closes #9774 from nongli/parquet.
2015-11-18 18:38:45 -08:00
Josh Rosen 4b11712190 [SPARK-11495] Fix potential socket / file handle leaks that were found via static analysis
The HP Fortify Opens Source Review team (https://www.hpfod.com/open-source-review-project) reported a handful of potential resource leaks that were discovered using their static analysis tool. We should fix the issues identified by their scan.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9455 from JoshRosen/fix-potential-resource-leaks.
2015-11-18 16:00:35 -08:00
Derek Dagit c07a50b862 [SPARK-10930] History "Stages" page "duration" can be confusing
Author: Derek Dagit <derekd@yahoo-inc.com>

Closes #9051 from d2r/spark-10930-ui-max-task-dur.
2015-11-18 15:56:54 -08:00
Josh Rosen 3a9851936d [SPARK-11649] Properly set Akka frame size in SparkListenerSuite test
SparkListenerSuite's _"onTaskGettingResult() called when result fetched remotely"_ test was extremely slow (1 to 4 minutes to run) and recently became extremely flaky, frequently failing with OutOfMemoryError.

The root cause was the fact that this was using `System.setProperty` to set the Akka frame size, which was not actually modifying the frame size. As a result, this test would allocate much more data than necessary. The fix here is to simply use SparkConf in order to configure the frame size.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9822 from JoshRosen/SPARK-11649.
2015-11-18 15:55:41 -08:00
Shixiong Zhu 7c5b641808 [SPARK-10745][CORE] Separate configs between shuffle and RPC
[SPARK-6028](https://issues.apache.org/jira/browse/SPARK-6028) uses network module to implement RPC. However, there are some configurations named with `spark.shuffle` prefix in the network module.

This PR refactors them to make sure the user can control them in shuffle and RPC separately. The user can use `spark.rpc.*` to set the configuration for netty RPC.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9481 from zsxwing/SPARK-10745.
2015-11-18 12:53:22 -08:00
Reynold Xin a416e41e28 [SPARK-11809] Switch the default Mesos mode to coarse-grained mode
Based on my conversions with people, I believe the consensus is that the coarse-grained mode is more stable and easier to reason about. It is best to use that as the default rather than the more flaky fine-grained mode.

Author: Reynold Xin <rxin@databricks.com>

Closes #9795 from rxin/SPARK-11809.
2015-11-18 12:50:29 -08:00
Bryan Cutler 31921e0f0b [SPARK-4557][STREAMING] Spark Streaming foreachRDD Java API method should accept a VoidFunction<...>
Currently streaming foreachRDD Java API uses a function prototype requiring a return value of null.  This PR deprecates the old method and uses VoidFunction to allow for more concise declaration.  Also added VoidFunction2 to Java API in order to use in Streaming methods.  Unit test is added for using foreachRDD with VoidFunction, and changes have been tested with Java 7 and Java 8 using lambdas.

Author: Bryan Cutler <bjcutler@us.ibm.com>

Closes #9488 from BryanCutler/foreachRDD-VoidFunction-SPARK-4557.
2015-11-18 12:09:54 -08:00
Yin Huai 6f99522d13 [SPARK-11792] [SQL] [FOLLOW-UP] Change SizeEstimation to KnownSizeEstimation and make estimatedSize return Long instead of Option[Long]
https://issues.apache.org/jira/browse/SPARK-11792

The main changes include:
* Renaming `SizeEstimation` to `KnownSizeEstimation`. Hopefully this new name has more information.
* Making `estimatedSize` return `Long` instead of `Option[Long]`.
* In `UnsaveHashedRelation`, `estimatedSize` will delegate the work to `SizeEstimator` if we have not created a `BytesToBytesMap`.

Since we will put `UnsaveHashedRelation` to `BlockManager`, it is generally good to let it provide a more accurate size estimation. Also, if we do not put `BytesToBytesMap` directly into `BlockerManager`, I feel it is not really necessary to make `BytesToBytesMap` extends `KnownSizeEstimation`.

Author: Yin Huai <yhuai@databricks.com>

Closes #9813 from yhuai/SPARK-11792-followup.
2015-11-18 11:49:12 -08:00
Hurshal Patel 3cca5ffb3d [SPARK-11195][CORE] Use correct classloader for TaskResultGetter
Make sure we are using the context classloader when deserializing failed TaskResults instead of the Spark classloader.

The issue is that `enqueueFailedTask` was using the incorrect classloader which results in `ClassNotFoundException`.

Adds a test in TaskResultGetterSuite that compiles a custom exception, throws it on the executor, and asserts that Spark handles the TaskResult deserialization instead of returning `UnknownReason`.

See #9367 for previous comments
See SPARK-11195 for a full repro

Author: Hurshal Patel <hpatel516@gmail.com>

Closes #9779 from choochootrain/spark-11195-master.
2015-11-18 09:28:59 -08:00
Jean-Baptiste Onofré e62820c85f [SPARK-6541] Sort executors by ID (numeric)
"Force" the executor ID sort with Int.

Author: Jean-Baptiste Onofré <jbonofre@apache.org>

Closes #9165 from jbonofre/SPARK-6541.
2015-11-18 08:57:58 +00:00
Yin Huai 1714350bdd [SPARK-11792][SQL] SizeEstimator cannot provide a good size estimation of UnsafeHashedRelations
https://issues.apache.org/jira/browse/SPARK-11792

Right now, SizeEstimator will "think" a small UnsafeHashedRelation is several GBs.

Author: Yin Huai <yhuai@databricks.com>

Closes #9788 from yhuai/SPARK-11792.
2015-11-18 00:42:52 -08:00
tedyu 446738e51f [SPARK-11761] Prevent the call to StreamingContext#stop() in the listener bus's thread
See discussion toward the tail of https://github.com/apache/spark/pull/9723
From zsxwing :
```
The user should not call stop or other long-time work in a listener since it will block the listener thread, and prevent from stopping SparkContext/StreamingContext.

I cannot see an approach since we need to stop the listener bus's thread before stopping SparkContext/StreamingContext totally.
```
Proposed solution is to prevent the call to StreamingContext#stop() in the listener bus's thread.

Author: tedyu <yuzhihong@gmail.com>

Closes #9741 from tedyu/master.
2015-11-17 22:47:53 -08:00
Kent Yao e33053ee00 [SPARK-11583] [CORE] MapStatus Using RoaringBitmap More Properly
This PR upgrade the version of RoaringBitmap to 0.5.10, to optimize the memory layout, will be much smaller when most of blocks are empty.

This PR is based on #9661 (fix conflicts), see all of the comments at https://github.com/apache/spark/pull/9661 .

Author: Kent Yao <yaooqinn@hotmail.com>
Author: Davies Liu <davies@databricks.com>
Author: Charles Allen <charles@allen-net.com>

Closes #9746 from davies/roaring_mapstatus.
2015-11-17 19:44:29 -08:00
Davies Liu bf25f9bdfc [SPARK-11016] Move RoaringBitmap to explicit Kryo serializer
Fix the serialization of RoaringBitmap with Kyro serializer

This PR came from https://github.com/metamx/spark/pull/1, thanks to drcrallen

Author: Davies Liu <davies@databricks.com>
Author: Charles Allen <charles@allen-net.com>

Closes #9748 from davies/SPARK-11016.
2015-11-17 19:39:39 -08:00
Jacek Lewandowski b362d50fca [SPARK-11726] Throw exception on timeout when waiting for REST server response
Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>

Closes #9692 from jacek-lewandowski/SPARK-11726.
2015-11-17 16:00:00 -08:00
Grace 965245d087 [SPARK-9552] Add force control for killExecutors to avoid false killing for those busy executors
By using the dynamic allocation, sometimes it occurs false killing for those busy executors. Some executors with assignments will be killed because of being idle for enough time (say 60 seconds). The root cause is that the Task-Launch listener event is asynchronized.

For example, some executors are under assigning tasks, but not sending out the listener notification yet. Meanwhile, the dynamic allocation's executor idle time is up (e.g., 60 seconds). It will trigger killExecutor event at the same time.
 1. the timer expiration starts before the listener event arrives.
 2. Then, the task is going to run on top of that killed/killing executor. It will lead to task failure finally.

Here is the proposal to fix it. We can add the force control for killExecutor. If the force control is not set (i.e., false), we'd better to check if the executor under killing is idle or busy. If the current executor has some assignment, we should not kill that executor and return back false (to indicate killing failure). In dynamic allocation, we'd better to turn off force killing (i.e., force = false), we will meet killing failure if tries to kill a busy executor. And then, the executor timer won't be invalid. Later on, the task assignment event arrives, we can remove the idle timer accordingly. So that we can avoid false killing for those busy executors in dynamic allocation.

For the rest of usages, the end users can decide if to use force killing or not by themselves.  If to turn on that option, the killExecutor will do the action without any status checking.

Author: Grace <jie.huang@intel.com>
Author: Andrew Or <andrew@databricks.com>
Author: Jie Huang <jie.huang@intel.com>

Closes #7888 from GraceH/forcekill.
2015-11-17 15:43:35 -08:00
Marcelo Vanzin 936bc0bcbf [SPARK-11786][CORE] Tone down messages from akka error monitor.
There events happen normally during the app's lifecycle, so printing
out ERROR logs all the time is misleading, and can actually affect usability
of interactive shells.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9772 from vanzin/SPARK-11786.
2015-11-17 14:23:28 -08:00
Chris Bannister cc567b6634 [SPARK-11695][CORE] Set s3a credentials
Set s3a credentials when creating a new default hadoop configuration.

Author: Chris Bannister <chris.bannister@swiftkey.com>

Closes #9663 from Zariel/set-s3a-creds.
2015-11-17 10:03:46 -08:00
Kousuke Saruta 30f3cfda1c [SPARK-11480][CORE][WEBUI] Wrong callsite is displayed when using AsyncRDDActions#takeAsync
When we call AsyncRDDActions#takeAsync, actually another DAGScheduler#runJob is called from another thread so we cannot get proper callsite infomation.

Following screenshots are before this patch applied and after.

Before:
<img width="1268" alt="2015-11-04 1 26 40" src="https://cloud.githubusercontent.com/assets/4736016/10914069/0ffc1306-8294-11e5-8e89-c4fadf58dd12.png">
<img width="1258" alt="2015-11-04 1 26 52" src="https://cloud.githubusercontent.com/assets/4736016/10914070/0ffe84ce-8294-11e5-8b2a-69d36276bedb.png">

After:
<img width="1268" alt="2015-11-04 0 48 07" src="https://cloud.githubusercontent.com/assets/4736016/10914080/1d8cfb7a-8294-11e5-9e09-ede25c2563e8.png">
<img width="1269" alt="2015-11-04 0 48 26" src="https://cloud.githubusercontent.com/assets/4736016/10914081/1d934e3a-8294-11e5-8b5e-e3dc37aaced3.png">

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #9437 from sarutak/SPARK-11480.
2015-11-16 16:59:16 -08:00
Shivaram Venkataraman ea6f53e48a [SPARKR][HOTFIX] Disable flaky SparkR package build test
See https://github.com/apache/spark/pull/9390#issuecomment-157160063 and https://gist.github.com/shivaram/3a2fecce60768a603dac for more information

Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu>

Closes #9744 from shivaram/sparkr-package-test-disable.
2015-11-16 16:57:50 -08:00
Wenchen Fan fd14936be7 [SPARK-11625][SQL] add java test for typed aggregate
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9591 from cloud-fan/agg-test.
2015-11-16 15:32:49 -08:00
Davies Liu 3c025087b5 Revert "[SPARK-11271][SPARK-11016][CORE] Use Spark BitSet instead of RoaringBitmap to reduce memory usage"
This reverts commit e209fa271a.
2015-11-16 14:50:38 -08:00
jerryshao 24477d2705 [SPARK-11718][YARN][CORE] Fix explicitly killed executor dies silently issue
Currently if dynamic allocation is enabled, explicitly killing executor will not get response, so the executor metadata is wrong in driver side. Which will make dynamic allocation on Yarn fail to work.

The problem is  `disableExecutor` returns false for pending killing executors when `onDisconnect` is detected, so no further implementation is done.

One solution is to bypass these explicitly killed executors to use `super.onDisconnect` to remove executor. This is simple.

Another solution is still querying the loss reason for these explicitly kill executors. Since executor may get killed and informed in the same AM-RM communication, so current way of adding pending loss reason request is not worked (container complete is already processed), here we should store this loss reason for later query.

Here this PR chooses solution 2.

Please help to review. vanzin I think this part is changed by you previously, would you please help to review? Thanks a lot.

Author: jerryshao <sshao@hortonworks.com>

Closes #9684 from jerryshao/SPARK-11718.
2015-11-16 11:43:18 -08:00