Commit graph

1192 commits

Author SHA1 Message Date
Kazuaki Ishizaki 7bf0794651 [SPARK-26463][CORE] Use ConfigEntry for hardcoded configs for scheduler categories.
## What changes were proposed in this pull request?

The PR makes hardcoded `spark.dynamicAllocation`, `spark.scheduler`, `spark.rpc`, `spark.task`, `spark.speculation`, and `spark.cleaner` configs to use `ConfigEntry`.

## How was this patch tested?

Existing tests

Closes #23416 from kiszk/SPARK-26463.

Authored-by: Kazuaki Ishizaki <ishizaki@jp.ibm.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-22 07:44:36 -06:00
Sean Owen c2d0d700b5 [SPARK-26640][CORE][ML][SQL][STREAMING][PYSPARK] Code cleanup from lgtm.com analysis
## What changes were proposed in this pull request?

Misc code cleanup from lgtm.com analysis. See comments below for details.

## How was this patch tested?

Existing tests.

Closes #23571 from srowen/SPARK-26640.

Lead-authored-by: Sean Owen <sean.owen@databricks.com>
Co-authored-by: Hyukjin Kwon <gurwls223@apache.org>
Co-authored-by: Sean Owen <srowen@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-17 19:40:39 -06:00
Jungtaek Lim (HeartSaVioR) d9e4cf67c0 [SPARK-26482][CORE] Use ConfigEntry for hardcoded configs for ui categories
## What changes were proposed in this pull request?

The PR makes hardcoded configs below to use `ConfigEntry`.

* spark.ui
* spark.ssl
* spark.authenticate
* spark.master.rest
* spark.master.ui
* spark.metrics
* spark.admin
* spark.modify.acl

This patch doesn't change configs which are not relevant to SparkConf (e.g. system properties).

## How was this patch tested?

Existing tests.

Closes #23423 from HeartSaVioR/SPARK-26466.

Authored-by: Jungtaek Lim (HeartSaVioR) <kabhwan@gmail.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2019-01-11 10:18:07 -08:00
Sean Owen 2f8a938805 [SPARK-26539][CORE] Remove spark.memory.useLegacyMode and StaticMemoryManager
## What changes were proposed in this pull request?

Remove spark.memory.useLegacyMode and StaticMemoryManager. Update tests that used the StaticMemoryManager to equivalent use of UnifiedMemoryManager.

## How was this patch tested?

Existing tests, with modifications to make them work with a different mem manager.

Closes #23457 from srowen/SPARK-26539.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2019-01-10 08:57:44 -06:00
Marcelo Vanzin 2783e4c45f [SPARK-24522][UI] Create filter to apply HTTP security checks consistently.
Currently there is code scattered in a bunch of places to do different
things related to HTTP security, such as access control, setting
security-related headers, and filtering out bad content. This makes it
really easy to miss these things when writing new UI code.

This change creates a new filter that does all of those things, and
makes sure that all servlet handlers that are attached to the UI get
the new filter and any user-defined filters consistently. The extent
of the actual features should be the same as before.

The new filter is added at the end of the filter chain, because authentication
is done by custom filters and thus needs to happen first. This means that
custom filters see unfiltered HTTP requests - which is actually the current
behavior anyway.

As a side-effect of some of the code refactoring, handlers added after
the initial set also get wrapped with a GzipHandler, which didn't happen
before.

Tested with added unit tests and in a history server with SPNEGO auth
configured.

Closes #23302 from vanzin/SPARK-24522.

Authored-by: Marcelo Vanzin <vanzin@cloudera.com>
Signed-off-by: Imran Rashid <irashid@cloudera.com>
2019-01-08 11:25:33 -06:00
Dongjoon Hyun e15a319ccd
[SPARK-26536][BUILD][TEST] Upgrade Mockito to 2.23.4
## What changes were proposed in this pull request?

This PR upgrades Mockito from 1.10.19 to 2.23.4. The following changes are required.

- Replace `org.mockito.Matchers` with `org.mockito.ArgumentMatchers`
- Replace `anyObject` with `any`
- Replace `getArgumentAt` with `getArgument` and add type annotation.
- Use `isNull` matcher in case of `null` is invoked.
```scala
     saslHandler.channelInactive(null);
-    verify(handler).channelInactive(any(TransportClient.class));
+    verify(handler).channelInactive(isNull());
```

