Commit graph

5457 commits

Author SHA1 Message Date
Kay Ousterhout 2e139eed31 [SPARK-17931] Eliminate unnecessary task (de) serialization
In the existing code, there are three layers of serialization
    involved in sending a task from the scheduler to an executor:
        - A Task object is serialized
        - The Task object is copied to a byte buffer that also
          contains serialized information about any additional JARs,
          files, and Properties needed for the task to execute. This
          byte buffer is stored as the member variable serializedTask
          in the TaskDescription class.
        - The TaskDescription is serialized (in addition to the serialized
          task + JARs, the TaskDescription class contains the task ID and
          other metadata) and sent in a LaunchTask message.

While it *is* necessary to have two layers of serialization, so that
the JAR, file, and Property info can be deserialized prior to
deserializing the Task object, the third layer of deserialization is
unnecessary.  This commit eliminates a layer of serialization by moving
the JARs, files, and Properties into the TaskDescription class.

This commit also serializes the Properties manually (by traversing the map),
as is done with the JARs and files, which reduces the final serialized size.

Unit tests

This is a simpler alternative to the approach proposed in #15505.

shivaram and I did some benchmarking of this and #15505 on a 20-machine m2.4xlarge EC2 machines (160 cores). We ran ~30 trials of code [1] (a very simple job with 10K tasks per stage) and measured the average time per stage:

Before this change: 2490ms
With this change: 2345 ms (so ~6% improvement over the baseline)
With witgo's approach in #15505: 2046 ms (~18% improvement over baseline)

The reason that #15505 has a more significant improvement is that it also moves the serialization from the TaskSchedulerImpl thread to the CoarseGrainedSchedulerBackend thread. I added that functionality on top of this change, and got almost the same improvement [1] as #15505 (average of 2103ms). I think we should decouple these two changes, both so we have some record of the improvement form each individual improvement, and because this change is more about simplifying the code base (the improvement is negligible) while the other is about performance improvement.  The plan, currently, is to merge this PR and then merge the remaining part of #15505 that moves serialization.

