Commit graph

5944 commits

Author SHA1 Message Date
Ryan Blue 2dc0480816 [SPARK-17532] Add lock debugging info to thread dumps.
## What changes were proposed in this pull request?

This adds information to the web UI thread dump page about the JVM locks
held by threads and the locks that threads are blocked waiting to
acquire. This should help find cases where lock contention is causing
Spark applications to run slowly.
## How was this patch tested?

Tested by applying this patch and viewing the change in the web UI.

![thread-lock-info](https://cloud.githubusercontent.com/assets/87915/18493057/6e5da870-79c3-11e6-8c20-f54c18a37544.png)

Additions:
- A "Thread Locking" column with the locks held by the thread or that are blocking the thread
- Links from the a blocked thread to the thread holding the lock
- Stack frames show where threads are inside `synchronized` blocks, "holding Monitor(...)"

Author: Ryan Blue <blue@apache.org>

Closes #15088 from rdblue/SPARK-17532-add-thread-lock-info.
2016-11-02 00:08:30 -07:00
Josh Rosen b929537b6e [SPARK-18182] Expose ReplayListenerBus.read() overload which takes string iterator
The `ReplayListenerBus.read()` method is used when implementing a custom `ApplicationHistoryProvider`. The current interface only exposes a `read()` method which takes an `InputStream` and performs stream-to-lines conversion itself, but it would also be useful to expose an overloaded method which accepts an iterator of strings, thereby enabling events to be provided from non-`InputStream` sources.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15698 from JoshRosen/replay-listener-bus-interface.
2016-11-01 16:49:41 -07:00
Shixiong Zhu d2923f1732 [SPARK-18143][SQL] Ignore Structured Streaming event logs to avoid breaking history server
## What changes were proposed in this pull request?

Because of the refactoring work in Structured Streaming, the event logs generated by Strucutred Streaming in Spark 2.0.0 and 2.0.1 cannot be parsed.

This PR just ignores these logs in ReplayListenerBus because no places use them.
## How was this patch tested?
- Generated events logs using Spark 2.0.0 and 2.0.1, and saved them as `structured-streaming-query-event-logs-2.0.0.txt` and `structured-streaming-query-event-logs-2.0.1.txt`
- The new added test makes sure ReplayListenerBus will skip these bad jsons.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15663 from zsxwing/fix-event-log.
2016-10-31 00:11:33 -07:00
Hossein 2881a2d1d1 [SPARK-17919] Make timeout to RBackend configurable in SparkR
## What changes were proposed in this pull request?

This patch makes RBackend connection timeout configurable by user.

## How was this patch tested?
N/A

Author: Hossein <hossein@databricks.com>

Closes #15471 from falaki/SPARK-17919.
2016-10-30 16:17:23 -07:00
Eric Liang 90d3b91f4c [SPARK-18103][SQL] Rename *FileCatalog to *FileIndex
## What changes were proposed in this pull request?

To reduce the number of components in SQL named *Catalog, rename *FileCatalog to *FileIndex. A FileIndex is responsible for returning the list of partitions / files to scan given a filtering expression.

```
TableFileCatalog => CatalogFileIndex
FileCatalog => FileIndex
ListingFileCatalog => InMemoryFileIndex
MetadataLogFileCatalog => MetadataLogFileIndex
PrunedTableFileCatalog => PrunedInMemoryFileIndex
```

cc yhuai marmbrus

## How was this patch tested?

N/A

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15634 from ericl/rename-file-provider.
2016-10-30 13:14:45 -07:00
wm624@hotmail.com 701a9d361b
[SPARK-CORE][TEST][MINOR] Fix the wrong comment in test
## What changes were proposed in this pull request?

While learning core scheduler code, I found two lines of wrong comments. This PR simply corrects the comments.

## How was this patch tested?

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

Closes #15631 from wangmiao1981/Rbug.
2016-10-27 10:00:37 +02:00
Yin Huai d3b4831d00 [SPARK-18132] Fix checkstyle
This PR fixes checkstyle.

Author: Yin Huai <yhuai@databricks.com>

Closes #15656 from yhuai/fix-format.
2016-10-26 22:22:23 -07:00
Miao Wang a76846cfb1 [SPARK-18126][SPARK-CORE] getIteratorZipWithIndex accepts negative value as index
## What changes were proposed in this pull request?

(Please fill in changes proposed in this fix)

`Utils.getIteratorZipWithIndex` was added to deal with number of records > 2147483647 in one partition.

method `getIteratorZipWithIndex` accepts `startIndex` < 0, which leads to negative index.

This PR just adds a defensive check on `startIndex` to make sure it is >= 0.

## How was this patch tested?

Add a new unit test.

Author: Miao Wang <miaowang@Miaos-MacBook-Pro.local>

Closes #15639 from wangmiao1981/zip.
2016-10-27 01:17:32 +02:00
Shixiong Zhu 7ac70e7ba8 [SPARK-13747][SQL] Fix concurrent executions in ForkJoinPool for SQL
## What changes were proposed in this pull request?

Calling `Await.result` will allow other tasks to be run on the same thread when using ForkJoinPool. However, SQL uses a `ThreadLocal` execution id to trace Spark jobs launched by a query, which doesn't work perfectly in ForkJoinPool.

This PR just uses `Awaitable.result` instead to  prevent ForkJoinPool from running other tasks in the current waiting thread.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15520 from zsxwing/SPARK-13747.
2016-10-26 10:36:36 -07:00
Shuai Lin 402205ddf7
[SPARK-17802] Improved caller context logging.
## What changes were proposed in this pull request?

[SPARK-16757](https://issues.apache.org/jira/browse/SPARK-16757) sets the hadoop `CallerContext` when calling hadoop/hdfs apis to make spark applications more diagnosable in hadoop/hdfs logs. However, the `org.apache.hadoop.ipc.CallerContext` class is only added since [hadoop 2.8](https://issues.apache.org/jira/browse/HDFS-9184), which is not officially releaed yet. So each time `utils.CallerContext.setCurrentContext()` is called (e.g [when a task is created](https://github.com/apache/spark/blob/b678e46/core/src/main/scala/org/apache/spark/scheduler/Task.scala#L95-L96)), a "java.lang.ClassNotFoundException: org.apache.hadoop.ipc.CallerContext"
error is logged, which pollutes the spark logs when there are lots of tasks.

This patch improves this behaviour by only logging the `ClassNotFoundException` once.

## How was this patch tested?

Existing tests.

Author: Shuai Lin <linshuai2012@gmail.com>

Closes #15377 from lins05/spark-17802-improve-callercontext-logging.
2016-10-26 14:31:47 +02:00
Alex Bozarth 5d0f81da49
[SPARK-4411][WEB UI] Add "kill" link for jobs in the UI
## What changes were proposed in this pull request?

Currently users can kill stages via the web ui but not jobs directly (jobs are killed if one of their stages is). I've added the ability to kill jobs via the web ui. This code change is based on #4823 by lianhuiwang and updated to work with the latest code matching how stages are currently killed. In general I've copied the kill stage code warning and note comments and all. I also updated applicable tests and documentation.

## How was this patch tested?

Manually tested and dev/run-tests

![screen shot 2016-10-11 at 4 49 43 pm](https://cloud.githubusercontent.com/assets/13952758/19292857/12f1b7c0-8fd4-11e6-8982-210249f7b697.png)

Author: Alex Bozarth <ajbozart@us.ibm.com>
Author: Lianhui Wang <lianhuiwang09@gmail.com>

Closes #15441 from ajbozarth/spark4411.
2016-10-26 14:26:54 +02:00
hayashidac c329a568b5 [SPARK-16988][SPARK SHELL] spark history server log needs to be fixed to show https url when ssl is enabled
spark history server log needs to be fixed to show https url when ssl is enabled

Author: chie8842 <chie@chie-no-Mac-mini.local>

Closes #15611 from hayashidac/SPARK-16988.
2016-10-26 07:13:48 +09:00
Vinayak c5fe3dd4f5 [SPARK-18010][CORE] Reduce work performed for building up the application list for the History Server app list UI page
## What changes were proposed in this pull request?
allow ReplayListenerBus to skip deserialising and replaying certain events using an inexpensive check of the event log entry. Use this to ensure that when event log replay is triggered for building the application list, we get the ReplayListenerBus to skip over all but the few events needed for our immediate purpose. Refer [SPARK-18010] for the motivation behind this change.

## How was this patch tested?

Tested with existing HistoryServer and ReplayListener unit test suites. All tests pass.

Please review https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark before opening a pull request.

Author: Vinayak <vijoshi5@in.ibm.com>

Closes #15556 from vijoshi/SAAS-467_master.
2016-10-25 10:36:03 -07:00
Kay Ousterhout 483c37c581 [SPARK-17894][HOTFIX] Fix broken build from
The named parameter in an overridden class isn't supported in Scala 2.10 so was breaking the build.

cc zsxwing

Author: Kay Ousterhout <kayousterhout@gmail.com>

Closes #15617 from kayousterhout/hotfix.
2016-10-24 20:16:00 -07:00
Eren Avsarogullari 81d6933e75 [SPARK-17894][CORE] Ensure uniqueness of TaskSetManager name.
`TaskSetManager` should have unique name to avoid adding duplicate ones to parent `Pool` via `SchedulableBuilder`. This problem has been surfaced with following discussion: [[PR: Avoid adding duplicate schedulables]](https://github.com/apache/spark/pull/15326)

**Proposal** :
There is 1x1 relationship between `stageAttemptId` and `TaskSetManager` so `taskSet.Id` covering both `stageId` and `stageAttemptId` looks to be used for uniqueness of `TaskSetManager` name instead of just `stageId`.

**Current TaskSetManager Name** :
`var name = "TaskSet_" + taskSet.stageId.toString`
**Sample**: TaskSet_0

**Proposed TaskSetManager Name** :
`val name = "TaskSet_" + taskSet.Id ` `// taskSet.Id = (stageId + "." + stageAttemptId)`
**Sample** : TaskSet_0.0

Added new Unit Test.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #15463 from erenavsarogullari/SPARK-17894.
2016-10-24 15:33:54 -07:00
Zheng RuiFeng c64a8ff397
[SPARK-18049][MLLIB][TEST] Add missing tests for truePositiveRate and weightedTruePositiveRate
## What changes were proposed in this pull request?
Add missing tests for `truePositiveRate` and `weightedTruePositiveRate` in `MulticlassMetricsSuite`

## How was this patch tested?
added testing

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15585 from zhengruifeng/mc_missing_test.
2016-10-24 10:25:24 +01:00
Sandeep Singh bc167a2a53 [SPARK-928][CORE] Add support for Unsafe-based serializer in Kryo
## What changes were proposed in this pull request?
Now since we have migrated to Kryo-3.0.0 in https://issues.apache.org/jira/browse/SPARK-11416, we can gives users option to use unsafe SerDer. It can turned by setting `spark.kryo.useUnsafe` to `true`

## How was this patch tested?
Ran existing tests

```
     Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms)    Rate(M/s)   Per Row(ns)   Relative
      ------------------------------------------------------------------------------------------------
      basicTypes: Int unsafe:true                    160 /  178         98.5          10.1       1.0X
      basicTypes: Long unsafe:true                   210 /  218         74.9          13.4       0.8X
      basicTypes: Float unsafe:true                  203 /  213         77.5          12.9       0.8X
      basicTypes: Double unsafe:true                 226 /  235         69.5          14.4       0.7X
      Array: Int unsafe:true                        1087 / 1101         14.5          69.1       0.1X
      Array: Long unsafe:true                       2758 / 2844          5.7         175.4       0.1X
      Array: Float unsafe:true                      1511 / 1552         10.4          96.1       0.1X
      Array: Double unsafe:true                     2942 / 2972          5.3         187.0       0.1X
      Map of string->Double unsafe:true             2645 / 2739          5.9         168.2       0.1X
      basicTypes: Int unsafe:false                   211 /  218         74.7          13.4       0.8X
      basicTypes: Long unsafe:false                  247 /  253         63.6          15.7       0.6X
      basicTypes: Float unsafe:false                 211 /  216         74.5          13.4       0.8X
      basicTypes: Double unsafe:false                227 /  233         69.2          14.4       0.7X
      Array: Int unsafe:false                       3012 / 3032          5.2         191.5       0.1X
      Array: Long unsafe:false                      4463 / 4515          3.5         283.8       0.0X
      Array: Float unsafe:false                     2788 / 2868          5.6         177.2       0.1X
      Array: Double unsafe:false                    3558 / 3752          4.4         226.2       0.0X
      Map of string->Double unsafe:false            2806 / 2933          5.6         178.4       0.1X
```

Author: Sandeep Singh <sandeep@techaddict.me>
Author: Sandeep Singh <sandeep@origamilogic.com>

Closes #12913 from techaddict/SPARK-928.
2016-10-22 12:03:37 -07:00
WeichenXu 4f1dcd3dce [SPARK-18051][SPARK CORE] fix bug of custom PartitionCoalescer causing serialization exception
## What changes were proposed in this pull request?

add a require check in `CoalescedRDD` to make sure the passed in `partitionCoalescer` to be `serializable`.
and update the document for api `RDD.coalesce`

## How was this patch tested?

Manual.(test code in jira [SPARK-18051])

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #15587 from WeichenXu123/fix_coalescer_bug.
2016-10-22 11:59:28 -07:00
Eric Liang 3eca283aca [SPARK-17994][SQL] Add back a file status cache for catalog tables
## What changes were proposed in this pull request?

In SPARK-16980, we removed the full in-memory cache of table partitions in favor of loading only needed partitions from the metastore. This greatly improves the initial latency of queries that only read a small fraction of table partitions.

However, since the metastore does not store file statistics, we need to discover those from remote storage. With the loss of the in-memory file status cache this has to happen on each query, increasing the latency of repeated queries over the same partitions.

The proposal is to add back a per-table cache of partition contents, i.e. Map[Path, Array[FileStatus]]. This cache would be retained per-table, and can be invalidated through refreshTable() and refreshByPath(). Unlike the prior cache, it can be incrementally updated as new partitions are read.

## How was this patch tested?

Existing tests and new tests in `HiveTablePerfStatsSuite`.

cc mallman

Author: Eric Liang <ekl@databricks.com>
Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15539 from ericl/meta-cache.
2016-10-22 22:08:28 +08:00
w00228970 c1f344f1a0 [SPARK-17929][CORE] Fix deadlock when CoarseGrainedSchedulerBackend reset
## What changes were proposed in this pull request?

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

Now `CoarseGrainedSchedulerBackend` reset will get the lock,
```
  protected def reset(): Unit = synchronized {
    numPendingExecutors = 0
    executorsPendingToRemove.clear()

    // Remove all the lingering executors that should be removed but not yet. The reason might be
    // because (1) disconnected event is not yet received; (2) executors die silently.
    executorDataMap.toMap.foreach { case (eid, _) =>
      driverEndpoint.askWithRetry[Boolean](
        RemoveExecutor(eid, SlaveLost("Stale executor after cluster manager re-registered.")))
    }
  }
```
 but on removeExecutor also need the lock "CoarseGrainedSchedulerBackend.this.synchronized", this will cause deadlock.

```
   private def removeExecutor(executorId: String, reason: ExecutorLossReason): Unit = {
      logDebug(s"Asked to remove executor $executorId with reason $reason")
      executorDataMap.get(executorId) match {
        case Some(executorInfo) =>
          // This must be synchronized because variables mutated
          // in this block are read when requesting executors
          val killed = CoarseGrainedSchedulerBackend.this.synchronized {
            addressToExecutorId -= executorInfo.executorAddress
            executorDataMap -= executorId
            executorsPendingLossReason -= executorId
            executorsPendingToRemove.remove(executorId).getOrElse(false)
          }
     ...

## How was this patch tested?

manual test.

Author: w00228970 <wangfei1@huawei.com>

Closes #15481 from scwf/spark-17929.
2016-10-21 14:43:55 -07:00
Hossein e371040a01 [SPARK-17811] SparkR cannot parallelize data.frame with NA or NULL in Date columns
## What changes were proposed in this pull request?
NA date values are serialized as "NA" and NA time values are serialized as NaN from R. In the backend we did not have proper logic to deal with them. As a result we got an IllegalArgumentException for Date and wrong value for time. This PR adds support for deserializing NA as Date and Time.

## How was this patch tested?
* [x] TODO

Author: Hossein <hossein@databricks.com>

Closes #15421 from falaki/SPARK-17811.
2016-10-21 12:38:52 -07:00
Alex Bozarth 3a237512b1
[SPARK-13275][WEB UI] Visually clarified executors start time in timeline
## What changes were proposed in this pull request?

Updated the Executors added/removed bubble in the time line so it's clearer where it starts. Now the bubble is left justified on the start time (still also denoted by the line) rather than center justified.

## How was this patch tested?

Manually tested UI

<img width="596" alt="screen shot 2016-10-17 at 6 04 36 pm" src="https://cloud.githubusercontent.com/assets/13952758/19496563/e6c9186e-953c-11e6-85e4-63309a553f65.png">
<img width="492" alt="screen shot 2016-10-17 at 5 54 09 pm" src="https://cloud.githubusercontent.com/assets/13952758/19496568/e9f06132-953c-11e6-8901-54405ebc7f5b.png">

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

Closes #15536 from ajbozarth/spark13275.
2016-10-21 11:39:32 +01:00
Zheng RuiFeng a8ea4da8d0
[SPARK-17331][FOLLOWUP][ML][CORE] Avoid allocating 0-length arrays
## What changes were proposed in this pull request?

`Array[T]()` -> `Array.empty[T]` to avoid allocating 0-length arrays.
Use regex `find . -name '*.scala' | xargs -i bash -c 'egrep "Array\[[A-Za-z]+\]\(\)" -n {} && echo {}'` to find modification candidates.

cc srowen

## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #15564 from zhengruifeng/avoid_0_length_array.
2016-10-21 09:49:37 +01:00
Jagadeesan 595893d33a
[SPARK-17960][PYSPARK][UPGRADE TO PY4J 0.10.4]
## What changes were proposed in this pull request?

1) Upgrade the Py4J version on the Java side
2) Update the py4j src zip file we bundle with Spark

## How was this patch tested?

Existing doctests & unit tests pass

Author: Jagadeesan <as2@us.ibm.com>

Closes #15514 from jagadeesanas2/SPARK-17960.
2016-10-21 09:48:24 +01:00
WeichenXu 39755169fb [SPARK-18003][SPARK CORE] Fix bug of RDD zipWithIndex & zipWithUniqueId index value overflowing
## What changes were proposed in this pull request?

- Fix bug of RDD `zipWithIndex` generating wrong result when one partition contains more than 2147483647 records.

- Fix bug of RDD `zipWithUniqueId` generating wrong result when one partition contains more than 2147483647 records.

## How was this patch tested?

test added.

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #15550 from WeichenXu123/fix_rdd_zipWithIndex_overflow.
2016-10-19 23:41:38 -07:00
Alex Bozarth 444c2d22e3 [SPARK-10541][WEB UI] Allow ApplicationHistoryProviders to provide their own text when there aren't any complete apps
## What changes were proposed in this pull request?

I've added a method to `ApplicationHistoryProvider` that returns the html paragraph to display when there are no applications. This allows providers other than `FsHistoryProvider` to determine what is printed. The current hard coded text is now moved into `FsHistoryProvider` since it assumed that's what was being used before.

I chose to make the function return html rather than text because the current text block had inline html in it and it allows a new implementation of `ApplicationHistoryProvider` more versatility. I did not see any security issues with this since injecting html here requires implementing `ApplicationHistoryProvider` and can't be done outside of code.

## How was this patch tested?

Manual testing and dev/run-tests

No visible changes to the UI

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

Closes #15490 from ajbozarth/spark10541.
2016-10-19 13:01:33 -07:00
Yu Peng 2629cd7460 [SPARK-17711][TEST-HADOOP2.2] Fix hadoop2.2 compilation error
## What changes were proposed in this pull request?

Fix hadoop2.2 compilation error.

## How was this patch tested?

Existing tests.

cc tdas zsxwing

Author: Yu Peng <loneknightpy@gmail.com>

Closes #15537 from loneknightpy/fix-17711.
2016-10-18 19:43:08 -07:00
Guoqiang Li 4518642abd [SPARK-17930][CORE] The SerializerInstance instance used when deserializing a TaskResult is not reused
## What changes were proposed in this pull request?
The following code is called when the DirectTaskResult instance is deserialized

```scala

  def value(): T = {
    if (valueObjectDeserialized) {
      valueObject
    } else {
      // Each deserialization creates a new instance of SerializerInstance, which is very time-consuming
      val resultSer = SparkEnv.get.serializer.newInstance()
      valueObject = resultSer.deserialize(valueBytes)
      valueObjectDeserialized = true
      valueObject
    }
  }

```

In the case of stage has a lot of tasks, reuse SerializerInstance instance can improve the scheduling performance of three times

The test data is TPC-DS 2T (Parquet) and  SQL statement as follows (query 2):

```sql

select  i_item_id,
        avg(ss_quantity) agg1,
        avg(ss_list_price) agg2,
        avg(ss_coupon_amt) agg3,
        avg(ss_sales_price) agg4
 from store_sales, customer_demographics, date_dim, item, promotion
 where ss_sold_date_sk = d_date_sk and
       ss_item_sk = i_item_sk and
       ss_cdemo_sk = cd_demo_sk and
       ss_promo_sk = p_promo_sk and
       cd_gender = 'M' and
       cd_marital_status = 'M' and
       cd_education_status = '4 yr Degree' and
       (p_channel_email = 'N' or p_channel_event = 'N') and
       d_year = 2001
 group by i_item_id
 order by i_item_id
 limit 100;

```

`spark-defaults.conf` file:

```
spark.master                           yarn-client
spark.executor.instances               20
spark.driver.memory                    16g
spark.executor.memory                  30g
spark.executor.cores                   5
spark.default.parallelism              100
spark.sql.shuffle.partitions           100000
spark.serializer                       org.apache.spark.serializer.KryoSerializer
spark.driver.maxResultSize              0
spark.rpc.netty.dispatcher.numThreads   8
spark.executor.extraJavaOptions          -XX:+UseG1GC -XX:+UseStringDeduplication -XX:G1HeapRegionSize=16M -XX:MetaspaceSize=256M
spark.cleaner.referenceTracking.blocking true
spark.cleaner.referenceTracking.blocking.shuffle true

```

Performance test results are as follows

[SPARK-17930](https://github.com/witgo/spark/tree/SPARK-17930)| [ed14633](ed14633414])
------------ | -------------
54.5 s|231.7 s

## How was this patch tested?

Existing tests.

Author: Guoqiang Li <witgo@qq.com>

Closes #15512 from witgo/SPARK-17930.
2016-10-18 13:46:57 -07:00
Yu Peng 231f39e3f6 [SPARK-17711] Compress rolled executor log
## What changes were proposed in this pull request?

This PR adds support for executor log compression.

## How was this patch tested?

Unit tests

cc: yhuai tdas mengxr

Author: Yu Peng <loneknightpy@gmail.com>

Closes #15285 from loneknightpy/compress-executor-log.
2016-10-18 13:23:31 -07:00
Liwei Lin 7d878cf2da [SQL][STREAMING][TEST] Fix flaky tests in StreamingQueryListenerSuite
This work has largely been done by lw-lin in his PR #15497. This is a slight refactoring of it.

## What changes were proposed in this pull request?
There were two sources of flakiness in StreamingQueryListener test.

- When testing with manual clock, consecutive attempts to advance the clock can occur without the stream execution thread being unblocked and doing some work between the two attempts. Hence the following can happen with the current ManualClock.
```
+-----------------------------------+--------------------------------+
|      StreamExecution thread       |         testing thread         |
+-----------------------------------+--------------------------------+
|  ManualClock.waitTillTime(100) {  |                                |
|        _isWaiting = true          |                                |
|            wait(10)               |                                |
|        still in wait(10)          |  if (_isWaiting) advance(100)  |
|        still in wait(10)          |  if (_isWaiting) advance(200)  | <- this should be disallowed !
|        still in wait(10)          |  if (_isWaiting) advance(300)  | <- this should be disallowed !
|      wake up from wait(10)        |                                |
|       current time is 600         |                                |
|       _isWaiting = false          |                                |
|  }                                |                                |
+-----------------------------------+--------------------------------+
```

- Second source of flakiness is that the adding data to memory stream may get processing in any trigger, not just the first trigger.

My fix is to make the manual clock wait for the other stream execution thread to start waiting for the clock at the right wait start time. That is, `advance(200)` (see above) will wait for stream execution thread to complete the wait that started at time 0, and start a new wait at time 200 (i.e. time stamp after the previous `advance(100)`).

In addition, since this is a feature that is solely used by StreamExecution, I removed all the non-generic code from ManualClock and put them in StreamManualClock inside StreamTest.

## How was this patch tested?
Ran existing unit test MANY TIME in Jenkins

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

Closes #15519 from tdas/metrics-flaky-test-fix.
2016-10-18 00:49:57 -07:00
Sital Kedia c7ac027d5f [SPARK-17839][CORE] Use Nio's directbuffer instead of BufferedInputStream in order to avoid additional copy from os buffer cache to user buffer
## What changes were proposed in this pull request?

Currently we use BufferedInputStream to read the shuffle file which copies the file content from os buffer cache to the user buffer. This adds additional latency in reading the spill files. We made a change to use java nio's direct buffer to read the spill files and for certain pipelines spilling significant amount of data, we see up to 7% speedup for the entire pipeline.

## How was this patch tested?
Tested by running the job in the cluster and observed up to 7% speedup.

Author: Sital Kedia <skedia@fb.com>

Closes #15408 from sitalkedia/skedia/nio_spill_read.
2016-10-17 11:03:04 -07:00
Reynold Xin 72a6e7a57a Revert "[SPARK-17637][SCHEDULER] Packed scheduling for Spark tasks across executors"
This reverts commit ed14633414.

The patch merged had obvious quality and documentation issue. The idea is useful, and we should work towards improving its quality and merging it in again.
2016-10-15 22:31:37 -07:00
Zhan Zhang ed14633414 [SPARK-17637][SCHEDULER] Packed scheduling for Spark tasks across executors
## What changes were proposed in this pull request?

Restructure the code and implement two new task assigner.
PackedAssigner: try to allocate tasks to the executors with least available cores, so that spark can release reserved executors when dynamic allocation is enabled.

BalancedAssigner: try to allocate tasks to the executors with more available cores in order to balance the workload across all executors.

By default, the original round robin assigner is used.

We test a pipeline, and new PackedAssigner  save around 45% regarding the reserved cpu and memory with dynamic allocation enabled.

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
Both unit test in TaskSchedulerImplSuite and manual tests in production pipeline.

Author: Zhan Zhang <zhanzhang@fb.com>

Closes #15218 from zhzhan/packed-scheduler.
2016-10-15 18:45:04 -07:00
Michael Allman 6ce1b675ee [SPARK-16980][SQL] Load only catalog table partition metadata required to answer a query
(This PR addresses https://issues.apache.org/jira/browse/SPARK-16980.)

## What changes were proposed in this pull request?

In a new Spark session, when a partitioned Hive table is converted to use Spark's `HadoopFsRelation` in `HiveMetastoreCatalog`, metadata for every partition of that table are retrieved from the metastore and loaded into driver memory. In addition, every partition's metadata files are read from the filesystem to perform schema inference.

If a user queries such a table with predicates which prune that table's partitions, we would like to be able to answer that query without consulting partition metadata which are not involved in the query. When querying a table with a large number of partitions for some data from a small number of partitions (maybe even a single partition), the current conversion strategy is highly inefficient. I suspect this scenario is not uncommon in the wild.

In addition to being inefficient in running time, the current strategy is inefficient in its use of driver memory. When the sum of the number of partitions of all tables loaded in a driver reaches a certain level (somewhere in the tens of thousands), their cached data exhaust all driver heap memory in the default configuration. I suspect this scenario is less common (in that not too many deployments work with tables with tens of thousands of partitions), however this does illustrate how large the memory footprint of this metadata can be. With tables with hundreds or thousands of partitions, I would expect the `HiveMetastoreCatalog` table cache to represent a significant portion of the driver's heap space.

This PR proposes an alternative approach. Basically, it makes four changes:

1. It adds a new method, `listPartitionsByFilter` to the Catalyst `ExternalCatalog` trait which returns the partition metadata for a given sequence of partition pruning predicates.
1. It refactors the `FileCatalog` type hierarchy to include a new `TableFileCatalog` to efficiently return files only for partitions matching a sequence of partition pruning predicates.
1. It removes partition loading and caching from `HiveMetastoreCatalog`.
1. It adds a new Catalyst optimizer rule, `PruneFileSourcePartitions`, which applies a plan's partition-pruning predicates to prune out unnecessary partition files from a `HadoopFsRelation`'s underlying file catalog.

The net effect is that when a query over a partitioned Hive table is planned, the analyzer retrieves the table metadata from `HiveMetastoreCatalog`. As part of this operation, the `HiveMetastoreCatalog` builds a `HadoopFsRelation` with a `TableFileCatalog`. It does not load any partition metadata or scan any files. The optimizer prunes-away unnecessary table partitions by sending the partition-pruning predicates to the relation's `TableFileCatalog `. The `TableFileCatalog` in turn calls the `listPartitionsByFilter` method on its external catalog. This queries the Hive metastore, passing along those filters.

As a bonus, performing partition pruning during optimization leads to a more accurate relation size estimate. This, along with c481bdf, can lead to automatic, safe application of the broadcast optimization in a join where it might previously have been omitted.

## Open Issues

1. This PR omits partition metadata caching. I can add this once the overall strategy for the cold path is established, perhaps in a future PR.
1. This PR removes and omits partitioned Hive table schema reconciliation. As a result, it fails to find Parquet schema columns with upper case letters because of the Hive metastore's case-insensitivity. This issue may be fixed by #14750, but that PR appears to have stalled. ericl has contributed to this PR a workaround for Parquet wherein schema reconciliation occurs at query execution time instead of planning. Whether ORC requires a similar patch is an open issue.
1. This PR omits an implementation of `listPartitionsByFilter` for the `InMemoryCatalog`.
1. This PR breaks parquet log output redirection during query execution. I can work around this by running `Class.forName("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$")` first thing in a Spark shell session, but I haven't figured out how to fix this properly.

## How was this patch tested?

The current Spark unit tests were run, and some ad-hoc tests were performed to validate that only the necessary partition metadata is loaded.

Author: Michael Allman <michael@videoamp.com>
Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #14690 from mallman/spark-16980-lazy_partition_fetching.
2016-10-14 18:26:18 -07:00
invkrh 28b645b1e6
[SPARK-17855][CORE] Remove query string from jar url
## What changes were proposed in this pull request?

Spark-submit support jar url with http protocol. However, if the url contains any query strings, `worker.DriverRunner.downloadUserJar()` method will throw "Did not see expected jar" exception. This is because this method checks the existance of a downloaded jar whose name contains query strings. This is a problem when your jar is located on some web service which requires some additional information to retrieve the file.

This pr just removes query strings before checking jar existance on worker.

## How was this patch tested?

For now, you can only test this patch by manual test.
* Deploy a spark cluster locally
* Make sure apache httpd service is on
* Save an uber jar, e.g spark-job.jar under `/var/www/html/`
* Use http://localhost/spark-job.jar?param=1 as jar url when running `spark-submit`
* Job should be launched

Author: invkrh <invkrh@gmail.com>

Closes #15420 from invkrh/spark-17855.
2016-10-14 12:52:08 +01:00
jerryshao 7bf8a40498 [SPARK-17686][CORE] Support printing out scala and java version with spark-submit --version command
## What changes were proposed in this pull request?

In our universal gateway service we need to specify different jars to Spark according to scala version. For now only after launching Spark application can we know which version of Scala it depends on. It makes hard for us to support different Scala + Spark versions to pick the right jars.

So here propose to print out Scala version according to Spark version in "spark-submit --version", so that user could leverage this output to make the choice without needing to launching application.

## How was this patch tested?

Manually verified in local environment.

Author: jerryshao <sshao@hortonworks.com>

Closes #15456 from jerryshao/SPARK-17686.
2016-10-13 03:29:14 -04:00
Alex Bozarth 6f2fa6c54a [SPARK-11272][WEB UI] Add support for downloading event logs from HistoryServer UI
## What changes were proposed in this pull request?

This is a reworked PR based on feedback in #9238 after it was closed and not reopened. As suggested in that PR I've only added the download feature. This functionality already exists in the api and this allows easier access to download event logs to share with others.

I've attached a screenshot of the committed version, but I will also include alternate options with screen shots in the comments below. I'm personally not sure which option is best.

## How was this patch tested?

Manual testing

![screen shot 2016-10-07 at 6 11 12 pm](https://cloud.githubusercontent.com/assets/13952758/19209213/832fe48e-8cba-11e6-9840-749b1be4d399.png)

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

Closes #15400 from ajbozarth/spark11272.
2016-10-13 03:24:37 -04:00
Imran Rashid 9ce7d3e542 [SPARK-17675][CORE] Expand Blacklist for TaskSets
## What changes were proposed in this pull request?

This is a step along the way to SPARK-8425.

To enable incremental review, the first step proposed here is to expand the blacklisting within tasksets. In particular, this will enable blacklisting for
* (task, executor) pairs (this already exists via an undocumented config)
* (task, node)
* (taskset, executor)
* (taskset, node)

Adding (task, node) is critical to making spark fault-tolerant of one-bad disk in a cluster, without requiring careful tuning of "spark.task.maxFailures". The other additions are also important to avoid many misleading task failures and long scheduling delays when there is one bad node on a large cluster.

Note that some of the code changes here aren't really required for just this -- they put pieces in place for SPARK-8425 even though they are not used yet (eg. the `BlacklistTracker` helper is a little out of place, `TaskSetBlacklist` holds onto a little more info than it needs to for just this change, and `ExecutorFailuresInTaskSet` is more complex than it needs to be).

## How was this patch tested?

Added unit tests, run tests via jenkins.

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

Closes #15249 from squito/taskset_blacklist_only.
2016-10-12 16:43:03 -05:00
Shixiong Zhu 47776e7c0c [SPARK-17850][CORE] Add a flag to ignore corrupt files
## What changes were proposed in this pull request?

Add a flag to ignore corrupt files. For Spark core, the configuration is `spark.files.ignoreCorruptFiles`. For Spark SQL, it's `spark.sql.files.ignoreCorruptFiles`.

## How was this patch tested?

The added unit tests

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15422 from zsxwing/SPARK-17850.
2016-10-12 13:51:53 -07:00
Hossein 5cc503f4fe [SPARK-17790][SPARKR] Support for parallelizing R data.frame larger than 2GB
## What changes were proposed in this pull request?
If the R data structure that is being parallelized is larger than `INT_MAX` we use files to transfer data to JVM. The serialization protocol mimics Python pickling. This allows us to simply call `PythonRDD.readRDDFromFile` to create the RDD.

I tested this on my MacBook. Following code works with this patch:
```R
intMax <- .Machine$integer.max
largeVec <- 1:intMax
rdd <- SparkR:::parallelize(sc, largeVec, 2)
```

## How was this patch tested?
* [x] Unit tests

Author: Hossein <hossein@databricks.com>

Closes #15375 from falaki/SPARK-17790.
2016-10-12 10:32:38 -07:00
Wenchen Fan b9a147181d [SPARK-17720][SQL] introduce static SQL conf
## What changes were proposed in this pull request?

SQLConf is session-scoped and mutable. However, we do have the requirement for a static SQL conf, which is global and immutable, e.g. the `schemaStringThreshold` in `HiveExternalCatalog`, the flag to enable/disable hive support, the global temp view database in https://github.com/apache/spark/pull/14897.

Actually we've already implemented static SQL conf implicitly via `SparkConf`, this PR just make it explicit and expose it to users, so that they can see the config value via SQL command or `SparkSession.conf`, and forbid users to set/unset static SQL conf.

## How was this patch tested?

new tests in SQLConfSuite

Author: Wenchen Fan <wenchen@databricks.com>

Closes #15295 from cloud-fan/global-conf.
2016-10-11 20:27:08 -07:00
Bryan Cutler 658c7147f5
[SPARK-17808][PYSPARK] Upgraded version of Pyrolite to 4.13
## What changes were proposed in this pull request?
Upgraded to a newer version of Pyrolite which supports serialization of a BinaryType StructField for PySpark.SQL

## How was this patch tested?
Added a unit test which fails with a raised ValueError when using the previous version of Pyrolite 4.9 and Python3

Author: Bryan Cutler <cutlerb@gmail.com>

Closes #15386 from BryanCutler/pyrolite-upgrade-SPARK-17808.
2016-10-11 08:29:52 +02:00
Ergin Seyfe 19a5bae47f [SPARK-17816][CORE] Fix ConcurrentModificationException issue in BlockStatusesAccumulator
## What changes were proposed in this pull request?
Change the BlockStatusesAccumulator to return immutable object when value method is called.

## How was this patch tested?
Existing tests plus I verified this change by running a pipeline which consistently repro this issue.

This is the stack trace for this exception:
`
java.util.ConcurrentModificationException
        at java.util.ArrayList$Itr.checkForComodification(ArrayList.java:901)
        at java.util.ArrayList$Itr.next(ArrayList.java:851)
        at scala.collection.convert.Wrappers$JIteratorWrapper.next(Wrappers.scala:43)
        at scala.collection.Iterator$class.foreach(Iterator.scala:893)
        at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
        at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
        at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
        at scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:59)
        at scala.collection.mutable.ListBuffer.$plus$plus$eq(ListBuffer.scala:183)
        at scala.collection.mutable.ListBuffer.$plus$plus$eq(ListBuffer.scala:45)
        at scala.collection.TraversableLike$class.to(TraversableLike.scala:590)
        at scala.collection.AbstractTraversable.to(Traversable.scala:104)
        at scala.collection.TraversableOnce$class.toList(TraversableOnce.scala:294)
        at scala.collection.AbstractTraversable.toList(Traversable.scala:104)
        at org.apache.spark.util.JsonProtocol$.accumValueToJson(JsonProtocol.scala:314)
        at org.apache.spark.util.JsonProtocol$$anonfun$accumulableInfoToJson$5.apply(JsonProtocol.scala:291)
        at org.apache.spark.util.JsonProtocol$$anonfun$accumulableInfoToJson$5.apply(JsonProtocol.scala:291)
        at scala.Option.map(Option.scala:146)
        at org.apache.spark.util.JsonProtocol$.accumulableInfoToJson(JsonProtocol.scala:291)
        at org.apache.spark.util.JsonProtocol$$anonfun$taskInfoToJson$12.apply(JsonProtocol.scala:283)
        at org.apache.spark.util.JsonProtocol$$anonfun$taskInfoToJson$12.apply(JsonProtocol.scala:283)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
        at scala.collection.immutable.List.foreach(List.scala:381)
        at scala.collection.generic.TraversableForwarder$class.foreach(TraversableForwarder.scala:35)
        at scala.collection.mutable.ListBuffer.foreach(ListBuffer.scala:45)
        at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
        at scala.collection.AbstractTraversable.map(Traversable.scala:104)
        at org.apache.spark.util.JsonProtocol$.taskInfoToJson(JsonProtocol.scala:283)
        at org.apache.spark.util.JsonProtocol$.taskEndToJson(JsonProtocol.scala:145)
        at org.apache.spark.util.JsonProtocol$.sparkEventToJson(JsonProtocol.scala:76)
`

Author: Ergin Seyfe <eseyfe@fb.com>

Closes #15371 from seyfe/race_cond_jsonprotocal.
2016-10-10 20:41:31 -07:00
Dhruve Ashar 4bafacaa5f [SPARK-17417][CORE] Fix # of partitions for Reliable RDD checkpointing
## What changes were proposed in this pull request?
Currently the no. of partition files are limited to 10000 files (%05d format). If there are more than 10000 part files, the logic goes for a toss while recreating the RDD as it sorts them by string. More details can be found in the JIRA desc [here](https://issues.apache.org/jira/browse/SPARK-17417).

## How was this patch tested?
I tested this patch by checkpointing a RDD and then manually renaming part files to the old format and tried to access the RDD. It was successfully created from the old format. Also verified loading a sample parquet file and saving it as multiple formats - CSV, JSON, Text, Parquet, ORC and read them successfully back from the saved files. I couldn't launch the unit test from my local box, so will wait for the Jenkins output.

Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #15370 from dhruve/bug/SPARK-17417.
2016-10-10 10:55:57 -05:00
Wenchen Fan 23ddff4b2b [SPARK-17338][SQL] add global temp view
## What changes were proposed in this pull request?

Global temporary view is a cross-session temporary view, which means it's shared among all sessions. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database `global_temp`(configurable via SparkConf), and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

changes for `SessionCatalog`:

1. add a new field `gloabalTempViews: GlobalTempViewManager`, to access the shared global temp views, and the global temp db name.
2. `createDatabase` will fail if users wanna create `global_temp`, which is system preserved.
3. `setCurrentDatabase` will fail if users wanna set `global_temp`, which is system preserved.
4. add `createGlobalTempView`, which is used in `CreateViewCommand` to create global temp views.
5. add `dropGlobalTempView`, which is used in `CatalogImpl` to drop global temp view.
6. add `alterTempViewDefinition`, which is used in `AlterViewAsCommand` to update the view definition for local/global temp views.
7. `renameTable`/`dropTable`/`isTemporaryTable`/`lookupRelation`/`getTempViewOrPermanentTableMetadata`/`refreshTable` will handle global temp views.

changes for SQL commands:

1. `CreateViewCommand`/`AlterViewAsCommand` is updated to support global temp views
2. `ShowTablesCommand` outputs a new column `database`, which is used to distinguish global and local temp views.
3. other commands can also handle global temp views if they call `SessionCatalog` APIs which accepts global temp views, e.g. `DropTableCommand`, `AlterTableRenameCommand`, `ShowColumnsCommand`, etc.

changes for other public API

1. add a new method `dropGlobalTempView` in `Catalog`
2. `Catalog.findTable` can find global temp view
3. add a new method `createGlobalTempView` in `Dataset`

## How was this patch tested?

new tests in `SQLViewSuite`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #14897 from cloud-fan/global-temp-view.
2016-10-10 15:48:57 +08:00
Weiqing Yang 8a6bbe095b
[MINOR][SQL] Use resource path for test_script.sh
## What changes were proposed in this pull request?
This PR modified the test case `test("script")` to use resource path for `test_script.sh`. Make the test case portable (even in IntelliJ).

## How was this patch tested?
Passed the test case.
Before:
Run `test("script")` in IntelliJ:
```
Caused by: org.apache.spark.SparkException: Subprocess exited with status 127. Error: bash: src/test/resources/test_script.sh: No such file or directory
```
After:
Test passed.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15246 from weiqingy/hivetest.
2016-10-08 12:12:35 +01:00
Sean Owen 4201ddcc07
[SPARK-17768][CORE] Small (Sum,Count,Mean)Evaluator problems and suboptimalities
## What changes were proposed in this pull request?

Fix:

- GroupedMeanEvaluator and GroupedSumEvaluator are unused, as is the StudentTCacher support class
- CountEvaluator can return a lower bound < 0, when counts can't be negative
- MeanEvaluator will actually fail on exactly 1 datum (yields t-test with 0 DOF)
- CountEvaluator uses a normal distribution, which may be an inappropriate approximation (leading to above)
- Test for SumEvaluator asserts incorrect expected sums – e.g. after observing 10% of data has sum of 2, expectation should be 20, not 38
- CountEvaluator, MeanEvaluator have no unit tests to catch these
- Duplication of distribution code across CountEvaluator, GroupedCountEvaluator
- The stats in each could use a bit of documentation as I had to guess at them
- (Code could use a few cleanups and optimizations too)

## How was this patch tested?

Existing and new tests

Author: Sean Owen <sowen@cloudera.com>

Closes #15341 from srowen/SPARK-17768.
2016-10-08 11:31:12 +01:00
Alex Bozarth 362ba4b6f8
[SPARK-17793][WEB UI] Sorting on the description on the Job or Stage page doesn’t always work
## What changes were proposed in this pull request?

Added secondary sorting on stage name for the description column. This provide a clearer behavior in the common case where the Description column only comprises of Stage names instead of the option description value.

## How was this patch tested?

manual testing and dev/run-tests

Screenshots of sorting on both description and stage name as well as an example of both:
![screen shot 2016-10-04 at 1 09 39 pm](https://cloud.githubusercontent.com/assets/13952758/19135523/067b042e-8b1a-11e6-912e-e6371d006d21.png)
![screen shot 2016-10-04 at 1 09 51 pm](https://cloud.githubusercontent.com/assets/13952758/19135526/06960936-8b1a-11e6-85e9-8aaf694c5f7b.png)
![screen shot 2016-10-05 at 1 14 45 pm](https://cloud.githubusercontent.com/assets/13952758/19135525/069547da-8b1a-11e6-8692-6524c75c4c07.png)
![screen shot 2016-10-05 at 1 14 51 pm](https://cloud.githubusercontent.com/assets/13952758/19135524/0694b4d2-8b1a-11e6-92dc-c8aa514e4f62.png)
![screen shot 2016-10-05 at 4 42 52 pm](https://cloud.githubusercontent.com/assets/13952758/19135618/e232eafe-8b1a-11e6-88b3-ff0bbb26b7f8.png)

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

Closes #15366 from ajbozarth/spark17793.
2016-10-08 11:24:00 +01:00
Sean Owen cff5607552 [SPARK-17707][WEBUI] Web UI prevents spark-submit application to be finished
## What changes were proposed in this pull request?

This expands calls to Jetty's simple `ServerConnector` constructor to explicitly specify a `ScheduledExecutorScheduler` that makes daemon threads. It should otherwise result in exactly the same configuration, because the other args are copied from the constructor that is currently called.

(I'm not sure we should change the Hive Thriftserver impl, but I did anyway.)

This also adds `sc.stop()` to the quick start guide example.

## How was this patch tested?

Existing tests; _pending_ at least manual verification of the fix.

Author: Sean Owen <sowen@cloudera.com>

Closes #15381 from srowen/SPARK-17707.
2016-10-07 10:31:41 -07:00
Brian Cho e56614cba9 [SPARK-16827] Stop reporting spill metrics as shuffle metrics
## What changes were proposed in this pull request?

Fix a bug where spill metrics were being reported as shuffle metrics. Eventually these spill metrics should be reported (SPARK-3577), but separate from shuffle metrics. The fix itself basically reverts the line to what it was in 1.6.

## How was this patch tested?

Tested on a job that was reporting shuffle writes even for the final stage, when no shuffle writes should take place. After the change the job no longer shows these writes.

Before:
![screen shot 2016-10-03 at 6 39 59 pm](https://cloud.githubusercontent.com/assets/1514239/19085897/dbf59a92-8a20-11e6-9f68-a978860c0d74.png)

After:
<img width="1052" alt="screen shot 2016-10-03 at 11 44 44 pm" src="https://cloud.githubusercontent.com/assets/1514239/19085903/e173a860-8a20-11e6-85e3-d47f9835f494.png">

Author: Brian Cho <bcho@fb.com>

Closes #15347 from dafrista/shuffle-metrics.
2016-10-07 11:37:18 -04:00
Alex Bozarth 24097d8474
[SPARK-17795][WEB UI] Sorting on stage or job tables doesn’t reload page on that table
## What changes were proposed in this pull request?

Added anchor on table header id to sorting links on job and stage tables. This make the page reload after a sort load the page at the sorted table.

This only changes page load behavior so no UI changes

## How was this patch tested?

manually tested and dev/run-tests

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

Closes #15369 from ajbozarth/spark17795.
2016-10-07 11:47:37 +01:00
Shixiong Zhu 9293734d35 [SPARK-17346][SQL] Add Kafka source for Structured Streaming
## What changes were proposed in this pull request?

This PR adds a new project ` external/kafka-0-10-sql` for Structured Streaming Kafka source.

It's based on the design doc: https://docs.google.com/document/d/19t2rWe51x7tq2e5AOfrsM9qb8_m7BRuv9fel9i0PqR8/edit?usp=sharing

tdas did most of work and part of them was inspired by koeninger's work.

### Introduction

The Kafka source is a structured streaming data source to poll data from Kafka. The schema of reading data is as follows:

Column | Type
---- | ----
key | binary
value | binary
topic | string
partition | int
offset | long
timestamp | long
timestampType | int

The source can deal with deleting topics. However, the user should make sure there is no Spark job processing the data when deleting a topic.

### Configuration

The user can use `DataStreamReader.option` to set the following configurations.

Kafka Source's options | value | default | meaning
------ | ------- | ------ | -----
startingOffset | ["earliest", "latest"] | "latest" | The start point when a query is started, either "earliest" which is from the earliest offset, or "latest" which is just from the latest offset. Note: This only applies when a new Streaming query is started, and that resuming will always pick up from where the query left off.
failOnDataLost | [true, false] | true | Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work as you expected.
subscribe | A comma-separated list of topics | (none) | The topic list to subscribe. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
subscribePattern | Java regex string | (none) | The pattern used to subscribe the topic. Only one of "subscribe" and "subscribeParttern" options can be specified for Kafka source.
kafka.consumer.poll.timeoutMs | long | 512 | The timeout in milliseconds to poll data from Kafka in executors
fetchOffset.numRetries | int | 3 | Number of times to retry before giving up fatch Kafka latest offsets.
fetchOffset.retryIntervalMs | long | 10 | milliseconds to wait before retrying to fetch Kafka offsets

Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, `stream.option("kafka.bootstrap.servers", "host:port")`

### Usage

* Subscribe to 1 topic
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1")
  .load()
```

* Subscribe to multiple topics
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribe", "topic1,topic2")
  .load()
```

* Subscribe to a pattern
```Scala
spark
  .readStream
  .format("kafka")
  .option("kafka.bootstrap.servers", "host:port")
  .option("subscribePattern", "topic.*")
  .load()
```

## How was this patch tested?

The new unit tests.

Author: Shixiong Zhu <shixiong@databricks.com>
Author: Tathagata Das <tathagata.das1565@gmail.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Author: cody koeninger <cody@koeninger.org>

Closes #15102 from zsxwing/kafka-source.
2016-10-05 16:45:45 -07:00
Shixiong Zhu 221b418b1c [SPARK-17778][TESTS] Mock SparkContext to reduce memory usage of BlockManagerSuite
## What changes were proposed in this pull request?

Mock SparkContext to reduce memory usage of BlockManagerSuite

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15350 from zsxwing/SPARK-17778.
2016-10-05 14:54:55 -07:00
sumansomasundar 7d51608835
[SPARK-16962][CORE][SQL] Fix misaligned record accesses for SPARC architectures
## What changes were proposed in this pull request?

Made changes to record length offsets to make them uniform throughout various areas of Spark core and unsafe

## How was this patch tested?

This change affects only SPARC architectures and was tested on X86 architectures as well for regression.

Author: sumansomasundar <suman.somasundar@oracle.com>

Closes #14762 from sumansomasundar/master.
2016-10-04 10:31:56 +01:00
Sean Owen 8e8de0073d
[SPARK-17671][WEBUI] Spark 2.0 history server summary page is slow even set spark.history.ui.maxApplications
## What changes were proposed in this pull request?

Return Iterator of applications internally in history server, for consistency and performance. See https://github.com/apache/spark/pull/15248 for some back-story.

The code called by and calling HistoryServer.getApplicationList wants an Iterator, but this method materializes an Iterable, which potentially causes a performance problem. It's simpler too to make this internal method also pass through an Iterator.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #15321 from srowen/SPARK-17671.
2016-10-04 10:29:22 +01:00
Tao LI 76dc2d9073 [SPARK-14914][CORE][SQL] Skip/fix some test cases on Windows due to limitation of Windows
## What changes were proposed in this pull request?

This PR proposes to fix/skip some tests failed on Windows. This PR takes over https://github.com/apache/spark/pull/12696.

**Before**

- **SparkSubmitSuite**

  ```
[info] - launch simple application with spark-submit *** FAILED *** (202 milliseconds)
[info]   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specifie

[info] - includes jars passed in through --jars *** FAILED *** (1 second, 625 milliseconds)
[info]   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```

- **DiskStoreSuite**

  ```
[info] - reads of memory-mapped and non memory-mapped files are equivalent *** FAILED *** (1 second, 78 milliseconds)
[info]   diskStoreMapped.remove(blockId) was false (DiskStoreSuite.scala:41)
```

**After**

- **SparkSubmitSuite**

  ```
[info] - launch simple application with spark-submit (578 milliseconds)
[info] - includes jars passed in through --jars (1 second, 875 milliseconds)
```

- **DiskStoreSuite**

  ```
[info] DiskStoreSuite:
[info] - reads of memory-mapped and non memory-mapped files are equivalent !!! CANCELED !!! (766 milliseconds
```

For `CreateTableAsSelectSuite` and `FsHistoryProviderSuite`, I could not reproduce as the Java version seems higher than the one that has the bugs about `setReadable(..)` and `setWritable(...)` but as they are bugs reported clearly, it'd be sensible to skip those. We should revert the changes for both back as soon as we drop the support of Java 7.

## How was this patch tested?

Manually tested via AppVeyor.

Closes #12696

Author: Tao LI <tl@microsoft.com>
Author: U-FAREAST\tl <tl@microsoft.com>
Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15320 from HyukjinKwon/SPARK-14914.
2016-10-02 16:01:02 -07:00
Eric Liang 4bcd9b728b [SPARK-17740] Spark tests should mock / interpose HDFS to ensure that streams are closed
## What changes were proposed in this pull request?

As a followup to SPARK-17666, ensure filesystem connections are not leaked at least in unit tests. This is done here by intercepting filesystem calls as suggested by JoshRosen . At the end of each test, we assert no filesystem streams are left open.

This applies to all tests using SharedSQLContext or SharedSparkContext.

## How was this patch tested?

I verified that tests in sql and core are indeed using the filesystem backend, and fixed the detected leaks. I also checked that reverting https://github.com/apache/spark/pull/15245 causes many actual test failures due to connection leaks.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #15306 from ericl/sc-4672.
2016-09-30 23:51:36 -07:00
Shubham Chopra a26afd5219 [SPARK-15353][CORE] Making peer selection for block replication pluggable
## What changes were proposed in this pull request?

This PR makes block replication strategies pluggable. It provides two trait that can be implemented, one that maps a host to its topology and is used in the master, and the second that helps prioritize a list of peers for block replication and would run in the executors.

This patch contains default implementations of these traits that make sure current Spark behavior is unchanged.

## How was this patch tested?

This patch should not change Spark behavior in any way, and was tested with unit tests for storage.

Author: Shubham Chopra <schopra31@bloomberg.net>

Closes #13152 from shubhamchopra/RackAwareBlockReplication.
2016-09-30 18:24:39 -07:00
Imran Rashid 3993ebca23 [SPARK-17676][CORE] FsHistoryProvider should ignore hidden files
## What changes were proposed in this pull request?

FsHistoryProvider was writing a hidden file (to check the fs's clock).
Even though it deleted the file immediately, sometimes another thread
would try to scan the files on the fs in-between, and then there would
be an error msg logged which was very misleading for the end-user.
(The logged error was harmless, though.)

## How was this patch tested?

I added one unit test, but to be clear, that test was passing before.  The actual change in behavior in that test is just logging (after the change, there is no more logged error), which I just manually verified.

Author: Imran Rashid <irashid@cloudera.com>

Closes #15250 from squito/SPARK-17676.
2016-09-29 15:40:35 -07:00
Brian Cho 027dea8f29 [SPARK-17715][SCHEDULER] Make task launch logs DEBUG
## What changes were proposed in this pull request?

Ramp down the task launch logs from INFO to DEBUG. Task launches can happen orders of magnitude more than executor registration so it makes the logs easier to handle if they are different log levels. For larger jobs, there can be 100,000s of task launches which makes the driver log huge.

## How was this patch tested?

No tests, as this is a trivial change.

Author: Brian Cho <bcho@fb.com>

Closes #15290 from dafrista/ramp-down-task-logging.
2016-09-29 15:59:17 -04:00
Gang Wu cb87b3ced9 [SPARK-17672] Spark 2.0 history server web Ui takes too long for a single application
Added a new API getApplicationInfo(appId: String) in class ApplicationHistoryProvider and class SparkUI to get app info. In this change, FsHistoryProvider can directly fetch one app info in O(1) time complexity compared to O(n) before the change which used an Iterator.find() interface.

Both ApplicationCache and OneApplicationResource classes adopt this new api.

 manual tests

Author: Gang Wu <wgtmac@uber.com>

Closes #15247 from wgtmac/SPARK-17671.
2016-09-29 15:51:38 -04:00
Imran Rashid 7f779e7439 [SPARK-17648][CORE] TaskScheduler really needs offers to be an IndexedSeq
## What changes were proposed in this pull request?

The Seq[WorkerOffer] is accessed by index, so it really should be an
IndexedSeq, otherwise an O(n) operation becomes O(n^2).  In practice
this hasn't been an issue b/c where these offers are generated, the call
to `.toSeq` just happens to create an IndexedSeq anyway.I got bitten by
this in performance tests I was doing, and its better for the types to be
more precise so eg. a change in Scala doesn't destroy performance.

## How was this patch tested?

Unit tests via jenkins.

Author: Imran Rashid <irashid@cloudera.com>

Closes #15221 from squito/SPARK-17648.
2016-09-29 15:36:40 -04:00
Weiqing Yang 7dfad4b132 [SPARK-17710][HOTFIX] Fix ClassCircularityError in ReplSuite tests in Maven build: use 'Class.forName' instead of 'Utils.classForName'
## What changes were proposed in this pull request?
Fix ClassCircularityError in ReplSuite tests when Spark is built by Maven build.

## How was this patch tested?
(1)
```
build/mvn -DskipTests -Phadoop-2.3 -Pyarn -Phive -Phive-thriftserver -Pkinesis-asl -Pmesos clean package
```
Then test:
```
build/mvn -Dtest=none -DwildcardSuites=org.apache.spark.repl.ReplSuite test
```
ReplSuite tests passed

(2)
Manual Tests against some Spark applications in Yarn client mode and Yarn cluster mode. Need to check if spark caller contexts are written into HDFS hdfs-audit.log and Yarn RM audit log successfully.

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15286 from Sherry302/SPARK-16757.
2016-09-28 20:20:03 -05:00
w00228970 46d1203bf2 [SPARK-17644][CORE] Do not add failedStages when abortStage for fetch failure
## What changes were proposed in this pull request?
| Time        |Thread 1 ,  Job1          | Thread 2 ,  Job2  |
|:-------------:|:-------------:|:-----:|
| 1 | abort stage due to FetchFailed |  |
| 2 | failedStages += failedStage |    |
| 3 |      |  task failed due to  FetchFailed |
| 4 |      |  can not post ResubmitFailedStages because failedStages is not empty |

Then job2 of thread2 never resubmit the failed stage and hang.

We should not add the failedStages when abortStage for fetch failure

## How was this patch tested?

added unit test

Author: w00228970 <wangfei1@huawei.com>
Author: wangfei <wangfei_hello@126.com>

Closes #15213 from scwf/dag-resubmit.
2016-09-28 12:02:59 -07:00
Liang-Chi Hsieh e7bce9e187 [SPARK-17056][CORE] Fix a wrong assert regarding unroll memory in MemoryStore
## What changes were proposed in this pull request?

There is an assert in MemoryStore's putIteratorAsValues method which is used to check if unroll memory is not released too much. This assert looks wrong.

## How was this patch tested?

Jenkins tests.

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

Closes #14642 from viirya/fix-unroll-memory.
2016-09-27 16:00:39 -07:00
Weiqing Yang 6a68c5d7b4 [SPARK-16757] Set up Spark caller context to HDFS and YARN
## What changes were proposed in this pull request?

1. Pass `jobId` to Task.
2. Invoke Hadoop APIs.
    * A new function `setCallerContext` is added in `Utils`. `setCallerContext` function invokes APIs of   `org.apache.hadoop.ipc.CallerContext` to set up spark caller contexts, which will be written into `hdfs-audit.log` and Yarn RM audit log.
    * For HDFS: Spark sets up its caller context by invoking`org.apache.hadoop.ipc.CallerContext` in `Task` and Yarn `Client` and `ApplicationMaster`.
    * For Yarn: Spark sets up its caller context by invoking `org.apache.hadoop.ipc.CallerContext` in Yarn `Client`.

## How was this patch tested?
Manual Tests against some Spark applications in Yarn client mode and Yarn cluster mode. Need to check if spark caller contexts are written into HDFS hdfs-audit.log and Yarn RM audit log successfully.

For example, run SparkKmeans in Yarn client mode:
```
./bin/spark-submit --verbose --executor-cores 3 --num-executors 1 --master yarn --deploy-mode client --class org.apache.spark.examples.SparkKMeans examples/target/original-spark-examples_2.11-2.1.0-SNAPSHOT.jar hdfs://localhost:9000/lr_big.txt 2 5
```

**Before**:
There will be no Spark caller context in records of `hdfs-audit.log` and Yarn RM audit log.

**After**:
Spark caller contexts will be written in records of `hdfs-audit.log` and Yarn RM audit log.

These are records in `hdfs-audit.log`:
```
2016-09-20 11:54:24,116 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_CLIENT_AppId_application_1474394339641_0005
2016-09-20 11:54:28,164 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0005_JobId_0_StageId_0_AttemptId_0_TaskId_2_AttemptNum_0
2016-09-20 11:54:28,164 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0005_JobId_0_StageId_0_AttemptId_0_TaskId_1_AttemptNum_0
2016-09-20 11:54:28,164 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0005_JobId_0_StageId_0_AttemptId_0_TaskId_0_AttemptNum_0
```
```
2016-09-20 11:59:33,868 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=mkdirs	src=/private/tmp/hadoop-wyang/nm-local-dir/usercache/wyang/appcache/application_1474394339641_0006/container_1474394339641_0006_01_000001/spark-warehouse	dst=null	perm=wyang:supergroup:rwxr-xr-x	proto=rpc	callerContext=SPARK_APPLICATION_MASTER_AppId_application_1474394339641_0006_AttemptId_1
2016-09-20 11:59:37,214 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_1_AttemptNum_0
2016-09-20 11:59:37,215 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_2_AttemptNum_0
2016-09-20 11:59:37,215 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_0_AttemptNum_0
2016-09-20 11:59:42,391 INFO FSNamesystem.audit: allowed=true	ugi=wyang (auth:SIMPLE)	ip=/127.0.0.1	cmd=open	src=/lr_big.txt	dst=null	perm=null	proto=rpc	callerContext=SPARK_TASK_AppId_application_1474394339641_0006_AttemptId_1_JobId_0_StageId_0_AttemptId_0_TaskId_3_AttemptNum_0
```
This is a record in Yarn RM log:
```
2016-09-20 11:59:24,050 INFO org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=wyang	IP=127.0.0.1	OPERATION=Submit Application Request	TARGET=ClientRMService	RESULT=SUCCESS	APPID=application_1474394339641_0006	CALLERCONTEXT=SPARK_CLIENT_AppId_application_1474394339641_0006
```

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #14659 from Sherry302/callercontextSubmit.
2016-09-27 08:10:38 -05:00
Ding Fei 6ee28423ad Fix two comments since Actor is not used anymore.
## What changes were proposed in this pull request?

Fix two comments since Actor is not used anymore.

Author: Ding Fei <danis@danix>

Closes #15251 from danix800/comment-fixing.
2016-09-26 23:09:51 -07:00
Shixiong Zhu bde85f8b70 [SPARK-17649][CORE] Log how many Spark events got dropped in LiveListenerBus
## What changes were proposed in this pull request?

Log how many Spark events got dropped in LiveListenerBus so that the user can get insights on how to set a correct event queue size.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15220 from zsxwing/SPARK-17649.
2016-09-26 10:44:35 -07:00
Burak Yavuz 59d87d2407 [SPARK-17650] malformed url's throw exceptions before bricking Executors
## What changes were proposed in this pull request?

When a malformed URL was sent to Executors through `sc.addJar` and `sc.addFile`, the executors become unusable, because they constantly throw `MalformedURLException`s and can never acknowledge that the file or jar is just bad input.

This PR tries to fix that problem by making sure MalformedURLs can never be submitted through `sc.addJar` and `sc.addFile`. Another solution would be to blacklist bad files and jars on Executors. Maybe fail the first time, and then ignore the second time (but print a warning message).

## How was this patch tested?

Unit tests in SparkContextSuite

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #15224 from brkyvz/SPARK-17650.
2016-09-25 22:57:31 -07:00
jisookim 90a30f4634 [SPARK-12221] add cpu time to metrics
Currently task metrics don't support executor CPU time, so there's no way to calculate how much CPU time a stage/task took from History Server metrics. This PR enables reporting CPU time.

Author: jisookim <jisookim0513@gmail.com>

Closes #10212 from jisookim0513/add-cpu-time-metric.
2016-09-23 13:43:47 -07:00
Holden Karau 90d5754212
[SPARK-16861][PYSPARK][CORE] Refactor PySpark accumulator API on top of Accumulator V2
## What changes were proposed in this pull request?

Move the internals of the PySpark accumulator API from the old deprecated API on top of the new accumulator API.

## How was this patch tested?

The existing PySpark accumulator tests (both unit tests and doc tests at the start of accumulator.py).

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

Closes #14467 from holdenk/SPARK-16861-refactor-pyspark-accumulator-api.
2016-09-23 09:44:30 +01:00
Marcelo Vanzin a4aeb7677b [SPARK-17639][BUILD] Add jce.jar to buildclasspath when building.
This was missing, preventing code that uses javax.crypto to properly
compile in Spark.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15204 from vanzin/SPARK-17639.
2016-09-22 21:35:25 -07:00
Dhruve Ashar 17b72d31e0 [SPARK-17365][CORE] Remove/Kill multiple executors together to reduce RPC call time.
## What changes were proposed in this pull request?
We are killing multiple executors together instead of iterating over expensive RPC calls to kill single executor.

## How was this patch tested?
Executed sample spark job to observe executors being killed/removed with dynamic allocation enabled.

Author: Dhruve Ashar <dashar@yahoo-inc.com>
Author: Dhruve Ashar <dhruveashar@gmail.com>

Closes #15152 from dhruve/impr/SPARK-17365.
2016-09-22 10:10:37 -07:00
Yanbo Liang c133907c5d [SPARK-17577][SPARKR][CORE] SparkR support add files to Spark job and get by executors
## What changes were proposed in this pull request?
Scala/Python users can add files to Spark job by submit options ```--files``` or ```SparkContext.addFile()```. Meanwhile, users can get the added file by ```SparkFiles.get(filename)```.
We should also support this function for SparkR users, since they also have the requirements for some shared dependency files. For example, SparkR users can download third party R packages to driver firstly, add these files to the Spark job as dependency by this API and then each executor can install these packages by ```install.packages```.

## How was this patch tested?
Add unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15131 from yanboliang/spark-17577.
2016-09-21 20:08:28 -07:00
jerryshao 8c3ee2bc42 [SPARK-17512][CORE] Avoid formatting to python path for yarn and mesos cluster mode
## What changes were proposed in this pull request?

Yarn and mesos cluster mode support remote python path (HDFS/S3 scheme) by their own mechanism, it is not necessary to check and format the python when running on these modes. This is a potential regression compared to 1.6, so here propose to fix it.

## How was this patch tested?

Unit test to verify SparkSubmit arguments, also with local cluster verification. Because of lack of `MiniDFSCluster` support in Spark unit test, there's no integration test added.

Author: jerryshao <sshao@hortonworks.com>

Closes #15137 from jerryshao/SPARK-17512.
2016-09-21 17:57:21 -04:00
Imran Rashid 9fcf1c51d5 [SPARK-17623][CORE] Clarify type of TaskEndReason with a failed task.
## What changes were proposed in this pull request?

In TaskResultGetter, enqueueFailedTask currently deserializes the result
as a TaskEndReason. But the type is actually more specific, its a
TaskFailedReason. This just leads to more blind casting later on – it
would be more clear if the msg was cast to the right type immediately,
so method parameter types could be tightened.

## How was this patch tested?

Existing unit tests via jenkins.  Note that the code was already performing a blind-cast to a TaskFailedReason before in any case, just in a different spot, so there shouldn't be any behavior change.

Author: Imran Rashid <irashid@cloudera.com>

Closes #15181 from squito/SPARK-17623.
2016-09-21 17:49:36 -04:00
Marcelo Vanzin 2cd1bfa4f0 [SPARK-4563][CORE] Allow driver to advertise a different network address.
The goal of this feature is to allow the Spark driver to run in an
isolated environment, such as a docker container, and be able to use
the host's port forwarding mechanism to be able to accept connections
from the outside world.

The change is restricted to the driver: there is no support for achieving
the same thing on executors (or the YARN AM for that matter). Those still
need full access to the outside world so that, for example, connections
can be made to an executor's block manager.

The core of the change is simple: add a new configuration that tells what's
the address the driver should bind to, which can be different than the address
it advertises to executors (spark.driver.host). Everything else is plumbing
the new configuration where it's needed.

To use the feature, the host starting the container needs to set up the
driver's port range to fall into a range that is being forwarded; this
required the block manager port to need a special configuration just for
the driver, which falls back to the existing spark.blockManager.port when
not set. This way, users can modify the driver settings without affecting
the executors; it would theoretically be nice to also have different
retry counts for driver and executors, but given that docker (at least)
allows forwarding port ranges, we can probably live without that for now.

Because of the nature of the feature it's kinda hard to add unit tests;
I just added a simple one to make sure the configuration works.

This was tested with a docker image running spark-shell with the following
command:

 docker blah blah blah \
   -p 38000-38100:38000-38100 \
   [image] \
   spark-shell \
     --num-executors 3 \
     --conf spark.shuffle.service.enabled=false \
     --conf spark.dynamicAllocation.enabled=false \
     --conf spark.driver.host=[host's address] \
     --conf spark.driver.port=38000 \
     --conf spark.driver.blockManager.port=38020 \
     --conf spark.ui.port=38040

Running on YARN; verified the driver works, executors start up and listen
on ephemeral ports (instead of using the driver's config), and that caching
and shuffling (without the shuffle service) works. Clicked through the UI
to make sure all pages (including executor thread dumps) worked. Also tested
apps without docker, and ran unit tests.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #15120 from vanzin/SPARK-4563.
2016-09-21 14:42:41 -07:00
erenavsarogullari dd7561d337
[CORE][MINOR] Add minor code change to TaskState and Task
## What changes were proposed in this pull request?
- TaskState and ExecutorState expose isFailed and isFinished functions. It can be useful to add test coverage for different states. Currently, Other enums do not expose any functions so this PR aims just these two enums.
- `private` access modifier is added for Finished Task States Set
- A minor doc change is added.

## How was this patch tested?
New Unit tests are added and run locally.

Author: erenavsarogullari <erenavsarogullari@gmail.com>

Closes #15143 from erenavsarogullari/SPARK-17584.
2016-09-21 14:47:18 +01:00
Yanbo Liang d3b8869763 [SPARK-17585][PYSPARK][CORE] PySpark SparkContext.addFile supports adding files recursively
## What changes were proposed in this pull request?
Users would like to add a directory as dependency in some cases, they can use ```SparkContext.addFile``` with argument ```recursive=true``` to recursively add all files under the directory by using Scala. But Python users can only add file not directory, we should also make it supported.

## How was this patch tested?
Unit test.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #15140 from yanboliang/spark-17585.
2016-09-21 01:37:03 -07:00
wm624@hotmail.com 61876a4279
[CORE][DOC] Fix errors in comments
## What changes were proposed in this pull request?
While reading source code of CORE and SQL core, I found some minor errors in comments such as extra space, missing blank line and grammar error.

I fixed these minor errors and might find more during my source code study.

## How was this patch tested?
Manually build

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

Closes #15151 from wangmiao1981/mem.
2016-09-21 09:33:29 +01:00
Weiqing Yang 1ea49916ac [MINOR][BUILD] Fix CheckStyle Error
## What changes were proposed in this pull request?
This PR is to fix the code style errors before 2.0.1 release.

## How was this patch tested?
Manual.

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

Author: Weiqing Yang <yangweiqing001@gmail.com>

Closes #15170 from Sherry302/fixjavastyle.
2016-09-20 21:48:25 -07:00
Shixiong Zhu 80d6655921 [SPARK-17438][WEBUI] Show Application.executorLimit in the application page
## What changes were proposed in this pull request?

This PR adds `Application.executorLimit` to the applicatino page

## How was this patch tested?

Checked the UI manually.

Screenshots:

1. Dynamic allocation is disabled

<img width="484" alt="screen shot 2016-09-07 at 4 21 49 pm" src="https://cloud.githubusercontent.com/assets/1000778/18332029/210056ea-7518-11e6-9f52-76d96046c1c0.png">

2. Dynamic allocation is enabled.

<img width="466" alt="screen shot 2016-09-07 at 4 25 30 pm" src="https://cloud.githubusercontent.com/assets/1000778/18332034/2c07700a-7518-11e6-8fce-aebe25014902.png">

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15001 from zsxwing/fix-core-info.
2016-09-19 14:00:42 -04:00
hyukjinkwon 7151011b38
[SPARK-17586][BUILD] Do not call static member via instance reference
## What changes were proposed in this pull request?

This PR fixes a warning message as below:

```
[WARNING] .../UnsafeInMemorySorter.java:284: warning: [static] static method should be qualified by type name, TaskMemoryManager, instead of by an expression
[WARNING]       currentPageNumber = memoryManager.decodePageNumber(recordPointer)
```

by referencing the static member via class not instance reference.

## How was this patch tested?

Existing tests should cover this - Jenkins tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #15141 from HyukjinKwon/SPARK-17586.
2016-09-18 19:18:49 +01:00
Josh Rosen 8faa5217b4 [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes()
## What changes were proposed in this pull request?

`MemoryStore.putIteratorAsBytes()` may silently lose values when used with `KryoSerializer` because it does not properly close the serialization stream before attempting to deserialize the already-serialized values, which may cause values buffered in Kryo's internal buffers to not be read.

This is the root cause behind a user-reported "wrong answer" bug in PySpark caching reported by bennoleslie on the Spark user mailing list in a thread titled "pyspark persist MEMORY_ONLY vs MEMORY_AND_DISK". Due to Spark 2.0's automatic use of KryoSerializer for "safe" types (such as byte arrays, primitives, etc.) this misuse of serializers manifested itself as silent data corruption rather than a StreamCorrupted error (which you might get from JavaSerializer).

The minimal fix, implemented here, is to close the serialization stream before attempting to deserialize written values. In addition, this patch adds several additional assertions / precondition checks to prevent misuse of `PartiallySerializedBlock` and `ChunkedByteBufferOutputStream`.

## How was this patch tested?

The original bug was masked by an invalid assert in the memory store test cases: the old assert compared two results record-by-record with `zip` but didn't first check that the lengths of the two collections were equal, causing missing records to go unnoticed. The updated test case reproduced this bug.

In addition, I added a new `PartiallySerializedBlockSuite` to unit test that component.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15043 from JoshRosen/partially-serialized-block-values-iterator-bugfix.
2016-09-17 11:46:15 -07:00
David Navas 9dbd4b864e
[SPARK-17529][CORE] Implement BitSet.clearUntil and use it during merge joins
## What changes were proposed in this pull request?

Add a clearUntil() method on BitSet (adapted from the pre-existing setUntil() method).
Use this method to clear the subset of the BitSet which needs to be used during merge joins.

## How was this patch tested?

dev/run-tests, as well as performance tests on skewed data as described in jira.

I expect there to be a small local performance hit using BitSet.clearUntil rather than BitSet.clear for normally shaped (unskewed) joins (additional read on the last long).  This is expected to be de-minimis and was not specifically tested.

Author: David Navas <davidn@clearstorydata.com>

Closes #15084 from davidnavas/bitSet.
2016-09-17 16:22:23 +01:00
Xin Ren f15d41be3c
[SPARK-17567][DOCS] Use valid url to Spark RDD paper
https://issues.apache.org/jira/browse/SPARK-17567

## What changes were proposed in this pull request?

Documentation (http://spark.apache.org/docs/latest/api/scala/#org.apache.spark.rdd.RDD) contains broken link to Spark paper (http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf).

I found it elsewhere (https://www.usenix.org/system/files/conference/nsdi12/nsdi12-final138.pdf) and I hope it is the same one. It should be uploaded to and linked from some Apache controlled storage, so it won't break again.

## How was this patch tested?

Tested manually on local laptop.

Author: Xin Ren <iamshrek@126.com>

Closes #15121 from keypointt/SPARK-17567.
2016-09-17 12:30:25 +01:00
Josh Rosen 1202075c95 [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions
## What changes were proposed in this pull request?

If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed.

This patch addresses this issue via multiple small changes:

- The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup).
- When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present.
- Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls.

This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix.

For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484).

## How was this patch tested?

Two new regression tests in BlockManagerSuite.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15085 from JoshRosen/SPARK-17484.
2016-09-15 11:54:17 -07:00
Josh Rosen 5b8f7377d5 [SPARK-17547] Ensure temp shuffle data file is cleaned up after error
SPARK-8029 (#9610) modified shuffle writers to first stage their data to a temporary file in the same directory as the final destination file and then to atomically rename this temporary file at the end of the write job. However, this change introduced the potential for the temporary output file to be leaked if an exception occurs during the write because the shuffle writers' existing error cleanup code doesn't handle deletion of the temp file.

This patch avoids this potential cause of disk-space leaks by adding `finally` blocks to ensure that temp files are always deleted if they haven't been renamed.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer.
2016-09-15 11:22:58 -07:00
Tejas Patil b479278142 [SPARK-17451][CORE] CoarseGrainedExecutorBackend should inform driver before self-kill
## What changes were proposed in this pull request?

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

`CoarseGrainedExecutorBackend` in some failure cases exits the JVM. While this does not have any issue, from the driver UI there is no specific reason captured for this. In this PR, I am adding functionality to `exitExecutor` to notify driver that the executor is exiting.

## How was this patch tested?

Ran the change over a test env and took down shuffle service before the executor could register to it. In the driver logs, where the job failure reason is mentioned (ie. `Job aborted due to stage ...` it gives the correct reason:

Before:
`ExecutorLostFailure (executor ZZZZZZZZZ exited caused by one of the running tasks) Reason: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.`

After:
`ExecutorLostFailure (executor ZZZZZZZZZ exited caused by one of the running tasks) Reason: Unable to create executor due to java.util.concurrent.TimeoutException: Timeout waiting for task.`

Author: Tejas Patil <tejasp@fb.com>

Closes #15013 from tejasapatil/SPARK-17451_inform_driver.
2016-09-15 10:23:41 -07:00
cenyuhai ad79fc0a84 [SPARK-17406][WEB UI] limit timeline executor events
## What changes were proposed in this pull request?
The job page will be too slow to open when there are thousands of executor events(added or removed). I found that in ExecutorsTab file, executorIdToData will not remove elements, it will increase all the time.Before this pr, it looks like [timeline1.png](https://issues.apache.org/jira/secure/attachment/12827112/timeline1.png). After this pr, it looks like [timeline2.png](https://issues.apache.org/jira/secure/attachment/12827113/timeline2.png)(we can set how many executor events will be displayed)

Author: cenyuhai <cenyuhai@didichuxing.com>

Closes #14969 from cenyuhai/SPARK-17406.
2016-09-15 09:58:53 +01:00
codlife 647ee05e58 [SPARK-17521] Error when I use sparkContext.makeRDD(Seq())
## What changes were proposed in this pull request?

 when i use sc.makeRDD below
```
val data3 = sc.makeRDD(Seq())
println(data3.partitions.length)
```
I got an error:
Exception in thread "main" java.lang.IllegalArgumentException: Positive number of slices required

We can fix this bug just modify the last line ,do a check of seq.size
```
  def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope {
    assertNotStopped()
    val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap
    new ParallelCollectionRDD[T](this, seq.map(_._1), math.max(seq.size, defaultParallelism), indexToPrefs)
  }
```

## How was this patch tested?

 manual tests

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

Author: codlife <1004910847@qq.com>
Author: codlife <wangjianfei15@otcaix.iscas.ac.cn>

Closes #15077 from codlife/master.
2016-09-15 09:38:13 +01:00
Xing SHI bb32294362 [SPARK-17465][SPARK CORE] Inappropriate memory management in org.apache.spark.storage.MemoryStore may lead to memory leak
The expression like `if (memoryMap(taskAttemptId) == 0) memoryMap.remove(taskAttemptId)` in method `releaseUnrollMemoryForThisTask` and `releasePendingUnrollMemoryForThisTask` should be called after release memory operation, whatever `memoryToRelease` is > 0 or not.

If the memory of a task has been set to 0 when calling a `releaseUnrollMemoryForThisTask` or a `releasePendingUnrollMemoryForThisTask` method, the key in the memory map corresponding to that task will never be removed from the hash map.

See the details in [SPARK-17465](https://issues.apache.org/jira/browse/SPARK-17465).

Author: Xing SHI <shi-kou@indetail.co.jp>

Closes #15022 from saturday-shi/SPARK-17465.
2016-09-14 13:59:57 -07:00
Shixiong Zhu e33bfaed3b [SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's value can be read thread-safely
## What changes were proposed in this pull request?

Make CollectionAccumulator and SetAccumulator's value can be read thread-safely to fix the ConcurrentModificationException reported in [JIRA](https://issues.apache.org/jira/browse/SPARK-17463).

## How was this patch tested?

Existing tests.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #15063 from zsxwing/SPARK-17463.
2016-09-14 13:33:51 -07:00
Xin Wu 040e46979d [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BY
## What changes were proposed in this pull request?
Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values.
However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC).

This PR is to support this new feature.

## How was this patch tested?
New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries.

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

Author: Xin Wu <xinwu@us.ibm.com>

Closes #14842 from xwu0226/SPARK-10747.
2016-09-14 21:14:29 +02:00
wm624@hotmail.com 18b4f035f4 [CORE][DOC] remove redundant comment
## What changes were proposed in this pull request?
In the comment, there is redundant `the estimated`.

This PR simply remove the redundant comment and adjusts format.

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

Closes #15091 from wangmiao1981/comment.
2016-09-14 09:49:15 +01:00
Jagadeesan def7c265f5 [SPARK-17449][DOCUMENTATION] Relation between heartbeatInterval and…
## What changes were proposed in this pull request?

The relation between spark.network.timeout and spark.executor.heartbeatInterval should be mentioned in the document.

… network timeout]

Author: Jagadeesan <as2@us.ibm.com>

Closes #15042 from jagadeesanas2/SPARK-17449.
2016-09-14 09:03:16 +01:00
Josh Rosen f9c580f110 [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job
## What changes were proposed in this pull request?

In Spark's `RDD.getOrCompute` we first try to read a local copy of a cached RDD block, then a remote copy, and only fall back to recomputing the block if no cached copy (local or remote) can be read. This logic works correctly in the case where no remote copies of the block exist, but if there _are_ remote copies and reads of those copies fail (due to network issues or internal Spark bugs) then the BlockManager will throw a `BlockFetchException` that will fail the task (and which could possibly fail the whole job if the read failures keep occurring).

In the cases of TorrentBroadcast and task result fetching we really do want to fail the entire job in case no remote blocks can be fetched, but this logic is inappropriate for reads of cached RDD blocks because those can/should be recomputed in case cached blocks are unavailable.

Therefore, I think that the `BlockManager.getRemoteBytes()` method should never throw on remote fetch errors and, instead, should handle failures by returning `None`.

## How was this patch tested?

Block manager changes should be covered by modified tests in `BlockManagerSuite`: the old tests expected exceptions to be thrown on failed remote reads, while the modified tests now expect `None` to be returned from the `getRemote*` method.

I also manually inspected all usages of `BlockManager.getRemoteValues()`, `getRemoteBytes()`, and `get()` to verify that they correctly pattern-match on the result and handle `None`. Note that these `None` branches are already exercised because the old `getRemoteBytes` returned `None` when no remote locations for the block could be found (which could occur if an executor died and its block manager de-registered with the master).

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15037 from JoshRosen/SPARK-17485.
2016-09-12 15:43:57 -07:00
Josh Rosen 3d40896f41 [SPARK-17483] Refactoring in BlockManager status reporting and block removal
This patch makes three minor refactorings to the BlockManager:

- Move the `if (info.tellMaster)` check out of `reportBlockStatus`; this fixes an issue where a debug logging message would incorrectly claim to have reported a block status to the master even though no message had been sent (in case `info.tellMaster == false`). This also makes it easier to write code which unconditionally sends block statuses to the master (which is necessary in another patch of mine).
- Split  `removeBlock()` into two methods, the existing method and an internal `removeBlockInternal()` method which is designed to be called by internal code that already holds a write lock on the block. This is also needed by a followup patch.
- Instead of calling `getCurrentBlockStatus()` in `removeBlock()`, just pass `BlockStatus.empty`; the block status should always be empty following complete removal of a block.

These changes were originally authored as part of a bug fix patch which is targeted at branch-2.0 and master; I've split them out here into their own separate PR in order to make them easier to review and so that the behavior-changing parts of my other patch can be isolated to their own PR.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #15036 from JoshRosen/cache-failure-race-conditions-refactorings-only.
2016-09-12 13:09:33 -07:00
Sean Zhong 1742c3ab86 [SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the whole RDD in memory
## What changes were proposed in this pull request?

   MemoryStore may throw OutOfMemoryError when trying to cache a super big RDD that cannot fit in memory.
   ```
   scala> sc.parallelize(1 to 1000000000, 100).map(x => new Array[Long](1000)).cache().count()

   java.lang.OutOfMemoryError: Java heap space
	at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:24)
	at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(<console>:23)
	at scala.collection.Iterator$$anon$11.next(Iterator.scala:409)
	at scala.collection.Iterator$JoinIterator.next(Iterator.scala:232)
	at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:683)
	at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43)
	at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1684)
	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
	at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
	at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915)
	at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70)
	at org.apache.spark.scheduler.Task.run(Task.scala:86)
	at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274)
	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)
   ```

Spark MemoryStore uses SizeTrackingVector as a temporary unrolling buffer to store all input values that it has read so far before transferring the values to storage memory cache. The problem is that when the input RDD is too big for caching in memory, the temporary unrolling memory SizeTrackingVector is not garbage collected in time. As SizeTrackingVector can occupy all available storage memory, it may cause the executor JVM to run out of memory quickly.

More info can be found at https://issues.apache.org/jira/browse/SPARK-17503

## How was this patch tested?

Unit test and manual test.

### Before change

Heap memory consumption
<img width="702" alt="screen shot 2016-09-12 at 4 16 15 pm" src="https://cloud.githubusercontent.com/assets/2595532/18429524/60d73a26-7906-11e6-9768-6f286f5c58c8.png">

Heap dump
<img width="1402" alt="screen shot 2016-09-12 at 4 34 19 pm" src="https://cloud.githubusercontent.com/assets/2595532/18429577/cbc1ef20-7906-11e6-847b-b5903f450b3b.png">

### After change

Heap memory consumption
<img width="706" alt="screen shot 2016-09-12 at 4 29 10 pm" src="https://cloud.githubusercontent.com/assets/2595532/18429503/4abe9342-7906-11e6-844a-b2f815072624.png">

Author: Sean Zhong <seanzhong@databricks.com>

Closes #15056 from clockfly/memory_store_leak.
2016-09-12 11:30:06 -07:00
WeichenXu 8087ecf8da [SPARK CORE][MINOR] fix "default partitioner cannot partition array keys" error message in PairRDDfunctions
## What changes were proposed in this pull request?

In order to avoid confusing user,
error message in `PairRDDfunctions`
`Default partitioner cannot partition array keys.`
is updated,
the one in `partitionBy` is replaced with
`Specified partitioner cannot partition array keys.`
other is replaced with
`Specified or default partitioner cannot partition array keys.`

## How was this patch tested?

N/A

Author: WeichenXu <WeichenXu123@outlook.com>

Closes #15045 from WeichenXu123/fix_partitionBy_error_message.
2016-09-12 12:23:16 +01:00