Commit graph

5038 commits

Author SHA1 Message Date
Josh Rosen 2d76e44b1a [SPARK-11647] Attempt to reduce time/flakiness of Thriftserver CLI and SparkSubmit tests
This patch aims to reduce the test time and flakiness of HiveSparkSubmitSuite, SparkSubmitSuite, and CliSuite.

Key changes:

- Disable IO synchronization calls for Derby writes, since durability doesn't matter for tests. This was done for HiveCompatibilitySuite in #6651 and resulted in huge test speedups.
- Add a few missing `--conf`s to disable various Spark UIs. The CliSuite, in particular, never disabled these UIs, leaving it prone to port-contention-related flakiness.
- Fix two instances where tests defined `beforeAll()` methods which were never called because the appropriate traits were not mixed in. I updated these tests suites to extend `BeforeAndAfterEach` so that they play nicely with our `ResetSystemProperties` trait.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9623 from JoshRosen/SPARK-11647.
2015-11-11 14:30:38 -08:00
Josh Rosen 529a1d3380 [SPARK-6152] Use shaded ASM5 to support closure cleaning of Java 8 compiled classes
This patch modifies Spark's closure cleaner (and a few other places) to use ASM 5, which is necessary in order to support cleaning of closures that were compiled by Java 8.

In order to avoid ASM dependency conflicts, Spark excludes ASM from all of its dependencies and uses a shaded version of ASM 4 that comes from `reflectasm` (see [SPARK-782](https://issues.apache.org/jira/browse/SPARK-782) and #232). This patch updates Spark to use a shaded version of ASM 5.0.4 that was published by the Apache XBean project; the POM used to create the shaded artifact can be found at https://github.com/apache/geronimo-xbean/blob/xbean-4.4/xbean-asm5-shaded/pom.xml.

http://movingfulcrum.tumblr.com/post/80826553604/asm-framework-50-the-missing-migration-guide was a useful resource while upgrading the code to use the new ASM5 opcodes.

I also added a new regression tests in the `java8-tests` subproject; the existing tests were insufficient to catch this bug, which only affected Scala 2.11 user code which was compiled targeting Java 8.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9512 from JoshRosen/SPARK-6152.
2015-11-11 11:16:39 -08:00
Reynold Xin 95daff6459 [SPARK-11646] WholeTextFileRDD should return Text rather than String
If it returns Text, we can reuse this in Spark SQL to provide a WholeTextFile data source and directly convert the Text into UTF8String without extra string decoding and encoding.

Author: Reynold Xin <rxin@databricks.com>

Closes #9622 from rxin/SPARK-11646.
2015-11-11 10:17:54 -08:00
Josh Rosen fac53d8ec0 [SPARK-10192][HOTFIX] Fix NPE in test that was added in #8402
This fixes an NPE introduced in SPARK-10192 / #8402.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9620 from JoshRosen/SPARK-10192-hotfix.
2015-11-10 22:24:00 -08:00
Tathagata Das 6600786ddd [SPARK-11361][STREAMING] Show scopes of RDD operations inside DStream.foreachRDD and DStream.transform in DAG viz
Currently, when a DStream sets the scope for RDD generated by it, that scope is not allowed to be overridden by the RDD operations. So in case of `DStream.foreachRDD`, all the RDDs generated inside the foreachRDD get the same scope - `foreachRDD  <time>`, as set by the `ForeachDStream`. So it is hard to debug generated RDDs in the RDD DAG viz in the Spark UI.

This patch allows the RDD operations inside `DStream.transform` and `DStream.foreachRDD` to append their own scopes to the earlier DStream scope.

I have also slightly tweaked how callsites are set such that the short callsite reflects the RDD operation name and line number. This tweak is necessary as callsites are not managed through scopes (which support nesting and overriding) and I didnt want to add another local property to control nesting and overriding of callsites.

## Before:
![image](https://cloud.githubusercontent.com/assets/663212/10808548/fa71c0c4-7da9-11e5-9af0-5737793a146f.png)

## After:
![image](https://cloud.githubusercontent.com/assets/663212/10808659/37bc45b6-7dab-11e5-8041-c20be6a9bc26.png)

The code that was used to generate this is:
```
    val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER)
    val words = lines.flatMap(_.split(" "))
    val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
    wordCounts.foreachRDD { rdd =>
      val temp = rdd.map { _ -> 1 }.reduceByKey( _ + _)
      val temp2 = temp.map { _ -> 1}.reduceByKey(_ + _)
      val count = temp2.count
      println(count)
    }
```

Note
- The inner scopes of the RDD operations map/reduceByKey inside foreachRDD is visible
- The short callsites of stages refers to the line number of the RDD ops rather than the same line number of foreachRDD in all three cases.

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

Closes #9315 from tdas/SPARK-11361.
2015-11-10 16:54:06 -08:00
tedyu 9009175416 [SPARK-11615] Drop @VisibleForTesting annotation
See http://search-hadoop.com/m/q3RTtjpe8r1iRbTj2 for discussion.

Summary: addition of VisibleForTesting annotation resulted in spark-shell malfunctioning.

Author: tedyu <yuzhihong@gmail.com>

Closes #9585 from tedyu/master.
2015-11-10 16:52:59 -08:00
tedyu 3e0a6cf1e0 [SPARK-11572] Exit AsynchronousListenerBus thread when stop() is called
As vonnagy reported in the following thread:
http://search-hadoop.com/m/q3RTtk982kvIow22

Attempts to join the thread in AsynchronousListenerBus resulted in lock up because AsynchronousListenerBus thread was still getting messages `SparkListenerExecutorMetricsUpdate` from the DAGScheduler

Author: tedyu <yuzhihong@gmail.com>

Closes #9546 from ted-yu/master.
2015-11-10 16:51:25 -08:00
Imran Rashid 33112f9c48 [SPARK-10192][CORE] simple test w/ failure involving a shared dependency
just trying to increase test coverage in the scheduler, this already works.  It includes a regression test for SPARK-9809

copied some test utils from https://github.com/apache/spark/pull/5636, we can wait till that is merged first

Author: Imran Rashid <irashid@cloudera.com>

Closes #8402 from squito/test_retry_in_shared_shuffle_dep.
2015-11-10 16:50:22 -08:00
Bryan Cutler a3989058c0 [SPARK-10827][CORE] AppClient should not use askWithReply in receiveAndReply
Changed AppClient to be non-blocking in `receiveAndReply` by using a separate thread to wait for response and reply to the context.  The threads are managed by a thread pool.  Also added unit tests for the AppClient interface.

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

Closes #9317 from BryanCutler/appClient-receiveAndReply-SPARK-10827.
2015-11-10 16:32:32 -08:00
Lianhui Wang 6e5fc37883 [SPARK-11252][NETWORK] ShuffleClient should release connection after fetching blocks had been completed for external shuffle
with yarn's external shuffle, ExternalShuffleClient of executors reserve its connections for yarn's NodeManager until application has been completed. so it will make NodeManager and executors have many socket connections.
in order to reduce network pressure of NodeManager's shuffleService, after registerWithShuffleServer or fetchBlocks have been completed in ExternalShuffleClient, connection for NM's shuffleService needs to be closed.andrewor14 rxin vanzin

Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #9227 from lianhuiwang/spark-11252.
2015-11-10 10:40:08 -08:00
Paul Chandler 5507a9d093 Fix typo in driver page
"Comamnd property" => "Command property"

Author: Paul Chandler <pestilence669@users.noreply.github.com>

Closes #9578 from pestilence669/fix_spelling.
2015-11-10 12:59:53 +01:00
Wenchen Fan fcb57e9c73 [SPARK-11564][SQL][FOLLOW-UP] improve java api for GroupedDataset
created `MapGroupFunction`, `FlatMapGroupFunction`, `CoGroupFunction`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9564 from cloud-fan/map.
2015-11-09 15:16:47 -08:00
Felix Bechstein 5039a49b63 [SPARK-10471][CORE][MESOS] prevent getting offers for unmet constraints
this change rejects offers for slaves with unmet constraints for 120s to mitigate offer starvation.
this prevents mesos to send us these offers again and again.
in return, we get more offers for slaves which might meet our constraints.
and it enables mesos to send the rejected offers to other frameworks.

Author: Felix Bechstein <felix.bechstein@otto.de>

Closes #8639 from felixb/decline_offers_constraint_mismatch.
2015-11-09 13:36:14 -08:00
tedyu 404a28f4ed [SPARK-11112] Fix Scala 2.11 compilation error in RDDInfo.scala
As shown in https://amplab.cs.berkeley.edu/jenkins/view/Spark-QA-Compile/job/Spark-Master-Scala211-Compile/1946/console , compilation fails with:
```
[error] /home/jenkins/workspace/Spark-Master-Scala211-Compile/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala:25: in class RDDInfo, multiple overloaded alternatives of constructor RDDInfo define default arguments.
[error] class RDDInfo(
[error]
```
This PR tries to fix the compilation error

Author: tedyu <yuzhihong@gmail.com>

Closes #9538 from tedyu/master.
2015-11-09 10:07:58 -08:00
Charles Yeh 08a7a836c3 [SPARK-10565][CORE] add missing web UI stats to /api/v1/applications JSON
I looked at the other endpoints, and they don't seem to be missing any fields.
Added fields:
![image](https://cloud.githubusercontent.com/assets/613879/10948801/58159982-82e4-11e5-86dc-62da201af910.png)

Author: Charles Yeh <charlesyeh@dropbox.com>

Closes #9472 from CharlesYeh/api_vars.
2015-11-09 11:59:32 -06:00
Reynold Xin 97b7080cf2 [SPARK-11564][SQL] Dataset Java API audit
A few changes:

1. Removed fold, since it can be confusing for distributed collections.
2. Created specific interfaces for each Dataset function (e.g. MapFunction, ReduceFunction, MapPartitionsFunction)
3. Added more documentation and test cases.

The other thing I'm considering doing is to have a "collector" interface for FlatMapFunction and MapPartitionsFunction, similar to MapReduce's map function.

Author: Reynold Xin <rxin@databricks.com>

Closes #9531 from rxin/SPARK-11564.
2015-11-08 20:57:09 -08:00
Andrew Or 7f741905b0 [SPARK-11112] DAG visualization: display RDD callsite
<img width="548" alt="screen shot 2015-11-01 at 9 42 33 am" src="https://cloud.githubusercontent.com/assets/2133137/10870343/2a8cd070-807d-11e5-857a-4ebcace77b5b.png">
mateiz sarutak

Author: Andrew Or <andrew@databricks.com>

Closes #9398 from andrewor14/rdd-callsite.
2015-11-07 05:35:53 +01:00
Josh Rosen 30b706b7b3 [SPARK-11389][CORE] Add support for off-heap memory to MemoryManager
In order to lay the groundwork for proper off-heap memory support in SQL / Tungsten, we need to extend our MemoryManager to perform bookkeeping for off-heap memory.

## User-facing changes

This PR introduces a new configuration, `spark.memory.offHeapSize` (name subject to change), which specifies the absolute amount of off-heap memory that Spark and Spark SQL can use. If Tungsten is configured to use off-heap execution memory for allocating data pages, then all data page allocations must fit within this size limit.

## Internals changes

This PR contains a lot of internal refactoring of the MemoryManager. The key change at the heart of this patch is the introduction of a `MemoryPool` class (name subject to change) to manage the bookkeeping for a particular category of memory (storage, on-heap execution, and off-heap execution). These MemoryPools are not fixed-size; they can be dynamically grown and shrunk according to the MemoryManager's policies. In StaticMemoryManager, these pools have fixed sizes, proportional to the legacy `[storage|shuffle].memoryFraction`. In the new UnifiedMemoryManager, the sizes of these pools are dynamically adjusted according to its policies.

There are two subclasses of `MemoryPool`: `StorageMemoryPool` manages storage memory and `ExecutionMemoryPool` manages execution memory. The MemoryManager creates two execution pools, one for on-heap memory and one for off-heap. Instances of `ExecutionMemoryPool` manage the logic for fair sharing of their pooled memory across running tasks (in other words, the ShuffleMemoryManager-like logic has been moved out of MemoryManager and pushed into these ExecutionMemoryPool instances).

I think that this design is substantially easier to understand and reason about than the previous design, where most of these responsibilities were handled by MemoryManager and its subclasses. To see this, take at look at how simple the logic in `UnifiedMemoryManager` has become: it's now very easy to see when memory is dynamically shifted between storage and execution.

## TODOs

- [x] Fix handful of test failures in the MemoryManagerSuites.
- [x] Fix remaining TODO comments in code.
- [ ] Document new configuration.
- [x] Fix commented-out tests / asserts:
  - [x] UnifiedMemoryManagerSuite.
- [x] Write tests that exercise the new off-heap memory management policies.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9344 from JoshRosen/offheap-memory-accounting.
2015-11-06 18:17:34 -08:00
Imran Rashid 49f1a82037 [SPARK-10116][CORE] XORShiftRandom.hashSeed is random in high bits
https://issues.apache.org/jira/browse/SPARK-10116

This is really trivial, just happened to notice it -- if `XORShiftRandom.hashSeed` is really supposed to have random bits throughout (as the comment implies), it needs to do something for the conversion to `long`.

mengxr mkolod

Author: Imran Rashid <irashid@cloudera.com>

Closes #8314 from squito/SPARK-10116.
2015-11-06 20:06:24 +00:00
Jacek Laskowski 62bb290773 Typo fixes + code readability improvements
Author: Jacek Laskowski <jacek.laskowski@deepsense.io>

Closes #9501 from jaceklaskowski/typos-with-style.
2015-11-06 20:05:18 +00:00
Davies Liu eec74ba8bd [SPARK-7542][SQL] Support off-heap index/sort buffer
This brings the support of off-heap memory for array inside BytesToBytesMap and InMemorySorter, then we could allocate all the memory from off-heap for execution.

Closes #8068

Author: Davies Liu <davies@databricks.com>

Closes #9477 from davies/unsafe_timsort.
2015-11-05 19:02:18 -08:00
Srinivasa Reddy Vundela c76865c622 [SPARK-11484][WEBUI] Using proxyBase set by spark AM
Use the proxyBase set by the AM, if not found then use env. This is to fix the issue if somebody accidentally set APPLICATION_WEB_PROXY_BASE to wrong proxyBase

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

Closes #9448 from vundela/master.
2015-11-05 11:30:44 -08:00
Nishkam Ravi a4b5cefcf1 [SPARK-11501][CORE][YARN] Propagate spark.rpc config to executors
spark.rpc is supposed to be configurable but is not currently (doesn't get propagated to executors because RpcEnv.create is done before driver properties are fetched).

Author: Nishkam Ravi <nishkamravi@gmail.com>

Closes #9460 from nishkamravi2/master_akka.
2015-11-05 09:35:49 -08:00
Herman van Hovell 7bdc92197c [SPARK-11449][CORE] PortableDataStream should be a factory
```PortableDataStream``` maintains some internal state. This makes it tricky to reuse a stream (one needs to call ```close``` on both the ```PortableDataStream``` and the ```InputStream``` it produces).

This PR removes all state from ```PortableDataStream``` and effectively turns it into an ```InputStream```/```Array[Byte]``` factory. This makes the user responsible for managing the ```InputStream``` it returns.

cc srowen

Author: Herman van Hovell <hvanhovell@questtec.nl>

Closes #9417 from hvanhovell/SPARK-11449.
2015-11-05 09:23:09 +00:00
Sean Owen 6f81eae24f [SPARK-11440][CORE][STREAMING][BUILD] Declare rest of @Experimental items non-experimental if they've existed since 1.2.0
Remove `Experimental` annotations in core, streaming for items that existed in 1.2.0 or before. The changes are:

* SparkContext
  * binary{Files,Records} : 1.2.0
  * submitJob : 1.0.0
* JavaSparkContext
  * binary{Files,Records} : 1.2.0
* DoubleRDDFunctions, JavaDoubleRDD
  * {mean,sum}Approx : 1.0.0
* PairRDDFunctions, JavaPairRDD
  * sampleByKeyExact : 1.2.0
  * countByKeyApprox : 1.0.0
* PairRDDFunctions
  * countApproxDistinctByKey : 1.1.0
* RDD
  * countApprox, countByValueApprox, countApproxDistinct : 1.0.0
* JavaRDDLike
  * countApprox : 1.0.0
* PythonHadoopUtil.Converter : 1.1.0
* PortableDataStream : 1.2.0 (related to binaryFiles)
* BoundedDouble : 1.0.0
* PartialResult : 1.0.0
* StreamingContext, JavaStreamingContext
  * binaryRecordsStream : 1.2.0
* HiveContext
  * analyze : 1.2.0

Author: Sean Owen <sowen@cloudera.com>

Closes #9396 from srowen/SPARK-11440.
2015-11-05 09:08:53 +00:00
Davies Liu 81498dd5c8 [SPARK-11425] [SPARK-11486] Improve hybrid aggregation
After aggregation, the dataset could be smaller than inputs, so it's better to do hash based aggregation for all inputs, then using sort based aggregation to merge them.

Author: Davies Liu <davies@databricks.com>

Closes #9383 from davies/fix_switch.
2015-11-04 21:30:21 -08:00
Josh Rosen d0b5633962 [SPARK-11307] Reduce memory consumption of OutputCommitCoordinator
OutputCommitCoordinator uses a map in a place where an array would suffice, increasing its memory consumption for result stages with millions of tasks.

This patch replaces that map with an array. The only tricky part of this is reasoning about the range of possible array indexes in order to make sure that we never index out of bounds.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9274 from JoshRosen/SPARK-11307.
2015-11-04 17:19:52 -08:00
Davies Liu 1b6a5d4af9 [SPARK-11493] remove bitset from BytesToBytesMap
Since we have 4 bytes as number of records in the beginning of a page, the address can not be zero, so we do not need the bitset.

For performance concerns, the bitset could help speed up false lookup if the slot is empty (because bitset is smaller than longArray, cache hit rate will be higher). In practice, the map is filled with 35% - 70% (use 50% as average), so only half of the false lookups can benefit of it, all others will pay the cost of load the bitset (still need to access the longArray anyway).

For aggregation, we always need to access the longArray (insert a new key after false lookup), also confirmed by a benchmark.

 For broadcast hash join, there could be a regression, but a simple benchmark showed that it may not (most of lookup are false):

```
sqlContext.range(1<<20).write.parquet("small")
df = sqlContext.read.parquet('small')
for i in range(3):
    t = time.time()
    df2 = sqlContext.range(1<<26).selectExpr("id * 1111111111 % 987654321 as id2")
    df2.join(df, df.id == df2.id2).count()
    print time.time() -t
```

Having bitset (used time in seconds):
```
17.5404241085
10.2758829594
10.5786800385
```
After removing bitset (used time in seconds):
```
21.8939979076
12.4132959843
9.97224712372
```

cc rxin nongli

Author: Davies Liu <davies@databricks.com>

Closes #9452 from davies/remove_bitset.
2015-11-04 14:45:02 -08:00
Adam Roberts 701fb50520 [SPARK-10949] Update Snappy version to 1.1.2
This is an updated version of #8995 by a-roberts. Original description follows:

Snappy now supports concatenation of serialized streams, this patch contains a version number change and the "does not support" test is now a "supports" test.

Snappy 1.1.2 changelog mentions:

> snappy-java-1.1.2 (22 September 2015)
> This is a backward compatible release for 1.1.x.
> Add AIX (32-bit) support.
> There is no upgrade for the native libraries of the other platforms.

> A major change since 1.1.1 is a support for reading concatenated results of SnappyOutputStream(s)
> snappy-java-1.1.2-RC2 (18 May 2015)
> Fix #107: SnappyOutputStream.close() is not idempotent
> snappy-java-1.1.2-RC1 (13 May 2015)
> SnappyInputStream now supports reading concatenated compressed results of SnappyOutputStream
> There has been no compressed format change since 1.0.5.x. So You can read the compressed results > interchangeablly between these versions.
> Fixes a problem when java.io.tmpdir does not exist.

Closes #8995.

Author: Adam Roberts <aroberts@uk.ibm.com>
Author: Josh Rosen <joshrosen@databricks.com>

Closes #9439 from JoshRosen/update-snappy.
2015-11-04 14:03:31 -08:00
Reynold Xin d19f4fda63 [SPARK-11505][SQL] Break aggregate functions into multiple files
functions.scala was getting pretty long. I broke it into multiple files.

I also added explicit data types for some public vals, and renamed aggregate function pretty names to lower case, which is more consistent with rest of the functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #9471 from rxin/SPARK-11505.
2015-11-04 13:44:07 -08:00
Marcelo Vanzin 8790ee6d69 [SPARK-10622][CORE][YARN] Differentiate dead from "mostly dead" executors.
In YARN mode, when preemption is enabled, we may leave executors in a
zombie state while we wait to retrieve the reason for which the executor
exited. This is so that we don't account for failed tasks that were
running on a preempted executor.

The issue is that while we wait for this information, the scheduler
might decide to schedule tasks on the executor, which will never be
able to run them. Other side effects include the block manager still
considering the executor available to cache blocks, for example.

So, when we know that an executor went down but we don't know why,
stop everything related to the executor, except its running tasks.
Only when we know the reason for the exit (or give up waiting for
it) we do update the running tasks.

This is achieved by a new `disableExecutor()` method in the
`Schedulable` interface. For managers that do not behave like this
(i.e. every one but YARN), the existing `executorLost()` method
will behave the same way it did before.

On top of that change, a few minor changes that made debugging easier,
and fixed some other minor issues:
- The cluster-mode AM was printing a misleading log message every
  time an executor disconnected from the driver (because the akka
  actor system was shared between driver and AM).
- Avoid sending unnecessary requests for an executor's exit reason
  when we already know it was explicitly disabled / killed. This
  avoids both multiple requests, and unnecessary requests that would
  just cause warning messages on the AM (in the explicit kill case).
- Tone down a log message about the executor being lost when it
  exited normally (e.g. preemption)
- Wake up the AM monitor thread when requests for executor loss
  reasons arrive too, so that we can more quickly remove executors
  from this zombie state.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8887 from vanzin/SPARK-10622.
2015-11-04 09:07:22 -08:00
tedyu c09e513987 [SPARK-11442] Reduce numSlices for local metrics test of SparkListenerSuite
In the thread, http://search-hadoop.com/m/q3RTtcQiFSlTxeP/test+failed+due+to+OOME&subj=test+failed+due+to+OOME, it was discussed that memory consumption for SparkListenerSuite should be brought down.

This is an attempt in that direction by reducing numSlices for local metrics test.

Author: tedyu <yuzhihong@gmail.com>

Closes #9384 from tedyu/master.
2015-11-04 10:51:40 +00:00
Marcelo Vanzin 53e9cee3e4 [SPARK-11466][CORE] Avoid mockito in multi-threaded FsHistoryProviderSuite test.
The test functionality should be the same, but without using mockito; logs don't
really say anything useful but I suspect it may be the cause of the flakiness,
since updating mocks when multiple threads may be using it doesn't work very
well. It also allows some other cleanup (= less test code in FsHistoryProvider).

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9425 from vanzin/SPARK-11466.
2015-11-03 16:26:28 -08:00
Jacek Laskowski 680b4e7bca Fix typo in WebUI
Author: Jacek Laskowski <jacek.laskowski@deepsense.io>

Closes #9444 from jaceklaskowski/TImely-fix.
2015-11-03 15:26:35 -08:00
Mark Grover b2e4b314d9 [SPARK-9790][YARN] Expose in WebUI if NodeManager is the reason why executors were killed.
Author: Mark Grover <grover.markgrover@gmail.com>

Closes #8093 from markgrover/nm2.
2015-11-03 08:51:40 -08:00
Reynold Xin 57446eb69c [SPARK-11256] Mark all Stage/ResultStage/ShuffleMapStage internal state as private.
Author: Reynold Xin <rxin@databricks.com>

Closes #9219 from rxin/stage-cleanup1.
2015-11-03 07:06:00 -08:00
Jacek Lewandowski 233e534ac4 [SPARK-11344] Made ApplicationDescription and DriverDescription case classes
DriverDescription refactored to case class because it included no mutable fields.

ApplicationDescription had one mutable field, which was appUiUrl. This field was set by the driver to point to the driver web UI. Master was modifying this field when the application was removed to redirect requests to history server. This was wrong because objects which are sent over the wire should be immutable. Now appUiUrl is immutable in ApplicationDescription and always points to the driver UI even if it is already shutdown. The UI url which master exposes to the user and modifies dynamically is now included into ApplicationInfo - a data object which describes the application state internally in master. That URL in ApplicationInfo is initialised with the value from ApplicationDescription.

ApplicationDescription also included value user, which is now a part of case class fields.

Author: Jacek Lewandowski <lewandowski.jacek@gmail.com>

Closes #9299 from jacek-lewandowski/SPARK-11344.
2015-11-03 12:46:11 +00:00
Calvin Jia 476f4348e2 [SPARK-11236] [TEST-MAVEN] [TEST-HADOOP1.0] [CORE] Update Tachyon dependency 0.7.1 -> 0.8.1
This is a reopening of #9204 which failed hadoop1 sbt tests.

With the original PR, a classpath issue would occur due to the MIMA plugin pulling in hadoop-2.2 dependencies regardless of the hadoop version when building the `oldDeps` project. These affect the hadoop1 sbt build because they are placed in `lib_managed` and Tachyon 0.8.0's default hadoop version is 2.2.

Author: Calvin Jia <jia.calvin@gmail.com>

Closes #9395 from calvinjia/spark-11236.
2015-11-02 17:02:31 -08:00
Marcelo Vanzin 71d1c907de [SPARK-10997][CORE] Add "client mode" to netty rpc env.
"Client mode" means the RPC env will not listen for incoming connections.
This allows certain processes in the Spark stack (such as Executors or
tha YARN client-mode AM) to act as pure clients when using the netty-based
RPC backend, reducing the number of sockets needed by the app and also the
number of open ports.

Client connections are also preferred when endpoints that actually have
a listening socket are involved; so, for example, if a Worker connects
to a Master and the Master needs to send a message to a Worker endpoint,
that client connection will be used, even though the Worker is also
listening for incoming connections.

With this change, the workaround for SPARK-10987 isn't necessary anymore, and
is removed. The AM connects to the driver in "client mode", and that connection
is used for all driver <-> AM communication, and so the AM is properly notified
when the connection goes down.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9210 from vanzin/SPARK-10997.
2015-11-02 10:26:36 -08:00
Liang-Chi Hsieh e209fa271a [SPARK-11271][SPARK-11016][CORE] Use Spark BitSet instead of RoaringBitmap to reduce memory usage
JIRA: https://issues.apache.org/jira/browse/SPARK-11271

As reported in the JIRA ticket, when there are too many tasks, the memory usage of MapStatus will cause problem. Use BitSet instead of RoaringBitMap should be more efficient in memory usage.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #9243 from viirya/mapstatus-bitset.
2015-11-02 08:52:52 +00:00
Marcelo Vanzin f8d93edec8 [SPARK-11073][CORE][YARN] Remove akka dependency in secret key generation.
Use standard JDK APIs for that (with a little help from Guava). Most of the
changes here are in test code, since there were no tests specific to that
part of the code.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9257 from vanzin/SPARK-11073.
2015-11-01 15:57:42 -08:00
Marcelo Vanzin cf04fdfe71 [SPARK-11020][CORE] Wait for HDFS to leave safe mode before initializing HS.
Large HDFS clusters may take a while to leave safe mode when starting; this change
makes the HS wait for that before doing checks about its configuraton. This means
the HS won't stop right away if HDFS is in safe mode and the configuration is not
correct, but that should be a very uncommon situation.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #9043 from vanzin/SPARK-11020.
2015-11-01 14:42:18 -08:00
Christian Kadner dc7e399fc0 [SPARK-11338] [WEBUI] Prepend app links on HistoryPage with uiRoot path
[SPARK-11338: HistoryPage not multi-tenancy enabled ...](https://issues.apache.org/jira/browse/SPARK-11338)
- `HistoryPage.scala` ...prepending all page links with the web proxy (`uiRoot`) path
- `HistoryServerSuite.scala` ...adding a test case to verify all site-relative links are prefixed when the environment variable `APPLICATION_WEB_PROXY_BASE` (or System property `spark.ui.proxyBase`) is set

Author: Christian Kadner <ckadner@us.ibm.com>

Closes #9291 from ckadner/SPARK-11338 and squashes the following commits:

01d2f35 [Christian Kadner] [SPARK-11338][WebUI] nit fixes
d054bd7 [Christian Kadner] [SPARK-11338][WebUI] prependBaseUri in method makePageLink
8bcb3dc [Christian Kadner] [SPARK-11338][WebUI] Prepend application links on HistoryPage with uiRoot path
2015-11-01 13:09:42 -08:00
Josh Rosen ac4118db2d [SPARK-11424] Guard against double-close() of RecordReaders
**TL;DR**: We can rule out one rare but potential cause of input stream corruption via defensive programming.

## Background

[MAPREDUCE-5918](https://issues.apache.org/jira/browse/MAPREDUCE-5918) is a bug where an instance of a decompressor ends up getting placed into a pool multiple times. Since the pool is backed by a list instead of a set, this can lead to the same decompressor being used in different places at the same time, which is not safe because those decompressors will overwrite each other's buffers. Sometimes this buffer sharing will lead to exceptions but other times it will might silently result in invalid / garbled input.

That Hadoop bug is fixed in Hadoop 2.7 but is still present in many Hadoop versions that we wish to support. As a result, I think that we should try to work around this issue in Spark via defensive programming to prevent RecordReaders from being closed multiple times.

So far, I've had a hard time coming up with explanations of exactly how double-`close()`s occur in practice, but I do have a couple of explanations that work on paper.

For instance, it looks like https://github.com/apache/spark/pull/7424, added in 1.5, introduces at least one extremely~rare corner-case path where Spark could double-close() a LineRecordReader instance in a way that triggers the bug. Here are the steps involved in the bad execution that I brainstormed up:

* [The task has finished reading input, so we call close()](https://github.com/apache/spark/blob/v1.5.1/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L168).
* [While handling the close call and trying to close the reader, reader.close() throws an exception]( https://github.com/apache/spark/blob/v1.5.1/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L190)
* We don't set `reader = null` after handling this exception, so the [TaskCompletionListener also ends up calling NewHadoopRDD.close()](https://github.com/apache/spark/blob/v1.5.1/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala#L156), which, in turn, closes the record reader again.

In this hypothetical situation, `LineRecordReader.close()` could [fail with an exception if its InputStream failed to close](https://github.com/apache/hadoop/blob/release-1.2.1/src/mapred/org/apache/hadoop/mapred/LineRecordReader.java#L212).
I googled for "Exception in RecordReader.close()" and it looks like it's possible for a closed Hadoop FileSystem to trigger an error there: [SPARK-757](https://issues.apache.org/jira/browse/SPARK-757), [SPARK-2491](https://issues.apache.org/jira/browse/SPARK-2491)

Looking at [SPARK-3052](https://issues.apache.org/jira/browse/SPARK-3052), it seems like it's possible to get spurious exceptions there when there is an error reading from Hadoop. If the Hadoop FileSystem were to get into an error state _right_ after reading the last record then it looks like we could hit the bug here in 1.5.

## The fix

This patch guards against these issues by modifying `HadoopRDD.close()` and `NewHadoopRDD.close()` so that they set `reader = null` even if an exception occurs in the `reader.close()` call. In addition, I modified `NextIterator. closeIfNeeded()` to guard against double-close if the first `close()` call throws an exception.

I don't have an easy way to test this, since I haven't been able to reproduce the bug that prompted this patch, but these changes seem safe and seem to rule out the on-paper reproductions that I was able to brainstorm up.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #9382 from JoshRosen/hadoop-decompressor-pooling-fix and squashes the following commits:

5ec97d7 [Josh Rosen] Add SqlNewHadoopRDD.unsetInputFileName() that I accidentally deleted.
ae46cf4 [Josh Rosen] Merge remote-tracking branch 'origin/master' into hadoop-decompressor-pooling-fix
087aa63 [Josh Rosen] Guard against double-close() of RecordReaders.
2015-10-31 10:47:22 -07:00
Yin Huai e8ec2a7b01 Revert "[SPARK-11236][CORE] Update Tachyon dependency from 0.7.1 -> 0.8.0."
This reverts commit 4f5e60c647.
2015-10-30 16:12:33 -07:00
Davies Liu 45029bfdea [SPARK-11423] remove MapPartitionsWithPreparationRDD
Since we do not need to preserve a page before calling compute(), MapPartitionsWithPreparationRDD is not needed anymore.

This PR basically revert #8543, #8511, #8038, #8011

Author: Davies Liu <davies@databricks.com>

Closes #9381 from davies/remove_prepare2.
2015-10-30 15:47:40 -07:00
Sun Rui fab710a917 [SPARK-11414][SPARKR] Forgot to update usage of 'spark.sparkr.r.command' in RRDD in the PR for SPARK-10971.
Author: Sun Rui <rui.sun@intel.com>

Closes #9368 from sun-rui/SPARK-11414.
2015-10-30 10:51:11 -07:00
Iulian Dragos 0451b00148 [SPARK-10986][MESOS] Set the context class loader in the Mesos executor backend.
See [SPARK-10986](https://issues.apache.org/jira/browse/SPARK-10986) for details.

This fixes the `ClassNotFoundException` for Spark classes in the serializer.

I am not sure this is the right way to handle the class loader, but I couldn't find any documentation on how the context class loader is used and who relies on it. It seems at least the serializer uses it to instantiate classes during deserialization.

I am open to suggestions (I tried this fix on a real Mesos cluster and it *does* fix the issue).

tnachen andrewor14

Author: Iulian Dragos <jaguarul@gmail.com>

Closes #9282 from dragos/issue/mesos-classloader.
2015-10-30 16:51:32 +00:00
Davies Liu 56419cf11f [SPARK-10342] [SPARK-10309] [SPARK-10474] [SPARK-10929] [SQL] Cooperative memory management
This PR introduce a mechanism to call spill() on those SQL operators that support spilling (for example, BytesToBytesMap, UnsafeExternalSorter and ShuffleExternalSorter) if there is not enough memory for execution. The preserved first page is needed anymore, so removed.

Other Spillable objects in Spark core (ExternalSorter and AppendOnlyMap) are not included in this PR, but those could benefit from this (trigger others' spilling).

The PrepareRDD may be not needed anymore, could be removed in follow up PR.

The following script will fail with OOM before this PR, finished in 150 seconds with 2G heap (also works in 1.5 branch, with similar duration).

```python
sqlContext.setConf("spark.sql.shuffle.partitions", "1")
df = sqlContext.range(1<<25).selectExpr("id", "repeat(id, 2) as s")
df2 = df.select(df.id.alias('id2'), df.s.alias('s2'))
j = df.join(df2, df.id==df2.id2).groupBy(df.id).max("id", "id2")
j.explain()
print j.count()
```

For thread-safety, here what I'm got:

1) Without calling spill(), the operators should only be used by single thread, no safety problems.

2) spill() could be triggered in two cases, triggered by itself, or by other operators. we can check trigger == this in spill(), so it's still in the same thread, so safety problems.

3) if it's triggered by other operators (right now cache will not trigger spill()), we only spill the data into disk when it's in scanning stage (building is finished), so the in-memory sorter or memory pages are read-only, we only need to synchronize the iterator and change it.

4) During scanning, the iterator will only use one record in one page, we can't free this page, because the downstream is currently using it (used by UnsafeRow or other objects). In BytesToBytesMap, we just skip the current page, and dump all others into disk. In UnsafeExternalSorter, we keep the page that is used by current record (having the same baseObject), free it when loading the next record. In ShuffleExternalSorter, the spill() will not trigger during scanning.

5) In order to avoid deadlock, we didn't call acquireMemory during spill (so we reused the pointer array in InMemorySorter).

Author: Davies Liu <davies@databricks.com>

Closes #9241 from davies/force_spill.
2015-10-29 23:38:06 -07:00
Calvin Jia 4f5e60c647 [SPARK-11236][CORE] Update Tachyon dependency from 0.7.1 -> 0.8.0.
Upgrades the tachyon-client version to the latest release.

No new dependencies are added and no spark facing APIs are changed. The removal of the `tachyon-underfs-s3` exclusion will enable users to use S3 out of the box and there are no longer any additional external dependencies added by the module.

Author: Calvin Jia <jia.calvin@gmail.com>

Closes #9204 from calvinjia/spark-11236.
2015-10-29 15:13:38 -07:00