Commit graph

6244 commits

Author SHA1 Message Date
Michael Mior 1437e344ec [SPARK-22050][CORE] Allow BlockUpdated events to be optionally logged to the event log
## What changes were proposed in this pull request?

I see that block updates are not logged to the event log.
This makes sense as a default for performance reasons.
However, I find it helpful when trying to get a better understanding of caching for a job to be able to log these updates.
This PR adds a configuration setting `spark.eventLog.blockUpdates` (defaulting to false) which allows block updates to be recorded in the log.
This contribution is original work which is licensed to the Apache Spark project.

## How was this patch tested?

Current and additional unit tests.

Author: Michael Mior <mmior@uwaterloo.ca>

Closes #19263 from michaelmior/log-block-updates.
2017-10-17 14:30:52 -07:00
jerryshao e1960c3d6f [SPARK-22062][CORE] Spill large block to disk in BlockManager's remote fetch to avoid OOM
## What changes were proposed in this pull request?

In the current BlockManager's `getRemoteBytes`, it will call `BlockTransferService#fetchBlockSync` to get remote block. In the `fetchBlockSync`, Spark will allocate a temporary `ByteBuffer` to store the whole fetched block. This will potentially lead to OOM if block size is too big or several blocks are fetched simultaneously in this executor.

So here leveraging the idea of shuffle fetch, to spill the large block to local disk before consumed by upstream code. The behavior is controlled by newly added configuration, if block size is smaller than the threshold, then this block will be persisted in memory; otherwise it will first spill to disk, and then read from disk file.

To achieve this feature, what I did is:

1. Rename `TempShuffleFileManager` to `TempFileManager`, since now it is not only used by shuffle.
2. Add a new `TempFileManager` to manage the files of fetched remote blocks, the files are tracked by weak reference, will be deleted when no use at all.

## How was this patch tested?

This was tested by adding UT, also manual verification in local test to perform GC to clean the files.

Author: jerryshao <sshao@hortonworks.com>

Closes #19476 from jerryshao/SPARK-22062.
2017-10-17 22:54:38 +08:00
Xingbo Jiang 0fa10666cf [SPARK-22233][CORE][FOLLOW-UP] Allow user to filter out empty split in HadoopRDD
## What changes were proposed in this pull request?

Update the config `spark.files.ignoreEmptySplits`, rename it and make it internal.

This is followup of #19464

## How was this patch tested?

Exsiting tests.

Author: Xingbo Jiang <xingbo.jiang@databricks.com>

Closes #19504 from jiangxb1987/partitionsplit.
2017-10-16 22:15:50 +08:00
Mridul Muralidharan 13c1559587 [SPARK-21549][CORE] Respect OutputFormats with no/invalid output directory provided
## What changes were proposed in this pull request?

PR #19294 added support for null's - but spark 2.1 handled other error cases where path argument can be invalid.
Namely:

* empty string
* URI parse exception while creating Path

This is resubmission of PR #19487, which I messed up while updating my repo.

## How was this patch tested?

Enhanced test to cover new support added.

Author: Mridul Muralidharan <mridul@gmail.com>

Closes #19497 from mridulm/master.
2017-10-15 18:40:53 -07:00
liulijia 014dc84712 [SPARK-22233][CORE] Allow user to filter out empty split in HadoopRDD
## What changes were proposed in this pull request?
Add a flag spark.files.ignoreEmptySplits. When true, methods like that use HadoopRDD and NewHadoopRDD such as SparkContext.textFiles will not create a partition for input splits that are empty.

Author: liulijia <liulijia@meituan.com>

Closes #19464 from liutang123/SPARK-22233.
2017-10-14 17:37:33 +09:00
Xianyang Liu b5c1ef7a8e [SPARK-22097][CORE] Request an accurate memory after we unrolled the block
## What changes were proposed in this pull request?

We only need request `bbos.size - unrollMemoryUsedByThisBlock` after unrolled the block.

## How was this patch tested?

Existing UT.

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #19316 from ConeyLiu/putIteratorAsBytes.
2017-10-12 20:26:51 +08:00
Marcelo Vanzin bd4eb9ce57 [SPARK-19558][SQL] Add config key to register QueryExecutionListeners automatically.
This change adds a new SQL config key that is equivalent to SparkContext's
"spark.extraListeners", allowing users to register QueryExecutionListener
instances through the Spark configuration system instead of having to
explicitly do it in code.

The code used by SparkContext to implement the feature was refactored into
a helper method in the Utils class, and SQL's ExecutionListenerManager was
modified to use it to initialize listener declared in the configuration.