[1] The reason the improvement wasn't quite as good as with #15505 when we ran the benchmarks is almost certainly because, at the point when we ran the benchmarks, I hadn't updated the code to manually serialize the Properties (instead the code was using Java's default serialization for the Properties object, whereas #15505 manually serialized the Properties).  This PR has since been updated to manually serialize the Properties, just like the other maps.

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #16053 from kayousterhout/SPARK-17931.
2017-01-06 10:48:08 -06:00
jerryshao 4a4c3dc9ca [SPARK-19033][CORE] Add admin acls for history server
## What changes were proposed in this pull request?

Current HistoryServer's ACLs is derived from application event-log, which means the newly changed ACLs cannot be applied to the old data, this will become a problem where newly added admin cannot access the old application history UI, only the new application can be affected.

So here propose to add admin ACLs for history server, any configured user/group could have the view access to all the applications, while the view ACLs derived from application run-time still take effect.

## How was this patch tested?

Unit test added.

Author: jerryshao <sshao@hortonworks.com>

Closes #16470 from jerryshao/SPARK-19033.
2017-01-06 10:07:54 -06:00
Rui Li f5d18af6a8 [SPARK-14958][CORE] Failed task not handled when there's error deserializing failure reason
## What changes were proposed in this pull request?

TaskResultGetter tries to deserialize the TaskEndReason before handling the failed task. If an error is thrown during deserialization, the failed task won't be handled, which leaves the job hanging.
The PR proposes to handle the failed task in a finally block.
## How was this patch tested?

In my case I hit a NoClassDefFoundError and the job hangs. Manually verified the patch can fix it.

Author: Rui Li <rui.li@intel.com>
Author: Rui Li <lirui@apache.org>
Author: Rui Li <shlr@cn.ibm.com>

Closes #12775 from lirui-intel/SPARK-14958.
2017-01-05 14:51:13 -08:00
Kay Ousterhout 00074b5778 [SPARK-19062] Utils.writeByteBuffer bug fix
This commit changes Utils.writeByteBuffer so that it does not change
the position of the ByteBuffer that it writes out, and adds a unit test for
this functionality.

cc mridulm

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #16462 from kayousterhout/SPARK-19062.
2017-01-04 11:21:09 -08:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

## How was this patch tested?
N/A since only docs or comments were updated.

Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com>

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
Sergei Lebedev 67fb33e7e0
[SPARK-19010][CORE] Include Kryo exception in case of overflow
## What changes were proposed in this pull request?

This is to workaround an implicit result of #4947 which suppressed the
original Kryo exception if the overflow happened during serialization.

## How was this patch tested?

`KryoSerializerSuite` was augmented to reflect this change.

Author: Sergei Lebedev <superbobry@gmail.com>

Closes #16416 from superbobry/patch-1.
2016-12-28 10:30:38 +00:00
Shixiong Zhu 7026ee23e0 [SPARK-17755][CORE] Use workerRef to send RegisterWorkerResponse to avoid the race condition
## What changes were proposed in this pull request?

The root cause of this issue is that RegisterWorkerResponse and LaunchExecutor are sent via two different channels (TCP connections) and their order is not guaranteed.

This PR changes the master and worker codes to use `workerRef` to send RegisterWorkerResponse, so that RegisterWorkerResponse and LaunchExecutor are sent via the same connection. Hence `LaunchExecutor` will always be after `RegisterWorkerResponse` and never be ignored.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16345 from zsxwing/SPARK-17755.
2016-12-25 23:48:14 -08:00
Kousuke Saruta f2ceb2abe9
[SPARK-18837][WEBUI] Very long stage descriptions do not wrap in the UI
## What changes were proposed in this pull request?

This issue was reported by wangyum.

In the AllJobsPage, JobPage and StagePage, the description length was limited before like as follows.

![ui-2 0 0](https://cloud.githubusercontent.com/assets/4736016/21319673/8b225246-c651-11e6-9041-4fcdd04f4dec.gif)

But recently, the limitation seems to have been accidentally removed.

![ui-2 1 0](https://cloud.githubusercontent.com/assets/4736016/21319825/104779f6-c652-11e6-8bfa-dfd800396352.gif)

The cause is that some tables are no longer `sortable` class although they were, and `sortable` class does not only mark tables as sortable but also limited the width of their child `td` elements.
The reason why now some tables are not `sortable` class is because another sortable mechanism was introduced by #13620 and #13708 with pagination feature.

To fix this issue, I've introduced new class `table-cell-width-limited` which limits the description cell width and the description is like what it was.

<img width="1260" alt="2016-12-20 1 00 34" src="https://cloud.githubusercontent.com/assets/4736016/21320478/89141c7a-c654-11e6-8494-f8f91325980b.png">

## How was this patch tested?

Tested manually with my browser.

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

Closes #16338 from sarutak/SPARK-18837.
2016-12-24 13:02:58 +00:00
Shixiong Zhu a848f0ba84 [SPARK-18991][CORE] Change ContextCleaner.referenceBuffer to use ConcurrentHashMap to make it faster
## What changes were proposed in this pull request?

The time complexity of ConcurrentHashMap's `remove` is O(1). Changing ContextCleaner.referenceBuffer's type from `ConcurrentLinkedQueue` to `ConcurrentHashMap's` will make the removal much faster.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16390 from zsxwing/SPARK-18991.
2016-12-23 15:38:41 -08:00
Shixiong Zhu f252cb5d16 [SPARK-18972][CORE] Fix the netty thread names for RPC
## What changes were proposed in this pull request?

Right now the name of threads created by Netty for Spark RPC are `shuffle-client-**` and `shuffle-server-**`. It's pretty confusing.

This PR just uses the module name in TransportConf to set the thread name. In addition, it also includes the following minor fixes:

- TransportChannelHandler.channelActive and channelInactive should call the corresponding super methods.
- Make ShuffleBlockFetcherIterator throw NoSuchElementException if it has no more elements. Otherwise,  if the caller calls `next` without `hasNext`, it will just hang.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16380 from zsxwing/SPARK-18972.
2016-12-22 16:22:55 -08:00
saturday_s ce99f51d2e [SPARK-18537][WEB UI] Add a REST api to serve spark streaming information
## What changes were proposed in this pull request?

This PR is an inheritance from #16000, and is a completion of #15904.

**Description**

- Augment the `org.apache.spark.status.api.v1` package for serving streaming information.
- Retrieve the streaming information through StreamingJobProgressListener.

> this api should cover exceptly the same amount of information as you can get from the web interface
> the implementation is base on the current REST implementation of spark-core
> and will be available for running applications only
>
> https://issues.apache.org/jira/browse/SPARK-18537

## How was this patch tested?

Local test.

Author: saturday_s <shi.indetail@gmail.com>
Author: Chan Chor Pang <ChorPang.Chan@access-company.com>
Author: peterCPChan <universknight@gmail.com>

Closes #16253 from saturday-shi/SPARK-18537.
2016-12-22 12:51:37 -08:00
jerryshao 31da755c80 [SPARK-18975][CORE] Add an API to remove SparkListener
## What changes were proposed in this pull request?

In current Spark we could add customized SparkListener through `SparkContext#addListener` API, but there's no equivalent API to remove the registered one. In our scenario SparkListener will be added repeatedly accordingly to the changed environment. If lacks the ability to remove listeners, there might be many registered listeners finally, this is unnecessary and potentially affects the performance. So here propose to add an API to remove registered listener.

## How was this patch tested?

Add an unit test to verify it.

Author: jerryshao <sshao@hortonworks.com>

Closes #16382 from jerryshao/SPARK-18975.
2016-12-22 11:18:22 -08:00
Dongjoon Hyun f489339c75
[SPARK-18953][CORE][WEB UI] Do now show the link to a dead worker on the master page
## What changes were proposed in this pull request?

For a dead worker, we will not be able to see its worker page anyway. This PR removes the links to dead workers from the master page.

## How was this patch tested?

Since this is UI change, please do the following steps manually.

**1. Start a master and a slave**

```
sbin/start-master.sh
sbin/start-slave.sh spark://10.22.16.140:7077
```

![1_live_worker_a](https://cloud.githubusercontent.com/assets/9700541/21373572/d7e187d6-c6d4-11e6-9110-f4371d215dec.png)

**2. Stop the slave**
```
sbin/stop-slave.sh
```

![2_dead_worker_a](https://cloud.githubusercontent.com/assets/9700541/21373579/dd9e9704-c6d4-11e6-9047-a22cb0aa83ed.png)

**3. Start a slave**

```
sbin/start-slave.sh spark://10.22.16.140:7077
```

![3_dead_worder_a_and_live_worker_b](https://cloud.githubusercontent.com/assets/9700541/21373582/e1b207f4-c6d4-11e6-89cb-6d8970175a5e.png)

**4. Stop the slave**

```
sbin/stop-slave.sh
```

![4_dead_worker_a_and_b](https://cloud.githubusercontent.com/assets/9700541/21373584/e5fecb4e-c6d4-11e6-95d3-49defe366946.png)

**5. Driver list testing**

Do the followings and stop the slave in a minute by `sbin/stop-slave.sh`.

```
sbin/start-master.sh
sbin/start-slave.sh spark://10.22.16.140:7077
bin/spark-submit --master=spark://10.22.16.140:7077 --deploy-mode=cluster --class org.apache.spark.examples.SparkPi examples/target/scala-2.11/jars/spark-examples_2.11-2.2.0-SNAPSHOT.jar 10000
```

![5_dead_worker_in_driver_list](https://cloud.githubusercontent.com/assets/9700541/21401320/be6cc9fc-c768-11e6-8de7-6512961296a5.png)

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #16366 from dongjoon-hyun/SPARK-18953.
2016-12-22 09:43:46 +00:00
Holden Karau 047a9d92ca [SPARK-18576][PYTHON] Add basic TaskContext information to PySpark
## What changes were proposed in this pull request?

Adds basic TaskContext information to PySpark.

## How was this patch tested?

New unit tests to `tests.py` & existing unit tests.

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

Closes #16211 from holdenk/SPARK-18576-pyspark-taskcontext.
2016-12-20 15:51:21 -08:00
Josh Rosen fa829ce21f [SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executors
## What changes were proposed in this pull request?

Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks.

This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks.

This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details.

## How was this patch tested?

Tested via a new test case in `JobCancellationSuite`, plus manual testing.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #16189 from JoshRosen/cancellation.
2016-12-19 18:43:59 -08:00
Josh Rosen 5857b9ac2d [SPARK-18928] Check TaskContext.isInterrupted() in FileScanRDD, JDBCRDD & UnsafeSorter
## What changes were proposed in this pull request?

In order to respond to task cancellation, Spark tasks must periodically check `TaskContext.isInterrupted()`, but this check is missing on a few critical read paths used in Spark SQL, including `FileScanRDD`, `JDBCRDD`, and UnsafeSorter-based sorts. This can cause interrupted / cancelled tasks to continue running and become zombies (as also described in #16189).

This patch aims to fix this problem by adding `TaskContext.isInterrupted()` checks to these paths. Note that I could have used `InterruptibleIterator` to simply wrap a bunch of iterators but in some cases this would have an adverse performance penalty or might not be effective due to certain special uses of Iterators in Spark SQL. Instead, I inlined `InterruptibleIterator`-style logic into existing iterator subclasses.

## How was this patch tested?

Tested manually in `spark-shell` with two different reproductions of non-cancellable tasks, one involving scans of huge files and another involving sort-merge joins that spill to disk. Both causes of zombie tasks are fixed by the changes added here.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #16340 from JoshRosen/sql-task-interruption.
2016-12-20 01:19:38 +01:00
Shivaram Venkataraman 4cb49412d1 [SPARK-18836][CORE] Serialize one copy of task metrics in DAGScheduler
## What changes were proposed in this pull request?

Right now we serialize the empty task metrics once per task – Since this is shared across all tasks we could use the same serialized task metrics across all tasks of a stage.

## How was this patch tested?

- [x] Run tests on EC2 to measure performance improvement

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

Closes #16261 from shivaram/task-metrics-one-copy.
2016-12-19 14:53:01 -08:00
Yuming Wang 1e5c51f336
[SPARK-18827][CORE] Fix cannot read broadcast on disk
## What changes were proposed in this pull request?
`NoSuchElementException` will throw since https://github.com/apache/spark/pull/15056 if a broadcast cannot cache in memory. The reason is that that change cannot cover `!unrolled.hasNext` in `next()` function.

This change is to cover the `!unrolled.hasNext` and check `hasNext` before calling `next` in `blockManager.getLocalValues` to make it  more robust.

We can cache and read broadcast even it cannot fit in memory from this pull request.

Exception log:
```
16/12/10 10:10:04 INFO UnifiedMemoryManager: Will not store broadcast_131 as the required space (1048576 bytes) exceeds our memory limit (122764 bytes)
16/12/10 10:10:04 WARN MemoryStore: Failed to reserve initial memory threshold of 1024.0 KB for computing block broadcast_131 in memory.
16/12/10 10:10:04 WARN MemoryStore: Not enough space to cache broadcast_131 in memory! (computed 384.0 B so far)
16/12/10 10:10:04 INFO MemoryStore: Memory use = 95.6 KB (blocks) + 0.0 B (scratch space shared across 0 tasks(s)) = 95.6 KB. Storage limit = 119.9 KB.
16/12/10 10:10:04 ERROR Utils: Exception encountered
java.util.NoSuchElementException
	at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58)
	at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269)
	at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206)
	at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96)
	at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
	at org.apache.spark.scheduler.Task.run(Task.scala:108)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
	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)
16/12/10 10:10:04 ERROR Executor: Exception in task 1.0 in stage 86.0 (TID 134423)
java.io.IOException: java.util.NoSuchElementException
	at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1276)
	at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206)
	at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66)
	at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96)
	at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86)
	at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)
	at org.apache.spark.scheduler.Task.run(Task.scala:108)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
	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)
Caused by: java.util.NoSuchElementException
	at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58)
	at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700)
	at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210)
	at scala.Option.map(Option.scala:146)
	at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210)
	at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269)
	... 12 more
```

## How was this patch tested?

Add unit test

Author: Yuming Wang <wgyumg@gmail.com>

Closes #16252 from wangyum/SPARK-18827.
2016-12-18 09:08:02 +00:00
uncleGen 6d2379b3b7
[SPARK-18485][CORE] Underlying integer overflow when create ChunkedByteBufferOutputStream in MemoryStore
## What changes were proposed in this pull request?

There is an underlying integer overflow when create ChunkedByteBufferOutputStream in MemoryStore. This PR provide a check before cast.

## How was this patch tested?

add new unit test

Author: uncleGen <hustyugm@gmail.com>

Closes #15915 from uncleGen/SPARK-18485.
2016-12-17 13:19:30 +00:00
hyukjinkwon 2bc1c95154 [SPARK-18895][TESTS] Fix resource-closing-related and path-related test failures in identified ones on Windows
## What changes were proposed in this pull request?

There are several tests failing due to resource-closing-related and path-related  problems on Windows as below.

- `RPackageUtilsSuite`:

```
- build an R package from a jar end to end *** FAILED *** (1 second, 625 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729427517-0\a\dep2\d\dep2-d.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- faulty R package shows documentation *** FAILED *** (359 milliseconds)
  java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729428970-0\dep1-c.jar
  at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279)
  at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653)
  at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535)

- SparkR zipping works properly *** FAILED *** (47 milliseconds)
  java.util.regex.PatternSyntaxException: Unknown character property name {r} near index 4

C:\projects\spark\target\tmp\1481729429282-0

    ^
  at java.util.regex.Pattern.error(Pattern.java:1955)
  at java.util.regex.Pattern.charPropertyNodeFor(Pattern.java:2781)
```

- `InputOutputMetricsSuite`:

```
- input metrics for old hadoop with coalesce *** FAILED *** (240 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with cache and coalesce *** FAILED *** (109 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics for new Hadoop API with coalesce *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-9366ec94-dac7-4a5c-a74b-3e7594a692ab\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics when reading text file *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - simple *** FAILED *** (125 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records read - more stages *** FAILED *** (110 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics on records - New Hadoop API *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-3f10a1a4-7820-4772-b821-25fd7523bf6f\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics on records read with cache *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input read/write and shuffle read/write metrics all line up *** FAILED *** (93 milliseconds)
  java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored
  at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277)
  at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202)
  at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252)

- input metrics with interleaved reads *** FAILED *** (0 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-2638d893-e89b-47ce-acd0-bbaeee78dd9b\InputOutputMetricsSuite_cart.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)

- input metrics with old CombineFileInputFormat *** FAILED *** (157 milliseconds)
  17947 was not greater than or equal to 300000 (InputOutputMetricsSuite.scala:324)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)

- input metrics with new CombineFileInputFormat *** FAILED *** (16 milliseconds)
  java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-11920c08-19d8-4c7c-9fba-28ed72b79f80\test\InputOutputMetricsSuite.txt, expected: file:///
  at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642)
  at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462)
  at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114)
```

- `ReplayListenerSuite`:

```
- End-to-end replay *** FAILED *** (121 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)

- End-to-end replay with compression *** FAILED *** (516 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)
```

- `EventLoggingListenerSuite`:

```
- End-to-end event logging *** FAILED *** (7 seconds, 435 milliseconds)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- End-to-end event logging with compression *** FAILED *** (1 second)
  java.io.IOException: No FileSystem for scheme: C
  at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421)
  at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428)
  at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88)

- Event log name *** FAILED *** (16 milliseconds)
  "file:/[]base-dir/app1" did not equal "file:/[C:/]base-dir/app1" (EventLoggingListenerSuite.scala:123)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555)
  at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466)
```

This PR proposes to fix the test failures on Windows

## How was this patch tested?

Manually tested via AppVeyor

**Before**

`RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/273-RPackageUtilsSuite-before
`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/272-InputOutputMetricsSuite-before
`ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/274-ReplayListenerSuite-before
`EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/275-EventLoggingListenerSuite-before

**After**

`RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/270-RPackageUtilsSuite
`InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/271-InputOutputMetricsSuite
`ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/277-ReplayListenerSuite-after
`EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/278-EventLoggingListenerSuite-after

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16305 from HyukjinKwon/RPackageUtilsSuite-InputOutputMetricsSuite.
2016-12-16 21:32:24 -08:00
Mark Hamstra 295db8259b [SPARK-17769][CORE][SCHEDULER] Some FetchFailure refactoring
## What changes were proposed in this pull request?

Readability rewrites.
Changed order of `failedStage.failedOnFetchAndShouldAbort(task.stageAttemptId)` and `disallowStageRetryForTest` evaluation.
Stage resubmission guard condition changed from `failedStages.isEmpty` to `!failedStages.contains(failedStage)`
Log all resubmission of stages
## How was this patch tested?

existing tests

Author: Mark Hamstra <markhamstra@gmail.com>

Closes #15335 from markhamstra/SPARK-17769.
2016-12-16 12:46:32 -08:00
Aliaksandr.Bedrytski f7a574a6cb
[SPARK-18708][CORE] Improvement/improve docs in spark context file
## What changes were proposed in this pull request?

SparkContext.scala was created a long time ago and contains several types of Scaladocs/Javadocs mixed together. Public methods/fields should have a Scaladoc that is formatted in the same way everywhere. This pull request also adds scaladoc to methods/fields that did not have it before.

## How was this patch tested?

No actual code was modified, only comments.

Please review http://spark.apache.org/contributing.html before opening a pull request.

Author: Aliaksandr.Bedrytski <aliaksandr.bedrytski@valtech.co.uk>

Closes #16137 from Mironor/improvement/improve-docs-in-spark-context-file.
2016-12-16 17:47:08 +00:00
Michal Senkyr 836c95b108
[SPARK-18723][DOC] Expanded programming guide information on wholeTex…
## What changes were proposed in this pull request?

Add additional information to wholeTextFiles in the Programming Guide. Also explain partitioning policy difference in relation to textFile and its impact on performance.

Also added reference to the underlying CombineFileInputFormat

## How was this patch tested?

Manual build of documentation and inspection in browser

```
cd docs
jekyll serve --watch
```

Author: Michal Senkyr <mike.senkyr@gmail.com>

Closes #16157 from michalsenkyr/wholeTextFilesExpandedDocs.
2016-12-16 17:43:39 +00:00
root 53ab8fb346
[SPARK-18742][CORE] Clarify that user-defined BroadcastFactory is not supported
## What changes were proposed in this pull request?
After SPARK-12588 Remove HTTPBroadcast [1], the one and only implementation of BroadcastFactory is TorrentBroadcastFactory and the spark.broadcast.factory conf has removed.

however the scaladoc says [2]:

/**
 * An interface for all the broadcast implementations in Spark (to allow
 * multiple broadcast implementations). SparkContext uses a user-specified
 * BroadcastFactory implementation to instantiate a particular broadcast for the
 * entire Spark job.
 */

so we should modify the comment that SparkContext will not use a  user-specified BroadcastFactory implementation

[1] https://issues.apache.org/jira/browse/SPARK-12588
[2] https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala#L25-L30

## How was this patch tested?
unit test added

Author: root <root@iZbp1gsnrlfzjxh82cz80vZ.(none)>
Author: windpiger <songjun@outlook.com>

Closes #16173 from windpiger/addBroadFactoryConf.
2016-12-16 10:16:57 +00:00
Imran Rashid 93cdb8a7d0 [SPARK-8425][CORE] Application Level Blacklisting
## What changes were proposed in this pull request?

This builds upon the blacklisting introduced in SPARK-17675 to add blacklisting of executors and nodes for an entire Spark application.  Resources are blacklisted based on tasks that fail, in tasksets that eventually complete successfully; they are automatically returned to the pool of active resources based on a timeout.  Full details are available in a design doc attached to the jira.
## How was this patch tested?

Added unit tests, ran them via Jenkins, also ran a handful of them in a loop to check for flakiness.

The added tests include:
- verifying BlacklistTracker works correctly
- verifying TaskSchedulerImpl interacts with BlacklistTracker correctly (via a mock BlacklistTracker)
- an integration test for the entire scheduler with blacklisting in a few different scenarios

Author: Imran Rashid <irashid@cloudera.com>
Author: mwws <wei.mao@intel.com>

Closes #14079 from squito/blacklist-SPARK-8425.
2016-12-15 08:29:56 -06:00
Alex Bozarth aebf44e50b
[SPARK-18816][WEB UI] Executors Logs column only ran visibility check on initial table load
## What changes were proposed in this pull request?

When I added a visibility check for the logs column on the executors page in #14382 the method I used only ran the check on the initial DataTable creation and not subsequent page loads. I moved the check out of the table definition and instead it runs on each page load. The jQuery DataTable functionality used is the same.

## How was this patch tested?

Tested Manually

No visible UI changes to screenshot.

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

Closes #16256 from ajbozarth/spark18816.
2016-12-13 21:37:46 +00:00
Shixiong Zhu fb3081d3b3 [SPARK-13747][CORE] Fix potential ThreadLocal leaks in RPC when using ForkJoinPool
## What changes were proposed in this pull request?

Some places in SQL may call `RpcEndpointRef.askWithRetry` (e.g., ParquetFileFormat.buildReader -> SparkContext.broadcast -> ... -> BlockManagerMaster.updateBlockInfo -> RpcEndpointRef.askWithRetry), which will finally call `Await.result`. It may cause `java.lang.IllegalArgumentException: spark.sql.execution.id is already set` when running in Scala ForkJoinPool.

This PR includes the following changes to fix this issue:

- Remove `ThreadUtils.awaitResult`
- Rename `ThreadUtils. awaitResultInForkJoinSafely` to `ThreadUtils.awaitResult`
- Replace `Await.result` in RpcTimeout with `ThreadUtils.awaitResult`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16230 from zsxwing/fix-SPARK-13747.
2016-12-13 09:53:22 -08:00
Jacek Laskowski 096f868b74
[MINOR][CORE][SQL] Remove explicit RDD and Partition overrides
## What changes were proposed in this pull request?

I **believe** that I _only_ removed duplicated code (that adds nothing but noise). I'm gonna remove the comment after Jenkins has built the changes with no issues and Spark devs has agreed to include the changes.

Remove explicit `RDD` and `Partition` overrides (that turn out code duplication)

## How was this patch tested?

Local build. Awaiting Jenkins.

…cation)

Author: Jacek Laskowski <jacek@japila.pl>

Closes #16145 from jaceklaskowski/rdd-overrides-removed.
2016-12-13 09:40:16 +00:00
Marcelo Vanzin bc59951bab [SPARK-18773][CORE] Make commons-crypto config translation consistent.
This change moves the logic that translates Spark configuration to
commons-crypto configuration to the network-common module. It also
extends TransportConf and ConfigProvider to provide the necessary
interfaces for the translation to work.

As part of the change, I removed SystemPropertyConfigProvider, which
was mostly used as an "empty config" in unit tests, and adjusted the
very few tests that required a specific config.

I also changed the config keys for AES encryption to live under the
"spark.network." namespace, which is more correct than their previous
names under "spark.authenticate.".

Tested via existing unit test.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16200 from vanzin/SPARK-18773.
2016-12-12 16:27:04 -08:00
Steve Loughran 586d198228 [SPARK-15844][CORE] HistoryServer doesn't come up if spark.authenticate = true
## What changes were proposed in this pull request?

During history server startup, the spark configuration is examined. If security.authentication is
set, log at debug and set the value to false, so that {{SecurityManager}} can be created.

## How was this patch tested?

A new test in `HistoryServerSuite` sets the `spark.authenticate` property to true, tries to create a security manager via a new package-private method `HistoryServer.createSecurityManager(SparkConf)`. This is the method used in `HistoryServer.main`. All other instantiations of a security manager in `HistoryServerSuite` have been switched to the new method, for consistency with the production code.

Author: Steve Loughran <stevel@apache.org>

Closes #13579 from steveloughran/history/SPARK-15844-security.
2016-12-12 12:24:53 -08:00
hyukjinkwon e094d01156
[SPARK-18803][TESTS] Fix JarEntry-related & path-related test failures and skip some tests by path length limitation on Windows
## What changes were proposed in this pull request?

This PR proposes to fix some tests being failed on Windows as below for several problems.

### Incorrect path handling

- FileSuite
  ```
  [info] - binary file input as byte array *** FAILED *** (500 milliseconds)
  [info]   "file:/C:/projects/spark/target/tmp/spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624/record-bytestream-00000.bin" did not contain "C:\projects\spark\target\tmp\spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624\record-bytestream-00000.bin" (FileSuite.scala:258)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  ...
  ```
  ```
  [info] - Get input files via old Hadoop API *** FAILED *** (1 second, 94 milliseconds)
  [info]   Set("/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00000", "/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00000", "C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00001") (FileSuite.scala:535)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500)
  ...
  ```

  ```
  [info] - Get input files via new Hadoop API *** FAILED *** (313 milliseconds)
  [info]   Set("/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00000", "/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00000", "C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00001") (FileSuite.scala:549)
  [info]   org.scalatest.exceptions.TestFailedException:
  ...
  ```

- TaskResultGetterSuite

  ```
  [info] - handling results larger than max RPC message size *** FAILED *** (1 second, 579 milliseconds)
  [info]   1 did not equal 0 Expect result to be removed from the block manager. (TaskResultGetterSuite.scala:129)
  [info]   org.scalatest.exceptions.TestFailedException:
  [info]   ...
  [info]   Cause: java.net.URISyntaxException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java
  [info]   at java.net.URI$Parser.fail(URI.java:2848)
  [info]   at java.net.URI$Parser.checkChars(URI.java:3021)
  ...
  ```
  ```
  [info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (0 milliseconds)
  [info]   java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java
  [info]   at java.net.URI.create(URI.java:852)
  ...
  ```

- SparkSubmitSuite

  ```
  [info]   java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\1481210831381-0\870903339\MyLib.java
  [info]   at java.net.URI.create(URI.java:852)
  [info]   at org.apache.spark.TestUtils$.org$apache$spark$TestUtils$$createURI(TestUtils.scala:112)
  ...
  ```

### Incorrect separate for JarEntry

After the path fix from above, then `TaskResultGetterSuite` throws another exception as below:

```
[info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (907 milliseconds)
[info]   java.lang.ClassNotFoundException: repro.MyException
[info]   at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
...
```

This is because `Paths.get` concatenates the given paths to an OS-specific path (Windows `\` and Linux `/`). However, for `JarEntry` we should comply ZIP specification meaning it should be always `/` according to ZIP specification.

See `4.4.17 file name: (Variable)` in https://pkware.cachefly.net/webdocs/casestudies/APPNOTE.TXT

### Long path problem on Windows

Some tests in `ShuffleSuite` via `ShuffleNettySuite` were skipped due to the same reason with SPARK-18718

## How was this patch tested?

Manually via AppVeyor.

**Before**

- `FileSuite`, `TaskResultGetterSuite`,`SparkSubmitSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/164-tmp-windows-base (please grep each to check each)
- `ShuffleSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/157-tmp-windows-base

**After**

- `FileSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/166-FileSuite
- `TaskResultGetterSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/173-TaskResultGetterSuite
- `SparkSubmitSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/167-SparkSubmitSuite
- `ShuffleSuite`
  https://ci.appveyor.com/project/spark-test/spark/build/176-ShuffleSuite

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16234 from HyukjinKwon/test-errors-windows.
2016-12-10 19:55:22 +00:00
WangTaoTheTonic 3a3e65adaf
[SPARK-18606][HISTORYSERVER] remove useless elements while searching
## What changes were proposed in this pull request?

When we search applications in HistoryServer, it will include all contents between <td> tag, which including useless elemtns like "<span title...", "a href" and making results confused.
We should remove those to make it clear.

## How was this patch tested?

manual tests.

Before:
![before](https://cloud.githubusercontent.com/assets/5276001/20662840/28bcc874-b590-11e6-9115-12fb64e49898.jpg)

After:
![after](https://cloud.githubusercontent.com/assets/5276001/20662844/2f717af2-b590-11e6-97dc-a48b08a54247.jpg)

Author: WangTaoTheTonic <wangtao111@huawei.com>

Closes #16031 from WangTaoTheTonic/span.
2016-12-10 16:43:08 +00:00
Davies Liu cf33a86285 [SPARK-4105] retry the fetch or stage if shuffle block is corrupt
## What changes were proposed in this pull request?

There is an outstanding issue that existed for a long time: Sometimes the shuffle blocks are corrupt and can't be decompressed. We recently hit this in three different workloads, sometimes we can reproduce it by every try, sometimes can't. I also found that when the corruption happened, the beginning and end of the blocks are correct, the corruption happen in the middle. There was one case that the string of block id is corrupt by one character. It seems that it's very likely the corruption is introduced by some weird machine/hardware, also the checksum (16 bits) in TCP is not strong enough to identify all the corruption.

Unfortunately, Spark does not have checksum for shuffle blocks or broadcast, the job will fail if any corruption happen in the shuffle block from disk, or broadcast blocks during network. This PR try to detect the corruption after fetching shuffle blocks by decompressing them, because most of the compression already have checksum in them. It will retry the block, or failed with FetchFailure, so the previous stage could be retried on different (still random) machines.

Checksum for broadcast will be added by another PR.

## How was this patch tested?

Added unit tests

Author: Davies Liu <davies@databricks.com>

Closes #15923 from davies/detect_corrupt.
2016-12-09 15:44:22 -08:00
Xiangrui Meng fd48d80a61 [SPARK-17822][R] Make JVMObjectTracker a member variable of RBackend
## What changes were proposed in this pull request?

* This PR changes `JVMObjectTracker` from `object` to `class` and let its instance associated with each RBackend. So we can manage the lifecycle of JVM objects when there are multiple `RBackend` sessions. `RBackend.close` will clear the object tracker explicitly.
* I assume that `SQLUtils` and `RRunner` do not need to track JVM instances, which could be wrong.
* Small refactor of `SerDe.sqlSerDe` to increase readability.

## How was this patch tested?

* Added unit tests for `JVMObjectTracker`.
* Wait for Jenkins to run full tests.

Author: Xiangrui Meng <meng@databricks.com>

Closes #16154 from mengxr/SPARK-17822.
2016-12-09 07:51:46 -08:00
Jacek Laskowski b162cc0c28
[MINOR][CORE][SQL][DOCS] Typo fixes
## What changes were proposed in this pull request?

Typo fixes

## How was this patch tested?

Local build. Awaiting the official build.

Author: Jacek Laskowski <jacek@japila.pl>

Closes #16144 from jaceklaskowski/typo-fixes.
2016-12-09 18:45:57 +08:00
Shixiong Zhu 26432df9cc [SPARK-18751][CORE] Fix deadlock when SparkContext.stop is called in Utils.tryOrStopSparkContext
## What changes were proposed in this pull request?

When `SparkContext.stop` is called in `Utils.tryOrStopSparkContext` (the following three places), it will cause deadlock because the `stop` method needs to wait for the thread running `stop` to exit.

- ContextCleaner.keepCleaning
- LiveListenerBus.listenerThread.run
- TaskSchedulerImpl.start

This PR adds `SparkContext.stopInNewThread` and uses it to eliminate the potential deadlock. I also removed my changes in #15775 since they are not necessary now.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16178 from zsxwing/fix-stop-deadlock.
2016-12-08 11:54:04 -08:00
Ergin Seyfe ed8869ebbf [SPARK-8617][WEBUI] HistoryServer: Include in-progress files during cleanup
## What changes were proposed in this pull request?
- Removed the`attempt.completed ` filter so cleaner would include the orphan inprogress files.
- Use loading time for inprogress files as lastUpdated. Keep using the modTime for completed files. First one will prevent deletion of inprogress job files. Second one will ensure that lastUpdated time won't change for completed jobs in an event of HistoryServer reboot.

## How was this patch tested?
Added new unittests and via existing tests.

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #16165 from seyfe/clear_old_inprogress_files.
2016-12-08 10:21:09 -08:00
Shixiong Zhu b47b892e45 [SPARK-18774][CORE][SQL] Ignore non-existing files when ignoreCorruptFiles is enabled
## What changes were proposed in this pull request?

When `ignoreCorruptFiles` is enabled, it's better to also ignore non-existing files.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16203 from zsxwing/ignore-file-not-found.
2016-12-07 22:37:04 -08:00
sarutak bb94f61a7a [SPARK-18762][WEBUI] Web UI should be http:4040 instead of https:4040
## What changes were proposed in this pull request?

When SSL is enabled, the Spark shell shows:
```
Spark context Web UI available at https://192.168.99.1:4040
```
This is wrong because 4040 is http, not https. It redirects to the https port.
More importantly, this introduces several broken links in the UI. For example, in the master UI, the worker link is https:8081 instead of http:8081 or https:8481.

CC: mengxr liancheng

I manually tested accessing by accessing MasterPage, WorkerPage and HistoryServer with SSL enabled.

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

Closes #16190 from sarutak/SPARK-18761.
2016-12-07 11:41:23 -08:00
Shixiong Zhu dbf3e298a1 [SPARK-18764][CORE] Add a warning log when skipping a corrupted file
## What changes were proposed in this pull request?

It's better to add a warning log when skipping a corrupted file. It will be helpful when we want to finish the job first, then find them in the log and fix these files.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16192 from zsxwing/SPARK-18764.
2016-12-07 10:30:05 -08:00
Jie Xiong c496d03b52 [SPARK-18208][SHUFFLE] Executor OOM due to a growing LongArray in BytesToBytesMap
## What changes were proposed in this pull request?

BytesToBytesMap currently does not release the in-memory storage (the longArray variable) after it spills to disk. This is typically not a problem during aggregation because the longArray should be much smaller than the pages, and because we grow the longArray at a conservative rate.

However this can lead to an OOM when an already running task is allocated more than its fair share, this can happen because of a scheduling delay. In this case the longArray can grow beyond the fair share of memory for the task. This becomes problematic when the task spills and the long array is not freed, that causes subsequent memory allocation requests to be denied by the memory manager resulting in an OOM.

This PR fixes this issuing by freeing the longArray when the BytesToBytesMap spills.

## How was this patch tested?

Existing tests and tested on realworld workloads.

Author: Jie Xiong <jiexiong@fb.com>
Author: jiexiong <jiexiong@gmail.com>

Closes #15722 from jiexiong/jie_oom_fix.
2016-12-07 04:33:30 -08:00
Sean Owen 79f5f281bb
[SPARK-18678][ML] Skewed reservoir sampling in SamplingUtils
## What changes were proposed in this pull request?

Fix reservoir sampling bias for small k. An off-by-one error meant that the probability of replacement was slightly too high -- k/(l-1) after l element instead of k/l, which matters for small k.

## How was this patch tested?

Existing test plus new test case.

Author: Sean Owen <sowen@cloudera.com>

Closes #16129 from srowen/SPARK-18678.
2016-12-07 17:34:45 +08:00
Peter Ableda 05d416ffc6 [SPARK-18740] Log spark.app.name in driver logs
## What changes were proposed in this pull request?

Added simple logInfo line to print out the `spark.app.name` in the driver logs

## How was this patch tested?

Spark was built and tested with SparkPi app. Example log:
```
16/12/06 05:49:50 INFO spark.SparkContext: Running Spark version 2.0.0
16/12/06 05:49:52 INFO spark.SparkContext: Submitted application: Spark Pi
16/12/06 05:49:52 INFO spark.SecurityManager: Changing view acls to: root
16/12/06 05:49:52 INFO spark.SecurityManager: Changing modify acls to: root
```

Author: Peter Ableda <peter.ableda@cloudera.com>

Closes #16172 from peterableda/feature/print_appname.
2016-12-06 10:12:27 -08:00
hyukjinkwon b8c7b8d31d
[SPARK-18672][CORE] Close recordwriter in SparkHadoopMapReduceWriter before committing
## What changes were proposed in this pull request?

It seems some APIs such as `PairRDDFunctions.saveAsHadoopDataset()` do not close the record writer before issuing the commit for the task.

On Windows, the output in the temp directory is being open and output committer tries to rename it from temp directory to the output directory after finishing writing.

So, it fails to move the file. It seems we should close the writer actually before committing the task like the other writers such as `FileFormatWriter`.

Identified failure was as below:

```
FAILURE! - in org.apache.spark.JavaAPISuite
writeWithNewAPIHadoopFile(org.apache.spark.JavaAPISuite)  Time elapsed: 0.25 sec  <<< ERROR!
org.apache.spark.SparkException: Job aborted.
	at org.apache.spark.JavaAPISuite.writeWithNewAPIHadoopFile(JavaAPISuite.java:1231)
Caused by: org.apache.spark.SparkException:
Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost, executor driver): org.apache.spark.SparkException: Task failed while writing rows
	at org.apache.spark.internal.io.SparkHadoopMapReduceWriter$.org$apache$spark$internal$io$SparkHadoopMapReduceWriter$$executeTask(SparkHadoopMapReduceWriter.scala:182)
	at org.apache.spark.internal.io.SparkHadoopMapReduceWriter$$anonfun$3.apply(SparkHadoopMapReduceWriter.scala:100)
	at org.apache.spark.internal.io.SparkHadoopMapReduceWriter$$anonfun$3.apply(SparkHadoopMapReduceWriter.scala:99)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:87)
	at org.apache.spark.scheduler.Task.run(Task.scala:108)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282)
	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)
Caused by: java.io.IOException: Could not rename file:/C:/projects/spark/core/target/tmp/1480553515529-0/output/_temporary/0/_temporary/attempt_20161201005155_0000_r_000000_0 to file:/C:/projects/spark/core/target/tmp/1480553515529-0/output/_temporary/0/task_20161201005155_0000_r_000000
	at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:436)
	at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.commitTask(FileOutputCommitter.java:415)
	at org.apache.spark.mapred.SparkHadoopMapRedUtil$.performCommit$1(SparkHadoopMapRedUtil.scala:50)
	at org.apache.spark.mapred.SparkHadoopMapRedUtil$.commitTask(SparkHadoopMapRedUtil.scala:76)
	at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.commitTask(HadoopMapReduceCommitProtocol.scala:153)
	at org.apache.spark.internal.io.SparkHadoopMapReduceWriter$$anonfun$4.apply(SparkHadoopMapReduceWriter.scala:167)
	at org.apache.spark.internal.io.SparkHadoopMapReduceWriter$$anonfun$4.apply(SparkHadoopMapReduceWriter.scala:156)
	at org.apache.spark.util.Utils$.tryWithSafeFinallyAndFailureCallbacks(Utils.scala:1341)
	at org.apache.spark.internal.io.SparkHadoopMapReduceWriter$.org$apache$spark$internal$io$SparkHadoopMapReduceWriter$$executeTask(SparkHadoopMapReduceWriter.scala:168)
	... 8 more
Driver stacktrace:
	at org.apache.spark.JavaAPISuite.writeWithNewAPIHadoopFile(JavaAPISuite.java:1231)
Caused by: org.apache.spark.SparkException: Task failed while writing rows
Caused by: java.io.IOException: Could not rename file:/C:/projects/spark/core/target/tmp/1480553515529-0/output/_temporary/0/_temporary/attempt_20161201005155_0000_r_000000_0 to file:/C:/projects/spark/core/target/tmp/1480553515529-0/output/_temporary/0/task_20161201005155_0000_r_000000
```

This PR proposes to close this before committing the task.

## How was this patch tested?

Manually tested via AppVeyor.

**Before**

https://ci.appveyor.com/project/spark-test/spark/build/94-scala-tests

**After**

https://ci.appveyor.com/project/spark-test/spark/build/93-scala-tests

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16098 from HyukjinKwon/close-wirter-first.
2016-12-06 12:20:16 +08:00
Reynold Xin e9730b707d [SPARK-18702][SQL] input_file_block_start and input_file_block_length
## What changes were proposed in this pull request?
We currently have function input_file_name to get the path of the input file, but don't have functions to get the block start offset and length. This patch introduces two functions:

1. input_file_block_start: returns the file block start offset, or -1 if not available.

2. input_file_block_length: returns the file block length, or -1 if not available.

## How was this patch tested?
Updated existing test cases in ColumnExpressionSuite that covered input_file_name to also cover the two new functions.

Author: Reynold Xin <rxin@databricks.com>

Closes #16133 from rxin/SPARK-18702.
2016-12-04 21:51:10 -08:00
Eric Liang 294163ee93 [SPARK-18679][SQL] Fix regression in file listing performance for non-catalog tables
## What changes were proposed in this pull request?

In Spark 2.1 ListingFileCatalog was significantly refactored (and renamed to InMemoryFileIndex). This introduced a regression where parallelism could only be introduced at the very top of the tree. However, in many cases (e.g. `spark.read.parquet(topLevelDir)`), the top of the tree is only a single directory.

This PR simplifies and fixes the parallel recursive listing code to allow parallelism to be introduced at any level during recursive descent (though note that once we decide to list a sub-tree in parallel, the sub-tree is listed in serial on executors).

cc mallman  cloud-fan

## How was this patch tested?

Checked metrics in unit tests.

Author: Eric Liang <ekl@databricks.com>

Closes #16112 from ericl/spark-18679.
2016-12-02 20:59:39 +08:00
Liang-Chi Hsieh dbf842b7a8 [SPARK-18666][WEB UI] Remove the codes checking deprecated config spark.sql.unsafe.enabled
## What changes were proposed in this pull request?

`spark.sql.unsafe.enabled` is deprecated since 1.6. There still are codes in UI to check it. We should remove it and clean the codes.

## How was this patch tested?

Changes to related existing unit test.

Please review http://spark.apache.org/contributing.html before opening a pull request.

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

Closes #16095 from viirya/remove-deprecated-config-code.
2016-12-01 01:57:58 -08:00
wm624@hotmail.com 2eb6764fbb [SPARK-18476][SPARKR][ML] SparkR Logistic Regression should should support output original label.
## What changes were proposed in this pull request?

Similar to SPARK-18401, as a classification algorithm, logistic regression should support output original label instead of supporting index label.

In this PR, original label output is supported and test cases are modified and added. Document is also modified.

## How was this patch tested?

Unit tests.

Author: wm624@hotmail.com <wm624@hotmail.com>

Closes #15910 from wangmiao1981/audit.
2016-11-30 20:32:17 -08:00
Shixiong Zhu c4979f6ea8 [SPARK-18655][SS] Ignore Structured Streaming 2.0.2 logs in history server
## What changes were proposed in this pull request?

As `queryStatus` in StreamingQueryListener events was removed in #15954, parsing 2.0.2 structured streaming logs will throw the following errror:

```
[info]   com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException: Unrecognized field "queryStatus" (class org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent), not marked as ignorable (2 known properties: "id", "exception"])
[info]  at [Source: {"Event":"org.apache.spark.sql.streaming.StreamingQueryListener$QueryTerminatedEvent","queryStatus":{"name":"query-1","id":1,"timestamp":1480491532753,"inputRate":0.0,"processingRate":0.0,"latency":null,"sourceStatuses":[{"description":"FileStreamSource[file:/Users/zsx/stream]","offsetDesc":"#0","inputRate":0.0,"processingRate":0.0,"triggerDetails":{"latency.getOffset.source":"1","triggerId":"1"}}],"sinkStatus":{"description":"FileSink[/Users/zsx/stream2]","offsetDesc":"[#0]"},"triggerDetails":{}},"exception":null}; line: 1, column: 521] (through reference chain: org.apache.spark.sql.streaming.QueryTerminatedEvent["queryStatus"])
[info]   at com.fasterxml.jackson.databind.exc.UnrecognizedPropertyException.from(UnrecognizedPropertyException.java:51)
[info]   at com.fasterxml.jackson.databind.DeserializationContext.reportUnknownProperty(DeserializationContext.java:839)
[info]   at com.fasterxml.jackson.databind.deser.std.StdDeserializer.handleUnknownProperty(StdDeserializer.java:1045)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperty(BeanDeserializerBase.java:1352)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.handleUnknownProperties(BeanDeserializerBase.java:1306)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializer._deserializeUsingPropertyBased(BeanDeserializer.java:453)
[info]   at com.fasterxml.jackson.databind.deser.BeanDeserializerBase.deserializeFromObjectUsingNonDefault(BeanDeserializerBase.java:1099)
...
```

This PR just ignores such errors and adds a test to make sure we can read 2.0.2 logs.

## How was this patch tested?

`query-event-logs-version-2.0.2.txt` has all types of events generated by Structured Streaming in Spark 2.0.2. `testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.2")` verified we can load them without any error.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16085 from zsxwing/SPARK-18655.
2016-11-30 16:18:53 -08:00
Marcelo Vanzin 93e9d880bf [SPARK-18546][CORE] Fix merging shuffle spills when using encryption.
The problem exists because it's not possible to just concatenate encrypted
partition data from different spill files; currently each partition would
have its own initial vector to set up encryption, and the final merged file
should contain a single initial vector for each merged partiton, otherwise
iterating over each record becomes really hard.

To fix that, UnsafeShuffleWriter now decrypts the partitions when merging,
so that the merged file contains a single initial vector at the start of
the partition data.

Because it's not possible to do that using the fast transferTo path, when
encryption is enabled UnsafeShuffleWriter will revert back to using file
streams when merging. It may be possible to use a hybrid approach when
using encryption, using an intermediate direct buffer when reading from
files and encrypting the data, but that's better left for a separate patch.

As part of the change I made DiskBlockObjectWriter take a SerializerManager
instead of a "wrap stream" closure, since that makes it easier to test the
code without having to mock SerializerManager functionality.

Tested with newly added unit tests (UnsafeShuffleWriterSuite for the write
side and ExternalAppendOnlyMapSuite for integration), and by running some
apps that failed without the fix.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15982 from vanzin/SPARK-18546.
2016-11-30 14:10:32 -08:00