aboutsummaryrefslogtreecommitdiff
path: root/core/src/test
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19365][CORE] Optimize RequestMessage serializationShixiong Zhu2017-01-272-2/+33
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Right now Netty PRC serializes `RequestMessage` using Java serialization, and the size of a single message (e.g., RequestMessage(..., "hello")`) is almost 1KB. This PR optimizes it by serializing `RequestMessage` manually (eliminate unnecessary information from most messages, e.g., class names of `RequestMessage`, `NettyRpcEndpointRef`, ...), and reduces the above message size to 100+ bytes. ## How was this patch tested? Jenkins I did a simple test to measure the improvement: Before ``` $ bin/spark-shell --master local-cluster[1,4,1024] ... scala> for (i <- 1 to 10) { | val start = System.nanoTime | val s = sc.parallelize(1 to 1000000, 10 * 1000).count() | val end = System.nanoTime | println(s"$i\t" + ((end - start)/1000/1000)) | } 1 6830 2 4353 3 3322 4 3107 5 3235 6 3139 7 3156 8 3166 9 3091 10 3029 ``` After: ``` $ bin/spark-shell --master local-cluster[1,4,1024] ... scala> for (i <- 1 to 10) { | val start = System.nanoTime | val s = sc.parallelize(1 to 1000000, 10 * 1000).count() | val end = System.nanoTime | println(s"$i\t" + ((end - start)/1000/1000)) | } 1 6431 2 3643 3 2913 4 2679 5 2760 6 2710 7 2747 8 2793 9 2679 10 2651 ``` I also captured the TCP packets for this test. Before this patch, the total size of TCP packets is ~1.5GB. After it, it reduces to ~1.2GB. Author: Shixiong Zhu <shixiong@databricks.com> Closes #16706 from zsxwing/rpc-opt.
* [SPARK-19220][UI] Make redirection to HTTPS apply to all URIs.Marcelo Vanzin2017-01-262-19/+56
| | | | | | | | | | | | | | | | | | | The redirect handler was installed only for the root of the server; any other context ended up being served directly through the HTTP port. Since every sub page (e.g. application UIs in the history server) is a separate servlet context, this meant that everything but the root was accessible via HTTP still. The change adds separate names to each connector, and binds contexts to specific connectors so that content is only served through the HTTPS connector when it's enabled. In that case, the only thing that binds to the HTTP connector is the redirect handler. Tested with new unit tests and by checking a live history server. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16582 from vanzin/SPARK-19220.
* [SPARK-19139][CORE] New auth mechanism for transport library.Marcelo Vanzin2017-01-243-10/+77
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This change introduces a new auth mechanism to the transport library, to be used when users enable strong encryption. This auth mechanism has better security than the currently used DIGEST-MD5. The new protocol uses symmetric key encryption to mutually authenticate the endpoints, and is very loosely based on ISO/IEC 9798. The new protocol falls back to SASL when it thinks the remote end is old. Because SASL does not support asking the server for multiple auth protocols, which would mean we could re-use the existing SASL code by just adding a new SASL provider, the protocol is implemented outside of the SASL API to avoid the boilerplate of adding a new provider. Details of the auth protocol are discussed in the included README.md file. This change partly undos the changes added in SPARK-13331; AES encryption is now decoupled from SASL authentication. The encryption code itself, though, has been re-used as part of this change. ## How was this patch tested? - Unit tests - Tested Spark 2.2 against Spark 1.6 shuffle service with SASL enabled - Tested Spark 2.2 against Spark 2.2 shuffle service with SASL fallback disabled Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16521 from vanzin/SPARK-19139.
* [SPARK-14049][CORE] Add functionality in spark history sever API to query ↵Parag Chaudhari2017-01-245-0/+277
| | | | | | | | | | | | | | | | | | | | | applications by end time ## What changes were proposed in this pull request? Currently, spark history server REST API provides functionality to query applications by application start time range based on minDate and maxDate query parameters, but it lacks support to query applications by their end time. In this pull request we are proposing optional minEndDate and maxEndDate query parameters and filtering capability based on these parameters to spark history server REST API. This functionality can be used for following queries, 1. Applications finished in last 'x' minutes 2. Applications finished before 'y' time 3. Applications finished between 'x' time to 'y' time 4. Applications started from 'x' time and finished before 'y' time. For backward compatibility, we can keep existing minDate and maxDate query parameters as they are and they can continue support filtering based on start time range. ## How was this patch tested? Existing unit tests and 4 new unit tests. Author: Parag Chaudhari <paragpc@amazon.com> Closes #11867 from paragpc/master-SHS-query-by-endtime_2.
* [SPARK-19146][CORE] Drop more elements when stageData.taskData.size > ↵Yuming Wang2017-01-231-0/+30
| | | | | | | | | | | | | | | | retainedTasks ## What changes were proposed in this pull request? Drop more elements when `stageData.taskData.size > retainedTasks` to reduce the number of times on call drop function. ## How was this patch tested? Jenkins Author: Yuming Wang <wgyumg@gmail.com> Closes #16527 from wangyum/SPARK-19146.
* [SPARK-19117][SPARK-18922][TESTS] Fix the rest of flaky, newly introduced ↵hyukjinkwon2017-01-211-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | and missed test failures on Windows ## What changes were proposed in this pull request? **Failed tests** ``` org.apache.spark.sql.hive.execution.HiveQuerySuite: - transform with SerDe3 *** FAILED *** - transform with SerDe4 *** FAILED *** ``` ``` org.apache.spark.sql.hive.execution.HiveDDLSuite: - create hive serde table with new syntax *** FAILED *** - add/drop partition with location - managed table *** FAILED *** ``` ``` org.apache.spark.sql.hive.ParquetMetastoreSuite: - Explicitly added partitions should be readable after load *** FAILED *** - Non-partitioned table readable after load *** FAILED *** ``` **Aborted tests** ``` Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.HiveSerDeSuite *** ABORTED *** (157 milliseconds) org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilessales.txt; ``` **Flaky tests(failed 9ish out of 10)** ``` org.apache.spark.scheduler.SparkListenerSuite: - local metrics *** FAILED *** ``` ## How was this patch tested? Manually tested via AppVeyor. **Failed tests** ``` org.apache.spark.sql.hive.execution.HiveQuerySuite: - transform with SerDe3 !!! CANCELED !!! (0 milliseconds) - transform with SerDe4 !!! CANCELED !!! (0 milliseconds) ``` ``` org.apache.spark.sql.hive.execution.HiveDDLSuite: - create hive serde table with new syntax (1 second, 672 milliseconds) - add/drop partition with location - managed table (2 seconds, 391 milliseconds) ``` ``` org.apache.spark.sql.hive.ParquetMetastoreSuite: - Explicitly added partitions should be readable after load (609 milliseconds) - Non-partitioned table readable after load (344 milliseconds) ``` **Aborted tests** ``` spark.sql.hive.execution.HiveSerDeSuite: - Read with RegexSerDe (2 seconds, 142 milliseconds) - Read and write with LazySimpleSerDe (tab separated) (2 seconds) - Read with AvroSerDe (1 second, 47 milliseconds) - Read Partitioned with AvroSerDe (1 second, 422 milliseconds) ``` **Flaky tests (failed 9ish out of 10)** ``` org.apache.spark.scheduler.SparkListenerSuite: - local metrics (4 seconds, 562 milliseconds) ``` Author: hyukjinkwon <gurwls223@gmail.com> Closes #16586 from HyukjinKwon/set-path-appveyor.
* [SPARK-19069][CORE] Expose task 'status' and 'duration' in spark history ↵Parag Chaudhari2017-01-2010-0/+340
| | | | | | | | | | | | | | | | server REST API. ## What changes were proposed in this pull request? Although Spark history server UI shows task ‘status’ and ‘duration’ fields, it does not expose these fields in the REST API response. For the Spark history server API users, it is not possible to determine task status and duration. Spark history server has access to task status and duration from event log, but it is not exposing these in API. This patch is proposed to expose task ‘status’ and ‘duration’ fields in Spark history server REST API. ## How was this patch tested? Modified existing test cases in org.apache.spark.deploy.history.HistoryServerSuite. Author: Parag Chaudhari <paragpc@amazon.com> Closes #16473 from paragpc/expose_task_status.
* [SPARK-16654][CORE] Add UI coverage for Application Level BlacklistingJosé Hiram Soltren2017-01-1921-211/+733
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Builds on top of work in SPARK-8425 to update Application Level Blacklisting in the scheduler. ## What changes were proposed in this pull request? Adds a UI to these patches by: - defining new listener events for blacklisting and unblacklisting, nodes and executors; - sending said events at the relevant points in BlacklistTracker; - adding JSON (de)serialization code for these events; - augmenting the Executors UI page to show which, and how many, executors are blacklisted; - adding a unit test to make sure events are being fired; - adding HistoryServerSuite coverage to verify that the SHS reads these events correctly. - updates the Executor UI to show Blacklisted/Active/Dead as a tri-state in Executors Status Updates .rat-excludes to pass tests. username squito ## How was this patch tested? ./dev/run-tests testOnly org.apache.spark.util.JsonProtocolSuite testOnly org.apache.spark.scheduler.BlacklistTrackerSuite testOnly org.apache.spark.deploy.history.HistoryServerSuite https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh ![blacklist-20161219](https://cloud.githubusercontent.com/assets/1208477/21335321/9eda320a-c623-11e6-8b8c-9c912a73c276.jpg) Author: José Hiram Soltren <jose@cloudera.com> Closes #16346 from jsoltren/SPARK-16654-submit.
* [SPARK-18113] Use ask to replace askWithRetry in canCommit and make receiver ↵jinxing2017-01-181-0/+16
| | | | | | | | | | | | | | | | | | | | idempotent. ## What changes were proposed in this pull request? Method canCommit sends AskPermissionToCommitOutput using askWithRetry. If timeout, it will send again. Thus AskPermissionToCommitOutput can be received multi times. Method canCommit should return the same value when called by the same attempt multi times. In implementation before this fix, method handleAskPermissionToCommit just check if there is committer already registered, which is not enough. When worker retries AskPermissionToCommitOutput it will get CommitDeniedException, then the task will fail with reason TaskCommitDenied, which is not regarded as a task failure(SPARK-11178), so TaskScheduler will schedule this task infinitely. In this fix, use `ask` to replace `askWithRetry` in `canCommit` and make receiver idempotent. ## How was this patch tested? Added a new unit test to OutputCommitCoordinatorSuite. Author: jinxing <jinxing@meituan.com> Closes #16503 from jinxing64/SPARK-18113.
* [SPARK-19227][SPARK-19251] remove unused imports and outdated commentsuncleGen2017-01-183-6/+0
| | | | | | | | | | | | ## What changes were proposed in this pull request? remove ununsed imports and outdated comments, and fix some minor code style issue. ## How was this patch tested? existing ut Author: uncleGen <hustyugm@gmail.com> Closes #16591 from uncleGen/SPARK-19227.
* [SPARK-17568][CORE][DEPLOY] Add spark-submit option to override ivy settings ↵Bryan Cutler2017-01-111-20/+80
| | | | | | | | | | | | | | | | | | used to resolve packages/artifacts ## What changes were proposed in this pull request? Adding option in spark-submit to allow overriding the default IvySettings used to resolve artifacts as part of the Spark Packages functionality. This will allow all artifact resolution to go through a central managed repository, such as Nexus or Artifactory, where site admins can better approve and control what is used with Spark apps. This change restructures the creation of the IvySettings object in two distinct ways. First, if the `spark.ivy.settings` option is not defined then `buildIvySettings` will create a default settings instance, as before, with defined repositories (Maven Central) included. Second, if the option is defined, the ivy settings file will be loaded from the given path and only repositories defined within will be used for artifact resolution. ## How was this patch tested? Existing tests for default behaviour, Manual tests that load a ivysettings.xml file with local and Nexus repositories defined. Added new test to load a simple Ivy settings file with a local filesystem resolver. Author: Bryan Cutler <cutlerb@gmail.com> Author: Ian Hummel <ian@themodernlife.net> Closes #15119 from BryanCutler/spark-custom-IvySettings.
* [SPARK-19117][TESTS] Skip the tests using script transformation on Windowshyukjinkwon2017-01-101-16/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to skip the tests for script transformation failed on Windows due to fixed bash location. ``` SQLQuerySuite: - script *** FAILED *** (553 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 56.0 failed 1 times, most recent failure: Lost task 0.0 in stage 56.0 (TID 54, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - Star Expansion - script transform *** FAILED *** (2 seconds, 375 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 389.0 failed 1 times, most recent failure: Lost task 0.0 in stage 389.0 (TID 725, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - test script transform for stdout *** FAILED *** (2 seconds, 813 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 391.0 failed 1 times, most recent failure: Lost task 0.0 in stage 391.0 (TID 726, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - test script transform for stderr *** FAILED *** (2 seconds, 407 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 393.0 failed 1 times, most recent failure: Lost task 0.0 in stage 393.0 (TID 727, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - test script transform data type *** FAILED *** (171 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 395.0 failed 1 times, most recent failure: Lost task 0.0 in stage 395.0 (TID 728, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ``` HiveQuerySuite: - transform *** FAILED *** (359 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1347.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1347.0 (TID 2395, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - schema-less transform *** FAILED *** (344 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1348.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1348.0 (TID 2396, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with custom field delimiter *** FAILED *** (296 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1349.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1349.0 (TID 2397, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with custom field delimiter2 *** FAILED *** (297 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1350.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1350.0 (TID 2398, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with custom field delimiter3 *** FAILED *** (312 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1351.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1351.0 (TID 2399, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with SerDe2 *** FAILED *** (437 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1355.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1355.0 (TID 2403, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ``` LogicalPlanToSQLSuite: - script transformation - schemaless *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1968.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1968.0 (TID 3932, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - alias list *** FAILED *** (94 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1969.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1969.0 (TID 3933, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - alias list with type *** FAILED *** (93 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1970.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1970.0 (TID 3934, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format delimited clause with only one format property *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1971.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1971.0 (TID 3935, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format delimited clause with multiple format properties *** FAILED *** (94 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1972.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1972.0 (TID 3936, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format serde clauses with SERDEPROPERTIES *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1973.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1973.0 (TID 3937, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format serde clauses without SERDEPROPERTIES *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1974.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1974.0 (TID 3938, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ``` ScriptTransformationSuite: - cat without SerDe *** FAILED *** (156 milliseconds) ... Caused by: java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - cat with LazySimpleSerDe *** FAILED *** (63 milliseconds) ... org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2383.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2383.0 (TID 4819, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation should not swallow errors from upstream operators (no serde) *** FAILED *** (78 milliseconds) ... org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2384.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2384.0 (TID 4820, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation should not swallow errors from upstream operators (with serde) *** FAILED *** (47 milliseconds) ... org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2385.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2385.0 (TID 4821, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - SPARK-14400 script transformation should fail for bad script command *** FAILED *** (47 milliseconds) "Job aborted due to stage failure: Task 0 in stage 2386.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2386.0 (TID 4822, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ## How was this patch tested? AppVeyor as below: ``` SQLQuerySuite: - script !!! CANCELED !!! (63 milliseconds) - Star Expansion - script transform !!! CANCELED !!! (0 milliseconds) - test script transform for stdout !!! CANCELED !!! (0 milliseconds) - test script transform for stderr !!! CANCELED !!! (0 milliseconds) - test script transform data type !!! CANCELED !!! (0 milliseconds) ``` ``` HiveQuerySuite: - transform !!! CANCELED !!! (31 milliseconds) - schema-less transform !!! CANCELED !!! (0 milliseconds) - transform with custom field delimiter !!! CANCELED !!! (0 milliseconds) - transform with custom field delimiter2 !!! CANCELED !!! (0 milliseconds) - transform with custom field delimiter3 !!! CANCELED !!! (0 milliseconds) - transform with SerDe2 !!! CANCELED !!! (0 milliseconds) ``` ``` LogicalPlanToSQLSuite: - script transformation - schemaless !!! CANCELED !!! (78 milliseconds) - script transformation - alias list !!! CANCELED !!! (0 milliseconds) - script transformation - alias list with type !!! CANCELED !!! (0 milliseconds) - script transformation - row format delimited clause with only one format property !!! CANCELED !!! (15 milliseconds) - script transformation - row format delimited clause with multiple format properties !!! CANCELED !!! (0 milliseconds) - script transformation - row format serde clauses with SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds) - script transformation - row format serde clauses without SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds) ``` ``` ScriptTransformationSuite: - cat without SerDe !!! CANCELED !!! (62 milliseconds) - cat with LazySimpleSerDe !!! CANCELED !!! (0 milliseconds) - script transformation should not swallow errors from upstream operators (no serde) !!! CANCELED !!! (0 milliseconds) - script transformation should not swallow errors from upstream operators (with serde) !!! CANCELED !!! (0 milliseconds) - SPARK-14400 script transformation should fail for bad script command !!! CANCELED !!! (0 milliseconds) ``` Jenkins tests Author: hyukjinkwon <gurwls223@gmail.com> Closes #16501 from HyukjinKwon/windows-bash.
* [SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed ↵hyukjinkwon2017-01-101-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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.
* [SPARK-17931] Eliminate unnecessary task (de) serializationKay Ousterhout2017-01-062-10/+85
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | In the existing code, there are three layers of serialization involved in sending a task from the scheduler to an executor: - A Task object is serialized - The Task object is copied to a byte buffer that also contains serialized information about any additional JARs, files, and Properties needed for the task to execute. This byte buffer is stored as the member variable serializedTask in the TaskDescription class. - The TaskDescription is serialized (in addition to the serialized task + JARs, the TaskDescription class contains the task ID and other metadata) and sent in a LaunchTask message. While it *is* necessary to have two layers of serialization, so that the JAR, file, and Property info can be deserialized prior to deserializing the Task object, the third layer of deserialization is unnecessary. This commit eliminates a layer of serialization by moving the JARs, files, and Properties into the TaskDescription class. This commit also serializes the Properties manually (by traversing the map), as is done with the JARs and files, which reduces the final serialized size. Unit tests This is a simpler alternative to the approach proposed in #15505. shivaram and I did some benchmarking of this and #15505 on a 20-machine m2.4xlarge EC2 machines (160 cores). We ran ~30 trials of code [1] (a very simple job with 10K tasks per stage) and measured the average time per stage: Before this change: 2490ms With this change: 2345 ms (so ~6% improvement over the baseline) With witgo's approach in #15505: 2046 ms (~18% improvement over baseline) The reason that #15505 has a more significant improvement is that it also moves the serialization from the TaskSchedulerImpl thread to the CoarseGrainedSchedulerBackend thread. I added that functionality on top of this change, and got almost the same improvement [1] as #15505 (average of 2103ms). I think we should decouple these two changes, both so we have some record of the improvement form each individual improvement, and because this change is more about simplifying the code base (the improvement is negligible) while the other is about performance improvement. The plan, currently, is to merge this PR and then merge the remaining part of #15505 that moves serialization. [1] The reason the improvement wasn't quite as good as with #15505 when we ran the benchmarks is almost certainly because, at the point when we ran the benchmarks, I hadn't updated the code to manually serialize the Properties (instead the code was using Java's default serialization for the Properties object, whereas #15505 manually serialized the Properties). This PR has since been updated to manually serialize the Properties, just like the other maps. Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #16053 from kayousterhout/SPARK-17931.
* [SPARK-19033][CORE] Add admin acls for history serverjerryshao2017-01-061-1/+110
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Current HistoryServer's ACLs is derived from application event-log, which means the newly changed ACLs cannot be applied to the old data, this will become a problem where newly added admin cannot access the old application history UI, only the new application can be affected. So here propose to add admin ACLs for history server, any configured user/group could have the view access to all the applications, while the view ACLs derived from application run-time still take effect. ## How was this patch tested? Unit test added. Author: jerryshao <sshao@hortonworks.com> Closes #16470 from jerryshao/SPARK-19033.
* [SPARK-14958][CORE] Failed task not handled when there's error deserializing ↵Rui Li2017-01-051-1/+20
| | | | | | | | | | | | | | | | | | failure reason ## What changes were proposed in this pull request? TaskResultGetter tries to deserialize the TaskEndReason before handling the failed task. If an error is thrown during deserialization, the failed task won't be handled, which leaves the job hanging. The PR proposes to handle the failed task in a finally block. ## How was this patch tested? In my case I hit a NoClassDefFoundError and the job hangs. Manually verified the patch can fix it. Author: Rui Li <rui.li@intel.com> Author: Rui Li <lirui@apache.org> Author: Rui Li <shlr@cn.ibm.com> Closes #12775 from lirui-intel/SPARK-14958.
* [SPARK-19062] Utils.writeByteBuffer bug fixKay Ousterhout2017-01-041-1/+24
| | | | | | | | | | | | This commit changes Utils.writeByteBuffer so that it does not change the position of the ByteBuffer that it writes out, and adds a unit test for this functionality. cc mridulm Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #16462 from kayousterhout/SPARK-19062.
* [MINOR][DOCS] Remove consecutive duplicated words/typo in Spark RepoNiranjan Padmanabhan2017-01-043-4/+4
| | | | | | | | | | | | ## 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.
* [MINOR] Add missing sc.stop() to end of examplesWeiqing Yang2017-01-031-11/+9
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Add `finally` clause for `sc.stop()` in the `test("register and deregister Spark listener from SparkContext")`. ## How was this patch tested? Pass the build and unit tests. Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #16426 from weiqingy/testIssue.
* [SPARK-19010][CORE] Include Kryo exception in case of overflowSergei Lebedev2016-12-281-1/+2
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This is to workaround an implicit result of #4947 which suppressed the original Kryo exception if the overflow happened during serialization. ## How was this patch tested? `KryoSerializerSuite` was augmented to reflect this change. Author: Sergei Lebedev <superbobry@gmail.com> Closes #16416 from superbobry/patch-1.
* [SPARK-17755][CORE] Use workerRef to send RegisterWorkerResponse to avoid ↵Shixiong Zhu2016-12-251-1/+1
| | | | | | | | | | | | | | | | | | the race condition ## What changes were proposed in this pull request? The root cause of this issue is that RegisterWorkerResponse and LaunchExecutor are sent via two different channels (TCP connections) and their order is not guaranteed. This PR changes the master and worker codes to use `workerRef` to send RegisterWorkerResponse, so that RegisterWorkerResponse and LaunchExecutor are sent via the same connection. Hence `LaunchExecutor` will always be after `RegisterWorkerResponse` and never be ignored. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16345 from zsxwing/SPARK-17755.
* [SPARK-18975][CORE] Add an API to remove SparkListenerjerryshao2016-12-221-0/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? In current Spark we could add customized SparkListener through `SparkContext#addListener` API, but there's no equivalent API to remove the registered one. In our scenario SparkListener will be added repeatedly accordingly to the changed environment. If lacks the ability to remove listeners, there might be many registered listeners finally, this is unnecessary and potentially affects the performance. So here propose to add an API to remove registered listener. ## How was this patch tested? Add an unit test to verify it. Author: jerryshao <sshao@hortonworks.com> Closes #16382 from jerryshao/SPARK-18975.
* [SPARK-18922][TESTS] Fix more resource-closing-related and path-related test ↵hyukjinkwon2016-12-221-0/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | failures in identified ones on Windows ## What changes were proposed in this pull request? There are several tests failing due to resource-closing-related and path-related problems on Windows as below. - `SQLQuerySuite`: ``` - specifying database name for a temporary table is not allowed *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-1f4471ab-aac0-4239-ae35-833d54b37e52; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370) ``` - `JsonSuite`: ``` - Loading a JSON dataset from a text file with SQL *** FAILED *** (94 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c918a8b7-fc09-433c-b9d0-36c0f78ae918; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370) ``` - `StateStoreSuite`: ``` - SPARK-18342: commit fails when rename fails *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: StateStoreSuite29777261fs://C:%5Cprojects%5Cspark%5Ctarget%5Ctmp%5Cspark-ef349862-7281-4963-aaf3-add0d670a4ad%5C?????-2218c2f8-2cf6-4f80-9cdf-96354e8246a77685899733421033312/0 at org.apache.hadoop.fs.Path.initialize(Path.java:206) at org.apache.hadoop.fs.Path.<init>(Path.java:116) at org.apache.hadoop.fs.Path.<init>(Path.java:89) ... Cause: java.net.URISyntaxException: Relative path in absolute URI: StateStoreSuite29777261fs://C:%5Cprojects%5Cspark%5Ctarget%5Ctmp%5Cspark-ef349862-7281-4963-aaf3-add0d670a4ad%5C?????-2218c2f8-2cf6-4f80-9cdf-96354e8246a77685899733421033312/0 at java.net.URI.checkPath(URI.java:1823) at java.net.URI.<init>(URI.java:745) at org.apache.hadoop.fs.Path.initialize(Path.java:203) ``` - `HDFSMetadataLogSuite`: ``` - FileManager: FileContextManager *** FAILED *** (94 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-415bb0bd-396b-444d-be82-04599e025f21 at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010) at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:127) at org.apache.spark.sql.execution.streaming.HDFSMetadataLogSuite.withTempDir(HDFSMetadataLogSuite.scala:38) - FileManager: FileSystemManager *** FAILED *** (78 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ef8222cd-85aa-47c0-a396-bc7979e15088 at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010) at org.apache.spark.sql.test.SQLTestUtils$class.withTempDir(SQLTestUtils.scala:127) at org.apache.spark.sql.execution.streaming.HDFSMetadataLogSuite.withTempDir(HDFSMetadataLogSuite.scala:38) ``` And, there are some tests being failed due to the length limitation on cmd in Windows as below: - `LauncherBackendSuite`: ``` - local: launcher handle *** FAILED *** (30 seconds, 120 milliseconds) The code passed to eventually never returned normally. Attempted 283 times over 30.0960053 seconds. Last failure message: The reference was null. (LauncherBackendSuite.scala:56) org.scalatest.exceptions.TestFailedDueToTimeoutException: at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) - standalone/client: launcher handle *** FAILED *** (30 seconds, 47 milliseconds) The code passed to eventually never returned normally. Attempted 282 times over 30.037987100000002 seconds. Last failure message: The reference was null. (LauncherBackendSuite.scala:56) org.scalatest.exceptions.TestFailedDueToTimeoutException: at org.scalatest.concurrent.Eventually$class.tryTryAgain$1(Eventually.scala:420) at org.scalatest.concurrent.Eventually$class.eventually(Eventually.scala:438) ``` The executed command is, https://gist.github.com/HyukjinKwon/d3fdd2e694e5c022992838a618a516bd, which is 16K length; however, the length limitation is 8K on Windows. So, it is being failed to launch. This PR proposes to fix the test failures on Windows and skip the tests failed due to the length limitation ## How was this patch tested? Manually tested via AppVeyor **Before** `SQLQuerySuite `: https://ci.appveyor.com/project/spark-test/spark/build/306-pr-references `JsonSuite`: https://ci.appveyor.com/project/spark-test/spark/build/307-pr-references `StateStoreSuite` : https://ci.appveyor.com/project/spark-test/spark/build/305-pr-references `HDFSMetadataLogSuite`: https://ci.appveyor.com/project/spark-test/spark/build/304-pr-references `LauncherBackendSuite`: https://ci.appveyor.com/project/spark-test/spark/build/303-pr-references **After** `SQLQuerySuite`: https://ci.appveyor.com/project/spark-test/spark/build/293-SQLQuerySuite `JsonSuite`: https://ci.appveyor.com/project/spark-test/spark/build/294-JsonSuite `StateStoreSuite`: https://ci.appveyor.com/project/spark-test/spark/build/297-StateStoreSuite `HDFSMetadataLogSuite`: https://ci.appveyor.com/project/spark-test/spark/build/319-pr-references `LauncherBackendSuite`: failed test skipped. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16335 from HyukjinKwon/more-fixes-on-windows.
* [SPARK-18761][CORE] Introduce "task reaper" to oversee task killing in executorsJosh Rosen2016-12-191-0/+77
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark's current task cancellation / task killing mechanism is "best effort" because some tasks may not be interruptible or may not respond to their "killed" flags being set. If a significant fraction of a cluster's task slots are occupied by tasks that have been marked as killed but remain running then this can lead to a situation where new jobs and tasks are starved of resources that are being used by these zombie tasks. This patch aims to address this problem by adding a "task reaper" mechanism to executors. At a high-level, task killing now launches a new thread which attempts to kill the task and then watches the task and periodically checks whether it has been killed. The TaskReaper will periodically re-attempt to call `TaskRunner.kill()` and will log warnings if the task keeps running. I modified TaskRunner to rename its thread at the start of the task, allowing TaskReaper to take a thread dump and filter it in order to log stacktraces from the exact task thread that we are waiting to finish. If the task has not stopped after a configurable timeout then the TaskReaper will throw an exception to trigger executor JVM death, thereby forcibly freeing any resources consumed by the zombie tasks. This feature is flagged off by default and is controlled by four new configurations under the `spark.task.reaper.*` namespace. See the updated `configuration.md` doc for details. ## How was this patch tested? Tested via a new test case in `JobCancellationSuite`, plus manual testing. Author: Josh Rosen <joshrosen@databricks.com> Closes #16189 from JoshRosen/cancellation.
* [SPARK-18836][CORE] Serialize one copy of task metrics in DAGSchedulerShivaram Venkataraman2016-12-193-4/+16
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Right now we serialize the empty task metrics once per task – Since this is shared across all tasks we could use the same serialized task metrics across all tasks of a stage. ## How was this patch tested? - [x] Run tests on EC2 to measure performance improvement Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #16261 from shivaram/task-metrics-one-copy.
* [SPARK-18827][CORE] Fix cannot read broadcast on diskYuming Wang2016-12-181-0/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `NoSuchElementException` will throw since https://github.com/apache/spark/pull/15056 if a broadcast cannot cache in memory. The reason is that that change cannot cover `!unrolled.hasNext` in `next()` function. This change is to cover the `!unrolled.hasNext` and check `hasNext` before calling `next` in `blockManager.getLocalValues` to make it more robust. We can cache and read broadcast even it cannot fit in memory from this pull request. Exception log: ``` 16/12/10 10:10:04 INFO UnifiedMemoryManager: Will not store broadcast_131 as the required space (1048576 bytes) exceeds our memory limit (122764 bytes) 16/12/10 10:10:04 WARN MemoryStore: Failed to reserve initial memory threshold of 1024.0 KB for computing block broadcast_131 in memory. 16/12/10 10:10:04 WARN MemoryStore: Not enough space to cache broadcast_131 in memory! (computed 384.0 B so far) 16/12/10 10:10:04 INFO MemoryStore: Memory use = 95.6 KB (blocks) + 0.0 B (scratch space shared across 0 tasks(s)) = 95.6 KB. Storage limit = 119.9 KB. 16/12/10 10:10:04 ERROR Utils: Exception encountered java.util.NoSuchElementException at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58) at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700) at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at scala.Option.map(Option.scala:146) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210) at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269) at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206) at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96) at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) 16/12/10 10:10:04 ERROR Executor: Exception in task 1.0 in stage 86.0 (TID 134423) java.io.IOException: java.util.NoSuchElementException at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1276) at org.apache.spark.broadcast.TorrentBroadcast.readBroadcastBlock(TorrentBroadcast.scala:206) at org.apache.spark.broadcast.TorrentBroadcast._value$lzycompute(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast._value(TorrentBroadcast.scala:66) at org.apache.spark.broadcast.TorrentBroadcast.getValue(TorrentBroadcast.scala:96) at org.apache.spark.broadcast.Broadcast.value(Broadcast.scala:70) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:86) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53) at org.apache.spark.scheduler.Task.run(Task.scala:108) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:282) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.util.NoSuchElementException at org.apache.spark.util.collection.PrimitiveVector$$anon$1.next(PrimitiveVector.scala:58) at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:700) at org.apache.spark.util.CompletionIterator.next(CompletionIterator.scala:30) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1$$anonfun$2.apply(TorrentBroadcast.scala:210) at scala.Option.map(Option.scala:146) at org.apache.spark.broadcast.TorrentBroadcast$$anonfun$readBroadcastBlock$1.apply(TorrentBroadcast.scala:210) at org.apache.spark.util.Utils$.tryOrIOException(Utils.scala:1269) ... 12 more ``` ## How was this patch tested? Add unit test Author: Yuming Wang <wgyumg@gmail.com> Closes #16252 from wangyum/SPARK-18827.
* [SPARK-18895][TESTS] Fix resource-closing-related and path-related test ↵hyukjinkwon2016-12-162-12/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | failures in identified ones on Windows ## What changes were proposed in this pull request? There are several tests failing due to resource-closing-related and path-related problems on Windows as below. - `RPackageUtilsSuite`: ``` - build an R package from a jar end to end *** FAILED *** (1 second, 625 milliseconds) java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729427517-0\a\dep2\d\dep2-d.jar at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279) at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653) at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535) - faulty R package shows documentation *** FAILED *** (359 milliseconds) java.io.IOException: Unable to delete file: C:\projects\spark\target\tmp\1481729428970-0\dep1-c.jar at org.apache.commons.io.FileUtils.forceDelete(FileUtils.java:2279) at org.apache.commons.io.FileUtils.cleanDirectory(FileUtils.java:1653) at org.apache.commons.io.FileUtils.deleteDirectory(FileUtils.java:1535) - SparkR zipping works properly *** FAILED *** (47 milliseconds) java.util.regex.PatternSyntaxException: Unknown character property name {r} near index 4 C:\projects\spark\target\tmp\1481729429282-0 ^ at java.util.regex.Pattern.error(Pattern.java:1955) at java.util.regex.Pattern.charPropertyNodeFor(Pattern.java:2781) ``` - `InputOutputMetricsSuite`: ``` - input metrics for old hadoop with coalesce *** FAILED *** (240 milliseconds) java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252) - input metrics with cache and coalesce *** FAILED *** (109 milliseconds) java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252) - input metrics for new Hadoop API with coalesce *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-9366ec94-dac7-4a5c-a74b-3e7594a692ab\test\InputOutputMetricsSuite.txt, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642) at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462) at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114) - input metrics when reading text file *** FAILED *** (110 milliseconds) java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252) - input metrics on records read - simple *** FAILED *** (125 milliseconds) java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252) - input metrics on records read - more stages *** FAILED *** (110 milliseconds) java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252) - input metrics on records - New Hadoop API *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-3f10a1a4-7820-4772-b821-25fd7523bf6f\test\InputOutputMetricsSuite.txt, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642) at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462) at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114) - input metrics on records read with cache *** FAILED *** (93 milliseconds) java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252) - input read/write and shuffle read/write metrics all line up *** FAILED *** (93 milliseconds) java.io.IOException: Not a file: file:/C:/projects/spark/core/ignored at org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:277) at org.apache.spark.rdd.HadoopRDD.getPartitions(HadoopRDD.scala:202) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:252) - input metrics with interleaved reads *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-2638d893-e89b-47ce-acd0-bbaeee78dd9b\InputOutputMetricsSuite_cart.txt, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642) at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462) at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114) - input metrics with old CombineFileInputFormat *** FAILED *** (157 milliseconds) 17947 was not greater than or equal to 300000 (InputOutputMetricsSuite.scala:324) org.scalatest.exceptions.TestFailedException: at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466) - input metrics with new CombineFileInputFormat *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Wrong FS: file://C:\projects\spark\target\tmp\spark-11920c08-19d8-4c7c-9fba-28ed72b79f80\test\InputOutputMetricsSuite.txt, expected: file:/// at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:642) at org.apache.hadoop.fs.FileSystem.makeQualified(FileSystem.java:462) at org.apache.hadoop.fs.FilterFileSystem.makeQualified(FilterFileSystem.java:114) ``` - `ReplayListenerSuite`: ``` - End-to-end replay *** FAILED *** (121 milliseconds) java.io.IOException: No FileSystem for scheme: C at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428) - End-to-end replay with compression *** FAILED *** (516 milliseconds) java.io.IOException: No FileSystem for scheme: C at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88) ``` - `EventLoggingListenerSuite`: ``` - End-to-end event logging *** FAILED *** (7 seconds, 435 milliseconds) java.io.IOException: No FileSystem for scheme: C at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88) - End-to-end event logging with compression *** FAILED *** (1 second) java.io.IOException: No FileSystem for scheme: C at org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2421) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2428) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:88) - Event log name *** FAILED *** (16 milliseconds) "file:/[]base-dir/app1" did not equal "file:/[C:/]base-dir/app1" (EventLoggingListenerSuite.scala:123) org.scalatest.exceptions.TestFailedException: at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) at org.scalatest.Assertions$AssertionsHelper.macroAssert(Assertions.scala:466) ``` This PR proposes to fix the test failures on Windows ## How was this patch tested? Manually tested via AppVeyor **Before** `RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/273-RPackageUtilsSuite-before `InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/272-InputOutputMetricsSuite-before `ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/274-ReplayListenerSuite-before `EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/275-EventLoggingListenerSuite-before **After** `RPackageUtilsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/270-RPackageUtilsSuite `InputOutputMetricsSuite`: https://ci.appveyor.com/project/spark-test/spark/build/271-InputOutputMetricsSuite `ReplayListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/277-ReplayListenerSuite-after `EventLoggingListenerSuite`: https://ci.appveyor.com/project/spark-test/spark/build/278-EventLoggingListenerSuite-after Author: hyukjinkwon <gurwls223@gmail.com> Closes #16305 from HyukjinKwon/RPackageUtilsSuite-InputOutputMetricsSuite.
* [SPARK-8425][CORE] Application Level BlacklistingImran Rashid2016-12-155-33/+487
| | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This builds upon the blacklisting introduced in SPARK-17675 to add blacklisting of executors and nodes for an entire Spark application. Resources are blacklisted based on tasks that fail, in tasksets that eventually complete successfully; they are automatically returned to the pool of active resources based on a timeout. Full details are available in a design doc attached to the jira. ## How was this patch tested? Added unit tests, ran them via Jenkins, also ran a handful of them in a loop to check for flakiness. The added tests include: - verifying BlacklistTracker works correctly - verifying TaskSchedulerImpl interacts with BlacklistTracker correctly (via a mock BlacklistTracker) - an integration test for the entire scheduler with blacklisting in a few different scenarios Author: Imran Rashid <irashid@cloudera.com> Author: mwws <wei.mao@intel.com> Closes #14079 from squito/blacklist-SPARK-8425.
* [SPARK-18830][TESTS] Fix tests in PipedRDDSuite to pass on Windowshyukjinkwon2016-12-141-157/+151
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to fix the tests failed on Windows as below: ``` [info] - pipe with empty partition *** FAILED *** (672 milliseconds) [info] Set(0, 4, 5) did not equal Set(0, 5, 6) (PipedRDDSuite.scala:145) [info] org.scalatest.exceptions.TestFailedException: ... ``` In this case, `wc -c` counts the characters on both Windows and Linux but the newlines characters on Windows are `\r\n` which are two. So, the counts ends up one more for each. ``` [info] - test pipe exports map_input_file *** FAILED *** (62 milliseconds) [info] java.lang.IllegalStateException: Subprocess exited with status 1. Command ran: printenv map_input_file [info] at org.apache.spark.rdd.PipedRDD$$anon$1.hasNext(PipedRDD.scala:178) ... ``` ``` [info] - test pipe exports mapreduce_map_input_file *** FAILED *** (172 milliseconds) [info] java.lang.IllegalStateException: Subprocess exited with status 1. Command ran: printenv mapreduce_map_input_file [info] at org.apache.spark.rdd.PipedRDD$$anon$1.hasNext(PipedRDD.scala:178) ... ``` `printenv` command prints the environment variables; however, when environment variables are set to `ProcessBuilder` as lower-cased keys, `printenv` in Windows ignores and does not print this although it is actually set and accessible. (this was tested in [here](https://ci.appveyor.com/project/spark-test/spark/build/208-PipedRDDSuite) for upper-cases with this [diff](https://github.com/apache/spark/compare/master...spark-test:74d39da) and [here](https://ci.appveyor.com/project/spark-test/spark/build/203-PipedRDDSuite) for lower-cases with this [diff](https://github.com/apache/spark/compare/master...spark-test:fde5e37f28032c15a8d8693ba033a8a779a26317). It seems a bug in `printenv`. (BTW, note that environment variables on Windows are case-insensitive). This is (I believe) a thirdparty tool on Windows that resembles `printenv` on Linux (installed in AppVeyor environment or Windows Server 2012 R2). This command does not exist, at least, for Windows 7 and 10 (manually tested). On Windows, we can use `cmd.exe /C set [varname]` officially for this purpose. We could fix the tests with this in order to test if the environment variable is set. ## How was this patch tested? Manually tested via AppVeyor. **Before** https://ci.appveyor.com/project/spark-test/spark/build/194-PipedRDDSuite **After** https://ci.appveyor.com/project/spark-test/spark/build/226-PipedRDDSuite Author: hyukjinkwon <gurwls223@gmail.com> Closes #16254 from HyukjinKwon/pipe-errors.
* [SPARK-18842][TESTS][LAUNCHER] De-duplicate paths in classpaths in commands ↵hyukjinkwon2016-12-141-12/+0
| | | | | | | | | | | | | | | | | | | | | | | | for local-cluster mode to work around the path length limitation on Windows ## What changes were proposed in this pull request? Currently, some tests are being failed and hanging on Windows due to this problem. For the reason in SPARK-18718, some tests using `local-cluster` mode were disabled on Windows due to the length limitation by paths given to classpaths. The limitation seems roughly 32K (see the [blog in MS](https://blogs.msdn.microsoft.com/oldnewthing/20031210-00/?p=41553/) and [another reference](https://support.thoughtworks.com/hc/en-us/articles/213248526-Getting-around-maximum-command-line-length-is-32767-characters-on-Windows)) but in `local-cluster` mode, executors were being launched as processes with the command such as [here](https://gist.github.com/HyukjinKwon/5bc81061c250d4af5a180869b59d42ea) in (only) tests. This length is roughly 40K due to the classpaths given to `java` command. However, it seems duplicates are almost half of them. So, if we deduplicate the paths, it seems reduced to roughly 20K with the command, [here](https://gist.github.com/HyukjinKwon/dad0c8db897e5e094684a2dc6a417790). Maybe, we should consider as some more paths are added in the future but it seems better than disabling all the tests for now with minimised changes. Therefore, this PR proposes to deduplicate the paths in classpaths in case of launching executors as processes in `local-cluster` mode. ## How was this patch tested? Existing tests in `ShuffleSuite` and `BroadcastJoinSuite` manually via AppVeyor Author: hyukjinkwon <gurwls223@gmail.com> Closes #16266 from HyukjinKwon/disable-local-cluster-tests.
* [SPARK-18846][SCHEDULER] Fix flakiness in SchedulerIntegrationSuiteImran Rashid2016-12-141-2/+12
| | | | | | | | | | | | | | | | | | | There is a small race in SchedulerIntegrationSuite. The test assumes that the taskscheduler thread processing that last task will finish before the DAGScheduler processes the task event and notifies the job waiter, but that is not 100% guaranteed. ran the test locally a bunch of times, never failed, though admittedly it never failed locally for me before either. However I am nearly 100% certain this is what caused the failure of one jenkins build https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/68694/consoleFull (which is long gone now, sorry -- I fixed it as part of https://github.com/apache/spark/pull/14079 initially) Author: Imran Rashid <irashid@cloudera.com> Closes #16270 from squito/sched_integ_flakiness.
* [SPARK-13747][CORE] Fix potential ThreadLocal leaks in RPC when using ↵Shixiong Zhu2016-12-132-4/+2
| | | | | | | | | | | | | | | | | | | | | | ForkJoinPool ## What changes were proposed in this pull request? Some places in SQL may call `RpcEndpointRef.askWithRetry` (e.g., ParquetFileFormat.buildReader -> SparkContext.broadcast -> ... -> BlockManagerMaster.updateBlockInfo -> RpcEndpointRef.askWithRetry), which will finally call `Await.result`. It may cause `java.lang.IllegalArgumentException: spark.sql.execution.id is already set` when running in Scala ForkJoinPool. This PR includes the following changes to fix this issue: - Remove `ThreadUtils.awaitResult` - Rename `ThreadUtils. awaitResultInForkJoinSafely` to `ThreadUtils.awaitResult` - Replace `Await.result` in RpcTimeout with `ThreadUtils.awaitResult`. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16230 from zsxwing/fix-SPARK-13747.
* [SPARK-18773][CORE] Make commons-crypto config translation consistent.Marcelo Vanzin2016-12-121-2/+3
| | | | | | | | | | | | | | | | | | | | | This change moves the logic that translates Spark configuration to commons-crypto configuration to the network-common module. It also extends TransportConf and ConfigProvider to provide the necessary interfaces for the translation to work. As part of the change, I removed SystemPropertyConfigProvider, which was mostly used as an "empty config" in unit tests, and adjusted the very few tests that required a specific config. I also changed the config keys for AES encryption to live under the "spark.network." namespace, which is more correct than their previous names under "spark.authenticate.". Tested via existing unit test. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16200 from vanzin/SPARK-18773.
* [SPARK-15844][CORE] HistoryServer doesn't come up if spark.authenticate = trueSteve Loughran2016-12-121-3/+14
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? During history server startup, the spark configuration is examined. If security.authentication is set, log at debug and set the value to false, so that {{SecurityManager}} can be created. ## How was this patch tested? A new test in `HistoryServerSuite` sets the `spark.authenticate` property to true, tries to create a security manager via a new package-private method `HistoryServer.createSecurityManager(SparkConf)`. This is the method used in `HistoryServer.main`. All other instantiations of a security manager in `HistoryServerSuite` have been switched to the new method, for consistency with the production code. Author: Steve Loughran <stevel@apache.org> Closes #13579 from steveloughran/history/SPARK-15844-security.
* [SPARK-18803][TESTS] Fix JarEntry-related & path-related test failures and ↵hyukjinkwon2016-12-105-7/+24
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | skip some tests by path length limitation on Windows ## What changes were proposed in this pull request? This PR proposes to fix some tests being failed on Windows as below for several problems. ### Incorrect path handling - FileSuite ``` [info] - binary file input as byte array *** FAILED *** (500 milliseconds) [info] "file:/C:/projects/spark/target/tmp/spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624/record-bytestream-00000.bin" did not contain "C:\projects\spark\target\tmp\spark-e7c3a3b8-0a4b-4a7f-9ebe-7c4883e48624\record-bytestream-00000.bin" (FileSuite.scala:258) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) ... ``` ``` [info] - Get input files via old Hadoop API *** FAILED *** (1 second, 94 milliseconds) [info] Set("/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00000", "/C:/projects/spark/target/tmp/spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00000", "C:\projects\spark\target\tmp\spark-cf5b1f8b-c5ed-43e0-8d17-546ebbfa8200\output/part-00001") (FileSuite.scala:535) [info] org.scalatest.exceptions.TestFailedException: [info] at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) ... ``` ``` [info] - Get input files via new Hadoop API *** FAILED *** (313 milliseconds) [info] Set("/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00000", "/C:/projects/spark/target/tmp/spark-12bc1540-1111-4df6-9c4d-79e0e614407c/output/part-00001") did not equal Set("C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00000", "C:\projects\spark\target\tmp\spark-12bc1540-1111-4df6-9c4d-79e0e614407c\output/part-00001") (FileSuite.scala:549) [info] org.scalatest.exceptions.TestFailedException: ... ``` - TaskResultGetterSuite ``` [info] - handling results larger than max RPC message size *** FAILED *** (1 second, 579 milliseconds) [info] 1 did not equal 0 Expect result to be removed from the block manager. (TaskResultGetterSuite.scala:129) [info] org.scalatest.exceptions.TestFailedException: [info] ... [info] Cause: java.net.URISyntaxException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java [info] at java.net.URI$Parser.fail(URI.java:2848) [info] at java.net.URI$Parser.checkChars(URI.java:3021) ... ``` ``` [info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (0 milliseconds) [info] java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\spark-93c485af-68da-440f-a907-aac7acd5fc25\repro\MyException.java [info] at java.net.URI.create(URI.java:852) ... ``` - SparkSubmitSuite ``` [info] java.lang.IllegalArgumentException: Illegal character in path at index 12: string:///C:\projects\spark\target\tmp\1481210831381-0\870903339\MyLib.java [info] at java.net.URI.create(URI.java:852) [info] at org.apache.spark.TestUtils$.org$apache$spark$TestUtils$$createURI(TestUtils.scala:112) ... ``` ### Incorrect separate for JarEntry After the path fix from above, then `TaskResultGetterSuite` throws another exception as below: ``` [info] - failed task deserialized with the correct classloader (SPARK-11195) *** FAILED *** (907 milliseconds) [info] java.lang.ClassNotFoundException: repro.MyException [info] at java.net.URLClassLoader.findClass(URLClassLoader.java:381) ... ``` This is because `Paths.get` concatenates the given paths to an OS-specific path (Windows `\` and Linux `/`). However, for `JarEntry` we should comply ZIP specification meaning it should be always `/` according to ZIP specification. See `4.4.17 file name: (Variable)` in https://pkware.cachefly.net/webdocs/casestudies/APPNOTE.TXT ### Long path problem on Windows Some tests in `ShuffleSuite` via `ShuffleNettySuite` were skipped due to the same reason with SPARK-18718 ## How was this patch tested? Manually via AppVeyor. **Before** - `FileSuite`, `TaskResultGetterSuite`,`SparkSubmitSuite` https://ci.appveyor.com/project/spark-test/spark/build/164-tmp-windows-base (please grep each to check each) - `ShuffleSuite` https://ci.appveyor.com/project/spark-test/spark/build/157-tmp-windows-base **After** - `FileSuite` https://ci.appveyor.com/project/spark-test/spark/build/166-FileSuite - `TaskResultGetterSuite` https://ci.appveyor.com/project/spark-test/spark/build/173-TaskResultGetterSuite - `SparkSubmitSuite` https://ci.appveyor.com/project/spark-test/spark/build/167-SparkSubmitSuite - `ShuffleSuite` https://ci.appveyor.com/project/spark-test/spark/build/176-ShuffleSuite Author: hyukjinkwon <gurwls223@gmail.com> Closes #16234 from HyukjinKwon/test-errors-windows.
* [SPARK-4105] retry the fetch or stage if shuffle block is corruptDavies Liu2016-12-091-9/+163
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? There is an outstanding issue that existed for a long time: Sometimes the shuffle blocks are corrupt and can't be decompressed. We recently hit this in three different workloads, sometimes we can reproduce it by every try, sometimes can't. I also found that when the corruption happened, the beginning and end of the blocks are correct, the corruption happen in the middle. There was one case that the string of block id is corrupt by one character. It seems that it's very likely the corruption is introduced by some weird machine/hardware, also the checksum (16 bits) in TCP is not strong enough to identify all the corruption. Unfortunately, Spark does not have checksum for shuffle blocks or broadcast, the job will fail if any corruption happen in the shuffle block from disk, or broadcast blocks during network. This PR try to detect the corruption after fetching shuffle blocks by decompressing them, because most of the compression already have checksum in them. It will retry the block, or failed with FetchFailure, so the previous stage could be retried on different (still random) machines. Checksum for broadcast will be added by another PR. ## How was this patch tested? Added unit tests Author: Davies Liu <davies@databricks.com> Closes #15923 from davies/detect_corrupt.
* [SPARK-17822][R] Make JVMObjectTracker a member variable of RBackendXiangrui Meng2016-12-092-0/+104
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? * This PR changes `JVMObjectTracker` from `object` to `class` and let its instance associated with each RBackend. So we can manage the lifecycle of JVM objects when there are multiple `RBackend` sessions. `RBackend.close` will clear the object tracker explicitly. * I assume that `SQLUtils` and `RRunner` do not need to track JVM instances, which could be wrong. * Small refactor of `SerDe.sqlSerDe` to increase readability. ## How was this patch tested? * Added unit tests for `JVMObjectTracker`. * Wait for Jenkins to run full tests. Author: Xiangrui Meng <meng@databricks.com> Closes #16154 from mengxr/SPARK-17822.
* [SPARK-18751][CORE] Fix deadlock when SparkContext.stop is called in ↵Shixiong Zhu2016-12-081-13/+0
| | | | | | | | | | | | | | | | | | | | | | Utils.tryOrStopSparkContext ## What changes were proposed in this pull request? When `SparkContext.stop` is called in `Utils.tryOrStopSparkContext` (the following three places), it will cause deadlock because the `stop` method needs to wait for the thread running `stop` to exit. - ContextCleaner.keepCleaning - LiveListenerBus.listenerThread.run - TaskSchedulerImpl.start This PR adds `SparkContext.stopInNewThread` and uses it to eliminate the potential deadlock. I also removed my changes in #15775 since they are not necessary now. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16178 from zsxwing/fix-stop-deadlock.
* [SPARK-8617][WEBUI] HistoryServer: Include in-progress files during cleanupErgin Seyfe2016-12-081-2/+48
| | | | | | | | | | | | | ## What changes were proposed in this pull request? - Removed the`attempt.completed ` filter so cleaner would include the orphan inprogress files. - Use loading time for inprogress files as lastUpdated. Keep using the modTime for completed files. First one will prevent deletion of inprogress job files. Second one will ensure that lastUpdated time won't change for completed jobs in an event of HistoryServer reboot. ## How was this patch tested? Added new unittests and via existing tests. Author: Ergin Seyfe <eseyfe@fb.com> Closes #16165 from seyfe/clear_old_inprogress_files.
* [SPARK-18718][TESTS] Skip some test failures due to path length limitation ↵hyukjinkwon2016-12-081-0/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | and fix tests to pass on Windows ## What changes were proposed in this pull request? There are some tests failed on Windows due to the wrong format of path and the limitation of path length as below: This PR proposes both to fix the failed tests by fixing the path for the tests below: - `InsertSuite` ``` Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.sources.InsertSuite *** ABORTED *** (12 seconds, 547 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-177945ef-9128-42b4-8c07-de31f78bbbd6; at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:382) at org.apache.spark.sql.execution.datasources.DataSource$$anonfun$14.apply(DataSource.scala:370) at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241) ``` - `PathOptionSuite` ``` - path option also exist for write path *** FAILED *** (1 second, 93 milliseconds) "C:[projectsspark arget mp]spark-5ab34a58-df8d-..." did not equal "C:[\projects\spark\target\tmp\]spark-5ab34a58-df8d-..." (PathOptionSuite.scala:93) org.scalatest.exceptions.TestFailedException: at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) ... ``` - `UDFSuite` ``` - SPARK-8005 input_file_name *** FAILED *** (2 seconds, 234 milliseconds) "file:///C:/projects/spark/target/tmp/spark-e4e5720a-2006-48f9-8b11-797bf59794bf/part-00001-26fb05e4-603d-471d-ae9d-b9549e0c7765.snappy.parquet" did not contain "C:\projects\spark\target\tmp\spark-e4e5720a-2006-48f9-8b11-797bf59794bf" (UDFSuite.scala:67) org.scalatest.exceptions.TestFailedException: at org.scalatest.Assertions$class.newAssertionFailedException(Assertions.scala:500) at org.scalatest.FunSuite.newAssertionFailedException(FunSuite.scala:1555) ... ``` and to skip the tests belows which are being failed on Windows due to path length limitation. - `SparkLauncherSuite` ``` Test org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher failed: java.lang.AssertionError: expected:<0> but was:<1>, took 0.062 sec at org.apache.spark.launcher.SparkLauncherSuite.testChildProcLauncher(SparkLauncherSuite.java:177) ... ``` The stderr from the process is `The filename or extension is too long` which is equivalent to the one below. - `BroadcastJoinSuite` ``` 04:09:40.882 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor java.io.IOException: Cannot run program "C:\Progra~1\Java\jdk1.8.0\bin\java" (in directory "C:\projects\spark\work\app-20161205040542-0000\51658"): CreateProcess error=206, The filename or extension is too long at java.lang.ProcessBuilder.start(ProcessBuilder.java:1048) at org.apache.spark.deploy.worker.ExecutorRunner.org$apache$spark$deploy$worker$ExecutorRunner$$fetchAndRunExecutor(ExecutorRunner.scala:167) at org.apache.spark.deploy.worker.ExecutorRunner$$anon$1.run(ExecutorRunner.scala:73) Caused by: java.io.IOException: CreateProcess error=206, The filename or extension is too long at java.lang.ProcessImpl.create(Native Method) at java.lang.ProcessImpl.<init>(ProcessImpl.java:386) at java.lang.ProcessImpl.start(ProcessImpl.java:137) at java.lang.ProcessBuilder.start(ProcessBuilder.java:1029) ... 2 more 04:09:40.929 ERROR org.apache.spark.deploy.worker.ExecutorRunner: Error running executor (appearently infinite same error messages) ... ``` ## How was this patch tested? Manually tested via AppVeyor. **Before** `InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/148-InsertSuite-pr `PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/139-PathOptionSuite-pr `UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/143-UDFSuite-pr `SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/141-SparkLauncherSuite-pr `BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/145-BroadcastJoinSuite-pr **After** `PathOptionSuite`: https://ci.appveyor.com/project/spark-test/spark/build/140-PathOptionSuite-pr `SparkLauncherSuite`: https://ci.appveyor.com/project/spark-test/spark/build/142-SparkLauncherSuite-pr `UDFSuite`: https://ci.appveyor.com/project/spark-test/spark/build/144-UDFSuite-pr `InsertSuite`: https://ci.appveyor.com/project/spark-test/spark/build/147-InsertSuite-pr `BroadcastJoinSuite`: https://ci.appveyor.com/project/spark-test/spark/build/149-BroadcastJoinSuite-pr Author: hyukjinkwon <gurwls223@gmail.com> Closes #16147 from HyukjinKwon/fix-tests.
* [SPARK-18678][ML] Skewed reservoir sampling in SamplingUtilsSean Owen2016-12-071-0/+13
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Fix reservoir sampling bias for small k. An off-by-one error meant that the probability of replacement was slightly too high -- k/(l-1) after l element instead of k/l, which matters for small k. ## How was this patch tested? Existing test plus new test case. Author: Sean Owen <sowen@cloudera.com> Closes #16129 from srowen/SPARK-18678.
* [SPARK-18666][WEB UI] Remove the codes checking deprecated config ↵Liang-Chi Hsieh2016-12-011-13/+3
| | | | | | | | | | | | | | | | | | spark.sql.unsafe.enabled ## What changes were proposed in this pull request? `spark.sql.unsafe.enabled` is deprecated since 1.6. There still are codes in UI to check it. We should remove it and clean the codes. ## How was this patch tested? Changes to related existing unit test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16095 from viirya/remove-deprecated-config-code.
* [SPARK-18546][CORE] Fix merging shuffle spills when using encryption.Marcelo Vanzin2016-11-306-91/+108
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | The problem exists because it's not possible to just concatenate encrypted partition data from different spill files; currently each partition would have its own initial vector to set up encryption, and the final merged file should contain a single initial vector for each merged partiton, otherwise iterating over each record becomes really hard. To fix that, UnsafeShuffleWriter now decrypts the partitions when merging, so that the merged file contains a single initial vector at the start of the partition data. Because it's not possible to do that using the fast transferTo path, when encryption is enabled UnsafeShuffleWriter will revert back to using file streams when merging. It may be possible to use a hybrid approach when using encryption, using an intermediate direct buffer when reading from files and encrypting the data, but that's better left for a separate patch. As part of the change I made DiskBlockObjectWriter take a SerializerManager instead of a "wrap stream" closure, since that makes it easier to test the code without having to mock SerializerManager functionality. Tested with newly added unit tests (UnsafeShuffleWriterSuite for the write side and ExternalAppendOnlyMapSuite for integration), and by running some apps that failed without the fix. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #15982 from vanzin/SPARK-18546.
* [SPARK-18640] Add synchronization to TaskScheduler.runningTasksByExecutorsJosh Rosen2016-11-301-2/+2
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The method `TaskSchedulerImpl.runningTasksByExecutors()` accesses the mutable `executorIdToRunningTaskIds` map without proper synchronization. In addition, as markhamstra pointed out in #15986, the signature's use of parentheses is a little odd given that this is a pure getter method. This patch fixes both issues. ## How was this patch tested? Covered by existing tests. Author: Josh Rosen <joshrosen@databricks.com> Closes #16073 from JoshRosen/runningTasksByExecutors-thread-safety.
* [SPARK-18617][CORE][STREAMING] Close "kryo auto pick" feature for Spark ↵uncleGen2016-11-291-2/+4
| | | | | | | | | | | | | | | | 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.
* [SPARK-18553][CORE] Fix leak of TaskSetManager following executor lossJosh Rosen2016-11-292-3/+72
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? _This is the master branch version of #15986; the original description follows:_ This patch fixes a critical resource leak in the TaskScheduler which could cause RDDs and ShuffleDependencies to be kept alive indefinitely if an executor with running tasks is permanently lost and the associated stage fails. This problem was originally identified by analyzing the heap dump of a driver belonging to a cluster that had run out of shuffle space. This dump contained several `ShuffleDependency` instances that were retained by `TaskSetManager`s inside the scheduler but were not otherwise referenced. Each of these `TaskSetManager`s was considered a "zombie" but had no running tasks and therefore should have been cleaned up. However, these zombie task sets were still referenced by the `TaskSchedulerImpl.taskIdToTaskSetManager` map. Entries are added to the `taskIdToTaskSetManager` map when tasks are launched and are removed inside of `TaskScheduler.statusUpdate()`, which is invoked by the scheduler backend while processing `StatusUpdate` messages from executors. The problem with this design is that a completely dead executor will never send a `StatusUpdate`. There is [some code](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L338) in `statusUpdate` which handles tasks that exit with the `TaskState.LOST` state (which is supposed to correspond to a task failure triggered by total executor loss), but this state only seems to be used in Mesos fine-grained mode. There doesn't seem to be any code which performs per-task state cleanup for tasks that were running on an executor that completely disappears without sending any sort of final death message. The `executorLost` and [`removeExecutor`](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L527) methods don't appear to perform any cleanup of the `taskId -> *` mappings, causing the leaks observed here. This patch's fix is to maintain a `executorId -> running task id` mapping so that these `taskId -> *` maps can be properly cleaned up following an executor loss. There are some potential corner-case interactions that I'm concerned about here, especially some details in [the comment](https://github.com/apache/spark/blob/072f4c518cdc57d705beec6bcc3113d9a6740819/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala#L523) in `removeExecutor`, so I'd appreciate a very careful review of these changes. ## How was this patch tested? I added a new unit test to `TaskSchedulerImplSuite`. /cc kayousterhout and markhamstra, who reviewed #15986. Author: Josh Rosen <joshrosen@databricks.com> Closes #16045 from JoshRosen/fix-leak-following-total-executor-loss-master.
* [SPARK-18547][CORE] Propagate I/O encryption key when executors register.Marcelo Vanzin2016-11-281-56/+79
| | | | | | | | | | | | | | | | | | | | | | | | This change modifies the method used to propagate encryption keys used during shuffle. Instead of relying on YARN's UserGroupInformation credential propagation, this change explicitly distributes the key using the messages exchanged between driver and executor during registration. When RPC encryption is enabled, this means key propagation is also secure. This allows shuffle encryption to work in non-YARN mode, which means that it's easier to write unit tests for areas of the code that are affected by the feature. The key is stored in the SecurityManager; because there are many instances of that class used in the code, the key is only guaranteed to exist in the instance managed by the SparkEnv. This path was chosen to avoid storing the key in the SparkConf, which would risk having the key being written to disk as part of the configuration (as, for example, is done when starting YARN applications). Tested by new and existing unit tests (which were moved from the YARN module to core), and by running apps with shuffle encryption enabled. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #15981 from vanzin/SPARK-18547.
* [SPARK-18117][CORE] Add test for TaskSetBlacklistImran Rashid2016-11-282-8/+291
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This adds tests to verify the interaction between TaskSetBlacklist and TaskSchedulerImpl. TaskSetBlacklist was introduced by SPARK-17675 but it neglected to add these tests. This change does not fix any bugs -- it is just for increasing test coverage. ## How was this patch tested? Jenkins Author: Imran Rashid <irashid@cloudera.com> Closes #15644 from squito/taskset_blacklist_test_update.
* [SPARK-18535][UI][YARN] Redact sensitive information from Spark logs and UIMark Grover2016-11-282-0/+32
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds a new property called `spark.secret.redactionPattern` that allows users to specify a scala regex to decide which Spark configuration properties and environment variables in driver and executor environments contain sensitive information. When this regex matches the property or environment variable name, its value is redacted from the environment UI and various logs like YARN and event logs. This change uses this property to redact information from event logs and YARN logs. It also, updates the UI code to adhere to this property instead of hardcoding the logic to decipher which properties are sensitive. Here's an image of the UI post-redaction: ![image](https://cloud.githubusercontent.com/assets/1709451/20506215/4cc30654-b007-11e6-8aee-4cde253fba2f.png) Here's the text in the YARN logs, post-redaction: ``HADOOP_CREDSTORE_PASSWORD -> *********(redacted)`` Here's the text in the event logs, post-redaction: ``...,"spark.executorEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)","spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD":"*********(redacted)",...`` ## How was this patch tested? 1. Unit tests are added to ensure that redaction works. 2. A YARN job reading data off of S3 with confidential information (hadoop credential provider password) being provided in the environment variables of driver and executor. And, afterwards, logs were grepped to make sure that no mention of secret password was present. It was also ensure that the job was able to read the data off of S3 correctly, thereby ensuring that the sensitive information was being trickled down to the right places to read the data. 3. The event logs were checked to make sure no mention of secret password was present. 4. UI environment tab was checked to make sure there was no secret information being displayed. Author: Mark Grover <mark@apache.org> Closes #15971 from markgrover/master_redaction.
* [SPARK-18458][CORE] Fix signed integer overflow problem at an expression in ↵Kazuaki Ishizaki2016-11-191-14/+14
| | | | | | | | | | | | | | | | RadixSort.java ## What changes were proposed in this pull request? This PR avoids that a result of an expression is negative due to signed integer overflow (e.g. 0x10?????? * 8 < 0). This PR casts each operand to `long` before executing a calculation. Since the result is interpreted as long, the result of the expression is positive. ## How was this patch tested? Manually executed query82 of TPC-DS with 100TB Author: Kazuaki Ishizaki <ishizaki@jp.ibm.com> Closes #15907 from kiszk/SPARK-18458.