Unit tests were added to verify all the new functionality.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19309 from vanzin/SPARK-19558.
2017-10-10 15:50:37 -07:00
Eyal Farago 2028e5a82b [SPARK-21907][CORE] oom during spill
## What changes were proposed in this pull request?
1. a test reproducing [SPARK-21907](https://issues.apache.org/jira/browse/SPARK-21907)
2. a fix for the root cause of the issue.

`org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill` calls `org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset` which may trigger another spill,
when this happens the `array` member is already de-allocated but still referenced by the code, this causes the nested spill to fail with an NPE in `org.apache.spark.memory.TaskMemoryManager.getPage`.
This patch introduces a reproduction in a test case and a fix, the fix simply sets the in-mem sorter's array member to an empty array before actually performing the allocation. This prevents the spilling code from 'touching' the de-allocated array.

## How was this patch tested?
introduced a new test case: `org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorterSuite#testOOMDuringSpill`.

Author: Eyal Farago <eyal@nrgene.com>

Closes #19181 from eyalfa/SPARK-21907__oom_during_spill.
2017-10-10 22:49:47 +02:00
Prashant Sharma 23af2d79ad [SPARK-20025][CORE] Ignore SPARK_LOCAL* env, while deploying via cluster mode.
## What changes were proposed in this pull request?

In a bare metal system with No DNS setup, spark may be configured with SPARK_LOCAL* for IP and host properties.
During a driver failover, in cluster deployment mode. SPARK_LOCAL* should be ignored while restarting on another node and should be picked up from target system's local environment.

## How was this patch tested?
Distributed deployment against a spark standalone cluster of  6 Workers. Tested by killing JVM's running driver and verified the restarted JVMs have right configurations on them.

Author: Prashant Sharma <prashsh1@in.ibm.com>
Author: Prashant Sharma <prashant@apache.org>

Closes #17357 from ScrapCodes/driver-failover-fix.
2017-10-10 20:48:42 +08:00
liuxian b8a08f25cc [SPARK-21506][DOC] The description of "spark.executor.cores" may be not correct
## What changes were proposed in this pull request?

The number of cores assigned to each executor is configurable. When this is not explicitly set,  multiple executors from the same application may be launched on the same worker too.

## How was this patch tested?
N/A

Author: liuxian <liu.xian3@zte.com.cn>

Closes #18711 from 10110346/executorcores.
2017-10-10 20:44:33 +08:00
Feng Liu bebd2e1ce1 [SPARK-22222][CORE] Fix the ARRAY_MAX in BufferHolder and add a test
## What changes were proposed in this pull request?

We should not break the assumption that the length of the allocated byte array is word rounded:
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java#L170
So we want to use `Integer.MAX_VALUE - 15` instead of `Integer.MAX_VALUE - 8` as the upper bound of an allocated byte array.

cc: srowen gatorsmile
## How was this patch tested?

Since the Spark unit test JVM has less than 1GB heap, here we run the test code as a submit job, so it can run on a JVM has 4GB memory.

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

Author: Feng Liu <fengliu@databricks.com>

Closes #19460 from liufengdb/fix_array_max.
2017-10-09 21:34:37 -07:00
Dongjoon Hyun f31e11404d [SPARK-21568][CORE] ConsoleProgressBar should only be enabled in shells
## What changes were proposed in this pull request?

This PR disables console progress bar feature in non-shell environment by overriding the configuration.

## How was this patch tested?

Manual. Run the following examples with and without `spark.ui.showConsoleProgress` in order to see progress bar on master branch and this PR.

**Scala Shell**
```scala
spark.range(1000000000).map(_ + 1).count
```

**PySpark**
```python
spark.range(10000000).rdd.map(lambda x: len(x)).count()
```

**Spark Submit**
```python
from pyspark.sql import SparkSession

if __name__ == "__main__":
    spark = SparkSession.builder.getOrCreate()
    spark.range(2000000).rdd.map(lambda row: len(row)).count()
    spark.stop()
```

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19061 from dongjoon-hyun/SPARK-21568.
2017-10-09 12:53:10 -07:00
Yuanjian Li fe7b219ae3 [SPARK-22074][CORE] Task killed by other attempt task should not be resubmitted
## What changes were proposed in this pull request?

As the detail scenario described in [SPARK-22074](https://issues.apache.org/jira/browse/SPARK-22074), unnecessary resubmitted may cause stage hanging in currently release versions. This patch add a new var in TaskInfo to mark this task killed by other attempt or not.

## How was this patch tested?

Add a new UT `[SPARK-22074] Task killed by other attempt task should not be resubmitted` in TaskSetManagerSuite, this UT recreate the scenario in JIRA description, it failed without the changes in this PR and passed conversely.

Author: Yuanjian Li <xyliyuanjian@gmail.com>

Closes #19287 from xuanyuanking/SPARK-22074.
2017-10-09 14:16:25 +08:00
Sergei Lebedev c998a2ae0e [SPARK-22147][CORE] Removed redundant allocations from BlockId
## What changes were proposed in this pull request?

Prior to this commit BlockId.hashCode and BlockId.equals were defined
in terms of BlockId.name. This allowed the subclasses to be concise and
enforced BlockId.name as a single unique identifier for a block. All
subclasses override BlockId.name with an expression involving an
allocation of StringBuilder and ultimatelly String. This is suboptimal
since it induced unnecessary GC pressure on the dirver, see
BlockManagerMasterEndpoint.

The commit removes the definition of hashCode and equals from the base
class. No other change is necessary since all subclasses are in fact
case classes and therefore have auto-generated hashCode and equals. No
change of behaviour is expected.

Sidenote: you might be wondering, why did the subclasses use the base
implementation and the auto-generated one? Apparently, this behaviour
is documented in the spec. See this SO answer for details
https://stackoverflow.com/a/44990210/262432.

## How was this patch tested?

BlockIdSuite

Author: Sergei Lebedev <s.lebedev@criteo.com>

Closes #19369 from superbobry/blockid-equals-hashcode.
2017-10-08 12:58:39 +01:00
Sergey Zhemzhitsky 2030f19511 [SPARK-21549][CORE] Respect OutputFormats with no output directory provided
## What changes were proposed in this pull request?

Fix for https://issues.apache.org/jira/browse/SPARK-21549 JIRA issue.

Since version 2.2 Spark does not respect OutputFormat with no output paths provided.
The examples of such formats are [Cassandra OutputFormat](08dfa3a7ac/src/main/java/org/apache/cassandra/hadoop2/AbstractColumnFamilyOutputFormat.java), [Aerospike OutputFormat](https://github.com/aerospike/aerospike-hadoop/blob/master/mapreduce/src/main/java/com/aerospike/hadoop/mapreduce/AerospikeOutputFormat.java), etc. which do not have an ability to rollback the results written to an external systems on job failure.

Provided output directory is required by Spark to allows files to be committed to an absolute output location, that is not the case for output formats which write data to external systems.

This pull request prevents accessing `absPathStagingDir` method that causes the error described in SPARK-21549 unless there are files to rename in `addedAbsPathFiles`.

## How was this patch tested?

Unit tests

Author: Sergey Zhemzhitsky <szhemzhitski@gmail.com>

Closes #19294 from szhem/SPARK-21549-abs-output-commits.
2017-10-06 20:43:53 -07:00
Sahil Takiar e36ec38d89 [SPARK-20466][CORE] HadoopRDD#addLocalConfiguration throws NPE
## What changes were proposed in this pull request?

Fix for SPARK-20466, full description of the issue in the JIRA. To summarize, `HadoopRDD` uses a metadata cache to cache `JobConf` objects. The cache uses soft-references, which means the JVM can delete entries from the cache whenever there is GC pressure. `HadoopRDD#getJobConf` had a bug where it would check if the cache contained the `JobConf`, if it did it would get the `JobConf` from the cache and return it. This doesn't work when soft-references are used as the JVM can delete the entry between the existence check and the get call.

## How was this patch tested?

Haven't thought of a good way to test this yet given the issue only occurs sometimes, and happens during high GC pressure. Was thinking of using mocks to verify `#getJobConf` is doing the right thing. I deleted the method `HadoopRDD#containsCachedMetadata` so that we don't hit this issue again.

Author: Sahil Takiar <stakiar@cloudera.com>

Closes #19413 from sahilTakiar/master.
2017-10-03 16:53:32 -07:00
Gene Pang 4329eb2e73 [SPARK-16944][Mesos] Improve data locality when launching new executors when dynamic allocation is enabled
## What changes were proposed in this pull request?

Improve the Spark-Mesos coarse-grained scheduler to consider the preferred locations when dynamic allocation is enabled.

## How was this patch tested?

Added a unittest, and performed manual testing on AWS.

Author: Gene Pang <gene.pang@gmail.com>

Closes #18098 from gpang/mesos_data_locality.
2017-10-02 15:09:11 -07:00
guoxiaolong 405c0e99e7 [SPARK-22173][WEB-UI] Table CSS style needs to be adjusted in History Page and in Executors Page.
## What changes were proposed in this pull request?

There is a problem with table CSS style.

1. At present, table CSS style is too crowded, and the table width cannot adapt itself.

2. Table CSS style is different from job page, stage page, task page, master page, worker page, etc. The Spark web UI needs to be consistent.

fix before:
![01](https://user-images.githubusercontent.com/26266482/31041261-c6766c3a-a5c4-11e7-97a7-96bd51ef12bd.png)

![02](https://user-images.githubusercontent.com/26266482/31041266-d75b6a32-a5c4-11e7-8071-e3bbbba39b80.png)

----------------------------------------------------------------------------------------------------------

fix after:
![1](https://user-images.githubusercontent.com/26266482/31041162-808a5a3e-a5c3-11e7-8d92-d763b500ce53.png)

![2](https://user-images.githubusercontent.com/26266482/31041166-86e583e0-a5c3-11e7-949c-11c370db9e27.png)

## How was this patch tested?

manual tests

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

Author: guoxiaolong <guo.xiaolong1@zte.com.cn>

Closes #19397 from guoxiaolongzte/SPARK-22173.
2017-10-02 08:07:56 +01:00
Reynold Xin 323806e68f [SPARK-22160][SQL] Make sample points per partition (in range partitioner) configurable and bump the default value up to 100
## What changes were proposed in this pull request?
Spark's RangePartitioner hard codes the number of sampling points per partition to be 20. This is sometimes too low. This ticket makes it configurable, via spark.sql.execution.rangeExchange.sampleSizePerPartition, and raises the default in Spark SQL to be 100.

## How was this patch tested?
Added a pretty sophisticated test based on chi square test ...

Author: Reynold Xin <rxin@databricks.com>

Closes #19387 from rxin/SPARK-22160.
2017-09-28 21:07:12 -07:00
zhoukang 3b117d631e [SPARK-22123][CORE] Add latest failure reason for task set blacklist
## What changes were proposed in this pull request?
This patch add latest failure reason for task set blacklist.Which can be showed on spark ui and let user know failure reason directly.
Till now , every job which aborted by completed blacklist just show log like below which has no more information:
`Aborting $taskSet because task $indexInTaskSet (partition $partition) cannot run anywhere due to node and executor blacklist.  Blacklisting behavior cannot run anywhere due to node and executor blacklist.Blacklisting behavior can be configured via spark.blacklist.*."`
**After modify:**
```
Aborting TaskSet 0.0 because task 0 (partition 0)
cannot run anywhere due to node and executor blacklist.
Most recent failure:
Some(Lost task 0.1 in stage 0.0 (TID 3,xxx, executor 1): java.lang.Exception: Fake error!
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:73)
at org.apache.spark.scheduler.Task.run(Task.scala:99)
at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:305)
at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
at java.lang.Thread.run(Thread.java:745)
).

Blacklisting behavior can be configured via spark.blacklist.*.

```

## How was this patch tested?

Unit test and manually test.

Author: zhoukang <zhoukang199191@gmail.com>

Closes #19338 from caneGuy/zhoukang/improve-blacklist.
2017-09-28 09:25:21 +08:00
Sean Owen 9b98aef6a3 [HOTFIX][BUILD] Fix finalizer checkstyle error and re-disable checkstyle
## What changes were proposed in this pull request?

Fix finalizer checkstyle violation by just turning it off; re-disable checkstyle as it won't be run by SBT PR builder. See https://github.com/apache/spark/pull/18887#issuecomment-332580700

## How was this patch tested?

`./dev/lint-java` runs successfully

Author: Sean Owen <sowen@cloudera.com>

Closes #19371 from srowen/HotfixFinalizerCheckstlye.
2017-09-27 13:40:21 -07:00
Takuya UESHIN 09cbf3df20 [SPARK-22125][PYSPARK][SQL] Enable Arrow Stream format for vectorized UDF.
## What changes were proposed in this pull request?

Currently we use Arrow File format to communicate with Python worker when invoking vectorized UDF but we can use Arrow Stream format.

This pr replaces the Arrow File format with the Arrow Stream format.

## How was this patch tested?

Existing tests.

Author: Takuya UESHIN <ueshin@databricks.com>

Closes #19349 from ueshin/issues/SPARK-22125.
2017-09-27 23:21:44 +09:00
Marcelo Vanzin 74daf622de [SPARK-20642][CORE] Store FsHistoryProvider listing data in a KVStore.
The application listing is still generated from event logs, but is now stored
in a KVStore instance. By default an in-memory store is used, but a new config
allows setting a local disk path to store the data, in which case a LevelDB
store will be created.

The provider stores things internally using the public REST API types; I believe
this is better going forward since it will make it easier to get rid of the
internal history server API which is mostly redundant at this point.

I also added a finalizer to LevelDBIterator, to make sure that resources are
eventually released. This helps when code iterates but does not exhaust the
iterator, thus not triggering the auto-close code.

HistoryServerSuite was modified to not re-start the history server unnecessarily;
this makes the json validation tests run more quickly.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18887 from vanzin/SPARK-20642.
2017-09-27 20:33:41 +08:00
Imran Rashid 2c5b9b1173 [SPARK-22083][CORE] Release locks in MemoryStore.evictBlocksToFreeSpace
## What changes were proposed in this pull request?

MemoryStore.evictBlocksToFreeSpace acquires write locks for all the
blocks it intends to evict up front.  If there is a failure to evict
blocks (eg., some failure dropping a block to disk), then we have to
release the lock.  Otherwise the lock is never released and an executor
trying to get the lock will wait forever.

## How was this patch tested?

Added unit test.

Author: Imran Rashid <irashid@cloudera.com>

Closes #19311 from squito/SPARK-22083.
2017-09-25 12:02:30 -07:00
Sean Owen 576c43fb42 [SPARK-22087][SPARK-14650][WIP][BUILD][REPL][CORE] Compile Spark REPL for Scala 2.12 + other 2.12 fixes
## What changes were proposed in this pull request?

Enable Scala 2.12 REPL. Fix most remaining issues with 2.12 compilation and warnings, including:

- Selecting Kafka 0.10.1+ for Scala 2.12 and patching over a minor API difference
- Fixing lots of "eta expansion of zero arg method deprecated" warnings
- Resolving the SparkContext.sequenceFile implicits compile problem
- Fixing an odd but valid jetty-server missing dependency in hive-thriftserver

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19307 from srowen/Scala212.
2017-09-24 09:40:13 +01:00
zuotingbing 4943ea5984 [SPARK-22058][CORE] the BufferedInputStream will not be closed if an exception occurs.
## What changes were proposed in this pull request?

EventLoggingListener use `val in = new BufferedInputStream(fs.open(log))` and will close it if `codec.map(_.compressedInputStream(in)).getOrElse(in)`  occurs an exception .
But, if `CompressionCodec.createCodec(new SparkConf, c)` throws an exception, the BufferedInputStream `in` will not be closed anymore.

## How was this patch tested?

exist tests

Author: zuotingbing <zuo.tingbing9@zte.com.cn>

Closes #19277 from zuotingbing/SPARK-22058.
2017-09-24 09:38:46 +01:00
hyukjinkwon 9d48bd0b34 [SPARK-22093][TESTS] Fixes assume in UtilsSuite and HiveDDLSuite
## What changes were proposed in this pull request?

This PR proposes to remove `assume` in `Utils.resolveURIs` and replace `assume` to `assert` in `Utils.resolveURI` in the test cases in `UtilsSuite`.

It looks `Utils.resolveURIs` supports multiple but also single paths as input. So, it looks not meaningful to check if the input has `,`.

For the test for `Utils.resolveURI`, I replaced it to `assert` because it looks taking single path and in order to prevent future mistakes when adding more tests here.

For `assume` in `HiveDDLSuite`, it looks it should be `assert` to test at the last
## How was this patch tested?

Fixed unit tests.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19332 from HyukjinKwon/SPARK-22093.
2017-09-24 17:11:29 +09:00
Sean Owen 50ada2a4d3 [SPARK-22033][CORE] BufferHolder, other size checks should account for the specific VM array size limitations
## What changes were proposed in this pull request?

Try to avoid allocating an array bigger than Integer.MAX_VALUE - 8, which is the actual max size on some JVMs, in several places

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19266 from srowen/SPARK-22033.
2017-09-23 15:40:59 +01:00
Bryan Cutler 27fc536d9a [SPARK-21190][PYSPARK] Python Vectorized UDFs
This PR adds vectorized UDFs to the Python API

**Proposed API**
Introduce a flag to turn on vectorization for a defined UDF, for example:

```
pandas_udf(DoubleType())
def plus(a, b)
    return a + b
```
or

```
plus = pandas_udf(lambda a, b: a + b, DoubleType())
```
Usage is the same as normal UDFs

0-parameter UDFs
pandas_udf functions can declare an optional `**kwargs` and when evaluated, will contain a key "size" that will give the required length of the output.  For example:

```
pandas_udf(LongType())
def f0(**kwargs):
    return pd.Series(1).repeat(kwargs["size"])

df.select(f0())
```

Added new unit tests in pyspark.sql that are enabled if pyarrow and Pandas are available.

- [x] Fix support for promoted types with null values
- [ ] Discuss 0-param UDF API (use of kwargs)
- [x] Add tests for chained UDFs
- [ ] Discuss behavior when pyarrow not installed / enabled
- [ ] Cleanup pydoc and add user docs

Author: Bryan Cutler <cutlerb@gmail.com>
Author: Takuya UESHIN <ueshin@databricks.com>

Closes #18659 from BryanCutler/arrow-vectorized-udfs-SPARK-21404.
2017-09-22 16:17:50 +08:00
Imran Rashid b75bd17774 [SPARK-21928][CORE] Set classloader on SerializerManager's private kryo
## What changes were proposed in this pull request?

We have to make sure that SerializerManager's private instance of
kryo also uses the right classloader, regardless of the current thread
classloader.  In particular, this fixes serde during remote cache
fetches, as those occur in netty threads.

## How was this patch tested?

Manual tests & existing suite via jenkins.  I haven't been able to reproduce this is in a unit test, because when a remote RDD partition can be fetched, there is a warning message and then the partition is just recomputed locally.  I manually verified the warning message is no longer present.

Author: Imran Rashid <irashid@cloudera.com>

Closes #19280 from squito/SPARK-21928_ser_classloader.
2017-09-21 10:20:19 -07:00
jerryshao 1da5822e6a [SPARK-21934][CORE] Expose Shuffle Netty memory usage to MetricsSystem
## What changes were proposed in this pull request?

This is a followup work of SPARK-9104 to expose the Netty memory usage to MetricsSystem. Current the shuffle Netty memory usage of `NettyBlockTransferService` will be exposed, if using external shuffle, then the Netty memory usage of `ExternalShuffleClient` and `ExternalShuffleService` will be exposed instead. Currently I don't expose Netty memory usage of `YarnShuffleService`, because `YarnShuffleService` doesn't have `MetricsSystem` itself, and is better to connect to Hadoop's MetricsSystem.

## How was this patch tested?

Manually verified in local cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #19160 from jerryshao/SPARK-21934.
2017-09-21 13:54:30 +08:00
Sean Owen 3d4dd14cd5 [SPARK-22066][BUILD] Update checkstyle to 8.2, enable it, fix violations
## What changes were proposed in this pull request?

Update plugins, including scala-maven-plugin, to latest versions. Update checkstyle to 8.2. Remove bogus checkstyle config and enable it. Fix existing and new Java checkstyle errors.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19282 from srowen/SPARK-22066.
2017-09-20 10:01:46 +01:00
Marcelo Vanzin c6ff59a230 [SPARK-18838][CORE] Add separate listener queues to LiveListenerBus.
This change modifies the live listener bus so that all listeners are
added to queues; each queue has its own thread to dispatch events,
making it possible to separate slow listeners from other more
performance-sensitive ones.

The public API has not changed - all listeners added with the existing
"addListener" method, which after this change mostly means all
user-defined listeners, end up in a default queue. Internally, there's
an API allowing listeners to be added to specific queues, and that API
is used to separate the internal Spark listeners into 3 categories:
application status listeners (e.g. UI), executor management (e.g. dynamic
allocation), and the event log.

The queueing logic, while abstracted away in a separate class, is kept
as much as possible hidden away from consumers. Aside from choosing their
queue, there's no code change needed to take advantage of queues.

Test coverage relies on existing tests; a few tests had to be tweaked
because they relied on `LiveListenerBus.postToAll` being synchronous,
and the change makes that method asynchronous. Other tests were simplified
not to use the asynchronous LiveListenerBus.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19211 from vanzin/SPARK-18838.
2017-09-20 13:41:29 +08:00
jerryshao 8319432af6 [SPARK-21917][CORE][YARN] Supporting adding http(s) resources in yarn mode
## What changes were proposed in this pull request?
In the current Spark, when submitting application on YARN with remote resources `./bin/spark-shell --jars http://central.maven.org/maven2/com/github/swagger-akka-http/swagger-akka-http_2.11/0.10.1/swagger-akka-http_2.11-0.10.1.jar --master yarn-client -v`, Spark will be failed with:

```
java.io.IOException: No FileSystem for scheme: http
	at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2586)
	at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2593)
	at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:91)
	at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2632)
	at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2614)
	at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:370)
	at org.apache.hadoop.fs.Path.getFileSystem(Path.java:296)
	at org.apache.spark.deploy.yarn.Client.copyFileToRemote(Client.scala:354)
	at org.apache.spark.deploy.yarn.Client.org$apache$spark$deploy$yarn$Client$$distribute$1(Client.scala:478)
	at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11$$anonfun$apply$6.apply(Client.scala:600)
	at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11$$anonfun$apply$6.apply(Client.scala:599)
	at scala.collection.mutable.ArraySeq.foreach(ArraySeq.scala:74)
	at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11.apply(Client.scala:599)
	at org.apache.spark.deploy.yarn.Client$$anonfun$prepareLocalResources$11.apply(Client.scala:598)
	at scala.collection.immutable.List.foreach(List.scala:381)
	at org.apache.spark.deploy.yarn.Client.prepareLocalResources(Client.scala:598)
	at org.apache.spark.deploy.yarn.Client.createContainerLaunchContext(Client.scala:848)
	at org.apache.spark.deploy.yarn.Client.submitApplication(Client.scala:173)
```

This is because `YARN#client` assumes resources are on the Hadoop compatible FS. To fix this problem, here propose to download remote http(s) resources to local and add this local downloaded resources to dist cache. This solution has one downside: remote resources are downloaded and uploaded again, but it only restricted to only remote http(s) resources, also the overhead is not so big. The advantages of this solution is that it is simple and the code changes restricts to only `SparkSubmit`.

## How was this patch tested?

Unit test added, also verified in local cluster.

Author: jerryshao <sshao@hortonworks.com>

Closes #19130 from jerryshao/SPARK-21917.
2017-09-19 22:20:05 +08:00
Armin 7c92351f43 [MINOR][CORE] Cleanup dead code and duplication in Mem. Management
## What changes were proposed in this pull request?

* Removed the method `org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter#alignToWords`.
It became unused as a result of 85b0a15754
(SPARK-15962) introducing word alignment for unsafe arrays.
* Cleaned up duplicate code in memory management and unsafe sorters
  * The change extracting the exception paths is more than just cosmetics since it def. reduces the size the affected methods compile to

## How was this patch tested?

* Build still passes after removing the method, grepping the codebase for `alignToWords` shows no reference to it anywhere either.
* Dried up code is covered by existing tests.

Author: Armin <me@obrown.io>

Closes #19254 from original-brownbear/cleanup-mem-consumer.
2017-09-19 10:06:32 +01:00
Xianyang Liu a11db942aa [SPARK-21923][CORE] Avoid calling reserveUnrollMemoryForThisTask for every record
## What changes were proposed in this pull request?
When Spark persist data to Unsafe memory, we call  the method `MemoryStore.putIteratorAsBytes`, which need synchronize the `memoryManager` for every record write. This implementation is not necessary, we can apply for more memory at a time to reduce unnecessary synchronization.

## How was this patch tested?

Test case (with 1 executor 20 core):
```scala
val start = System.currentTimeMillis()
val data = sc.parallelize(0 until Integer.MAX_VALUE, 100)
      .persist(StorageLevel.OFF_HEAP)
      .count()

println(System.currentTimeMillis() - start)

```

Test result:

before

|  27647  |  29108  |  28591  |  28264  |  27232  |

after

|  26868  |  26358  |  27767  |  26653  |  26693  |

Author: Xianyang Liu <xianyang.liu@intel.com>

Closes #19135 from ConeyLiu/memorystore.
2017-09-19 14:51:27 +08:00
alexmnyc 94f7e046a2 [SPARK-22030][CORE] GraphiteSink fails to re-connect to Graphite instances behind an ELB or any other auto-scaled LB
## What changes were proposed in this pull request?

Upgrade codahale metrics library so that Graphite constructor can re-resolve hosts behind a CNAME with re-tried DNS lookups. When Graphite is deployed behind an ELB, ELB may change IP addresses based on auto-scaling needs. Using current approach yields Graphite usage impossible, fixing for that use case

- Upgrade to codahale 3.1.5
- Use new Graphite(host, port) constructor instead of new Graphite(new InetSocketAddress(host, port)) constructor

## How was this patch tested?

The same logic is used for another project that is using the same configuration and code path, and graphite re-connect's behind ELB's are no longer an issue

This are proposed changes for codahale lib - https://github.com/dropwizard/metrics/compare/v3.1.2...v3.1.5#diff-6916c85d2dd08d89fe771c952e3b8512R120. Specifically, b4d246d34e/metrics-graphite/src/main/java/com/codahale/metrics/graphite/Graphite.java (L120)

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

Author: alexmnyc <project@alexandermarkham.com>

Closes #19210 from alexmnyc/patch-1.
2017-09-19 10:05:59 +08:00
Sital Kedia 1e978b17d6 [SPARK-21113][CORE] Read ahead input stream to amortize disk IO cost …
Profiling some of our big jobs, we see that around 30% of the time is being spent in reading the spill files from disk. In order to amortize the disk IO cost, the idea is to implement a read ahead input stream which asynchronously reads ahead from the underlying input stream when specified amount of data has been read from the current buffer. It does it by maintaining two buffer - active buffer and read ahead buffer. The active buffer contains data which should be returned when a read() call is issued. The read-ahead buffer is used to asynchronously read from the underlying input stream and once the active buffer is exhausted, we flip the two buffers so that we can start reading from the read ahead buffer without being blocked in disk I/O.

## How was this patch tested?

Tested by running a job on the cluster and could see up to 8% CPU improvement.

Author: Sital Kedia <skedia@fb.com>
Author: Shixiong Zhu <zsxwing@gmail.com>
Author: Sital Kedia <sitalkedia@users.noreply.github.com>

Closes #18317 from sitalkedia/read_ahead_buffer.
2017-09-17 23:15:08 -07:00
Andrew Ash 6308c65f08 [SPARK-21953] Show both memory and disk bytes spilled if either is present
As written now, there must be both memory and disk bytes spilled to show either of them. If there is only one of those types of spill recorded, it will be hidden.

Author: Andrew Ash <andrew@andrewash.com>

Closes #19164 from ash211/patch-3.
2017-09-18 10:42:24 +08:00
zhoukang 22b111ef9d [SPARK-21902][CORE] Print root cause for BlockManager#doPut
## What changes were proposed in this pull request?

As logging below, actually exception will be hidden when removeBlockInternal throw an exception.
`2017-08-31,10:26:57,733 WARN org.apache.spark.storage.BlockManager: Putting block broadcast_110 failed due to an exception
2017-08-31,10:26:57,734 WARN org.apache.spark.broadcast.BroadcastManager: Failed to create a new broadcast in 1 attempts
java.io.IOException: Failed to create local dir in /tmp/blockmgr-5bb5ac1e-c494-434a-ab89-bd1808c6b9ed/2e.
        at org.apache.spark.storage.DiskBlockManager.getFile(DiskBlockManager.scala:70)
        at org.apache.spark.storage.DiskStore.remove(DiskStore.scala:115)
        at org.apache.spark.storage.BlockManager.removeBlockInternal(BlockManager.scala:1339)
        at org.apache.spark.storage.BlockManager.doPut(BlockManager.scala:910)
        at org.apache.spark.storage.BlockManager.doPutIterator(BlockManager.scala:948)
        at org.apache.spark.storage.BlockManager.putIterator(BlockManager.scala:726)
        at org.apache.spark.storage.BlockManager.putSingle(BlockManager.scala:1233)
        at org.apache.spark.broadcast.TorrentBroadcast.writeBlocks(TorrentBroadcast.scala:122)
        at org.apache.spark.broadcast.TorrentBroadcast.<init>(TorrentBroadcast.scala:88)
        at org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast(TorrentBroadcastFactory.scala:34)
        at org.apache.spark.broadcast.BroadcastManager$$anonfun$newBroadcast$1.apply$mcVI$sp(BroadcastManager.scala:60)
        at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)
        at org.apache.spark.broadcast.BroadcastManager.newBroadcast(BroadcastManager.scala:58)
        at org.apache.spark.SparkContext.broadcast(SparkContext.scala:1415)
        at org.apache.spark.scheduler.DAGScheduler.submitMissingTasks(DAGScheduler.scala:1002)
        at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$submitStage(DAGScheduler.scala:924)
        at org.apache.spark.scheduler.DAGScheduler$$anonfun$submitWaitingChildStages$6.apply(DAGScheduler.scala:771)
        at org.apache.spark.scheduler.DAGScheduler$$anonfun$submitWaitingChildStages$6.apply(DAGScheduler.scala:770)
        at scala.collection.IndexedSeqOptimized$class.foreach(IndexedSeqOptimized.scala:33)
        at scala.collection.mutable.ArrayOps$ofRef.foreach(ArrayOps.scala:186)
        at org.apache.spark.scheduler.DAGScheduler.submitWaitingChildStages(DAGScheduler.scala:770)
        at org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:1235)
        at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1662)
        at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1620)
        at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1609)
        at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48)`

In this pr i will print exception first make troubleshooting more conveniently.
PS:
This one split from [PR-19133](https://github.com/apache/spark/pull/19133)

## How was this patch tested?
Exsist unit test

Author: zhoukang <zhoukang199191@gmail.com>

Closes #19171 from caneGuy/zhoukang/print-rootcause.
2017-09-15 14:03:26 +08:00
zhoukang 4b88393cb9 [SPARK-21922] Fix duration always updating when task failed but status is still RUN…
…NING

## What changes were proposed in this pull request?
When driver quit abnormally which cause executor shutdown and task metrics can not be sent to driver for updating.In this case the status will always be 'RUNNING' and the duration on history UI will be 'CurrentTime - launchTime' which increase infinitely.
We can fix this time by modify time of event log since this time has gotten when `FSHistoryProvider` fetch event log from File System.
And the result picture is uploaded in [SPARK-21922](https://issues.apache.org/jira/browse/SPARK-21922).
How to reproduce?
(1) Submit a job to spark on yarn
(2) Mock an oom(or other case can make driver quit abnormally)  senario for driver
(3) Make sure executor is running task when driver quitting
(4) Open the history server and checkout result
It is not a corner case since there are many such jobs in our current cluster.

## How was this patch tested?
Deploy historyserver and open a job has this problem.

Author: zhoukang <zhoukang199191@gmail.com>

Closes #19132 from caneGuy/zhoukang/fix-duration.
2017-09-14 20:40:33 +08:00
Zheng RuiFeng 66cb72d7b9 [MINOR][DOC] Add missing call of update() in examples of PeriodicGraphCheckpointer & PeriodicRDDCheckpointer
## What changes were proposed in this pull request?
forgot to call `update()` with `graph1` & `rdd1` in examples for `PeriodicGraphCheckpointer` & `PeriodicRDDCheckpoin`
## How was this patch tested?
existing tests

Author: Zheng RuiFeng <ruifengz@foxmail.com>

Closes #19198 from zhengruifeng/fix_doc_checkpointer.
2017-09-14 14:04:43 +08:00
Armin b6ef1f57bc [SPARK-21970][CORE] Fix Redundant Throws Declarations in Java Codebase
## What changes were proposed in this pull request?

1. Removing all redundant throws declarations from Java codebase.
2. Removing dead code made visible by this from `ShuffleExternalSorter#closeAndGetSpills`

## How was this patch tested?

Build still passes.

Author: Armin <me@obrown.io>

Closes #19182 from original-brownbear/SPARK-21970.
2017-09-13 14:04:26 +01:00
caoxuewen ca00cc70d6 [SPARK-21963][CORE][TEST] Create temp file should be delete after use
## What changes were proposed in this pull request?

After you create a temporary table, you need to delete it, otherwise it will leave a file similar to the file name ‘SPARK194465907929586320484966temp’.

## How was this patch tested?

N / A

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #19174 from heary-cao/DeleteTempFile.
2017-09-13 13:01:30 +01:00
German Schiavon a1d98c6dcd [SPARK-21982] Set locale to US
## What changes were proposed in this pull request?

In UtilsSuite Locale was set by default to US, but at the moment of using format function it wasn't, taking by default JVM locale which could be different than US making this test fail.

## How was this patch tested?
Unit test (UtilsSuite)

Author: German Schiavon <germanschiavon@gmail.com>

Closes #19205 from Gschiavon/fix/test-locale.
2017-09-13 09:52:45 +01:00
caoxuewen dc74c0e67d [MINOR][SQL] remove unuse import class
## What changes were proposed in this pull request?

this PR describe remove the import class that are unused.

## How was this patch tested?

N/A

Author: caoxuewen <cao.xuewen@zte.com.cn>

Closes #19131 from heary-cao/unuse_import.
2017-09-11 10:09:20 +01:00
Dongjoon Hyun c26976fe14 [SPARK-21939][TEST] Use TimeLimits instead of Timeouts
Since ScalaTest 3.0.0, `org.scalatest.concurrent.Timeouts` is deprecated.
This PR replaces the deprecated one with `org.scalatest.concurrent.TimeLimits`.

```scala
-import org.scalatest.concurrent.Timeouts._
+import org.scalatest.concurrent.TimeLimits._
```

Pass the existing test suites.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19150 from dongjoon-hyun/SPARK-21939.

Change-Id: I1a1b07f1b97e51e2263dfb34b7eaaa099b2ded5e
2017-09-08 09:31:13 +08:00
Sanket Chintapalli b9ab791a9e [SPARK-21890] Credentials not being passed to add the tokens
I observed this while running a oozie job trying to connect to hbase via spark.
It look like the creds are not being passed in thehttps://github.com/apache/spark/blob/branch-2.2/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/HadoopFSCredentialProvider.scala#L53 for 2.2 release.
More Info as to why it fails on secure grid:
Oozie client gets the necessary tokens the application needs before launching. It passes those tokens along to the oozie launcher job (MR job) which will then actually call the Spark client to launch the spark app and pass the tokens along.
The oozie launcher job cannot get anymore tokens because all it has is tokens ( you can't get tokens with tokens, you need tgt or keytab).
The error here is because the launcher job runs the Spark Client to submit the spark job but the spark client doesn't see that it already has the hdfs tokens so it tries to get more, which ends with the exception.
There was a change with SPARK-19021 to generalize the hdfs credentials provider that changed it so we don't pass the existing credentials into the call to get tokens so it doesn't realize it already has the necessary tokens.

https://issues.apache.org/jira/browse/SPARK-21890
Modified to pass creds to get delegation tokens

Author: Sanket Chintapalli <schintap@yahoo-inc.com>

Closes #19140 from redsanket/SPARK-21890-master.
2017-09-07 11:25:24 -05:00
Sean Owen ca59445adb [SPARK-21418][SQL] NoSuchElementException: None.get in DataSourceScanExec with sun.io.serialization.extendedDebugInfo=true
## What changes were proposed in this pull request?

If no SparkConf is available to Utils.redact, simply don't redact.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19123 from srowen/SPARK-21418.
2017-09-04 23:02:59 +02:00
Sean Owen 12ab7f7e89 [SPARK-14280][BUILD][WIP] Update change-version.sh and pom.xml to add Scala 2.12 profiles and enable 2.12 compilation
…build; fix some things that will be warnings or errors in 2.12; restore Scala 2.12 profile infrastructure

## What changes were proposed in this pull request?

This change adds back the infrastructure for a Scala 2.12 build, but does not enable it in the release or Python test scripts.

In order to make that meaningful, it also resolves compile errors that the code hits in 2.12 only, in a way that still works with 2.11.

It also updates dependencies to the earliest minor release of dependencies whose current version does not yet support Scala 2.12. This is in a sense covered by other JIRAs under the main umbrella, but implemented here. The versions below still work with 2.11, and are the _latest_ maintenance release in the _earliest_ viable minor release.

- Scalatest 2.x -> 3.0.3
- Chill 0.8.0 -> 0.8.4
- Clapper 1.0.x -> 1.1.2
- json4s 3.2.x -> 3.4.2
- Jackson 2.6.x -> 2.7.9 (required by json4s)

This change does _not_ fully enable a Scala 2.12 build:

- It will also require dropping support for Kafka before 0.10. Easy enough, just didn't do it yet here
- It will require recreating `SparkILoop` and `Main` for REPL 2.12, which is SPARK-14650. Possible to do here too.

What it does do is make changes that resolve much of the remaining gap without affecting the current 2.11 build.

## How was this patch tested?

Existing tests and build. Manually tested with `./dev/change-scala-version.sh 2.12` to verify it compiles, modulo the exceptions above.

Author: Sean Owen <sowen@cloudera.com>

Closes #18645 from srowen/SPARK-14280.
2017-09-01 19:21:21 +01:00