- Make and use `doReturn` wrapper to avoid [SI-4775](https://issues.scala-lang.org/browse/SI-4775)
```scala
private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*)
```

## How was this patch tested?

Pass the Jenkins with the existing tests.

Closes #23452 from dongjoon-hyun/SPARK-26536.

Authored-by: Dongjoon Hyun <dongjoon@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2019-01-04 19:23:38 -08:00
caoxuewen 327ac83f5c [SPARK-26180][CORE][TEST] Reuse withTempDir function to the SparkCore test case
## What changes were proposed in this pull request?

Currently, the common `withTempDir` function is used in Spark SQL test cases. To handle `val dir = Utils. createTempDir()` and `Utils. deleteRecursively (dir)`. Unfortunately, the `withTempDir` function cannot be used in the Spark Core test case. This PR Sharing `withTempDir` function in Spark Sql and SparkCore  to clean up SparkCore test cases. thanks.

## How was this patch tested?

N / A

Closes #23151 from heary-cao/withCreateTempDir.

Authored-by: caoxuewen <cao.xuewen@zte.com.cn>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
2018-12-01 16:34:11 +08:00
DB Tsai ad853c5678
[SPARK-25956] Make Scala 2.12 as default Scala version in Spark 3.0
## What changes were proposed in this pull request?

This PR makes Spark's default Scala version as 2.12, and Scala 2.11 will be the alternative version. This implies that Scala 2.12 will be used by our CI builds including pull request builds.

We'll update the Jenkins to include a new compile-only jobs for Scala 2.11 to ensure the code can be still compiled with Scala 2.11.

## How was this patch tested?

existing tests

Closes #22967 from dbtsai/scala2.12.

Authored-by: DB Tsai <d_tsai@apple.com>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
2018-11-14 16:22:23 -08:00
Yuanjian Li 2977e2312d [SPARK-25986][BUILD] Add rules to ban throw Errors in application code
## What changes were proposed in this pull request?

Add scala and java lint check rules to ban the usage of `throw new xxxErrors` and fix up all exists instance followed by https://github.com/apache/spark/pull/22989#issuecomment-437939830. See more details in https://github.com/apache/spark/pull/22969.

## How was this patch tested?

Local test with lint-scala and lint-java.

Closes #22989 from xuanyuanking/SPARK-25986.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-14 13:05:18 -08:00
gss2002 2b671e7292 [SPARK-25778] WriteAheadLogBackedBlockRDD in YARN Cluster Mode Fails …
…due lack of access to tmpDir from $PWD to HDFS

WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode.

## What changes were proposed in this pull request?
This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir

## How was this patch tested?
Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode.

Closes #22867 from gss2002/SPARK-25778.

Authored-by: gss2002 <greg@senia.org>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-14 13:02:13 -08:00
Sean Owen 2d085c13b7 [SPARK-25984][CORE][SQL][STREAMING] Remove deprecated .newInstance(), primitive box class constructor calls
## What changes were proposed in this pull request?

Deprecated in Java 11, replace Class.newInstance with Class.getConstructor.getInstance, and primtive wrapper class constructors with valueOf or equivalent

## How was this patch tested?

Existing tests.

Closes #22988 from srowen/SPARK-25984.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-10 09:52:14 -06:00
Imran Rashid cee230160b [SPARK-25871][STREAMING] Don't use EC for streaming WAL
The write ahead log expects to be able to call hflush, but that is a no-op when writing to a file with hdfs erasure coding.  So ensure that file is always written with replication instead, regardless of filesystem defaults.

None yet.  I'm posting this mostly to make it visible.

Closes #22882 from squito/SPARK-25871.

Authored-by: Imran Rashid <irashid@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-11-06 10:52:42 -08:00
Shahid ed0c57e10d [SPARK-25861][MINOR][WEBUI] Remove unused refreshInterval parameter from the headerSparkPage method.
## What changes were proposed in this pull request?
'refreshInterval' is not used any where in the headerSparkPage method. So, we don't need to pass the parameter while calling the  'headerSparkPage' method.

## How was this patch tested?
Existing tests

Closes #22864 from shahidki31/unusedCode.

Authored-by: Shahid <shahidki31@gmail.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-11-02 17:17:48 -05:00
Sean Owen f83fedc9f2 [SPARK-25737][CORE] Remove JavaSparkContextVarargsWorkaround
## What changes were proposed in this pull request?

Remove JavaSparkContextVarargsWorkaround

## How was this patch tested?

Existing tests.

Closes #22729 from srowen/SPARK-25737.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-24 14:43:51 -05:00
Sean Owen 1ddfab8c4f [SPARK-19287][CORE][STREAMING] JavaPairRDD flatMapValues requires function returning Iterable, not Iterator
## What changes were proposed in this pull request?

Fix old oversight in API: Java `flatMapValues` needs a `FlatMapFunction`

## How was this patch tested?

Existing tests.

Closes #22690 from srowen/SPARK-19287.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-12 18:10:59 -05:00
Sean Owen a001814189 [SPARK-25598][STREAMING][BUILD][TEST-MAVEN] Remove flume connector in Spark 3
## What changes were proposed in this pull request?

Removes all vestiges of Flume in the build, for Spark 3.
I don't think this needs Jenkins config changes.

## How was this patch tested?

Existing tests.

Closes #22692 from srowen/SPARK-25598.

Authored-by: Sean Owen <sean.owen@databricks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-11 14:28:06 -07:00
Prashant Sharma 3ae4f07de0 [SPARK-17159][STREAM] Significant speed up for running spark streaming against Object store.
## What changes were proposed in this pull request?

Original work by Steve Loughran.
Based on #17745.

This is a minimal patch of changes to FileInputDStream to reduce File status requests when querying files. Each call to file status is 3+ http calls to object store. This patch eliminates the need for it, by using FileStatus objects.

This is a minor optimisation when working with filesystems, but significant when working with object stores.

## How was this patch tested?

Tests included. Existing tests pass.

Closes #22339 from ScrapCodes/PR_17745.

Lead-authored-by: Prashant Sharma <prashant@apache.org>
Co-authored-by: Steve Loughran <stevel@hortonworks.com>
Signed-off-by: Sean Owen <sean.owen@databricks.com>
2018-10-05 02:22:06 +01:00
gatorsmile 9bf397c0e4 [SPARK-25592] Setting version to 3.0.0-SNAPSHOT
## What changes were proposed in this pull request?

This patch is to bump the master branch version to 3.0.0-SNAPSHOT.

## How was this patch tested?
N/A

Closes #22606 from gatorsmile/bump3.0.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-10-02 08:48:24 -07:00
Ilan Filonenko 51540c2fa6 [SPARK-25372][YARN][K8S] Deprecate and generalize keytab / principal config
## What changes were proposed in this pull request?

SparkSubmit already logs in the user if a keytab is provided, the only issue is that it uses the existing configs which have "yarn" in their name. As such, the configs were changed to:

`spark.kerberos.keytab` and `spark.kerberos.principal`.

## How was this patch tested?

Will be tested with K8S tests, but needs to be tested with Yarn

- [x] K8S Secure HDFS tests
- [x] Yarn Secure HDFS tests vanzin

Closes #22362 from ifilonenko/SPARK-25372.

Authored-by: Ilan Filonenko <if56@cornell.edu>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-26 17:24:52 -07:00
Santiago Saavedra 497f00f62b [SPARK-23200] Reset Kubernetes-specific config on Checkpoint restore
Several configuration parameters related to Kubernetes need to be
reset, as they are changed with each invokation of spark-submit and
thus prevents recovery of Spark Streaming tasks.

## What changes were proposed in this pull request?

When using the Kubernetes cluster-manager and spawning a Streaming workload, it is important to reset many spark.kubernetes.* properties that are generated by spark-submit but which would get rewritten when restoring a Checkpoint. This is so, because the spark-submit codepath creates Kubernetes resources, such as a ConfigMap, a Secret and other variables, which have an autogenerated name and the previous one will not resolve anymore.

In short, this change enables checkpoint restoration for streaming workloads, and thus enables Spark Streaming workloads in Kubernetes, which were not possible to restore from a checkpoint before if the workload went down.

## How was this patch tested?

This patch needs would benefit from testing in different k8s clusters.

This is similar to the YARN related code for resetting a Spark Streaming workload, but for the Kubernetes scheduler. This PR removes the initcontainers properties that existed before because they are now removed in master.

For a previous discussion, see the non-rebased work at: apache-spark-on-k8s#516

Closes #22392 from ssaavedra/fix-checkpointing-master.

Authored-by: Santiago Saavedra <santiagosaavedra@gmail.com>
Signed-off-by: Yinan Li <ynli@google.com>
2018-09-18 22:08:50 -07:00
gatorsmile bb2f069cf2 [SPARK-25436] Bump master branch version to 2.5.0-SNAPSHOT
## What changes were proposed in this pull request?
In the dev list, we can still discuss whether the next version is 2.5.0 or 3.0.0. Let us first bump the master branch version to `2.5.0-SNAPSHOT`.

## How was this patch tested?
N/A

Closes #22426 from gatorsmile/bumpVersionMaster.

Authored-by: gatorsmile <gatorsmile@gmail.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
2018-09-15 16:24:02 -07:00
ankurgupta 39a02d8f75 [SPARK-24415][CORE] Fixed the aggregated stage metrics by retaining stage objects in liveStages until all tasks are complete
The problem occurs because stage object is removed from liveStages in
AppStatusListener onStageCompletion. Because of this any onTaskEnd event
received after onStageCompletion event do not update stage metrics.

The fix is to retain stage objects in liveStages until all tasks are complete.

1. Fixed the reproducible example posted in the JIRA
2. Added unit test

Closes #22209 from ankuriitg/ankurgupta/SPARK-24415.

Authored-by: ankurgupta <ankur.gupta@cloudera.com>
Signed-off-by: Marcelo Vanzin <vanzin@cloudera.com>
2018-09-05 09:52:04 -07:00
Maxim Gekk 32da87dfa4 [SPARK-25286][CORE] Removing the dangerous parmap
## What changes were proposed in this pull request?

I propose to remove one of `parmap` methods which accepts an execution context as a parameter. The method should be removed to eliminate any deadlocks that can occur if `parmap` is called recursively on thread pools restricted by size.

Closes #22292 from MaxGekk/remove-overloaded-parmap.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Xiao Li <gatorsmile@gmail.com>
2018-08-31 10:43:30 -07:00
Marco Gaido 55f36641ff [SPARK-25093][SQL] Avoid recompiling regexp for comments multiple times
## What changes were proposed in this pull request?

The PR moves the compilation of the regexp for code formatting outside the method which is called for each code block when splitting expressions, in order to avoid recompiling the regexp every time.

Credit should be given to Izek Greenfield.

## How was this patch tested?

existing UTs

Closes #22135 from mgaido91/SPARK-25093.

Authored-by: Marco Gaido <marcogaido91@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-22 14:31:51 +08:00
Maxim Gekk 131ca146ed [SPARK-24005][CORE] Remove usage of Scala’s parallel collection
## What changes were proposed in this pull request?

In the PR, I propose to replace Scala parallel collections by new methods `parmap()`. The methods use futures to transform a sequential collection by applying a lambda function to each element in parallel. The result of `parmap` is another regular (sequential) collection.

The proposed `parmap` method aims to solve the problem of impossibility to interrupt parallel Scala collection. This possibility is needed for reliable task preemption.

## How was this patch tested?

A test was added to `ThreadUtilsSuite`

Closes #21913 from MaxGekk/par-map.

Authored-by: Maxim Gekk <maxim.gekk@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
2018-08-07 17:14:30 +08:00
Karthik Palaniappan ee5a5a0925 [SPARK-21960][STREAMING] Spark Streaming Dynamic Allocation should respect spark.executor.instances
## What changes were proposed in this pull request?

Removes check that `spark.executor.instances` is set to 0 when using Streaming DRA.

## How was this patch tested?

Manual tests

My only concern with this PR is that `spark.executor.instances` (or the actual initial number of executors that the cluster manager gives Spark) can be outside of `spark.streaming.dynamicAllocation.minExecutors` to `spark.streaming.dynamicAllocation.maxExecutors`. I don't see a good way around that, because this code only runs after the SparkContext has been created.

Author: Karthik Palaniappan <karthikpal@google.com>

Closes #19183 from karth295/master.
2018-07-27 12:18:56 -05:00
Kallman, Steven c5aa54d54b [SPARK-24553][WEB-UI] http 302 fixes for href redirect
## What changes were proposed in this pull request?

Updated URL/href links to include a '/' before '?id' to make links consistent and avoid http 302 redirect errors within UI port 4040 tabs.

## How was this patch tested?

Built a runnable distribution and executed jobs. Validated that http 302 redirects are no longer encountered when clicking on links within UI port 4040 tabs.

Author: Steven Kallman <SJKallmangmail.com>

Author: Kallman, Steven <Steven.Kallman@CapitalOne.com>

Closes #21600 from SJKallman/{Spark-24553}{WEB-UI}-redirect-href-fixes.
2018-06-27 15:36:59 -07:00
Kazuaki Ishizaki 90da7dc241 [SPARK-24452][SQL][CORE] Avoid possible overflow in int add or multiple
## What changes were proposed in this pull request?

This PR fixes possible overflow in int add or multiply. In particular, their overflows in multiply are detected by [Spotbugs](https://spotbugs.github.io/)

The following assignments may cause overflow in right hand side. As a result, the result may be negative.
```
long = int * int
long = int + int
```

To avoid this problem, this PR performs cast from int to long in right hand side.

## How was this patch tested?

Existing UTs.

Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com>

Closes #21481 from kiszk/SPARK-24452.
2018-06-15 13:47:48 -07:00
Jacek Laskowski 495d8cf09a [SPARK-24490][WEBUI] Use WebUI.addStaticHandler in web UIs
`WebUI` defines `addStaticHandler` that web UIs don't use (and simply introduce duplication). Let's clean them up and remove duplications.

Local build and waiting for Jenkins

Author: Jacek Laskowski <jacek@japila.pl>

Closes #21510 from jaceklaskowski/SPARK-24490-Use-WebUI.addStaticHandler.
2018-06-15 09:59:02 -07:00
Gabor Somogyi aca65c63cb [SPARK-23991][DSTREAMS] Fix data loss when WAL write fails in allocateBlocksToBatch
When blocks tried to get allocated to a batch and WAL write fails then the blocks will be removed from the received block queue. This fact simply produces data loss because the next allocation will not find the mentioned blocks in the queue.

In this PR blocks will be removed from the received queue only if WAL write succeded.

Additional unit test.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #21430 from gaborgsomogyi/SPARK-23991.

Change-Id: I5ead84f0233f0c95e6d9f2854ac2ff6906f6b341
2018-05-29 20:10:59 +08:00
Marco Gaido 84d31aa5d4 [SPARK-24209][SHS] Automatic retrieve proxyBase from Knox headers
## What changes were proposed in this pull request?

The PR retrieves the proxyBase automatically from the header `X-Forwarded-Context` (if available). This is the header used by Knox to inform the proxied service about the base path.

This provides 0-configuration support for Knox gateway (instead of having to properly set `spark.ui.proxyBase`) and it allows to access directly SHS when it is proxied by Knox. In the previous scenario, indeed, after setting `spark.ui.proxyBase`, direct access to SHS was not working fine (due to bad link generated).

## How was this patch tested?

added UT + manual tests

Author: Marco Gaido <marcogaido91@gmail.com>

Closes #21268 from mgaido91/SPARK-24209.
2018-05-21 18:11:05 -07:00
Marcelo Vanzin 5fa4384711 [SPARK-23361][YARN] Allow AM to restart after initial tokens expire.
Currently, the Spark AM relies on the initial set of tokens created by
the submission client to be able to talk to HDFS and other services that
require delegation tokens. This means that after those tokens expire, a
new AM will fail to start (e.g. when there is an application failure and
re-attempts are enabled).

This PR makes it so that the first thing the AM does when the user provides
a principal and keytab is to create new delegation tokens for use. This
makes sure that the AM can be started irrespective of how old the original
token set is. It also allows all of the token management to be done by the
AM - there is no need for the submission client to set configuration values
to tell the AM when to renew tokens.

Note that even though in this case the AM will not be using the delegation
tokens created by the submission client, those tokens still need to be provided
to YARN, since they are used to do log aggregation.

To be able to re-use the code in the AMCredentialRenewal for the above
purposes, I refactored that class a bit so that it can fetch tokens into
a pre-defined UGI, insted of always logging in.

Another issue with re-attempts is that, after the fix that allows the AM
to restart correctly, new executors would get confused about when to
update credentials, because the credential updater used the update time
initially set up by the submission code. This could make the executor
fail to update credentials in time, since that value would be very out
of date in the situation described in the bug.

To fix that, I changed the YARN code to use the new RPC-based mechanism
for distributing tokens to executors. This allowed the old credential
updater code to be removed, and a lot of code in the renewer to be
simplified.

I also made two currently hardcoded values (the renewal time ratio, and
the retry wait) configurable; while this probably never needs to be set
by anyone in a production environment, it helps with testing; that's also
why they're not documented.

Tested on real cluster with a specially crafted application to test this
functionality: checked proper access to HDFS, Hive and HBase in cluster
mode with token renewal on and AM restarts. Tested things still work in
client mode too.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20657 from vanzin/SPARK-23361.
2018-03-23 13:59:21 +08:00
Marcelo Vanzin c99fc9ad9b [SPARK-23550][CORE] Cleanup Utils.
A few different things going on:
- Remove unused methods.
- Move JSON methods to the only class that uses them.
- Move test-only methods to TestUtils.
- Make getMaxResultSize() a config constant.
- Reuse functionality from existing libraries (JRE or JavaUtils) where possible.

The change also includes changes to a few tests to call `Utils.createTempFile` correctly,
so that temp dirs are created under the designated top-level temp dir instead of
potentially polluting git index.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20706 from vanzin/SPARK-23550.
2018-03-07 13:42:06 -08:00
Gabor Somogyi b308182f23 [SPARK-23438][DSTREAMS] Fix DStreams data loss with WAL when driver crashes
## What changes were proposed in this pull request?

There is a race condition introduced in SPARK-11141 which could cause data loss.
The problem is that ReceivedBlockTracker.insertAllocatedBatch function assumes that all blocks from streamIdToUnallocatedBlockQueues allocated to the batch and clears the queue.

In this PR only the allocated blocks will be removed from the queue which will prevent data loss.

## How was this patch tested?

Additional unit test + manually.

Author: Gabor Somogyi <gabor.g.somogyi@gmail.com>

Closes #20620 from gaborgsomogyi/SPARK-23438.
2018-02-26 08:39:44 -08:00
guoxiaolong bd24731722 [SPARK-23382][WEB-UI] Spark Streaming ui about the contents of the for need to have hidden and show features, when the table records very much.
## What changes were proposed in this pull request?
Spark Streaming ui about the contents of the for need to have hidden and show features, when the table records very much.
please refer to https://github.com/apache/spark/pull/20216

fix after:
![1](https://user-images.githubusercontent.com/26266482/36068644-df029328-0f14-11e8-8350-cfdde9733ffc.png)

## How was this patch tested?

(Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests)
(If this patch involves UI changes, please attach a screenshot; otherwise, remove this)

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

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

Closes #20570 from guoxiaolongzte/SPARK-23382.
2018-02-13 11:39:33 -06:00
jerryshao 4b7cd479a2 Revert "[SPARK-23200] Reset Kubernetes-specific config on Checkpoint restore"
This reverts commit d1721816d2.

The patch is not fully tested and out-of-date. So revert it.
2018-02-01 14:00:08 +08:00
Marcelo Vanzin dd242bad39 [SPARK-21525][STREAMING] Check error code from supervisor RPC.
The code was ignoring the error code from the AddBlock RPC, which
means that a failure to write to the WAL was being ignored by the
receiver, and would lead to the block being acked (in the case of
the Flume receiver) and data potentially lost.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #20161 from vanzin/SPARK-21525.
2018-01-31 11:48:19 -08:00
Santiago Saavedra d1721816d2 [SPARK-23200] Reset Kubernetes-specific config on Checkpoint restore
## What changes were proposed in this pull request?

When using the Kubernetes cluster-manager and spawning a Streaming workload, it is important to reset many spark.kubernetes.* properties that are generated by spark-submit but which would get rewritten when restoring a Checkpoint. This is so, because the spark-submit codepath creates Kubernetes resources, such as a ConfigMap, a Secret and other variables, which have an autogenerated name and the previous one will not resolve anymore.

In short, this change enables checkpoint restoration for streaming workloads, and thus enables Spark Streaming workloads in Kubernetes, which were not possible to restore from a checkpoint before if the workload went down.

## How was this patch tested?

This patch was tested with the twitter-streaming example in AWS, using checkpoints in s3 with the s3a:// protocol, as supported by Hadoop.

This is similar to the YARN related code for resetting a Spark Streaming workload, but for the Kubernetes scheduler. I'm adding the initcontainers properties because even if the discussion is not completely settled on the mailing list, my understanding is that at this moment they are going forward for the moment.

For a previous discussion, see the non-rebased work at: https://github.com/apache-spark-on-k8s/spark/pull/516

Author: Santiago Saavedra <ssaavedra@openshine.com>

Closes #20383 from ssaavedra/fix-k8s-checkpointing.
2018-01-26 15:24:06 +08:00
Tathagata Das 15adcc8273 [SPARK-23197][DSTREAMS] Increased timeouts to resolve flakiness
## What changes were proposed in this pull request?

Increased timeout from 50 ms to 300 ms (50 ms was really too low).

## How was this patch tested?
Multiple rounds of tests.

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

Closes #20371 from tdas/SPARK-23197.
2018-01-23 16:24:20 -08:00
gatorsmile 651f76153f [SPARK-23028] Bump master branch version to 2.4.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps the master branch version to `2.4.0-SNAPSHOT`.

## How was this patch tested?
N/A

Author: gatorsmile <gatorsmile@gmail.com>

Closes #20222 from gatorsmile/bump24.
2018-01-13 00:37:59 +08:00
Sean Owen c284c4e1f6 [MINOR] Fix a bunch of typos 2018-01-02 07:10:19 +09:00
Marcelo Vanzin 7570eab6be [SPARK-22788][STREAMING] Use correct hadoop config for fs append support.
Still look at the old one in case any Spark user is setting it
explicitly, though.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19983 from vanzin/SPARK-22788.
2017-12-20 11:31:11 -06:00
kellyzly f41c0a93fd [SPARK-22660][BUILD] Use position() and limit() to fix ambiguity issue in scala-2.12
…a-2.12 and JDK9

## What changes were proposed in this pull request?
Some compile error after upgrading to scala-2.12
```javascript
spark_source/core/src/main/scala/org/apache/spark/executor/Executor.scala:455: ambiguous reference to overloaded definition, method limit in class ByteBuffer of type (x$1: Int)java.nio.ByteBuffer
method limit in class Buffer of type ()Int
match expected type ?
     val resultSize = serializedDirectResult.limit
error
```
The limit method was moved from ByteBuffer to the superclass Buffer and it can no longer be called without (). The same reason for position method.

```javascript
/home/zly/prj/oss/jdk9_HOS_SOURCE/spark_source/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala:427: ambiguous reference to overloaded definition, [error] both method putAll in class Properties of type (x$1: java.util.Map[_, _])Unit [error] and  method putAll in class Hashtable of type (x$1: java.util.Map[_ <: Object, _ <: Object])Unit [error] match argument types (java.util.Map[String,String])
 [error]       props.putAll(outputSerdeProps.toMap.asJava)
 [error]             ^
 ```
This is because the key type is Object instead of String which is unsafe.

## How was this patch tested?

running tests

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

Author: kellyzly <kellyzly@126.com>

Closes #19854 from kellyzly/SPARK-22660.
2017-12-07 10:04:04 -06:00
Marcelo Vanzin 8ff474f6e5 [SPARK-20650][CORE] Remove JobProgressListener.
The only remaining use of this class was the SparkStatusTracker, which
was modified to use the new status store. The test code to wait for
executors was moved to TestUtils and now uses the SparkStatusTracker API.

Indirectly, ConsoleProgressBar also uses this data. Because it has
some lower latency requirements, a shortcut to efficiently get the
active stages from the active listener was added to the AppStateStore.

Now that all UI code goes through the status store to get its data,
the FsHistoryProvider can be cleaned up to only replay event logs
when needed - that is, when there is no pre-existing disk store for
the application.

As part of this change I also modified the streaming UI to read the needed
data from the store, which was missed in the previous patch that made
JobProgressListener redundant.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19750 from vanzin/SPARK-20650.
2017-11-29 14:34:41 -08:00
Dongjoon Hyun b10837ab1a [SPARK-22557][TEST] Use ThreadSignaler explicitly
## What changes were proposed in this pull request?

ScalaTest 3.0 uses an implicit `Signaler`. This PR makes it sure all Spark tests uses `ThreadSignaler` explicitly which has the same default behavior of interrupting a thread on the JVM like ScalaTest 2.2.x. This will reduce potential flakiness.

## How was this patch tested?

This is testsuite-only update. This should passes the Jenkins tests.

Author: Dongjoon Hyun <dongjoon@apache.org>

Closes #19784 from dongjoon-hyun/use_thread_signaler.
2017-11-20 13:32:01 +09:00
Marcelo Vanzin 39b3f10dda [SPARK-20649][CORE] Simplify REST API resource structure.
With the new UI store, the API resource classes have a lot less code,
since there's no need for complicated translations between the UI
types and the API types. So the code ended up with a bunch of files
with a single method declared in them.

This change re-structures the API code so that it uses less classes;
mainly, most sub-resources were removed, and the code to deal with
single-attempt and multi-attempt apps was simplified.

The only change was the addition of a method to return a single
attempt's information; that was missing in the old API, so trying
to retrieve "/v1/applications/appId/attemptId" would result in a
404 even if the attempt existed (and URIs under that one would
return valid data).

The streaming API resources also overtook the same treatment, even
though the data is not stored in the new UI store.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #19748 from vanzin/SPARK-20649.
2017-11-15 15:41:53 -06:00
Santiago Saavedra 5ebdcd185f [SPARK-22294][DEPLOY] Reset spark.driver.bindAddress when starting a Checkpoint
## What changes were proposed in this pull request?

It seems that recovering from a checkpoint can replace the old
driver and executor IP addresses, as the workload can now be taking
place in a different cluster configuration. It follows that the
bindAddress for the master may also have changed. Thus we should not be
keeping the old one, and instead be added to the list of properties to
reset and recreate from the new environment.

## How was this patch tested?

This patch was tested via manual testing on AWS, using the experimental (not yet merged) Kubernetes scheduler, which uses bindAddress to bind to a Kubernetes service (and thus was how I first encountered the bug too), but it is not a code-path related to the scheduler and this may have slipped through when merging SPARK-4563.

Author: Santiago Saavedra <ssaavedra@openshine.com>

Closes #19427 from ssaavedra/fix-checkpointing-master.
2017-11-10 10:57:58 -08:00
Sean Owen 51debf8b1f [SPARK-14540][BUILD] Support Scala 2.12 closures and Java 8 lambdas in ClosureCleaner (step 0)
## What changes were proposed in this pull request?

Preliminary changes to get ClosureCleaner to work with Scala 2.12. Makes many usages just work, but not all. This does _not_ resolve the JIRA.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #19675 from srowen/SPARK-14540.0.
2017-11-08 10:24:40 +00:00
Tristan Stevens fe258a7963 [SPARK-22429][STREAMING] Streaming checkpointing code does not retry after failure
## What changes were proposed in this pull request?

SPARK-14930/SPARK-13693 put in a change to set the fs object to null after a failure, however the retry loop does not include initialization. Moved fs initialization inside the retry while loop to aid recoverability.

## How was this patch tested?

Passes all existing unit tests.

Author: Tristan Stevens <tristan@cloudera.com>

Closes #19645 from tmgstevens/SPARK-22429.
2017-11-05 09:10:40 +00:00
xubo245 7a8412352e [SPARK-22423][SQL] Scala test source files like TestHiveSingleton.scala should be in scala source root
## What changes were proposed in this pull request?

  Scala test source files like TestHiveSingleton.scala should be in scala source root

## How was this patch tested?

Just move scala file from java directory to scala directory
No new test case in this PR.

```
	renamed:    mllib/src/test/java/org/apache/spark/ml/util/IdentifiableSuite.scala -> mllib/src/test/scala/org/apache/spark/ml/util/IdentifiableSuite.scala
	renamed:    streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala -> streaming/src/test/scala/org/apache/spark/streaming/JavaTestUtils.scala
	renamed:    streaming/src/test/java/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala -> streaming/src/test/scala/org/apache/spark/streaming/api/java/JavaStreamingListenerWrapperSuite.scala
	renamed:   sql/hive/src/test/java/org/apache/spark/sql/hive/test/TestHiveSingleton.scala  sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala
```

Author: xubo245 <601450868@qq.com>

Closes #19639 from xubo245/scalaDirectory.
2017-11-04 11:51:10 +00:00
ZouChenjun 882079f5c6 [SPARK-22243][DSTREAM] spark.yarn.jars should reload from config when checkpoint recovery
## What changes were proposed in this pull request?
the previous [PR](https://github.com/apache/spark/pull/19469) is deleted by mistake.
the solution is straight forward.
adding  "spark.yarn.jars" to propertiesToReload so this property will load from config.

## How was this patch tested?

manual tests

Author: ZouChenjun <zouchenjun@youzan.com>

Closes #19637 from ChenjunZou/checkpoint-yarn-jars.
2017-11-02 11:06:37 -07:00
Stavros Kontopoulos b2463fad71 [SPARK-22145][MESOS] fix supervise with checkpointing on mesos
## What changes were proposed in this pull request?

- Fixes the issue with the frameworkId being recovered by checkpointed data overwriting the one sent by the dipatcher.
- Keeps submission driver id as the only index for all data structures in the dispatcher.
Allocates a different task id per driver retry to satisfy the mesos requirements. Check the relevant ticket for the details on that.
## How was this patch tested?

Manually tested this with DC/OS 1.10. Launched a streaming job with checkpointing to hdfs, made the driver fail several times and observed behavior:
![image](https://user-images.githubusercontent.com/7945591/30940500-f7d2a744-a3e9-11e7-8c56-f2ccbb271e80.png)

![image](https://user-images.githubusercontent.com/7945591/30940550-19bc15de-a3ea-11e7-8a11-f48abfe36720.png)

![image](https://user-images.githubusercontent.com/7945591/30940524-083ea308-a3ea-11e7-83ae-00d3fa17b928.png)

![image](https://user-images.githubusercontent.com/7945591/30940579-2f0fb242-a3ea-11e7-82f9-86179da28b8c.png)

![image](https://user-images.githubusercontent.com/7945591/30940591-3b561b0e-a3ea-11e7-9dbd-e71912bb2ef3.png)

![image](https://user-images.githubusercontent.com/7945591/30940605-49c810ca-a3ea-11e7-8af5-67930851fd38.png)

![image](https://user-images.githubusercontent.com/7945591/30940631-59f4a288-a3ea-11e7-88cb-c3741b72bb13.png)

![image](https://user-images.githubusercontent.com/7945591/30940642-62346c9e-a3ea-11e7-8935-82e494925f67.png)

![image](https://user-images.githubusercontent.com/7945591/30940653-6c46d53c-a3ea-11e7-8dd1-5840d484d28c.png)

Author: Stavros Kontopoulos <st.kontopoulos@gmail.com>

Closes #19374 from skonto/fix_retry.
2017-11-02 13:25:48 +00:00
Sean Owen 6c6950839d [SPARK-22322][CORE] Update FutureAction for compatibility with Scala 2.12 Future
## What changes were proposed in this pull request?

Scala 2.12's `Future` defines two new methods to implement, `transform` and `transformWith`. These can be implemented naturally in Spark's `FutureAction` extension and subclasses, but, only in terms of the new methods that don't exist in Scala 2.11. To support both at the same time, reflection is used to implement these.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #19561 from srowen/SPARK-22322.
2017-10-25 12:51:20 +01: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
guoxiaolong 10e37f6eb6 [UI][STREAMING] Modify the title, 'Records' instead of 'Input Size'
## What changes were proposed in this pull request?
Spark Streaming is processing data should be record, so the title should be 'Records', and should not be 'Input Size'.

Fix before:
![1](https://user-images.githubusercontent.com/26266482/30099599-c64d4a8a-9318-11e7-8a8d-1ca99b409323.png)

Fix after:
![2](https://user-images.githubusercontent.com/26266482/30099609-cd4df7d0-9318-11e7-8a27-dbaec6797bb1.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 #19144 from guoxiaolongzte/streamingUI.
2017-09-22 11:51:57 +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
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
hyukjinkwon 7f3c6ff4ff [SPARK-21903][BUILD] Upgrade scalastyle to 1.0.0.
## What changes were proposed in this pull request?

1.0.0 fixes an issue with import order, explicit type for public methods, line length limitation and comment validation:

```
[error] .../spark/repl/scala-2.11/src/main/scala/org/apache/spark/repl/Main.scala:50:16: Are you sure you want to println? If yes, wrap the code block with
[error]       // scalastyle:off println
[error]       println(...)
[error]       // scalastyle:on println
[error] .../spark/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala:49: File line length exceeds 100 characters
[error] .../spark/repl/scala-2.11/src/main/scala/org/apache/spark/repl/SparkILoop.scala:22:21: Are you sure you want to println? If yes, wrap the code block with
[error]       // scalastyle:off println
[error]       println(...)
[error]       // scalastyle:on println
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:35:6: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:51:6: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:93:15: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:98:15: Public method must have explicit type
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:47:2: Insert a space after the start of the comment
[error] .../spark/streaming/src/test/java/org/apache/spark/streaming/JavaTestUtils.scala:26:43: JavaDStream should come before JavaDStreamLike.
```

This PR also fixes the workaround added in SPARK-16877 for `org.scalastyle.scalariform.OverrideJavaChecker` feature, added from 0.9.0.

## How was this patch tested?

Manually tested.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #19116 from HyukjinKwon/scalastyle-1.0.0.
2017-09-05 19:40:05 +09: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
Marcelo Vanzin 3f958a9992 [SPARK-21731][BUILD] Upgrade scalastyle to 0.9.
This version fixes a few issues in the import order checker; it provides
better error messages, and detects more improper ordering (thus the need
to change a lot of files in this patch). The main fix is that it correctly
complains about the order of packages vs. classes.

As part of the above, I moved some "SparkSession" import in ML examples
inside the "$example on$" blocks; that didn't seem consistent across
different source files to start with, and avoids having to add more on/off blocks
around specific imports.

The new scalastyle also seems to have a better header detector, so a few
license headers had to be updated to match the expected indentation.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #18943 from vanzin/SPARK-21731.
2017-08-15 13:59:00 -07:00
shaofei007 60e9b2bdd5 [SPARK-21357][DSTREAMS] FileInputDStream not remove out of date RDD
## What changes were proposed in this pull request?

```DStreams
         class FileInputDStream

 [line 162]   protected[streaming] override def clearMetadata(time: Time) {
    batchTimeToSelectedFiles.synchronized {
      val oldFiles = batchTimeToSelectedFiles.filter(_._1 < (time - rememberDuration))
      batchTimeToSelectedFiles --= oldFiles.keys

```
The above code does not remove the old generatedRDDs. "super.clearMetadata(time)" was added to the beginning of clearMetadata to remove the old generatedRDDs.

## How was this patch tested?

At the end of clearMetadata, the testing code (print the number of generatedRDDs) was added to check the old RDDS were removed manually.

Author: shaofei007 <1427357147@qq.com>
Author: Fei Shao <1427357147@qq.com>

Closes #18718 from shaofei007/master.
2017-07-29 13:27:39 +01:00
Sean Owen e26dac5feb [SPARK-21415] Triage scapegoat warnings, part 1
## What changes were proposed in this pull request?

Address scapegoat warnings for:
- BigDecimal double constructor
- Catching NPE
- Finalizer without super
- List.size is O(n)
- Prefer Seq.empty
- Prefer Set.empty
- reverse.map instead of reverseMap
- Type shadowing
- Unnecessary if condition.
- Use .log1p
- Var could be val

In some instances like Seq.empty, I avoided making the change even where valid in test code to keep the scope of the change smaller. Those issues are concerned with performance and it won't matter for tests.

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #18635 from srowen/Scapegoat1.
2017-07-18 08:47:17 +01:00
saturday_s e92ffe6f17 [SPARK-19688][STREAMING] Not to read spark.yarn.credentials.file from checkpoint.
## What changes were proposed in this pull request?

Reload the `spark.yarn.credentials.file` property when restarting a streaming application from checkpoint.

## How was this patch tested?

Manual tested with 1.6.3 and 2.1.1.
I didn't test this with master because of some compile problems, but I think it will be the same result.

## Notice

This should be merged into maintenance branches too.

jira: [SPARK-21008](https://issues.apache.org/jira/browse/SPARK-21008)

Author: saturday_s <shi.indetail@gmail.com>

Closes #18230 from saturday-shi/SPARK-21008.
2017-06-19 10:24:29 -07:00
hyukjinkwon eb3ea3a083 [SPARK-20935][STREAMING] Always close WriteAheadLog and make it idempotent
## What changes were proposed in this pull request?

This PR proposes to stop `ReceiverTracker` to close `WriteAheadLog` whenever it is and make `WriteAheadLog` and its implementations idempotent.

## How was this patch tested?

Added a test in `WriteAheadLogSuite`. Note that  the added test looks passing even if it closes twice (namely even without the changes in `FileBasedWriteAheadLog` and `BatchedWriteAheadLog`. It looks both are already idempotent but this is a rather sanity check.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #18224 from HyukjinKwon/streaming-closing.
2017-06-11 09:54:57 +01:00
Josh Rosen 2a23cdd078 [SPARK-20863] Add metrics/instrumentation to LiveListenerBus
## What changes were proposed in this pull request?

This patch adds Coda Hale metrics for instrumenting the `LiveListenerBus` in order to track the number of events received, dropped, and processed. In addition, it adds per-SparkListener-subclass timers to track message processing time. This is useful for identifying when slow third-party SparkListeners cause performance bottlenecks.

See the new `LiveListenerBusMetrics` for a complete description of the new metrics.

## How was this patch tested?

New tests in SparkListenerSuite, including a test to ensure proper counting of dropped listener events.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #18083 from JoshRosen/listener-bus-metrics.
2017-06-08 18:08:25 -07:00
Wil Selwood b6f2017a6a [MINOR] document edge case of updateFunc usage
## What changes were proposed in this pull request?

Include documentation of the fact that the updateFunc is sometimes called with no new values. This is documented in the main documentation here: https://spark.apache.org/docs/latest/streaming-programming-guide.html#updatestatebykey-operation however from the docs included with the code it is not clear that this is the case.

## How was this patch tested?

PR only changes comments. Confirmed code still builds.

Author: Wil Selwood <wil.selwood@sa.catapult.org.uk>

Closes #18088 from wselwood/note-edge-case-in-docs.
2017-05-26 11:29:52 +01:00
Shixiong Zhu 324a904d8e [SPARK-13747][CORE] Add ThreadUtils.awaitReady and disallow Await.ready
## What changes were proposed in this pull request?

Add `ThreadUtils.awaitReady` similar to `ThreadUtils.awaitResult` and disallow `Await.ready`.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #17763 from zsxwing/awaitready.
2017-05-17 17:21:46 -07:00
Sean Owen fc8a2b6ee6 [SPARK-20554][BUILD] Remove usage of scala.language.reflectiveCalls
## What changes were proposed in this pull request?

Remove uses of scala.language.reflectiveCalls that are either unnecessary or probably resulting in more complex code. This turned out to be less significant than I thought, but, still worth a touch-up.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17949 from srowen/SPARK-20554.
2017-05-12 09:55:04 +01:00
NICHOLAS T. MARION b512233a45 [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities
## What changes were proposed in this pull request?

Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these functions at any point that getParameter is called against a HttpServletRequest.

## How was this patch tested?

Unit tests, IBM Security AppScan Standard no longer showing vulnerabilities, manual verification of WebUI pages.

Author: NICHOLAS T. MARION <nmarion@us.ibm.com>

Closes #17686 from n-marion/xss-fix.
2017-05-10 10:59:57 +01:00
Josh Rosen f44c8a843c [SPARK-20453] Bump master branch version to 2.3.0-SNAPSHOT
This patch bumps the master branch version to `2.3.0-SNAPSHOT`.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #17753 from JoshRosen/SPARK-20453.
2017-04-24 21:48:04 -07:00
hyukjinkwon ceaf77ae43 [SPARK-18692][BUILD][DOCS] Test Java 8 unidoc build on Jenkins
## What changes were proposed in this pull request?

This PR proposes to run Spark unidoc to test Javadoc 8 build as Javadoc 8 is easily re-breakable.

There are several problems with it:

- It introduces little extra bit of time to run the tests. In my case, it took 1.5 mins more (`Elapsed :[94.8746569157]`). How it was tested is described in "How was this patch tested?".

- > One problem that I noticed was that Unidoc appeared to be processing test sources: if we can find a way to exclude those from being processed in the first place then that might significantly speed things up.

  (see  joshrosen's [comment](https://issues.apache.org/jira/browse/SPARK-18692?focusedCommentId=15947627&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15947627))

To complete this automated build, It also suggests to fix existing Javadoc breaks / ones introduced by test codes as described above.

There fixes are similar instances that previously fixed. Please refer https://github.com/apache/spark/pull/15999 and https://github.com/apache/spark/pull/16013

Note that this only fixes **errors** not **warnings**. Please see my observation https://github.com/apache/spark/pull/17389#issuecomment-288438704 for spurious errors by warnings.

## How was this patch tested?

Manually via `jekyll build` for building tests. Also, tested via running `./dev/run-tests`.

This was tested via manually adding `time.time()` as below:

```diff
     profiles_and_goals = build_profiles + sbt_goals

     print("[info] Building Spark unidoc (w/Hive 1.2.1) using SBT with these arguments: ",
           " ".join(profiles_and_goals))

+    import time
+    st = time.time()
     exec_sbt(profiles_and_goals)
+    print("Elapsed :[%s]" % str(time.time() - st))
```

produces

```
...
========================================================================
Building Unidoc API Documentation
========================================================================
...
[info] Main Java API documentation successful.
...
Elapsed :[94.8746569157]
...

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17477 from HyukjinKwon/SPARK-18692.
2017-04-12 12:38:48 +01:00
Sean Owen a26e3ed5e4 [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String toLowerCase "Turkish locale bug" causes Spark problems
## What changes were proposed in this pull request?

Add Locale.ROOT to internal calls to String `toLowerCase`, `toUpperCase`, to avoid inadvertent locale-sensitive variation in behavior (aka the "Turkish locale problem").

The change looks large but it is just adding `Locale.ROOT` (the locale with no country or language specified) to every call to these methods.

## How was this patch tested?

Existing tests.

Author: Sean Owen <sowen@cloudera.com>

Closes #17527 from srowen/SPARK-20156.
2017-04-10 20:11:56 +01:00
Jacek Laskowski 0197262a35 [DOCS] Docs-only improvements
…adoc

## What changes were proposed in this pull request?

Use recommended values for row boundaries in Window's scaladoc, i.e. `Window.unboundedPreceding`, `Window.unboundedFollowing`, and `Window.currentRow` (that were introduced in 2.1.0).

## How was this patch tested?

Local build

Author: Jacek Laskowski <jacek@japila.pl>

Closes #17417 from jaceklaskowski/window-expression-scaladoc.
2017-03-30 16:07:27 +01:00
Marcelo Vanzin b56ad2b1ec [SPARK-19556][CORE] Do not encrypt block manager data in memory.
This change modifies the way block data is encrypted to make the more
common cases faster, while penalizing an edge case. As a side effect
of the change, all data that goes through the block manager is now
encrypted only when needed, including the previous path (broadcast
variables) where that did not happen.

The way the change works is by not encrypting data that is stored in
memory; so if a serialized block is in memory, it will only be encrypted
once it is evicted to disk.

The penalty comes when transferring that encrypted data from disk. If the
data ends up in memory again, it is as efficient as before; but if the
evicted block needs to be transferred directly to a remote executor, then
there's now a performance penalty, since the code now uses a custom
FileRegion implementation to decrypt the data before transferring.

This also means that block data transferred between executors now is
not encrypted (and thus relies on the network library encryption support
for secrecy). Shuffle blocks are still transferred in encrypted form,
since they're handled in a slightly different way by the code. This also
keeps compatibility with existing external shuffle services, which transfer
encrypted shuffle blocks, and avoids having to make the external service
aware of encryption at all.

The serialization and deserialization APIs in the SerializerManager now
do not do encryption automatically; callers need to explicitly wrap their
streams with an appropriate crypto stream before using those.

As a result of these changes, some of the workarounds added in SPARK-19520
are removed here.

Testing: a new trait ("EncryptionFunSuite") was added that provides an easy
way to run a test twice, with encryption on and off; broadcast, block manager
and caching tests were modified to use this new trait so that the existing
tests exercise both encrypted and non-encrypted paths. I also ran some
applications with encryption turned on to verify that they still work,
including streaming tests that failed without the fix for SPARK-19520.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #17295 from vanzin/SPARK-19556.
2017-03-29 20:27:41 +08:00
Eric Liang 8e558041aa [SPARK-19820][CORE] Add interface to kill tasks w/ a reason
This commit adds a killTaskAttempt method to SparkContext, to allow users to
kill tasks so that they can be re-scheduled elsewhere.

This also refactors the task kill path to allow specifying a reason for the task kill. The reason is propagated opaquely through events, and will show up in the UI automatically as `(N killed: $reason)` and `TaskKilled: $reason`. Without this change, there is no way to provide the user feedback through the UI.

Currently used reasons are "stage cancelled", "another attempt succeeded", and "killed via SparkContext.killTask". The user can also specify a custom reason through `SparkContext.killTask`.

cc rxin

In the stage overview UI the reasons are summarized:
![1](https://cloud.githubusercontent.com/assets/14922/23929209/a83b2862-08e1-11e7-8b3e-ae1967bbe2e5.png)

Within the stage UI you can see individual task kill reasons:
![2](https://cloud.githubusercontent.com/assets/14922/23929200/9a798692-08e1-11e7-8697-72b27ad8a287.png)

Existing tests, tried killing some stages in the UI and verified the messages are as expected.

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekl@google.com>

Closes #17166 from ericl/kill-reason.
2017-03-23 23:30:44 -07:00
uncleGen e29a74d5b1 [DOCS][SS] fix structured streaming python example
## What changes were proposed in this pull request?

- SS python example: `TypeError: 'xxx' object is not callable`
- some other doc issue.

## How was this patch tested?

Jenkins.

Author: uncleGen <hustyugm@gmail.com>

Closes #17257 from uncleGen/docs-ss-python.
2017-03-12 08:29:37 +00:00
uncleGen 207067ead6 [SPARK-19822][TEST] CheckpointSuite.testCheckpointedOperation: should not filter checkpointFilesOfLatestTime with the PATH string.
## What changes were proposed in this pull request?

https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73800/testReport/

```
sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException: The code
passed to eventually never returned normally. Attempted 617 times over 10.003740484 seconds.
Last failure message: 8 did not equal 2.
	at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420)
	at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:336)
	at org.scalatest.concurrent.Eventually$.eventually(Eventually.scala:478)
	at org.apache.spark.streaming.DStreamCheckpointTester$class.generateOutput(CheckpointSuite
.scala:172)
	at org.apache.spark.streaming.CheckpointSuite.generateOutput(CheckpointSuite.scala:211)
```

the check condition is:

```
val checkpointFilesOfLatestTime = Checkpoint.getCheckpointFiles(checkpointDir).filter {
     _.toString.contains(clock.getTimeMillis.toString)
}
// Checkpoint files are written twice for every batch interval. So assert that both
// are written to make sure that both of them have been written.
assert(checkpointFilesOfLatestTime.size === 2)
```

the path string may contain the `clock.getTimeMillis.toString`, like `3500` :

```
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-1500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-2500
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3000
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500.bk
file:/root/dev/spark/assembly/CheckpointSuite/spark-20035007-9891-4fb6-91c1-cc15b7ccaf15/checkpoint-3500
                                                       ▲▲▲▲
```

so we should only check the filename, but not the whole path.

## How was this patch tested?

Jenkins.

Author: uncleGen <hustyugm@gmail.com>

Closes #17167 from uncleGen/flaky-CheckpointSuite.
2017-03-05 18:17:30 -08:00
hyukjinkwon 4ba9c6c453 [MINOR][BUILD] Fix lint-java breaks in Java
## What changes were proposed in this pull request?

This PR proposes to fix the lint-breaks as below:

```
[ERROR] src/test/java/org/apache/spark/network/TransportResponseHandlerSuite.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.network.buffer.ManagedBuffer.
[ERROR] src/main/java/org/apache/spark/unsafe/types/UTF8String.java:[156,10] (modifier) ModifierOrder: 'Nonnull' annotation modifier does not precede non-annotation modifiers.
[ERROR] src/main/java/org/apache/spark/SparkFirehoseListener.java:[122] (sizes) LineLength: Line is longer than 100 characters (found 105).
[ERROR] src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java:[164,78] (coding) OneStatementPerLine: Only one statement per line allowed.
[ERROR] src/test/java/test/org/apache/spark/JavaAPISuite.java:[1157] (sizes) LineLength: Line is longer than 100 characters (found 121).
[ERROR] src/test/java/org/apache/spark/streaming/JavaMapWithStateSuite.java:[149] (sizes) LineLength: Line is longer than 100 characters (found 113).
[ERROR] src/test/java/test/org/apache/spark/streaming/Java8APISuite.java:[146] (sizes) LineLength: Line is longer than 100 characters (found 122).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[32,8] (imports) UnusedImports: Unused import - org.apache.spark.streaming.Time.
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[611] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java:[1317] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetAggregatorSuite.java:[91] (sizes) LineLength: Line is longer than 100 characters (found 102).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[113] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[164] (sizes) LineLength: Line is longer than 100 characters (found 110).
[ERROR] src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java:[212] (sizes) LineLength: Line is longer than 100 characters (found 114).
[ERROR] src/test/java/org/apache/spark/mllib/tree/JavaDecisionTreeSuite.java:[36] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java:[26,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[20,8] (imports) UnusedImports: Unused import - com.amazonaws.regions.RegionUtils.
[ERROR] src/test/java/org/apache/spark/streaming/kinesis/JavaKinesisStreamSuite.java:[94] (sizes) LineLength: Line is longer than 100 characters (found 103).
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[30,8] (imports) UnusedImports: Unused import - org.apache.spark.sql.api.java.UDF1.
[ERROR] src/main/java/org/apache/spark/examples/ml/JavaTokenizerExample.java:[72] (sizes) LineLength: Line is longer than 100 characters (found 104).
[ERROR] src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java:[121] (sizes) LineLength: Line is longer than 100 characters (found 101).
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[28,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaRDD.
[ERROR] src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java:[29,8] (imports) UnusedImports: Unused import - org.apache.spark.api.java.JavaSparkContext.
```

## How was this patch tested?

Manually via

```bash
./dev/lint-java
```

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #17072 from HyukjinKwon/java-lint.
2017-02-27 08:44:26 +00:00
Marcelo Vanzin 17d83e1ee5 [SPARK-19652][UI] Do auth checks for REST API access.
The REST API has a security filter that performs auth checks
based on the UI root's security manager. That works fine when
the UI root is the app's UI, but not when it's the history server.

In the SHS case, all users would be allowed to see all applications
through the REST API, even if the UI itself wouldn't be available
to them.

This change adds auth checks for each app access through the API
too, so that only authorized users can see the app's data.

The change also modifies the existing security filter to use
`HttpServletRequest.getRemoteUser()`, which is used in other
places. That is not necessarily the same as the principal's
name; for example, when using Hadoop's SPNEGO auth filter,
the remote user strips the realm information, which then matches
the user name registered as the owner of the application.

I also renamed the UIRootFromServletContext trait to a more generic
name since I'm using it to store more context information now.

Tested manually with an authentication filter enabled.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16978 from vanzin/SPARK-19652.
2017-02-21 16:14:34 -08:00
Sean Owen d0ecca6075
[SPARK-19646][CORE][STREAMING] binaryRecords replicates records in scala API
## What changes were proposed in this pull request?

Use `BytesWritable.copyBytes`, not `getBytes`, because `getBytes` returns the underlying array, which may be reused when repeated reads don't need a different size, as is the case with binaryRecords APIs

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16974 from srowen/SPARK-19646.
2017-02-20 09:02:09 -08:00
Sean Owen 1487c9af20
[SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 features
## What changes were proposed in this pull request?

Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code.

## How was this patch tested?

Jenkins tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16964 from srowen/SPARK-19534.
2017-02-19 09:42:50 -08:00
jinxing ba8912e5f3
[SPARK-19450] Replace askWithRetry with askSync.
## What changes were proposed in this pull request?

`askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated.
As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218):

>askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it.

Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`.

## How was this patch tested?
This PR doesn't change code logic, existing unit test can cover.

Author: jinxing <jinxing@meituan.com>

Closes #16790 from jinxing64/SPARK-19450.
2017-02-19 04:34:07 -08:00
Sean Owen 0e2405490f
[SPARK-19550][BUILD][CORE][WIP] Remove Java 7 support
- Move external/java8-tests tests into core, streaming, sql and remove
- Remove MaxPermGen and related options
- Fix some reflection / TODOs around Java 8+ methods
- Update doc references to 1.7/1.8 differences
- Remove Java 7/8 related build profiles
- Update some plugins for better Java 8 compatibility
- Fix a few Java-related warnings

For the future:

- Update Java 8 examples to fully use Java 8
- Update Java tests to use lambdas for simplicity
- Update Java internal implementations to use lambdas

## How was this patch tested?

Existing tests

Author: Sean Owen <sowen@cloudera.com>

Closes #16871 from srowen/SPARK-19493.
2017-02-16 12:32:45 +00:00
Marcelo Vanzin 0169360ef5 [SPARK-19520][STREAMING] Do not encrypt data written to the WAL.
Spark's I/O encryption uses an ephemeral key for each driver instance.
So driver B cannot decrypt data written by driver A since it doesn't
have the correct key.

The write ahead log is used for recovery, thus needs to be readable by
a different driver. So it cannot be encrypted by Spark's I/O encryption
code.

The BlockManager APIs used by the WAL code to write the data automatically
encrypt data, so changes are needed so that callers can to opt out of
encryption.

Aside from that, the "putBytes" API in the BlockManager does not do
encryption, so a separate situation arised where the WAL would write
unencrypted data to the BM and, when those blocks were read, decryption
would fail. So the WAL code needs to ask the BM to encrypt that data
when encryption is enabled; this code is not optimal since it results
in a (temporary) second copy of the data block in memory, but should be
OK for now until a more performant solution is added. The non-encryption
case should not be affected.

Tested with new unit tests, and by running streaming apps that do
recovery using the WAL data with I/O encryption turned on.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #16862 from vanzin/SPARK-19520.
2017-02-13 14:19:41 -08:00
jinxing c5fcb7f68b [SPARK-19347] ReceiverSupervisorImpl can add block to ReceiverTracker multiple times because of askWithRetry.
## What changes were proposed in this pull request?

`ReceiverSupervisorImpl` on executor side reports block's meta back to `ReceiverTracker` on driver side. In current code, `askWithRetry` is used. However, for `AddBlock`, `ReceiverTracker` is not idempotent, which may result in messages are processed multiple times.

**To reproduce**:

1. Check if it is the first time receiving `AddBlock` in `ReceiverTracker`, if so sleep long enough(say 200 seconds), thus the first RPC call will be timeout in `askWithRetry`, then `AddBlock` will be resent.
2. Rebuild Spark and run following job:
```
  def streamProcessing(): Unit = {
    val conf = new SparkConf()
      .setAppName("StreamingTest")
      .setMaster(masterUrl)
    val ssc = new StreamingContext(conf, Seconds(200))
    val stream = ssc.socketTextStream("localhost", 1234)
    stream.print()
    ssc.start()
    ssc.awaitTermination()
  }
```
**To fix**:

It makes sense to provide a blocking version `ask` in RpcEndpointRef, as mentioned in SPARK-18113 (https://github.com/apache/spark/pull/16503#event-927953218). Because Netty RPC layer will not drop messages. `askWithRetry` is a leftover from akka days. It imposes restrictions on the caller(e.g. idempotency) and other things that people generally don't pay that much attention to when using it.

## How was this patch tested?
Test manually. The scenario described above doesn't happen with this patch.

Author: jinxing <jinxing@meituan.com>

Closes #16690 from jinxing64/SPARK-19347.
2017-02-01 13:54:37 -08:00
hyukjinkwon f1a1f2607d
[SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings in Scala/Java APIs generation
## What changes were proposed in this pull request?

This PR proposes three things as below:

- Support LaTex inline-formula, `\( ... \)` in Scala API documentation
  It seems currently,

  ```
  \( ... \)
  ```

  are rendered as they are, for example,

  <img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png">

  It seems mistakenly more backslashes were added.

- Fix warnings Scaladoc/Javadoc generation
  This PR fixes t two types of warnings as below:

  ```
  [warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException".
  [warn]   /**
  [warn]   ^
  ```

  ```
  [warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution
  [warn]  * `${var}`, `${system:var}` and `${env:var}`.
  [warn]      ^
  ```

- Fix Javadoc8 break
  ```
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found
  [error]    *                          E.g., {link VectorUDT} for vector features.
  [error]                                            ^
  [error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found
  [error]  *                       E.g., {link VectorUDT} for vector features.
  [error]                                       ^
  [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found
  [error]  * Note that, this rule must be run after {link PreprocessTableInsertion}.
  [error]                                                  ^
  ```

## How was this patch tested?

Manually via `sbt unidoc` and `jeykil build`.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16741 from HyukjinKwon/warn-and-break.
2017-02-01 13:26:16 +00:00
Liwei Lin 40a4cfc7c7 [SPARK-19330][DSTREAMS] Also show tooltip for successful batches
## What changes were proposed in this pull request?

### Before
![_streaming_before](https://cloud.githubusercontent.com/assets/15843379/22181462/1e45c20c-e0c8-11e6-831c-8bf69722a4ee.png)

### After
![_streaming_after](https://cloud.githubusercontent.com/assets/15843379/22181464/23f38a40-e0c8-11e6-9a87-e27b1ffb1935.png)

## How was this patch tested?

Manually

Author: Liwei Lin <lwlin7@gmail.com>

Closes #16673 from lw-lin/streaming.
2017-01-24 16:36:17 -08:00
uncleGen a81e336f1e [SPARK-19182][DSTREAM] Optimize the lock in StreamingJobProgressListener to not block UI when generating Streaming jobs
## What changes were proposed in this pull request?

When DStreamGraph is generating a job, it will hold a lock and block other APIs. Because StreamingJobProgressListener (numInactiveReceivers, streamName(streamId: Int), streamIds) needs to call DStreamGraph's methods to access some information, the UI may hang if generating a job is very slow (e.g., talking to the slow Kafka cluster to fetch metadata).
It's better to optimize the locks in DStreamGraph and StreamingJobProgressListener to make the UI not block by job generation.

## How was this patch tested?
existing ut

cc zsxwing

Author: uncleGen <hustyugm@gmail.com>

Closes #16601 from uncleGen/SPARK-19182.
2017-01-18 10:55:31 -08:00
CodingCat f8db8945f2 [SPARK-18905][STREAMING] Fix the issue of removing a failed jobset from JobScheduler.jobSets
## What changes were proposed in this pull request?

the current implementation of Spark streaming considers a batch is completed no matter the results of the jobs (1169db44bc/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala (L203))
Let's consider the following case:
A micro batch contains 2 jobs and they read from two different kafka topics respectively. One of these jobs is failed due to some problem in the user defined logic, after the other one is finished successfully.
1. The main thread in the Spark streaming application will execute the line mentioned above,
2. and another thread (checkpoint writer) will make a checkpoint file immediately after this line is executed.
3. Then due to the current error handling mechanism in Spark Streaming, StreamingContext will be closed (1169db44bc/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala (L214))
the user recovers from the checkpoint file, and because the JobSet containing the failed job has been removed (taken as completed) before the checkpoint is constructed, the data being processed by the failed job would never be reprocessed

This PR fix it by removing jobset from JobScheduler.jobSets only when all jobs in a jobset are successfully finished

## How was this patch tested?

existing tests

Author: CodingCat <zhunansjtu@gmail.com>
Author: Nan Zhu <zhunansjtu@gmail.com>

Closes #16542 from CodingCat/SPARK-18905.
2017-01-16 18:33:20 -08:00
hyukjinkwon 4e27578faa
[SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed due to path and resource-not-closed problems on Windows
## What changes were proposed in this pull request?

This PR proposes to fix all the test failures identified by testing with AppVeyor.

**Scala - aborted tests**

```
WindowQuerySuite:
  Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.WindowQuerySuite *** ABORTED *** (156 milliseconds)
   org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive   argetscala-2.11   est-classesdatafilespart_tiny.txt;

OrcSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.orc.OrcSourceSuite *** ABORTED *** (62 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

ParquetMetastoreSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetMetastoreSuite *** ABORTED *** (4 seconds, 703 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

ParquetSourceSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetSourceSuite *** ABORTED *** (3 seconds, 907 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark  arget mpspark-581a6575-454f-4f21-a516-a07f95266143;

KafkaRDDSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaRDDSuite *** ABORTED *** (5 seconds, 212 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-4722304d-213e-4296-b556-951df1a46807

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.DirectKafkaStreamSuite *** ABORTED *** (7 seconds, 127 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d0d3eba7-4215-4e10-b40e-bb797e89338e
   at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010)

ReliableKafkaStreamSuite
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.ReliableKafkaStreamSuite *** ABORTED *** (5 seconds, 498 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d33e45a0-287e-4bed-acae-ca809a89d888

KafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaStreamSuite *** ABORTED *** (2 seconds, 892 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-59c9d169-5a56-4519-9ef0-cefdbd3f2e6c

KafkaClusterSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaClusterSuite *** ABORTED *** (1 second, 690 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-3ef402b0-8689-4a60-85ae-e41e274f179d

DirectKafkaStreamSuite:
 Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.DirectKafkaStreamSuite *** ABORTED *** (59 seconds, 626 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-426107da-68cf-4d94-b0d6-1f428f1c53f6

KafkaRDDSuite:
Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.KafkaRDDSuite *** ABORTED *** (2 minutes, 6 seconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-b9ce7929-5dae-46ab-a0c4-9ef6f58fbc2
```

**Java - failed tests**

```
Test org.apache.spark.streaming.kafka.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-1cee32f4-4390-4321-82c9-e8616b3f0fb0, took 9.61 sec

Test org.apache.spark.streaming.kafka.JavaKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-f42695dd-242e-4b07-847c-f299b8e4676e, took 11.797 sec

Test org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-85c0d062-78cf-459c-a2dd-7973572101ce, took 1.581 sec

Test org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-49eb6b5c-8366-47a6-83f2-80c443c48280, took 17.895 sec

org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-898cf826-d636-4b1c-a61a-c12a364c02e7, took 8.858 sec
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
 - insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (828 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-bb6337b9-4f99-45ab-ad2c-a787ab965c09

 - SPARK-18635 special chars in partition values - partition management true *** FAILED *** (5 seconds, 360 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - SPARK-18635 special chars in partition values - partition management false *** FAILED *** (141 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
UtilsSuite:
 - reading offset bytes of a file (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ecb2b7d5-db8b-43a7-b268-1bf242b5a491

 - reading offset bytes across multiple files (compressed) *** FAILED *** (0 milliseconds)
   java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-25cc47a8-1faa-4da5-8862-cf174df63ce0
```

```
StatisticsSuite:
 - MetastoreRelations fallback to HDFS for size estimation *** FAILED *** (110 milliseconds)
   org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'csv_table' not found in database 'default';
```

```
SQLQuerySuite:
 - permanent UDTF *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count_temp'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 24

 - describe functions - user defined functions *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7

 - CTAS without serde with location *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-ed673d73-edfc-404e-829e-2e2b9725d94e/c1

 - derived from Hive query file: drop_database_removes_partition_dirs.q *** FAILED *** (47 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_database_removes_partition_dirs_table

 - derived from Hive query file: drop_table_removes_partition_dirs.q *** FAILED *** (0 milliseconds)
   java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_table_removes_partition_dirs_table2

 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH *** FAILED *** (109 milliseconds)
   java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/sql/hive/projectsspark	arget	mpspark-1a122f8c-dfb3-46c4-bab1-f30764baee0e/*part-r*
```

```
HiveDDLSuite:
 - drop external tables in default database *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - add/drop partitions - external table *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - create/drop database - location without pre-created directory *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - create/drop database - location with pre-created directory *** FAILED *** (32 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - drop database containing tables - CASCADE *** FAILED *** (94 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - CASCADE *** FAILED *** (63 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop database containing tables - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - drop an empty database - RESTRICT *** FAILED *** (47 milliseconds)
   CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675)

 - CREATE TABLE LIKE an external data source table *** FAILED *** (140 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-c5eba16d-07ae-4186-95bb-21c5811cf888;

 - CREATE TABLE LIKE an external Hive serde table *** FAILED *** (16 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - desc table for data source table - no user-defined schema *** FAILED *** (125 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e8bf5bf5-721a-4cbe-9d6	at scala.collection.immutable.List.foreach(List.scala:381)d-5543a8301c1d;
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table *** FAILED *** (16 milliseconds)
   java.lang.IllegalArgumentException: Can not create a Path from an empty string
```

```
ShowCreateTableSuite:
 - simple external hive table *** FAILED *** (0 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files *** FAILED *** (313 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: partitioned pruned table reports only selected files *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-311f45f8-d064-4023-a4bb-e28235bff64d;

 - hive table: lazy partition pruning reads only necessary partition data *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: lazy partition pruning reads only necessary partition data *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-fde874ca-66bd-4d0b-a40f-a043b65bf957;

 - hive table: lazy partition pruning with file status caching enabled *** FAILED *** (188 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: lazy partition pruning with file status caching enabled *** FAILED *** (187 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-e6d20183-dd68-4145-acbe-4a509849accd;

 - hive table: file status caching respects refresh table and refreshByPath *** FAILED *** (172 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: file status caching respects refresh table and refreshByPath *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-8b2c9651-2adf-4d58-874f-659007e21463;

 - hive table: file status cache respects size limit *** FAILED *** (219 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: file status cache respects size limit *** FAILED *** (171 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-7835ab57-cb48-4d2c-bb1d-b46d5a4c47e4;

 - datasource table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-20598d76-c004-42a7-8061-6c56f0eda5e2;

 - hive table: table setup does not scan filesystem *** FAILED *** (266 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - hive table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 281 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 422 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4cfed321-4d1d-4b48-8d34-5c169afff383;

 - hive table: files read and cached when filesource partition management is off *** FAILED *** (234 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);

 - datasource table: all partition data cached in memory when partition management is off *** FAILED *** (203 milliseconds)
   org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark	arget	mpspark-4bcc0398-15c9-4f6a-811e-12d40f3eec12;

 - SPARK-18700: table loaded only once even when resolved concurrently *** FAILED *** (1 second, 266 milliseconds)
   org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string);
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it *** FAILED *** (2 seconds, 94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: define a UDF and use it *** FAILED *** (281 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - permanent Hive UDF: use a already defined permanent function *** FAILED *** (718 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8368: includes jars passed in through --jars *** FAILED *** (3 seconds, 521 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8020: set sql conf in spark conf *** FAILED *** (0 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-8489: MissingRequirementError during reflection *** FAILED *** (94 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-9757 Persist Parquet relation with decimal column *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-11009 fix wrong result of Window function in cluster mode *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-14244 fix window partition size attribute binding failure *** FAILED *** (78 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set spark.sql.warehouse.dir *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - set hive.metastore.warehouse.dir *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-16901: set javax.jdo.option.ConnectionURL *** FAILED *** (16 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified

 - SPARK-18360: default table path of tables in default database should depend on the location of default database *** FAILED *** (15 milliseconds)
   java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified
```

```
UtilsSuite:
 - resolveURIs with multiple paths *** FAILED *** (0 milliseconds)
   ".../jar3,file:/C:/pi.py[%23]py.pi,file:/C:/path%..." did not equal ".../jar3,file:/C:/pi.py[#]py.pi,file:/C:/path%..." (UtilsSuite.scala:468)
```

```
CheckpointSuite:
 - recovery with file input stream *** FAILED *** (10 seconds, 205 milliseconds)
   The code passed to eventually never returned normally. Attempted 660 times over 10.014272499999999 seconds. Last failure message: Unexpected internal error near index 1
   \
    ^. (CheckpointSuite.scala:680)
```

## How was this patch tested?

Manually via AppVeyor as below:

**Scala - aborted tests**

```
WindowQuerySuite - all passed
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
ParquetMetastoreSuite - all passed
ParquetSourceSuite - all passed
KafkaRDDSuite - all passed
DirectKafkaStreamSuite - all passed
ReliableKafkaStreamSuite - all passed
KafkaStreamSuite - all passed
KafkaClusterSuite - all passed
DirectKafkaStreamSuite - all passed
KafkaRDDSuite - all passed
```

**Java - failed tests**

```
org.apache.spark.streaming.kafka.JavaKafkaRDDSuite - all passed
org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka.JavaKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite - all passed
org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite - all passed
```

**Scala - failed tests**

```
PartitionProviderCompatibilitySuite:
- insert overwrite partition of new datasource table overwrites just partition (1 second, 953 milliseconds)
- SPARK-18635 special chars in partition values - partition management true (6 seconds, 31 milliseconds)
- SPARK-18635 special chars in partition values - partition management false (4 seconds, 578 milliseconds)
```

```
UtilsSuite:
- reading offset bytes of a file (compressed) (203 milliseconds)
- reading offset bytes across multiple files (compressed) (0 milliseconds)
```

```
StatisticsSuite:
- MetastoreRelations fallback to HDFS for size estimation (94 milliseconds)
```

```
SQLQuerySuite:
 - permanent UDTF (407 milliseconds)
 - describe functions - user defined functions (441 milliseconds)
 - CTAS without serde with location (2 seconds, 831 milliseconds)
 - derived from Hive query file: drop_database_removes_partition_dirs.q (734 milliseconds)
 - derived from Hive query file: drop_table_removes_partition_dirs.q (563 milliseconds)
 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH (453 milliseconds)
```

```
HiveDDLSuite:
 - drop external tables in default database (3 seconds, 5 milliseconds)
 - add/drop partitions - external table (2 seconds, 750 milliseconds)
 - create/drop database - location without pre-created directory (500 milliseconds)
 - create/drop database - location with pre-created directory (407 milliseconds)
 - drop database containing tables - CASCADE (453 milliseconds)
 - drop an empty database - CASCADE (375 milliseconds)
 - drop database containing tables - RESTRICT (328 milliseconds)
 - drop an empty database - RESTRICT (391 milliseconds)
 - CREATE TABLE LIKE an external data source table (953 milliseconds)
 - CREATE TABLE LIKE an external Hive serde table (3 seconds, 782 milliseconds)
 - desc table for data source table - no user-defined schema (1 second, 150 milliseconds)
```

```
MetastoreDataSourcesSuite
 - CTAS: persisted bucketed data source table (875 milliseconds)
```

```
ShowCreateTableSuite:
 - simple external hive table (78 milliseconds)
```

```
PartitionedTablePerfStatsSuite:
 - hive table: partitioned pruned table reports only selected files (1 second, 109 milliseconds)
- datasource table: partitioned pruned table reports only selected files (860 milliseconds)
 - hive table: lazy partition pruning reads only necessary partition data (859 milliseconds)
 - datasource table: lazy partition pruning reads only necessary partition data (1 second, 219 milliseconds)
 - hive table: lazy partition pruning with file status caching enabled (875 milliseconds)
 - datasource table: lazy partition pruning with file status caching enabled (890 milliseconds)
 - hive table: file status caching respects refresh table and refreshByPath (922 milliseconds)
 - datasource table: file status caching respects refresh table and refreshByPath (640 milliseconds)
 - hive table: file status cache respects size limit (469 milliseconds)
 - datasource table: file status cache respects size limit (453 milliseconds)
 - datasource table: table setup does not scan filesystem (328 milliseconds)
 - hive table: table setup does not scan filesystem (313 milliseconds)
 - hive table: num hive client calls does not scale with partition count (5 seconds, 431 milliseconds)
 - datasource table: num hive client calls does not scale with partition count (4 seconds, 79 milliseconds)
 - hive table: files read and cached when filesource partition management is off (656 milliseconds)
 - datasource table: all partition data cached in memory when partition management is off (484 milliseconds)
 - SPARK-18700: table loaded only once even when resolved concurrently (2 seconds, 578 milliseconds)
```

```
HiveSparkSubmitSuite:
 - temporary Hive UDF: define a UDF and use it (1 second, 745 milliseconds)
 - permanent Hive UDF: define a UDF and use it (406 milliseconds)
 - permanent Hive UDF: use a already defined permanent function (375 milliseconds)
 - SPARK-8368: includes jars passed in through --jars (391 milliseconds)
 - SPARK-8020: set sql conf in spark conf (156 milliseconds)
 - SPARK-8489: MissingRequirementError during reflection (187 milliseconds)
 - SPARK-9757 Persist Parquet relation with decimal column (157 milliseconds)
 - SPARK-11009 fix wrong result of Window function in cluster mode (156 milliseconds)
 - SPARK-14244 fix window partition size attribute binding failure (156 milliseconds)
 - set spark.sql.warehouse.dir (172 milliseconds)
 - set hive.metastore.warehouse.dir (156 milliseconds)
 - SPARK-16901: set javax.jdo.option.ConnectionURL (157 milliseconds)
 - SPARK-18360: default table path of tables in default database should depend on the location of default database (172 milliseconds)
```

```
UtilsSuite:
 - resolveURIs with multiple paths (0 milliseconds)
```

```
CheckpointSuite:
 - recovery with file input stream (4 seconds, 452 milliseconds)
```

Note: after resolving the aborted tests, there is a test failure identified as below:

```
OrcSourceSuite:
- SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds)
  org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z
  at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625)
```

This does not look due to this problem so this PR does not fix it here.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #16451 from HyukjinKwon/all-path-resource-fixes.
2017-01-10 13:19:21 +00:00
Niranjan Padmanabhan a1e40b1f5d
[MINOR][DOCS] Remove consecutive duplicated words/typo in Spark Repo
## What changes were proposed in this pull request?
There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words.

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

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

Closes #16455 from neurons/np.structure_streaming_doc.
2017-01-04 15:07:29 +00:00
saturday_s ce99f51d2e [SPARK-18537][WEB UI] Add a REST api to serve spark streaming information
## What changes were proposed in this pull request?

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

**Description**

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

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

## How was this patch tested?

Local test.

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

Closes #16253 from saturday-shi/SPARK-18537.
2016-12-22 12:51:37 -08:00
Burak Yavuz afe36516e4 [FLAKY-TEST] InputStreamsSuite.socket input stream
## What changes were proposed in this pull request?

https://spark-tests.appspot.com/test-details?suite_name=org.apache.spark.streaming.InputStreamsSuite&test_name=socket+input+stream

## How was this patch tested?

Tested 2,000 times.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #16343 from brkyvz/sock.
2016-12-21 17:23:48 -08:00
Ryan Williams afd9bc1d8a [SPARK-17807][CORE] split test-tags into test-JAR
Remove spark-tag's compile-scope dependency (and, indirectly, spark-core's compile-scope transitive-dependency) on scalatest by splitting test-oriented tags into spark-tags' test JAR.

Alternative to #16303.

Author: Ryan Williams <ryan.blake.williams@gmail.com>

Closes #16311 from ryan-williams/tt.
2016-12-21 16:37:20 -08:00
Shixiong Zhu 078c71c2dc [SPARK-18954][TESTS] Fix flaky test: o.a.s.streaming.BasicOperationsSuite rdd cleanup - map and window
## What changes were proposed in this pull request?

The issue in this test is the cleanup of RDDs may not be able to finish before stopping StreamingContext. This PR basically just puts the assertions into `eventually` and runs it before stopping StreamingContext.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16362 from zsxwing/SPARK-18954.
2016-12-21 11:59:21 -08:00
Shixiong Zhu ccfe60a830 [SPARK-18031][TESTS] Fix flaky test ExecutorAllocationManagerSuite.basic functionality
## What changes were proposed in this pull request?

The failure is because in `test("basic functionality")`, it doesn't block until `ExecutorAllocationManager.manageAllocation` is called. This PR just adds StreamManualClock to allow the tests to block on expected wait time to make the test deterministic.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16321 from zsxwing/SPARK-18031.
2016-12-21 11:17:44 -08:00
Reynold Xin c7c7265950 [SPARK-18695] Bump master branch version to 2.2.0-SNAPSHOT
## What changes were proposed in this pull request?
This patch bumps master branch version to 2.2.0-SNAPSHOT.

## How was this patch tested?
N/A

Author: Reynold Xin <rxin@databricks.com>

Closes #16126 from rxin/SPARK-18695.
2016-12-02 21:09:37 -08:00
Shixiong Zhu 086b0c8f67 [SPARK-18617][SPARK-18560][TESTS] Fix flaky test: StreamingContextSuite. Receiver data should be deserialized properly
## What changes were proposed in this pull request?

Avoid to create multiple threads to stop StreamingContext. Otherwise, the latch added in #16091 can be passed too early.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16105 from zsxwing/SPARK-18617-2.
2016-12-01 14:22:49 -08:00
Shixiong Zhu 0a811210f8 [SPARK-18617][SPARK-18560][TEST] Fix flaky test: StreamingContextSuite. Receiver data should be deserialized properly
## What changes were proposed in this pull request?

Fixed the potential SparkContext leak in `StreamingContextSuite.SPARK-18560 Receiver data should be deserialized properly` which was added in #16052. I also removed FakeByteArrayReceiver and used TestReceiver directly.

## How was this patch tested?

Jenkins

Author: Shixiong Zhu <shixiong@databricks.com>

Closes #16091 from zsxwing/SPARK-18617-follow-up.
2016-11-30 17:41:43 -08:00
uncleGen 56c82edabd [SPARK-18617][CORE][STREAMING] Close "kryo auto pick" feature for Spark Streaming
## What changes were proposed in this pull request?

#15992 provided a solution to fix the bug, i.e. **receiver data can not be deserialized properly**. As zsxwing said, it is a critical bug, but we should not break APIs between maintenance releases. It may be a rational choice to close auto pick kryo serializer for Spark Streaming in the first step. I will continue #15992 to optimize the solution.

## How was this patch tested?

existing ut

Author: uncleGen <hustyugm@gmail.com>

Closes #16052 from uncleGen/SPARK-18617.
2016-11-29 23:45:06 -08:00