aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-19534][TESTS] Convert Java tests to use lambdas, Java 8 featuresSean Owen2017-02-197-497/+144
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Convert tests to use Java 8 lambdas, and modest related fixes to surrounding code. ## How was this patch tested? Jenkins tests Author: Sean Owen <sowen@cloudera.com> Closes #16964 from srowen/SPARK-19534.
* [SPARK-19450] Replace askWithRetry with askSync.jinxing2017-02-1919-108/+47
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `askSync` is already added in `RpcEndpointRef` (see SPARK-19347 and https://github.com/apache/spark/pull/16690#issuecomment-276850068) and `askWithRetry` is marked as deprecated. As mentioned SPARK-18113(https://github.com/apache/spark/pull/16503#event-927953218): >askWithRetry is basically an unneeded API, and a leftover from the akka days that doesn't make sense anymore. It's prone to cause deadlocks (exactly because it's blocking), it imposes restrictions on the caller (e.g. idempotency) and other things that people generally don't pay that much attention to when using it. Since `askWithRetry` is just used inside spark and not in user logic. It might make sense to replace all of them with `askSync`. ## How was this patch tested? This PR doesn't change code logic, existing unit test can cover. Author: jinxing <jinxing@meituan.com> Closes #16790 from jinxing64/SPARK-19450.
* [SPARK-19263] DAGScheduler should avoid sending conflicting task set.jinxing2017-02-183-3/+91
| | | | | | | | | | | | | | | | | | | | | | | | | In current `DAGScheduler handleTaskCompletion` code, when event.reason is `Success`, it will first do `stage.pendingPartitions -= task.partitionId`, which maybe a bug when `FetchFailed` happens. **Think about below** 1. Stage 0 runs and generates shuffle output data. 2. Stage 1 reads the output from stage 0 and generates more shuffle data. It has two tasks: ShuffleMapTask1 and ShuffleMapTask2, and these tasks are launched on executorA. 3. ShuffleMapTask1 fails to fetch blocks locally and sends a FetchFailed to the driver. The driver marks executorA as lost and updates failedEpoch; 4. The driver resubmits stage 0 so the missing output can be re-generated, and then once it completes, resubmits stage 1 with ShuffleMapTask1x and ShuffleMapTask2x. 5. ShuffleMapTask2 (from the original attempt of stage 1) successfully finishes on executorA and sends Success back to driver. This causes DAGScheduler::handleTaskCompletion to remove partition 2 from stage.pendingPartitions (line 1149), but it does not add the partition to the set of output locations (line 1192), because the task’s epoch is less than the failure epoch for the executor (because of the earlier failure on executor A) 6. ShuffleMapTask1x successfully finishes on executorB, causing the driver to remove partition 1 from stage.pendingPartitions. Combined with the previous step, this means that there are no more pending partitions for the stage, so the DAGScheduler marks the stage as finished (line 1196). However, the shuffle stage is not available (line 1215) because the completion for ShuffleMapTask2 was ignored because of its epoch, so the DAGScheduler resubmits the stage. 7. ShuffleMapTask2x is still running, so when TaskSchedulerImpl::submitTasks is called for the re-submitted stage, it throws an error, because there’s an existing active task set **In this fix** If a task completion is from a previous stage attempt and the epoch is too low (i.e., it was from a failed executor), don't remove the corresponding partition from pendingPartitions. Author: jinxing <jinxing@meituan.com> Author: jinxing <jinxing6042@126.com> Closes #16620 from jinxing64/SPARK-19263.
* [SPARK-18986][CORE] ExternalAppendOnlyMap shouldn't fail when forced to ↵Liang-Chi Hsieh2017-02-172-5/+23
| | | | | | | | | | | | | | | | | | | | | | | | | | spill before calling its iterator ## What changes were proposed in this pull request? `ExternalAppendOnlyMap.forceSpill` now uses an assert to check if an iterator is not null in the map. However, the assertion is only true after the map is asked for iterator. Before it, if another memory consumer asks more memory than currently available, `ExternalAppendOnlyMap.forceSpill` is also be called too. In this case, we will see failure like this: [info] java.lang.AssertionError: assertion failed [info] at scala.Predef$.assert(Predef.scala:156) [info] at org.apache.spark.util.collection.ExternalAppendOnlyMap.forceSpill(ExternalAppendOnlyMap.scala:196) [info] at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:111) [info] at org.apache.spark.util.collection.ExternalAppendOnlyMapSuite$$anonfun$13.apply$mcV$sp(ExternalAppendOnlyMapSuite.scala:294) This fixing is motivated by http://apache-spark-developers-list.1001551.n3.nabble.com/java-lang-AssertionError-assertion-failed-tc20277.html. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16387 from viirya/fix-externalappendonlymap.
* [SPARK-19500] [SQL] Fix off-by-one bug in BytesToBytesMapDavies Liu2017-02-171-2/+3
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Radix sort require that half of array as free (as temporary space), so we use 0.5 as the scale factor to make sure that BytesToBytesMap will not have more items than 1/2 of capacity. Turned out this is not true, the current implementation of append() could leave 1 more item than the threshold (1/2 of capacity) in the array, which break the requirement of radix sort (fail the assert in 2.2, or fail to insert into InMemorySorter in 2.1). This PR fix the off-by-one bug in BytesToBytesMap. This PR also fix a bug that the array will never grow if it fail to grow once (stay as initial capacity), introduced by #15722 . ## How was this patch tested? Added regression test. Author: Davies Liu <davies@databricks.com> Closes #16844 from davies/off_by_one.
* [SPARK-19622][WEBUI] Fix a http error in a paged table when using a `Go` ↵Stan Zhai2017-02-171-2/+3
| | | | | | | | | | | | | | | | | | button to search. ## What changes were proposed in this pull request? The search function of paged table is not available because of we don't skip the hash data of the reqeust path. ![](https://issues.apache.org/jira/secure/attachment/12852996/screenshot-1.png) ## How was this patch tested? Tested manually with my browser. Author: Stan Zhai <zhaishidan@haizhi.com> Closes #16953 from stanzhai/fix-webui-paged-table.
* [SPARK-18352][SQL] Support parsing multiline json filesNathan Howell2017-02-161-0/+7
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? If a new option `wholeFile` is set to `true` the JSON reader will parse each file (instead of a single line) as a value. This is done with Jackson streaming and it should be capable of parsing very large documents, assuming the row will fit in memory. Because the file is not buffered in memory the corrupt record handling is also slightly different when `wholeFile` is enabled: the corrupt column will contain the filename instead of the literal JSON if there is a parsing failure. It would be easy to extend this to add the parser location (line, column and byte offsets) to the output if desired. These changes have allowed types other than `String` to be parsed. Support for `UTF8String` and `Text` have been added (alongside `String` and `InputFormat`) and no longer require a conversion to `String` just for parsing. I've also included a few other changes that generate slightly better bytecode and (imo) make it more obvious when and where boxing is occurring in the parser. These are included as separate commits, let me know if they should be flattened into this PR or moved to a new one. ## How was this patch tested? New and existing unit tests. No performance or load tests have been run. Author: Nathan Howell <nhowell@godaddy.com> Closes #16386 from NathanHowell/SPARK-18352.
* [SPARK-19550][BUILD][CORE][WIP] Remove Java 7 supportSean Owen2017-02-1631-75/+409
| | | | | | | | | | | | | | | | | | | | | | | | - Move external/java8-tests tests into core, streaming, sql and remove - Remove MaxPermGen and related options - Fix some reflection / TODOs around Java 8+ methods - Update doc references to 1.7/1.8 differences - Remove Java 7/8 related build profiles - Update some plugins for better Java 8 compatibility - Fix a few Java-related warnings For the future: - Update Java 8 examples to fully use Java 8 - Update Java tests to use lambdas for simplicity - Update Java internal implementations to use lambdas ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16871 from srowen/SPARK-19493.
* [SPARK-19399][SPARKR] Add R coalesce API for DataFrame and ColumnFelix Cheung2017-02-151-1/+2
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Add coalesce on DataFrame for down partitioning without shuffle and coalesce on Column ## How was this patch tested? manual, unit tests Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16739 from felixcheung/rcoalesce.
* [SPARK-19520][STREAMING] Do not encrypt data written to the WAL.Marcelo Vanzin2017-02-133-9/+48
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Spark's I/O encryption uses an ephemeral key for each driver instance. So driver B cannot decrypt data written by driver A since it doesn't have the correct key. The write ahead log is used for recovery, thus needs to be readable by a different driver. So it cannot be encrypted by Spark's I/O encryption code. The BlockManager APIs used by the WAL code to write the data automatically encrypt data, so changes are needed so that callers can to opt out of encryption. Aside from that, the "putBytes" API in the BlockManager does not do encryption, so a separate situation arised where the WAL would write unencrypted data to the BM and, when those blocks were read, decryption would fail. So the WAL code needs to ask the BM to encrypt that data when encryption is enabled; this code is not optimal since it results in a (temporary) second copy of the data block in memory, but should be OK for now until a more performant solution is added. The non-encryption case should not be affected. Tested with new unit tests, and by running streaming apps that do recovery using the WAL data with I/O encryption turned on. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16862 from vanzin/SPARK-19520.
* [SPARK-17714][CORE][TEST-MAVEN][TEST-HADOOP2.6] Avoid using ↵Shixiong Zhu2017-02-131-12/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ExecutorClassLoader to load Netty generated classes ## What changes were proposed in this pull request? Netty's `MessageToMessageEncoder` uses [Javassist](https://github.com/netty/netty/blob/91a0bdc17a8298437d6de08a8958d753799bd4a6/common/src/main/java/io/netty/util/internal/JavassistTypeParameterMatcherGenerator.java#L62) to generate a matcher class and the implementation calls `Class.forName` to check if this class is already generated. If `MessageEncoder` or `MessageDecoder` is created in `ExecutorClassLoader.findClass`, it will cause `ClassCircularityError`. This is because loading this Netty generated class will call `ExecutorClassLoader.findClass` to search this class, and `ExecutorClassLoader` will try to use RPC to load it and cause to load the non-exist matcher class again. JVM will report `ClassCircularityError` to prevent such infinite recursion. ##### Why it only happens in Maven builds It's because Maven and SBT have different class loader tree. The Maven build will set a URLClassLoader as the current context class loader to run the tests and expose this issue. The class loader tree is as following: ``` bootstrap class loader ------ ... ----- REPL class loader ---- ExecutorClassLoader | | URLClasssLoader ``` The SBT build uses the bootstrap class loader directly and `ReplSuite.test("propagation of local properties")` is the first test in ReplSuite, which happens to load `io/netty/util/internal/__matchers__/org/apache/spark/network/protocol/MessageMatcher` into the bootstrap class loader (Note: in maven build, it's loaded into URLClasssLoader so it cannot be found in ExecutorClassLoader). This issue can be reproduced in SBT as well. Here are the produce steps: - Enable `hadoop.caller.context.enabled`. - Replace `Class.forName` with `Utils.classForName` in `object CallerContext`. - Ignore `ReplSuite.test("propagation of local properties")`. - Run `ReplSuite` using SBT. This PR just creates a singleton MessageEncoder and MessageDecoder and makes sure they are created before switching to ExecutorClassLoader. TransportContext will be created when creating RpcEnv and that happens before creating ExecutorClassLoader. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16859 from zsxwing/SPARK-17714.
* [SPARK-19537] Move pendingPartitions to ShuffleMapStage.Kay Ousterhout2017-02-103-9/+26
| | | | | | | | | | | | | | The pendingPartitions instance variable should be moved to ShuffleMapStage, because it is only used by ShuffleMapStages. This change is purely refactoring and does not change functionality. I fixed this in an attempt to clarify some of the discussion around #16620, which I was having trouble reasoning about. I stole the helpful comment Imran wrote for pendingPartitions and used it here. cc squito markhamstra jinxing64 Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #16876 from kayousterhout/SPARK-19537.
* [SPARK-19549] Allow providing reason for stage/job cancellingAla Luszczak2017-02-106-25/+123
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? This change add an optional argument to `SparkContext.cancelStage()` and `SparkContext.cancelJob()` functions, which allows the caller to provide exact reason for the cancellation. ## How was this patch tested? Adds unit test. Author: Ala Luszczak <ala@databricks.com> Closes #16887 from ala/cancel.
* [SPARK-19466][CORE][SCHEDULER] Improve Fair Scheduler LoggingEren Avsarogullari2017-02-101-21/+46
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | Fair Scheduler Logging for the following cases can be useful for the user. 1. If **valid** `spark.scheduler.allocation.file` property is set, user can be informed and aware which scheduler file is processed when `SparkContext` initializes. 2. If **invalid** `spark.scheduler.allocation.file` property is set, currently, the following stacktrace is shown to user. In addition to this, more meaningful message can be shown to user by emphasizing the problem at building level of fair scheduler. Also other potential issues can be covered at this level as **Fair Scheduler can not be built. + exception stacktrace** ``` Exception in thread "main" java.io.FileNotFoundException: INVALID_FILE (No such file or directory) at java.io.FileInputStream.open0(Native Method) at java.io.FileInputStream.open(FileInputStream.java:195) at java.io.FileInputStream.<init>(FileInputStream.java:138) at java.io.FileInputStream.<init>(FileInputStream.java:93) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$buildPools$1.apply(SchedulableBuilder.scala:76) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$buildPools$1.apply(SchedulableBuilder.scala:75) ``` 3. If `spark.scheduler.allocation.file` property is not set and **default** fair scheduler file (**fairscheduler.xml**) is found in classpath, it will be loaded but currently, user is not informed for using default file so logging can be useful as **Fair Scheduler file: fairscheduler.xml is found successfully and will be parsed.** 4. If **spark.scheduler.allocation.file** property is not set and **default** fair scheduler file does not exist in classpath, currently, user is not informed so logging can be useful as **No Fair Scheduler file found.** Also this PR is related with https://github.com/apache/spark/pull/15237 to emphasize fileName in warning logs when fair scheduler file has invalid minShare, weight or schedulingMode values. ## How was this patch tested? Added new Unit Tests. Author: erenavsarogullari <erenavsarogullari@gmail.com> Closes #16813 from erenavsarogullari/SPARK-19466.
* [SPARK-19263] Fix race in SchedulerIntegrationSuite.jinxing2017-02-091-7/+7
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? All the process of offering resource and generating `TaskDescription` should be guarded by taskScheduler.synchronized in `reviveOffers`, otherwise there is race condition. ## How was this patch tested? Existing unit tests. Author: jinxing <jinxing@meituan.com> Closes #16831 from jinxing64/SPARK-19263-FixRaceInTest.
* [SPARK-19481] [REPL] [MAVEN] Avoid to leak SparkContext in ↵Shixiong Zhu2017-02-091-0/+7
| | | | | | | | | | | | | | | | | | Signaling.cancelOnInterrupt ## What changes were proposed in this pull request? `Signaling.cancelOnInterrupt` leaks a SparkContext per call and it makes ReplSuite unstable. This PR adds `SparkContext.getActive` to allow `Signaling.cancelOnInterrupt` to get the active `SparkContext` to avoid the leak. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16825 from zsxwing/SPARK-19481.
* [SPARK-16554][CORE] Automatically Kill Executors and Nodes when they are ↵José Hiram Soltren2017-02-0910-26/+239
| | | | | | | | | | | | | | | | | | | | | | | Blacklisted ## What changes were proposed in this pull request? In SPARK-8425, we introduced a mechanism for blacklisting executors and nodes (hosts). After a certain number of failures, these resources would be "blacklisted" and no further work would be assigned to them for some period of time. In some scenarios, it is better to fail fast, and to simply kill these unreliable resources. This changes proposes to do so by having the BlacklistTracker kill unreliable resources when they would otherwise be "blacklisted". In order to be thread safe, this code depends on the CoarseGrainedSchedulerBackend sending a message to the driver backend in order to do the actual killing. This also helps to prevent a race which would permit work to begin on a resource (executor or node), between the time the resource is marked for killing and the time at which it is finally killed. ## How was this patch tested? ./dev/run-tests Ran https://github.com/jsoltren/jose-utils/blob/master/blacklist/test-blacklist.sh, and checked logs to see executors and nodes being killed. Testing can likely be improved here; suggestions welcome. Author: José Hiram Soltren <jose@cloudera.com> Closes #16650 from jsoltren/SPARK-16554-submit.
* [SPARK-17874][CORE] Add SSL port configuration.Marcelo Vanzin2017-02-095-88/+149
| | | | | | | | | | | | | | | | | | | | | | Make the SSL port configuration explicit, instead of deriving it from the non-SSL port, but retain the existing functionality in case anyone depends on it. The change starts the HTTPS and HTTP connectors separately, so that it's possible to use independent ports for each. For that to work, the initialization of the server needs to be shuffled around a bit. The change also makes it so the initialization of both connectors is similar, and end up using the same Scheduler - previously only the HTTP connector would use the correct one. Also fixed some outdated documentation about a couple of services that were removed long ago. Tested with unit tests and by running spark-shell with SSL configs. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #16625 from vanzin/SPARK-17874.
* [MINOR][CORE] Fix incorrect documentation of WritableConverterLee Dongjin2017-02-091-8/+10
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? `WritableConverter` and `WritableFactory` work in opposite directions. But both of them are documented with same description: > A class encapsulating how to convert some type T to Writable. It stores both the Writable class corresponding to T (e.g. IntWritable for Int) and a function for doing the conversion. This error is a result of commit 2604939. As a note, `WritableFactory` was added from commit d37978d, which resolves [SPARK-4795](https://issues.apache.org/jira/browse/SPARK-4795) with the correct description. This PR fix the documentation of `WritableConverter`, along with some improvements on type description. ## How was this patch tested? `build/mvn clean checkstyle:checkstyle` Author: Lee Dongjin <dongjin@apache.org> Closes #16830 from dongjinleekr/feature/fix-writableconverter-doc.
* [SPARK-19265][SQL][FOLLOW-UP] Configurable `tableRelationCache` maximum sizeLiwei Lin2017-02-092-0/+30
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable. In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`. ## How was this patch tested? new test case: `test("conf entry: checkValue()")` Author: Liwei Lin <lwlin7@gmail.com> Closes #16736 from lw-lin/conf.
* [SPARK-19464][CORE][YARN][TEST-HADOOP2.6] Remove support for Hadoop 2.5 and ↵Sean Owen2017-02-086-189/+79
| | | | | | | | | | | | | | | | | | | earlier ## What changes were proposed in this pull request? - Remove support for Hadoop 2.5 and earlier - Remove reflection and code constructs only needed to support multiple versions at once - Update docs to reflect newer versions - Remove older versions' builds and profiles. ## How was this patch tested? Existing tests Author: Sean Owen <sowen@cloudera.com> Closes #16810 from srowen/SPARK-19464.
* [SPARK-19260] Spaces or "%20" in path parameter are not correctly handled ↵zuotingbing2017-02-072-3/+2
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | with… JIRA Issue: https://issues.apache.org/jira/browse/SPARK-19260 ## What changes were proposed in this pull request? 1. “spark.history.fs.logDirectory” supports with space character and “%20” characters. 2. As usually, if the run classpath includes hdfs-site.xml and core-site.xml files, the supplied path eg."/test" which does not contain a scheme should be taken as a HDFS path rather than a local path since the path parameter is a Hadoop dir. ## How was this patch tested? Update Unit Test and take some manual tests local: .sbin/start-history-server.sh "file:/a b" .sbin/start-history-server.sh "/abc%20c" (without hdfs-site.xml,core-site.xml) .sbin/start-history-server.sh "/a b" (without hdfs-site.xml,core-site.xml) .sbin/start-history-server.sh "/a b/a bc%20c" (without hdfs-site.xml,core-site.xml) hdfs: .sbin/start-history-server.sh "hdfs:/namenode:9000/a b" .sbin/start-history-server.sh "/a b" (with hdfs-site.xml,core-site.xml) .sbin/start-history-server.sh "/a b/a bc%20c" (with hdfs-site.xml,core-site.xml) Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #16614 from zuotingbing/SPARK-19260.
* [SPARK-18967][SCHEDULER] compute locality levels even if delay = 0Imran Rashid2017-02-063-8/+105
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Before this change, with delay scheduling off, spark would effectively ignore locality preferences for bulk scheduling. With this change, locality preferences are used when multiple offers are made simultaneously. ## How was this patch tested? Test case added which fails without this change. All unit tests run via jenkins. Author: Imran Rashid <irashid@cloudera.com> Closes #16376 from squito/locality_without_delay.
* [SPARK-19398] Change one misleading log in TaskSetManager.Jin Xing2017-02-061-4/+8
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Log below is misleading: ``` if (successful(index)) { logInfo( s"Task ${info.id} in stage ${taskSet.id} (TID $tid) failed, " + "but another instance of the task has already succeeded, " + "so not re-queuing the task to be re-executed.") } ``` If fetch failed, the task is marked as successful in `TaskSetManager:: handleFailedTask`. Then log above will be printed. The `successful` just means task will not be scheduled any longer, not a real success. ## How was this patch tested? Existing unit tests can cover this. Author: jinxing <jinxing@meituan.com> Closes #16738 from jinxing64/SPARK-19398.
* [SPARK-17663][CORE] SchedulableBuilder should handle invalid data access via ↵erenavsarogullari2017-02-063-51/+182
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | scheduler.allocation.file ## What changes were proposed in this pull request? If `spark.scheduler.allocation.file` has invalid `minShare` or/and `weight` values, these cause : - `NumberFormatException` due to `toInt` function - `SparkContext` can not be initialized. - It does not show meaningful error message to user. In a nutshell, this functionality can be more robust by selecting one of the following flows : **1-** Currently, if `schedulingMode` has an invalid value, a warning message is logged and default value is set as `FIFO`. Same pattern can be used for `minShare`(default: 0) and `weight`(default: 1) as well **2-** Meaningful error message can be shown to the user for all invalid cases. PR offers : - `schedulingMode` handles just empty values. It also needs to be supported for **whitespace**, **non-uppercase**(fair, FaIr etc...) or `SchedulingMode.NONE` cases by setting default value(`FIFO`) - `minShare` and `weight` handle just empty values. They also need to be supported for **non-integer** cases by setting default values. - Some refactoring of `PoolSuite`. **Code to Reproduce :** ``` val conf = new SparkConf().setAppName("spark-fairscheduler").setMaster("local") conf.set("spark.scheduler.mode", "FAIR") conf.set("spark.scheduler.allocation.file", "src/main/resources/fairscheduler-invalid-data.xml") val sc = new SparkContext(conf) ``` **fairscheduler-invalid-data.xml :** ``` <allocations> <pool name="production"> <schedulingMode>FIFO</schedulingMode> <weight>invalid_weight</weight> <minShare>2</minShare> </pool> </allocations> ``` **Stacktrace :** ``` Exception in thread "main" java.lang.NumberFormatException: For input string: "invalid_weight" at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) at java.lang.Integer.parseInt(Integer.java:580) at java.lang.Integer.parseInt(Integer.java:615) at scala.collection.immutable.StringLike$class.toInt(StringLike.scala:272) at scala.collection.immutable.StringOps.toInt(StringOps.scala:29) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$org$apache$spark$scheduler$FairSchedulableBuilder$$buildFairSchedulerPool$1.apply(SchedulableBuilder.scala:127) at org.apache.spark.scheduler.FairSchedulableBuilder$$anonfun$org$apache$spark$scheduler$FairSchedulableBuilder$$buildFairSchedulerPool$1.apply(SchedulableBuilder.scala:102) ``` ## How was this patch tested? Added Unit Test Case. Author: erenavsarogullari <erenavsarogullari@gmail.com> Closes #15237 from erenavsarogullari/SPARK-17663.
* [SPARK-10063] Follow-up: remove dead code related to an old output committer.Reynold Xin2017-02-032-18/+0
| | | | | | | | | | | | ## What changes were proposed in this pull request? DirectParquetOutputCommitter was removed from Spark as it was deemed unsafe to use. We however still have some code to generate warning. This patch removes those code as well. ## How was this patch tested? N/A Author: Reynold Xin <rxin@databricks.com> Closes #16796 from rxin/remove-direct.
* [SPARK-19244][CORE] Sort MemoryConsumers according to their memory usage ↵Liang-Chi Hsieh2017-02-032-13/+78
| | | | | | | | | | | | | | | | | | | | | | | | | | when spilling ## What changes were proposed in this pull request? In `TaskMemoryManager `, when we acquire memory by calling `acquireExecutionMemory` and we can't acquire required memory, we will try to spill other memory consumers. Currently, we simply iterates the memory consumers in a hash set. Normally each time the consumer will be iterated in the same order. The first issue is that we might spill additional consumers. For example, if consumer 1 uses 10MB, consumer 2 uses 50MB, then consumer 3 acquires 100MB but we can only get 60MB and spilling is needed. We might spill both consumer 1 and consumer 2. But we actually just need to spill consumer 2 and get the required 100MB. The second issue is that if we spill consumer 1 in first time spilling. After a while, consumer 1 now uses 5MB. Then consumer 4 may acquire some memory and spilling is needed again. Because we iterate the memory consumers in the same order, we will spill consumer 1 again. So for consumer 1, we will produce many small spilling files. This patch modifies the way iterating the memory consumers. It sorts the memory consumers by their memory usage. So the consumer using more memory will spill first. Once it is spilled, even it acquires few memory again, in next time spilling happens it will not be the consumers to spill again if there are other consumers using more memory than it. ## How was this patch tested? Jenkins tests. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16603 from viirya/sort-memoryconsumer-when-spill.
* [SPARK-19437] Rectify spark executor id in HeartbeatReceiverSuite.jinxing2017-02-021-13/+13
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? The current code in `HeartbeatReceiverSuite`, executorId is set as below: ``` private val executorId1 = "executor-1" private val executorId2 = "executor-2" ``` The executorId is sent to driver when register as below: ``` test("expire dead hosts should kill executors with replacement (SPARK-8119)") { ... fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty)) ... } ``` Receiving `RegisterExecutor` in `CoarseGrainedSchedulerBackend`, the executorId will be compared with `currentExecutorIdCounter` as below: ``` case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls) => if (executorDataMap.contains(executorId)) { executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) context.reply(true) } else { ... executorDataMap.put(executorId, data) if (currentExecutorIdCounter < executorId.toInt) { currentExecutorIdCounter = executorId.toInt } ... ``` `executorId.toInt` will cause NumberformatException. This unit test can pass currently because of `askWithRetry`, when catching exception, RPC will call again, thus it will go `if` branch and return true. **To fix** Rectify executorId and replace `askWithRetry` with `askSync`, refer to https://github.com/apache/spark/pull/16690 ## How was this patch tested? This fix is for unit test and no need to add another one.(If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: jinxing <jinxing@meituan.com> Closes #16779 from jinxing64/SPARK-19437.
* [SPARK-19432][CORE] Fix an unexpected failure when connecting timeoutShixiong Zhu2017-02-011-3/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? When connecting timeout, `ask` may fail with a confusing message: ``` 17/02/01 23:15:19 INFO Worker: Connecting to master ... java.lang.IllegalArgumentException: requirement failed: TransportClient has not yet been set. at scala.Predef$.require(Predef.scala:224) at org.apache.spark.rpc.netty.RpcOutboxMessage.onTimeout(Outbox.scala:70) at org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$ask$1.applyOrElse(NettyRpcEnv.scala:232) at org.apache.spark.rpc.netty.NettyRpcEnv$$anonfun$ask$1.applyOrElse(NettyRpcEnv.scala:231) at scala.concurrent.Future$$anonfun$onFailure$1.apply(Future.scala:138) at scala.concurrent.Future$$anonfun$onFailure$1.apply(Future.scala:136) at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32) ``` It's better to provide a meaningful message. ## How was this patch tested? Jenkins Author: Shixiong Zhu <shixiong@databricks.com> Closes #16773 from zsxwing/connect-timeout.
* [SPARK-19347] ReceiverSupervisorImpl can add block to ReceiverTracker ↵jinxing2017-02-011-3/+35
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | multiple times because of askWithRetry. ## What changes were proposed in this pull request? `ReceiverSupervisorImpl` on executor side reports block's meta back to `ReceiverTracker` on driver side. In current code, `askWithRetry` is used. However, for `AddBlock`, `ReceiverTracker` is not idempotent, which may result in messages are processed multiple times. **To reproduce**: 1. Check if it is the first time receiving `AddBlock` in `ReceiverTracker`, if so sleep long enough(say 200 seconds), thus the first RPC call will be timeout in `askWithRetry`, then `AddBlock` will be resent. 2. Rebuild Spark and run following job: ``` def streamProcessing(): Unit = { val conf = new SparkConf() .setAppName("StreamingTest") .setMaster(masterUrl) val ssc = new StreamingContext(conf, Seconds(200)) val stream = ssc.socketTextStream("localhost", 1234) stream.print() ssc.start() ssc.awaitTermination() } ``` **To fix**: It makes sense to provide a blocking version `ask` in RpcEndpointRef, as mentioned in SPARK-18113 (https://github.com/apache/spark/pull/16503#event-927953218). Because Netty RPC layer will not drop messages. `askWithRetry` is a leftover from akka days. It imposes restrictions on the caller(e.g. idempotency) and other things that people generally don't pay that much attention to when using it. ## How was this patch tested? Test manually. The scenario described above doesn't happen with this patch. Author: jinxing <jinxing@meituan.com> Closes #16690 from jinxing64/SPARK-19347.
* [SPARK-19377][WEBUI][CORE] Killed tasks should have the status as KILLEDDevaraj K2017-02-011-0/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Copying of the killed status was missing while getting the newTaskInfo object by dropping the unnecessary details to reduce the memory usage. This patch adds the copying of the killed status to newTaskInfo object, this will correct the display of the status from wrong status to KILLED status in Web UI. ## How was this patch tested? Current behaviour of displaying tasks in stage UI page, | Index | ID | Attempt | Status | Locality Level | Executor ID / Host | Launch Time | Duration | GC Time | Input Size / Records | Write Time | Shuffle Write Size / Records | Errors | | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | |143 |10 |0 |SUCCESS |NODE_LOCAL |6 / x.xx.x.x stdout stderr|2017/01/25 07:49:27 |0 ms | |0.0 B / 0 | |0.0 B / 0 |TaskKilled (killed intentionally)| |156 |11 |0 |SUCCESS |NODE_LOCAL |5 / x.xx.x.x stdout stderr|2017/01/25 07:49:27 |0 ms | |0.0 B / 0 | |0.0 B / 0 |TaskKilled (killed intentionally)| Web UI display after applying the patch, | Index | ID | Attempt | Status | Locality Level | Executor ID / Host | Launch Time | Duration | GC Time | Input Size / Records | Write Time | Shuffle Write Size / Records | Errors | | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | --- | |143 |10 |0 |KILLED |NODE_LOCAL |6 / x.xx.x.x stdout stderr|2017/01/25 07:49:27 |0 ms | |0.0 B / 0 | | 0.0 B / 0 | TaskKilled (killed intentionally)| |156 |11 |0 |KILLED |NODE_LOCAL |5 / x.xx.x.x stdout stderr|2017/01/25 07:49:27 |0 ms | |0.0 B / 0 | |0.0 B / 0 | TaskKilled (killed intentionally)| Author: Devaraj K <devaraj@apache.org> Closes #16725 from devaraj-kavali/SPARK-19377.
* [SPARK-19402][DOCS] Support LaTex inline formula correctly and fix warnings ↵hyukjinkwon2017-02-015-12/+12
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | in Scala/Java APIs generation ## What changes were proposed in this pull request? This PR proposes three things as below: - Support LaTex inline-formula, `\( ... \)` in Scala API documentation It seems currently, ``` \( ... \) ``` are rendered as they are, for example, <img width="345" alt="2017-01-30 10 01 13" src="https://cloud.githubusercontent.com/assets/6477701/22423960/ab37d54a-e737-11e6-9196-4f6229c0189c.png"> It seems mistakenly more backslashes were added. - Fix warnings Scaladoc/Javadoc generation This PR fixes t two types of warnings as below: ``` [warn] .../spark/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala:335: Could not find any member to link for "UnsupportedOperationException". [warn] /** [warn] ^ ``` ``` [warn] .../spark/sql/core/src/main/scala/org/apache/spark/sql/internal/VariableSubstitution.scala:24: Variable var undefined in comment for class VariableSubstitution in class VariableSubstitution [warn] * `${var}`, `${system:var}` and `${env:var}`. [warn] ^ ``` - Fix Javadoc8 break ``` [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictionModel.java:7: error: reference not found [error] * E.g., {link VectorUDT} for vector features. [error] ^ [error] .../spark/mllib/target/java/org/apache/spark/ml/PredictorParams.java:12: error: reference not found [error] * E.g., {link VectorUDT} for vector features. [error] ^ [error] .../spark/mllib/target/java/org/apache/spark/ml/Predictor.java:10: error: reference not found [error] * E.g., {link VectorUDT} for vector features. [error] ^ [error] .../spark/sql/hive/target/java/org/apache/spark/sql/hive/HiveAnalysis.java:5: error: reference not found [error] * Note that, this rule must be run after {link PreprocessTableInsertion}. [error] ^ ``` ## How was this patch tested? Manually via `sbt unidoc` and `jeykil build`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16741 from HyukjinKwon/warn-and-break.
* [SPARK-19365][CORE] Optimize RequestMessage serializationShixiong Zhu2017-01-274-27/+132
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## 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.
* [CORE][DOCS] Update a help message for --files in spark-submitTakeshi YAMAMURO2017-01-261-1/+2
| | | | | | | | | ## What changes were proposed in this pull request? This pr is to update a help message for `--files` in spark-submit because it seems users get confused about how to get full paths of the files that one adds via the option. Author: Takeshi YAMAMURO <linguin.m.s@gmail.com> Closes #16698 from maropu/SparkFilesDoc.
* [SPARK-19220][UI] Make redirection to HTTPS apply to all URIs.Marcelo Vanzin2017-01-265-59/+155
| | | | | | | | | | | | | | | | | | | 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-18495][UI] Document meaning of green dot in DAG visualizationuncleGen2017-01-251-1/+6
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? A green dot in the DAG visualization apparently means that the referenced RDD is cached. This is not documented anywhere except in this [blog post](https://databricks.com/blog/2015/06/22/understanding-your-spark-application-through-visualization.html). It would be good if the Web UI itself documented this somehow (perhaps in the tooltip?) so that the user can naturally learn what it means while using the Web UI. before pr: ![dingtalk20170125180158](https://cloud.githubusercontent.com/assets/7402327/22286167/37910ea2-e329-11e6-9aae-03dce6fceee2.png) ![dingtalk20170125180218](https://cloud.githubusercontent.com/assets/7402327/22286178/421f3132-e329-11e6-8283-ba6bbd15cfb0.png) after pr: ![dingtalk20170125175704](https://cloud.githubusercontent.com/assets/7402327/22286216/56a0050a-e329-11e6-813d-2be96b5ab7f1.png) ![dingtalk20170125175749](https://cloud.githubusercontent.com/assets/7402327/22286233/618cd646-e329-11e6-9ce5-10322b169dcb.png) ## How was this patch tested? Author: uncleGen <hustyugm@gmail.com> Closes #16702 from uncleGen/SPARK-18495.
* [SPARK-14804][SPARK][GRAPHX] Fix checkpointing of VertexRDD/EdgeRDDTathagata Das2017-01-251-2/+3
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? EdgeRDD/VertexRDD overrides checkpoint() and isCheckpointed() to forward these to the internal partitionRDD. So when checkpoint() is called on them, its the partitionRDD that actually gets checkpointed. However since isCheckpointed() also overridden to call partitionRDD.isCheckpointed, EdgeRDD/VertexRDD.isCheckpointed returns true even though this RDD is actually not checkpointed. This would have been fine except the RDD's internal logic for computing the RDD depends on isCheckpointed(). So for VertexRDD/EdgeRDD, since isCheckpointed is true, when computing Spark tries to read checkpoint data of VertexRDD/EdgeRDD even though they are not actually checkpointed. Through a crazy sequence of call forwarding, it reads checkpoint data of partitionsRDD and tries to cast it to types in Vertex/EdgeRDD. This leads to ClassCastException. The minimal fix that does not change any public behavior is to modify RDD internal to not use public override-able API for internal logic. ## How was this patch tested? New unit tests. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #15396 from tdas/SPARK-14804.
* [SPARK-19139][CORE] New auth mechanism for transport library.Marcelo Vanzin2017-01-2411-31/+118
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-246-2/+297
| | | | | | | | | | | | | | | | | | | | | 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-19306][CORE] Fix inconsistent state in DiskBlockObject when expection ↵jerryshao2017-01-231-19/+25
| | | | | | | | | | | | | | | | occurred ## What changes were proposed in this pull request? In `DiskBlockObjectWriter`, when some errors happened during writing, it will call `revertPartialWritesAndClose`, if this method again failed due to some issues like out of disk, it will throw exception without resetting the state of this writer, also skipping the revert. So here propose to fix this issue to offer user a chance to recover from such issue. ## How was this patch tested? Existing test. Author: jerryshao <sshao@hortonworks.com> Closes #16657 from jerryshao/SPARK-19306.
* [SPARK-19146][CORE] Drop more elements when stageData.taskData.size > ↵Yuming Wang2017-01-232-3/+41
| | | | | | | | | | | | | | | | 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-17724][STREAMING][WEBUI] Unevaluated new lines in tooltip in DAG ↵Xin Ren2017-01-211-2/+32
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Visualization of a job https://issues.apache.org/jira/browse/SPARK-17724 ## What changes were proposed in this pull request? For unevaluated `\n`, evaluate it and enable line break, for Streaming WebUI `stages` page and `job` page. (I didn't change Scala source file, since Jetty server has to somehow indicate line break and js to code display it.) (This PR is a continue from previous PR https://github.com/apache/spark/pull/15353 for the same issue, sorry being so long time) Two changes: 1. RDD Node tooltipText is actually showing the `<circle>` `title` property, so I set extra attribute in `spark-dag-viz.js`: `.attr("data-html", "true")` `<circle x="-5" y="-5" r="5" data-toggle="tooltip" data-placement="bottom" title="" data-original-title="ParallelCollectionRDD [9]\nmakeRDD at QueueStream.scala:49"></circle>` 2. Static `<tspan>` text of each stage, split by `/n`, and append an extra `<tspan>` element to its parentNode `<text><tspan xml:space="preserve" dy="1em" x="1">reduceByKey</tspan><tspan xml:space="preserve" dy="1em" x="1">reduceByKey/n 23:34:49</tspan></text> ` ## UI changes Screenshot **before fix**, `\n` is not evaluated in both circle tooltipText and static text: ![screen shot 2017-01-19 at 12 21 54 am](https://cloud.githubusercontent.com/assets/3925641/22098829/53c7f49c-dddd-11e6-9daa-b3ddb6044114.png) Screenshot **after fix**: ![screen shot 2017-01-19 at 12 20 30 am](https://cloud.githubusercontent.com/assets/3925641/22098806/294910d4-dddd-11e6-9948-d942e09f545e.png) ## How was this patch tested? Tested locally. For Streaming WebUI `stages` page and `job` page, on multiple browsers: - Chrome - Firefox - Safari Author: Xin Ren <renxin.ubc@gmail.com> Closes #16643 from keypointt/SPARK-17724-2nd.
* [SPARK-19069][CORE] Expose task 'status' and 'duration' in spark history ↵Parag Chaudhari2017-01-2014-4/+354
| | | | | | | | | | | | | | | | 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-1935-235/+948
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | 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-182-4/+31
| | | | | | | | | | | | | | | | | | | | 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-19223][SQL][PYSPARK] Fix InputFileBlockHolder for datasources which ↵Liang-Chi Hsieh2017-01-181-3/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | are based on HadoopRDD or NewHadoopRDD ## What changes were proposed in this pull request? For some datasources which are based on HadoopRDD or NewHadoopRDD, such as spark-xml, InputFileBlockHolder doesn't work with Python UDF. The method to reproduce it is, running the following codes with `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`: from pyspark.sql.functions import udf,input_file_name from pyspark.sql.types import StringType from pyspark.sql import SparkSession def filename(path): return path session = SparkSession.builder.appName('APP').getOrCreate() session.udf.register('sameText', filename) sameText = udf(filename, StringType()) df = session.read.format('xml').load('a.xml', rowTag='root').select('*', input_file_name().alias('file')) df.select('file').show() # works df.select(sameText(df['file'])).show() # returns empty content The issue is because in `HadoopRDD` and `NewHadoopRDD` we set the file block's info in `InputFileBlockHolder` before the returned iterator begins consuming. `InputFileBlockHolder` will record this info into thread local variable. When running Python UDF in batch, we set up another thread to consume the iterator from child plan's output rdd, so we can't read the info back in another thread. To fix this, we have to set the info in `InputFileBlockHolder` after the iterator begins consuming. So the info can be read in correct thread. ## How was this patch tested? Manual test with above example codes for spark-xml package on pyspark: `bin/pyspark --packages com.databricks:spark-xml_2.11:0.4.1`. Added pyspark test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #16585 from viirya/fix-inputfileblock-hadooprdd.
* [SPARK-19227][SPARK-19251] remove unused imports and outdated commentsuncleGen2017-01-1820-38/+12
| | | | | | | | | | | | ## 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-18243][SQL] Port Hive writing to use FileFormat interfaceWenchen Fan2017-01-171-1/+1
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Inserting data into Hive tables has its own implementation that is distinct from data sources: `InsertIntoHiveTable`, `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`. Note that one other major difference is that data source tables write directly to the final destination without using some staging directory, and then Spark itself adds the partitions/tables to the catalog. Hive tables actually write to some staging directory, and then call Hive metastore's loadPartition/loadTable function to load those data in. So we still need to keep `InsertIntoHiveTable` to put this special logic. In the future, we should think of writing to the hive table location directly, so that we don't need to call `loadTable`/`loadPartition` at the end and remove `InsertIntoHiveTable`. This PR removes `SparkHiveWriterContainer` and `SparkHiveDynamicPartitionWriterContainer`, and create a `HiveFileFormat` to implement the write logic. In the future, we should also implement the read logic in `HiveFileFormat`. ## How was this patch tested? existing tests Author: Wenchen Fan <wenchen@databricks.com> Closes #16517 from cloud-fan/insert-hive.
* [SPARK-19179][YARN] Change spark.yarn.access.namenodes config and update docsjerryshao2017-01-171-2/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? `spark.yarn.access.namenodes` configuration cannot actually reflects the usage of it, inside the code it is the Hadoop filesystems we get tokens, not NNs. So here propose to update the name of this configuration, also change the related code and doc. ## How was this patch tested? Local verification. Author: jerryshao <sshao@hortonworks.com> Closes #16560 from jerryshao/SPARK-19179.