aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-17113] [SHUFFLE] Job failure due to Executor OOM in offheap modeSital Kedia2016-08-192-1/+8
| | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR fixes executor OOM in offheap mode due to bug in Cooperative Memory Management for UnsafeExternSorter. UnsafeExternalSorter was checking if memory page is being used by upstream by comparing the base object address of the current page with the base object address of upstream. However, in case of offheap memory allocation, the base object addresses are always null, so there was no spilling happening and eventually the operator would OOM. Following is the stack trace this issue addresses - java.lang.OutOfMemoryError: Unable to acquire 1220 bytes of memory, got 0 at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:120) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.acquireNewPageIfNecessary(UnsafeExternalSorter.java:341) at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.insertRecord(UnsafeExternalSorter.java:362) at org.apache.spark.sql.execution.UnsafeExternalRowSorter.insertRow(UnsafeExternalRowSorter.java:93) at org.apache.spark.sql.execution.UnsafeExternalRowSorter.sort(UnsafeExternalRowSorter.java:170) ## How was this patch tested? Tested by running the failing job. Author: Sital Kedia <skedia@fb.com> Closes #14693 from sitalkedia/fix_offheap_oom.
* [SPARK-11227][CORE] UnknownHostException can be thrown when NameNode HA is ↵Kousuke Saruta2016-08-191-1/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | enabled. ## What changes were proposed in this pull request? If the following conditions are satisfied, executors don't load properties in `hdfs-site.xml` and UnknownHostException can be thrown. (1) NameNode HA is enabled (2) spark.eventLogging is disabled or logging path is NOT on HDFS (3) Using Standalone or Mesos for the cluster manager (4) There are no code to load `HdfsCondition` class in the driver regardless of directly or indirectly. (5) The tasks access to HDFS (There might be some more conditions...) For example, following code causes UnknownHostException when the conditions above are satisfied. ``` sc.textFile("<path on HDFS>").collect ``` ``` java.lang.IllegalArgumentException: java.net.UnknownHostException: hacluster at org.apache.hadoop.security.SecurityUtil.buildTokenService(SecurityUtil.java:378) at org.apache.hadoop.hdfs.NameNodeProxies.createNonHAProxy(NameNodeProxies.java:310) at org.apache.hadoop.hdfs.NameNodeProxies.createProxy(NameNodeProxies.java:176) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:678) at org.apache.hadoop.hdfs.DFSClient.<init>(DFSClient.java:619) at org.apache.hadoop.hdfs.DistributedFileSystem.initialize(DistributedFileSystem.java:149) at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2653) at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:92) at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2687) at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2669) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:371) at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:170) at org.apache.hadoop.mapred.JobConf.getWorkingDirectory(JobConf.java:656) at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:438) at org.apache.hadoop.mapred.FileInputFormat.setInputPaths(FileInputFormat.java:411) at org.apache.spark.SparkContext$$anonfun$hadoopFile$1$$anonfun$32.apply(SparkContext.scala:986) at org.apache.spark.SparkContext$$anonfun$hadoopFile$1$$anonfun$32.apply(SparkContext.scala:986) at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:177) at org.apache.spark.rdd.HadoopRDD$$anonfun$getJobConf$6.apply(HadoopRDD.scala:177) at scala.Option.map(Option.scala:146) at org.apache.spark.rdd.HadoopRDD.getJobConf(HadoopRDD.scala:177) at org.apache.spark.rdd.HadoopRDD$$anon$1.<init>(HadoopRDD.scala:213) at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:209) at org.apache.spark.rdd.HadoopRDD.compute(HadoopRDD.scala:102) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:38) at org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:318) at org.apache.spark.rdd.RDD.iterator(RDD.scala:282) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:85) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) Caused by: java.net.UnknownHostException: hacluster ``` But following code doesn't cause the Exception because `textFile` method loads `HdfsConfiguration` indirectly. ``` sc.textFile("<path on HDFS>").collect ``` When a job includes some operations which access to HDFS, the object of `org.apache.hadoop.Configuration` is wrapped by `SerializableConfiguration`, serialized and broadcasted from driver to executors and each executor deserialize the object with `loadDefaults` false so HDFS related properties should be set before broadcasted. ## How was this patch tested? Tested manually on my standalone cluster. Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp> Closes #13738 from sarutak/SPARK-11227.
* [SPARK-16673][WEB UI] New Executor Page removed conditional for Logs and ↵Alex Bozarth2016-08-192-11/+34
| | | | | | | | | | | | | | | | | | | | Thread Dump columns ## What changes were proposed in this pull request? When #13670 switched `ExecutorsPage` to use JQuery DataTables it incidentally removed the conditional for the Logs and Thread Dump columns. I reimplemented the conditional display of the Logs and Thread dump columns as it was before the switch. ## How was this patch tested? Manually tested and dev/run-tests ![both](https://cloud.githubusercontent.com/assets/13952758/17186879/da8dd1a8-53eb-11e6-8b0c-d0ff0156a9a7.png) ![dump](https://cloud.githubusercontent.com/assets/13952758/17186881/dab08a04-53eb-11e6-8b1c-50ffd0bf2ae8.png) ![logs](https://cloud.githubusercontent.com/assets/13952758/17186880/dab04d00-53eb-11e6-8754-68dd64d6d9f4.png) Author: Alex Bozarth <ajbozart@us.ibm.com> Closes #14382 from ajbozarth/spark16673.
* [SPARK-16961][CORE] Fixed off-by-one error that biased randomizeInPlaceNick Lavers2016-08-192-1/+36
| | | | | | | | | | | | | JIRA issue link: https://issues.apache.org/jira/browse/SPARK-16961 Changed one line of Utils.randomizeInPlace to allow elements to stay in place. Created a unit test that runs a Pearson's chi squared test to determine whether the output diverges significantly from a uniform distribution. Author: Nick Lavers <nick.lavers@videoamp.com> Closes #14551 from nicklavers/SPARK-16961-randomizeInPlace.
* [SPARK-16736][CORE][SQL] purge superfluous fs callsSteve Loughran2016-08-175-53/+28
| | | | | | | | | | | | | | | A review of the code, working back from Hadoop's `FileSystem.exists()` and `FileSystem.isDirectory()` code, then removing uses of the calls when superfluous. 1. delete is harmless if called on a nonexistent path, so don't do any checks before deletes 1. any `FileSystem.exists()` check before `getFileStatus()` or `open()` is superfluous as the operation itself does the check. Instead the `FileNotFoundException` is caught and triggers the downgraded path. When a `FileNotFoundException` was thrown before, the code still creates a new FNFE with the error messages. Though now the inner exceptions are nested, for easier diagnostics. Initially, relying on Jenkins test runs. One troublespot here is that some of the codepaths are clearly error situations; it's not clear that they have coverage anyway. Trying to create the failure conditions in tests would be ideal, but it will also be hard. Author: Steve Loughran <stevel@apache.org> Closes #14371 from steveloughran/cloud/SPARK-16736-superfluous-fs-calls.
* [SPARK-16671][CORE][SQL] Consolidate code to do variable substitution.Marcelo Vanzin2016-08-156-127/+294
| | | | | | | | | | | | | | | | Both core and sql have slightly different code that does variable substitution of config values. This change refactors that code and encapsulates the logic of reading config values and expading variables in a new helper class, which can be configured so that both core and sql can use it without losing existing functionality, and allows for easier testing and makes it easier to add more features in the future. Tested with existing and new unit tests, and by running spark-shell with some configs referencing variables and making sure it behaved as expected. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14468 from vanzin/SPARK-16671.
* [SPARK-11714][MESOS] Make Spark on Mesos honor port restrictions on coarse ↵Stavros Kontopoulos2016-08-156-25/+336
| | | | | | | | | | | | | grain mode - Make mesos coarse grained scheduler accept port offers and pre-assign ports Previous attempt was for fine grained: https://github.com/apache/spark/pull/10808 Author: Stavros Kontopoulos <stavros.kontopoulos@lightbend.com> Author: Stavros Kontopoulos <stavros.kontopoulos@typesafe.com> Closes #11157 from skonto/honour_ports_coarse.
* [WIP][MINOR][TYPO] Fix several trivival typosZhenglai Zhang2016-08-142-2/+2
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? * Fixed one typo `"overriden"` as `"overridden"`, also make sure no other same typo. * Fixed one typo `"lowcase"` as `"lowercase"`, also make sure no other same typo. ## How was this patch tested? Since the change is very tiny, so I just make sure compilation is successful. I am new to the spark community, please feel free to let me do other necessary steps. Thanks in advance! ---- Updated: Found another typo `lowcase` later and fixed then in the same patch Author: Zhenglai Zhang <zhenglaizhang@hotmail.com> Closes #14622 from zhenglaizhang/fixtypo.
* [MINOR][CORE] fix warnings on depreciated methods in ↵Xin Ren2016-08-132-5/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | MesosClusterSchedulerSuite and DiskBlockObjectWriterSuite ## What changes were proposed in this pull request? Fixed warnings below after scanning through warnings during build: ``` [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala:34: imported `Utils' is permanently hidden by definition of object Utils in package mesos [warn] import org.apache.spark.scheduler.cluster.mesos.Utils [warn] ^ ``` and ``` [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:113: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:119: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:131: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ [warn] /home/jenkins/workspace/SparkPullRequestBuilder/core/src/test/scala/org/apache/spark/storage/DiskBlockObjectWriterSuite.scala:135: method shuffleBytesWritten in class ShuffleWriteMetrics is deprecated: use bytesWritten instead [warn] assert(writeMetrics.shuffleBytesWritten === file.length()) [warn] ^ ``` ## How was this patch tested? Tested manually on local laptop. Author: Xin Ren <iamshrek@126.com> Closes #14609 from keypointt/suiteWarnings.
* [SPARK-16985] Change dataFormat from yyyyMMddHHmm to yyyyMMddHHmmsshongshen2016-08-124-5/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In our cluster, sometimes the sql output maybe overrided. When I submit some sql, all insert into the same table, and the sql will cost less one minute, here is the detail, 1 sql1, 11:03 insert into table. 2 sql2, 11:04:11 insert into table. 3 sql3, 11:04:48 insert into table. 4 sql4, 11:05 insert into table. 5 sql5, 11:06 insert into table. The sql3's output file will override the sql2's output file. here is the log: ``` 16/05/04 11:04:11 INFO hive.SparkHiveHadoopWriter: XXfinalPath=hdfs://tl-sng-gdt-nn-tdw.tencent-distribute.com:54310/tmp/assorz/tdw-tdwadmin/20160504/04559505496526517_-1_1204544348/10000/_tmp.p_20160428/attempt_201605041104_0001_m_000000_1 16/05/04 11:04:48 INFO hive.SparkHiveHadoopWriter: XXfinalPath=hdfs://tl-sng-gdt-nn-tdw.tencent-distribute.com:54310/tmp/assorz/tdw-tdwadmin/20160504/04559505496526517_-1_212180468/10000/_tmp.p_20160428/attempt_201605041104_0001_m_000000_1 ``` The reason is the output file use SimpleDateFormat("yyyyMMddHHmm"), if two sql insert into the same table in the same minute, the output will be overrite. I think we should change dateFormat to "yyyyMMddHHmmss", in our cluster, we can't finished a sql in one second. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: hongshen <shenh062326@126.com> Closes #14574 from shenh062326/SPARK-16985.
* [SPARK-13081][PYSPARK][SPARK_SUBMIT] Allow set pythonExec of driver and ↵Jeff Zhang2016-08-115-3/+34
| | | | | | | | | | | | executor through conf… Before this PR, user have to export environment variable to specify the python of driver & executor which is not so convenient for users. This PR is trying to allow user to specify python through configuration "--pyspark-driver-python" & "--pyspark-executor-python" Manually test in local & yarn mode for pyspark-shell and pyspark batch mode. Author: Jeff Zhang <zjffdu@apache.org> Closes #13146 from zjffdu/SPARK-13081.
* [SPARK-16868][WEB UI] Fix executor be both dead and alive on executor ui.huangzhaowei2016-08-111-0/+4
| | | | | | | | | | | | | | | ## What changes were proposed in this pull request? In a heavy pressure of the spark application, since the executor will register it to driver block manager twice(because of heart beats), the executor will show as picture show: ![image](https://cloud.githubusercontent.com/assets/7404824/17467245/c1359094-5d4e-11e6-843a-f6d6347e1bf6.png) ## How was this patch tested? NA Details in: [SPARK-16868](https://issues.apache.org/jira/browse/SPARK-16868) Author: huangzhaowei <carlmartinmax@gmail.com> Closes #14530 from SaintBacchus/SPARK-16868.
* [SPARK-13602][CORE] Add shutdown hook to DriverRunner to prevent driver ↵Bryan Cutler2016-08-112-50/+142
| | | | | | | | | | | | | | | | | | process leak ## What changes were proposed in this pull request? Added shutdown hook to DriverRunner to kill the driver process in case the Worker JVM exits suddenly and the `WorkerWatcher` was unable to properly catch this. Did some cleanup to consolidate driver state management and setting of finalized vars within the running thread. ## How was this patch tested? Added unit tests to verify that final state and exception variables are set accordingly for successfull, failed, and errors in the driver process. Retrofitted existing test to verify killing of mocked process ends with the correct state and stops properly Manually tested (with deploy-mode=cluster) that the shutdown hook is called by forcibly exiting the `Worker` and various points in the code with the `WorkerWatcher` both disabled and enabled. Also, manually killed the driver through the ui and verified that the `DriverRunner` interrupted, killed the process and exited properly. Author: Bryan Cutler <cutlerb@gmail.com> Closes #11746 from BryanCutler/DriverRunner-shutdown-hook-SPARK-13602.
* [SPARK-16952] don't lookup spark home directory when executor uri is setMichael Gummelt2016-08-112-7/+26
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? remove requirement to set spark.mesos.executor.home when spark.executor.uri is used ## How was this patch tested? unit tests Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #14552 from mgummelt/fix-spark-home.
* [SPARK-14743][YARN] Add a configurable credential manager for Spark running ↵jerryshao2016-08-103-41/+8
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | on YARN ## What changes were proposed in this pull request? Add a configurable token manager for Spark on running on yarn. ### Current Problems ### 1. Supported token provider is hard-coded, currently only hdfs, hbase and hive are supported and it is impossible for user to add new token provider without code changes. 2. Also this problem exits in timely token renewer and updater. ### Changes In This Proposal ### In this proposal, to address the problems mentioned above and make the current code more cleaner and easier to understand, mainly has 3 changes: 1. Abstract a `ServiceTokenProvider` as well as `ServiceTokenRenewable` interface for token provider. Each service wants to communicate with Spark through token way needs to implement this interface. 2. Provide a `ConfigurableTokenManager` to manage all the register token providers, also token renewer and updater. Also this class offers the API for other modules to obtain tokens, get renewal interval and so on. 3. Implement 3 built-in token providers `HDFSTokenProvider`, `HiveTokenProvider` and `HBaseTokenProvider` to keep the same semantics as supported today. Whether to load in these built-in token providers is controlled by configuration "spark.yarn.security.tokens.${service}.enabled", by default for all the built-in token providers are loaded. ### Behavior Changes ### For the end user there's no behavior change, we still use the same configuration `spark.yarn.security.tokens.${service}.enabled` to decide which token provider is enabled (hbase or hive). For user implemented token provider (assume the name of token provider is "test") needs to add into this class should have two configurations: 1. `spark.yarn.security.tokens.test.enabled` to true 2. `spark.yarn.security.tokens.test.class` to the full qualified class name. So we still keep the same semantics as current code while add one new configuration. ### Current Status ### - [x] token provider interface and management framework. - [x] implement built-in token providers (hdfs, hbase, hive). - [x] Coverage of unit test. - [x] Integrated test with security cluster. ## How was this patch tested? Unit test and integrated test. Please suggest and review, any comment is greatly appreciated. Author: jerryshao <sshao@hortonworks.com> Closes #14065 from jerryshao/SPARK-16342.
* [SPARK-12920][CORE] Honor "spark.ui.retainedStages" to reduce mem-pressureRajesh Balamohan2016-08-102-18/+36
| | | | | | | | | | When large number of jobs are run concurrently with Spark thrift server, thrift server starts running at high CPU due to GC pressure. Job UI retention causes memory pressure with large jobs. https://issues.apache.org/jira/secure/attachment/12783302/SPARK-12920.profiler_job_progress_listner.png has the profiler snapshot. This PR honors `spark.ui.retainedStages` strictly to reduce memory pressure. Manual and unit tests Author: Rajesh Balamohan <rbalamohan@apache.org> Closes #10846 from rajeshbalamohan/SPARK-12920.
* [SPARK-15639] [SPARK-16321] [SQL] Push down filter at RowGroups level for ↵Liang-Chi Hsieh2016-08-102-0/+21
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | parquet reader ## What changes were proposed in this pull request? The base class `SpecificParquetRecordReaderBase` used for vectorized parquet reader will try to get pushed-down filters from the given configuration. This pushed-down filters are used for RowGroups-level filtering. However, we don't set up the filters to push down into the configuration. In other words, the filters are not actually pushed down to do RowGroups-level filtering. This patch is to fix this and tries to set up the filters for pushing down to configuration for the reader. The benchmark that excludes the time of writing Parquet file: test("Benchmark for Parquet") { val N = 500 << 12 withParquetTable((0 until N).map(i => (101, i)), "t") { val benchmark = new Benchmark("Parquet reader", N) benchmark.addCase("reading Parquet file", 10) { iter => sql("SELECT _1 FROM t where t._1 < 100").collect() } benchmark.run() } } `withParquetTable` in default will run tests for vectorized reader non-vectorized readers. I only let it run vectorized reader. When we set the block size of parquet as 1024 to have multiple row groups. The benchmark is: Before this patch: The retrieved row groups: 8063 Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 825 / 1233 2.5 402.6 1.0X After this patch: The retrieved row groups: 0 Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 306 / 503 6.7 149.6 1.0X Next, I run the benchmark for non-pushdown case using the same benchmark code but with disabled pushdown configuration. This time the parquet block size is default value. Before this patch: Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 136 / 238 15.0 66.5 1.0X After this patch: Java HotSpot(TM) 64-Bit Server VM 1.8.0_71-b15 on Linux 3.19.0-25-generic Intel(R) Core(TM) i7-5557U CPU 3.10GHz Parquet reader: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ reading Parquet file 124 / 193 16.5 60.7 1.0X For non-pushdown case, from the results, I think this patch doesn't affect normal code path. I've manually output the `totalRowCount` in `SpecificParquetRecordReaderBase` to see if this patch actually filter the row-groups. When running the above benchmark: After this patch: `totalRowCount = 0` Before this patch: `totalRowCount = 1024000` ## How was this patch tested? Existing tests should be passed. Author: Liang-Chi Hsieh <simonh@tw.ibm.com> Closes #13701 from viirya/vectorized-reader-push-down-filter2.
* [SPARK-16927][SPARK-16923] Override task properties at dispatcher.Timothy Chen2016-08-101-22/+22
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? - enable setting default properties for all jobs submitted through the dispatcher [SPARK-16927] - remove duplication of conf vars on cluster submitted jobs [SPARK-16923] (this is a small fix, so I'm including in the same PR) ## How was this patch tested? mesos/spark integration test suite manual testing Author: Timothy Chen <tnachen@gmail.com> Closes #14511 from mgummelt/override-props.
* Make logDir easily copy/paste-ableAndrew Ash2016-08-091-1/+1
| | | | | | | | | | | In many terminals double-clicking and dragging also includes the trailing period. Simply remove this to make the value more easily copy/pasteable. Example value: `hdfs://mybox-123.net.example.com:8020/spark-events.` Author: Andrew Ash <andrew@andrewash.com> Closes #14566 from ash211/patch-9.
* [SPARK-16956] Make ApplicationState.MAX_NUM_RETRY configurableJosh Rosen2016-08-092-3/+6
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch introduces a new configuration, `spark.deploy.maxExecutorRetries`, to let users configure an obscure behavior in the standalone master where the master will kill Spark applications which have experienced too many back-to-back executor failures. The current setting is a hardcoded constant (10); this patch replaces that with a new cluster-wide configuration. **Background:** This application-killing was added in 6b5980da796e0204a7735a31fb454f312bc9daac (from September 2012) and I believe that it was designed to prevent a faulty application whose executors could never launch from DOS'ing the Spark cluster via an infinite series of executor launch attempts. In a subsequent patch (#1360), this feature was refined to prevent applications which have running executors from being killed by this code path. **Motivation for making this configurable:** Previously, if a Spark Standalone application experienced more than `ApplicationState.MAX_NUM_RETRY` executor failures and was left with no executors running then the Spark master would kill that application, but this behavior is problematic in environments where the Spark executors run on unstable infrastructure and can all simultaneously die. For instance, if your Spark driver runs on an on-demand EC2 instance while all workers run on ephemeral spot instances then it's possible for all executors to die at the same time while the driver stays alive. In this case, it may be desirable to keep the Spark application alive so that it can recover once new workers and executors are available. In order to accommodate this use-case, this patch modifies the Master to never kill faulty applications if `spark.deploy.maxExecutorRetries` is negative. I'd like to merge this patch into master, branch-2.0, and branch-1.6. ## How was this patch tested? I tested this manually using `spark-shell` and `local-cluster` mode. This is a tricky feature to unit test and historically this code has not changed very often, so I'd prefer to skip the additional effort of adding a testing framework and would rather rely on manual tests and review for now. Author: Josh Rosen <joshrosen@databricks.com> Closes #14544 from JoshRosen/add-setting-for-max-executor-failures.
* [SPARK-16809] enable history server links in dispatcher UIMichael Gummelt2016-08-096-10/+65
| | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Links the Spark Mesos Dispatcher UI to the history server UI - adds spark.mesos.dispatcher.historyServer.url - explicitly generates frameworkIDs for the launched drivers, so the dispatcher knows how to correlate drivers and frameworkIDs ## How was this patch tested? manual testing Author: Michael Gummelt <mgummelt@mesosphere.io> Author: Sergiusz Urbaniak <sur@mesosphere.io> Closes #14414 from mgummelt/history-server.
* [SPARK-16522][MESOS] Spark application throws exception on exit.Sun Rui2016-08-092-1/+39
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Spark applications running on Mesos throw exception upon exit. For details, refer to https://issues.apache.org/jira/browse/SPARK-16522. I am not sure if there is any better fix, so wait for review comments. ## How was this patch tested? Manual test. Observed that the exception is gone upon application exit. Author: Sun Rui <sunrui2016@gmail.com> Closes #14175 from sun-rui/SPARK-16522.
* [SPARK-16606][CORE] Misleading warning for SparkContext.getOrCreate "WARN ↵Sean Owen2016-08-091-4/+10
| | | | | | | | | | | | | | | | | | | SparkContext: Use an existing SparkContext, some configuration may not take effect." ## What changes were proposed in this pull request? SparkContext.getOrCreate shouldn't warn about ignored config if - it wasn't ignored because a new context is created with it or - no config was actually provided ## How was this patch tested? Jenkins + existing tests. Author: Sean Owen <sowen@cloudera.com> Closes #14533 from srowen/SPARK-16606.
* [SPARK-16779][TRIVIAL] Avoid using postfix operators where they do not add ↵Holden Karau2016-08-087-9/+2
| | | | | | | | | | | | | | | | much and remove whitelisting ## What changes were proposed in this pull request? Avoid using postfix operation for command execution in SQLQuerySuite where it wasn't whitelisted and audit existing whitelistings removing postfix operators from most places. Some notable places where postfix operation remains is in the XML parsing & time units (seconds, millis, etc.) where it arguably can improve readability. ## How was this patch tested? Existing tests. Author: Holden Karau <holden@us.ibm.com> Closes #14407 from holdenk/SPARK-16779.
* [SPARK-16953] Make requestTotalExecutors public Developer API to be ↵Tathagata Das2016-08-081-1/+2
| | | | | | | | | | | | consistent with requestExecutors/killExecutors ## What changes were proposed in this pull request? RequestExecutors and killExecutor are public developer APIs for managing the number of executors allocated to the SparkContext. For consistency, requestTotalExecutors should also be a public Developer API, as it provides similar functionality. In fact, using requestTotalExecutors is more convenient that requestExecutors as the former is idempotent and the latter is not. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #14541 from tdas/SPARK-16953.
* [SPARK-16919] Configurable update interval for console progress barTejas Patil2016-08-081-11/+12
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently the update interval for the console progress bar is hardcoded. This PR makes it configurable for users. ## How was this patch tested? Ran a long running job and with a high value of update interval, the updates were shown less frequently. Author: Tejas Patil <tejasp@fb.com> Closes #14507 from tejasapatil/SPARK-16919.
* [SPARK-16909][SPARK CORE] Streaming for postgreSQL JDBC driverPrince J Wesley2016-08-071-5/+11
| | | | | | | | | | As per the postgreSQL JDBC driver [implementation](https://github.com/pgjdbc/pgjdbc/blob/ab2a6d89081fc2c1fdb2a8600f413db33669022c/pgjdbc/src/main/java/org/postgresql/PGProperty.java#L99), the default record fetch size is 0(which means, it caches all record) This fix enforces default record fetch size as 10 to enable streaming of data. Author: Prince J Wesley <princejohnwesley@gmail.com> Closes #14502 from princejwesley/spark-postgres.
* [SPARK-16925] Master should call schedule() after all executor exit events, ↵Josh Rosen2016-08-062-10/+22
| | | | | | | | | | | | | | | | | | | | | | | | | | not only failures ## What changes were proposed in this pull request? This patch fixes a bug in Spark's standalone Master which could cause applications to hang if tasks cause executors to exit with zero exit codes. As an example of the bug, run ``` sc.parallelize(1 to 1, 1).foreachPartition { _ => System.exit(0) } ``` on a standalone cluster which has a single Spark application. This will cause all executors to die but those executors won't be replaced unless another Spark application or worker joins or leaves the cluster (or if an executor exits with a non-zero exit code). This behavior is caused by a bug in how the Master handles the `ExecutorStateChanged` event: the current implementation calls `schedule()` only if the executor exited with a non-zero exit code, so a task which causes a JVM to unexpectedly exit "cleanly" will skip the `schedule()` call. This patch addresses this by modifying the `ExecutorStateChanged` to always unconditionally call `schedule()`. This should be safe because it should always be safe to call `schedule()`; adding extra `schedule()` calls can only affect performance and should not introduce correctness bugs. ## How was this patch tested? I added a regression test in `DistributedSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes #14510 from JoshRosen/SPARK-16925.
* [SPARK-16796][WEB UI] Mask spark.authenticate.secret on Spark environ…Artur Sukhenko2016-08-061-1/+3
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Mask `spark.authenticate.secret` on Spark environment page (Web UI). This is addition to https://github.com/apache/spark/pull/14409 ## How was this patch tested? `./dev/run-tests` [info] ScalaTest [info] Run completed in 1 hour, 8 minutes, 38 seconds. [info] Total number of tests run: 2166 [info] Suites: completed 65, aborted 0 [info] Tests: succeeded 2166, failed 0, canceled 0, ignored 590, pending 0 [info] All tests passed. Author: Artur Sukhenko <artur.sukhenko@gmail.com> Closes #14484 from Devian-ua/SPARK-16796.
* [MINOR] Update AccumulatorV2 doc to not mention "+=".petermaxlee2016-08-051-1/+1
| | | | | | | | | | | | ## What changes were proposed in this pull request? As reported by Bryan Cutler on the mailing list, AccumulatorV2 does not have a += method, yet the documentation still references it. ## How was this patch tested? N/A Author: petermaxlee <petermaxlee@gmail.com> Closes #14466 from petermaxlee/accumulator.
* [SPARK-16875][SQL] Add args checking for DataSet randomSplit and sampleZheng RuiFeng2016-08-041-12/+25
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Add the missing args-checking for randomSplit and sample ## How was this patch tested? unit tests Author: Zheng RuiFeng <ruifengz@foxmail.com> Closes #14478 from zhengruifeng/fix_randomSplit.
* [SPARK-16873][CORE] Fix SpillReader NPE when spillFile has no datasharkd2016-08-031-1/+3
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? SpillReader NPE when spillFile has no data. See follow logs: 16/07/31 20:54:04 INFO collection.ExternalSorter: spill memory to file:/data4/yarnenv/local/usercache/tesla/appcache/application_1465785263942_56138/blockmgr-db5f46c3-d7a4-4f93-8b77-565e469696fb/09/temp_shuffle_ec3ece08-4569-4197-893a-4a5dfcbbf9fa, fileSize:0.0 B 16/07/31 20:54:04 WARN memory.TaskMemoryManager: leak 164.3 MB memory from org.apache.spark.util.collection.ExternalSorter3db4b52d 16/07/31 20:54:04 ERROR executor.Executor: Managed memory leak detected; size = 190458101 bytes, TID = 2358516/07/31 20:54:04 ERROR executor.Executor: Exception in task 1013.0 in stage 18.0 (TID 23585) java.lang.NullPointerException at org.apache.spark.util.collection.ExternalSorter$SpillReader.cleanup(ExternalSorter.scala:624) at org.apache.spark.util.collection.ExternalSorter$SpillReader.nextBatchStream(ExternalSorter.scala:539) at org.apache.spark.util.collection.ExternalSorter$SpillReader.<init>(ExternalSorter.scala:507) at org.apache.spark.util.collection.ExternalSorter$SpillableIterator.spill(ExternalSorter.scala:816) at org.apache.spark.util.collection.ExternalSorter.forceSpill(ExternalSorter.scala:251) at org.apache.spark.util.collection.Spillable.spill(Spillable.scala:109) at org.apache.spark.memory.TaskMemoryManager.acquireExecutionMemory(TaskMemoryManager.java:154) at org.apache.spark.memory.TaskMemoryManager.allocatePage(TaskMemoryManager.java:249) at org.apache.spark.memory.MemoryConsumer.allocatePage(MemoryConsumer.java:112) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.acquireNewPageIfNecessary(ShuffleExternalSorter.java:346) at org.apache.spark.shuffle.sort.ShuffleExternalSorter.insertRecord(ShuffleExternalSorter.java:367) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.insertRecordIntoSorter(UnsafeShuffleWriter.java:237) at org.apache.spark.shuffle.sort.UnsafeShuffleWriter.write(UnsafeShuffleWriter.java:164) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:73) at org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:41) at org.apache.spark.scheduler.Task.run(Task.scala:89) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:227) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at java.lang.Thread.run(Thread.java:744) 16/07/31 20:54:30 INFO executor.Executor: Executor is trying to kill task 1090.1 in stage 18.0 (TID 23793) 16/07/31 20:54:30 INFO executor.CoarseGrainedExecutorBackend: Driver commanded a shutdown ## How was this patch tested? Manual test. Author: sharkd <sharkd.tu@gmail.com> Author: sharkdtu <sharkdtu@tencent.com> Closes #14479 from sharkdtu/master.
* [SPARK-16796][WEB UI] Visible passwords on Spark environment pageArtur Sukhenko2016-08-021-1/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Mask spark.ssl.keyPassword, spark.ssl.keyStorePassword, spark.ssl.trustStorePassword in Web UI environment page. (Changes their values to ***** in env. page) ## How was this patch tested? I've built spark, run spark shell and checked that this values have been masked with *****. Also run tests: ./dev/run-tests [info] ScalaTest [info] Run completed in 1 hour, 9 minutes, 5 seconds. [info] Total number of tests run: 2166 [info] Suites: completed 65, aborted 0 [info] Tests: succeeded 2166, failed 0, canceled 0, ignored 590, pending 0 [info] All tests passed. ![mask](https://cloud.githubusercontent.com/assets/15244468/17262154/7641e132-55e2-11e6-8a6c-30ead77c7372.png) Author: Artur Sukhenko <artur.sukhenko@gmail.com> Closes #14409 from Devian-ua/maskpass.
* [SPARK-16787] SparkContext.addFile() should not throw if called twice with ↵Josh Rosen2016-08-024-26/+78
| | | | | | | | | | | | | | | | | | | | | | | | | | | | the same file ## What changes were proposed in this pull request? The behavior of `SparkContext.addFile()` changed slightly with the introduction of the Netty-RPC-based file server, which was introduced in Spark 1.6 (where it was disabled by default) and became the default / only file server in Spark 2.0.0. Prior to 2.0, calling `SparkContext.addFile()` with files that have the same name and identical contents would succeed. This behavior was never explicitly documented but Spark has behaved this way since very early 1.x versions. In 2.0 (or 1.6 with the Netty file server enabled), the second `addFile()` call will fail with a requirement error because NettyStreamManager tries to guard against duplicate file registration. This problem also affects `addJar()` in a more subtle way: the `fileServer.addJar()` call will also fail with an exception but that exception is logged and ignored; I believe that the problematic exception-catching path was mistakenly copied from some old code which was only relevant to very old versions of Spark and YARN mode. I believe that this change of behavior was unintentional, so this patch weakens the `require` check so that adding the same filename at the same path will succeed. At file download time, Spark tasks will fail with exceptions if an executor already has a local copy of a file and that file's contents do not match the contents of the file being downloaded / added. As a result, it's important that we prevent files with the same name and different contents from being served because allowing that can effectively brick an executor by preventing it from successfully launching any new tasks. Before this patch's change, this was prevented by forbidding `addFile()` from being called twice on files with the same name. Because Spark does not defensively copy local files that are passed to `addFile` it is vulnerable to files' contents changing, so I think it's okay to rely on an implicit assumption that these files are intended to be immutable (since if they _are_ mutable then this can lead to either explicit task failures or implicit incorrectness (in case new executors silently get newer copies of the file while old executors continue to use an older version)). To guard against this, I have decided to only update the file addition timestamps on the first call to `addFile()`; duplicate calls will succeed but will not update the timestamp. This behavior is fine as long as we assume files are immutable, which seems reasonable given the behaviors described above. As part of this change, I also improved the thread-safety of the `addedJars` and `addedFiles` maps; this is important because these maps may be concurrently read by a task launching thread and written by a driver thread in case the user's driver code is multi-threaded. ## How was this patch tested? I added regression tests in `SparkContextSuite`. Author: Josh Rosen <joshrosen@databricks.com> Closes #14396 from JoshRosen/SPARK-16787.
* [SPARK-15541] Casting ConcurrentHashMap to ConcurrentMap (master branch)Maciej Brynski2016-08-021-3/+5
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Casting ConcurrentHashMap to ConcurrentMap allows to run code compiled with Java 8 on Java 7 ## How was this patch tested? Compilation. Existing automatic tests Author: Maciej Brynski <maciej.brynski@adpilot.pl> Closes #14459 from maver1ck/spark-15541-master.
* [SPARK-16694][CORE] Use for/foreach rather than map for Unit expressions ↵Sean Owen2016-07-309-29/+29
| | | | | | | | | | | | | | | | whose side effects are required ## What changes were proposed in this pull request? Use foreach/for instead of map where operation requires execution of body, not actually defining a transformation ## How was this patch tested? Jenkins Author: Sean Owen <sowen@cloudera.com> Closes #14332 from srowen/SPARK-16694.
* [SPARK-16637] Unified containerizerMichael Gummelt2016-07-2910-72/+132
| | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? New config var: spark.mesos.docker.containerizer={"mesos","docker" (default)} This adds support for running docker containers via the Mesos unified containerizer: http://mesos.apache.org/documentation/latest/container-image/ The benefit is losing the dependency on `dockerd`, and all the costs which it incurs. I've also updated the supported Mesos version to 0.28.2 for support of the required protobufs. This is blocked on: https://github.com/apache/spark/pull/14167 ## How was this patch tested? - manually testing jobs submitted with both "mesos" and "docker" settings for the new config var. - spark/mesos integration test suite Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #14275 from mgummelt/unified-containerizer.
* [SPARK-5847][CORE] Allow for configuring MetricsSystem's use of app ID to ↵Mark Grover2016-07-275-26/+158
| | | | | | | | | | | | | | | | | | | | | | | | | | namespace all metrics ## What changes were proposed in this pull request? Adding a new property to SparkConf called spark.metrics.namespace that allows users to set a custom namespace for executor and driver metrics in the metrics systems. By default, the root namespace used for driver or executor metrics is the value of `spark.app.id`. However, often times, users want to be able to track the metrics across apps for driver and executor metrics, which is hard to do with application ID (i.e. `spark.app.id`) since it changes with every invocation of the app. For such use cases, users can set the `spark.metrics.namespace` property to another spark configuration key like `spark.app.name` which is then used to populate the root namespace of the metrics system (with the app name in our example). `spark.metrics.namespace` property can be set to any arbitrary spark property key, whose value would be used to set the root namespace of the metrics system. Non driver and executor metrics are never prefixed with `spark.app.id`, nor does the `spark.metrics.namespace` property have any such affect on such metrics. ## How was this patch tested? Added new unit tests, modified existing unit tests. Author: Mark Grover <mark@apache.org> Closes #14270 from markgrover/spark-5847.
* [SPARK-15703][SCHEDULER][CORE][WEBUI] Make ListenerBus event queue size ↵Dhruve Ashar2016-07-268-36/+56
| | | | | | | | | | | | | | | | | configurable ## What changes were proposed in this pull request? This change adds a new configuration entry to specify the size of the spark listener bus event queue. The value for this config ("spark.scheduler.listenerbus.eventqueue.size") is set to a default to 10000. Note: I haven't currently documented the configuration entry. We can decide whether it would be appropriate to make it a public configuration or keep it as an undocumented one. Refer JIRA for more details. ## How was this patch tested? Ran existing jobs and verified the event queue size with debug logs and from the Spark WebUI Environment tab. Author: Dhruve Ashar <dhruveashar@gmail.com> Closes #14269 from dhruve/bug/SPARK-15703.
* [SPARK-15271][MESOS] Allow force pulling executor docker imagesPhilipp Hoffmann2016-07-266-22/+91
| | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? (Please fill in changes proposed in this fix) ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Mesos agents by default will not pull docker images which are cached locally already. In order to run Spark executors from mutable tags like `:latest` this commit introduces a Spark setting (`spark.mesos.executor.docker.forcePullImage`). Setting this flag to true will tell the Mesos agent to force pull the docker image (default is `false` which is consistent with the previous implementation and Mesos' default behaviour). Author: Philipp Hoffmann <mail@philipphoffmann.de> Closes #14348 from philipphoffmann/force-pull-image.
* [SPARK-15590][WEBUI] Paginate Job Table in Jobs tabTao Lin2016-07-252-62/+312
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This patch adds pagination support for the Job Tables in the Jobs tab. Pagination is provided for all of the three Job Tables (active, completed, and failed). Interactions (jumping, sorting, and setting page size) for paged tables are also included. The diff didn't keep track of some lines based on the original ones. The function `makeRow`of the original `AllJobsPage.scala` is reused. They are separated at the beginning of the function `jobRow` (L427-439) and the function `row`(L594-618) in the new `AllJobsPage.scala`. ## How was this patch tested? Tested manually by using checking the Web UI after completing and failing hundreds of jobs. Generate completed jobs by: ```scala val d = sc.parallelize(Array(1,2,3,4,5)) for(i <- 1 to 255){ var b = d.collect() } ``` Generate failed jobs by calling the following code multiple times: ```scala var b = d.map(_/0).collect() ``` Interactions like jumping, sorting, and setting page size are all tested. This shows the pagination for completed jobs: ![paginate success jobs](https://cloud.githubusercontent.com/assets/5558370/15986498/efa12ef6-303b-11e6-8b1d-c3382aeb9ad0.png) This shows the sorting works in job tables: ![sorting](https://cloud.githubusercontent.com/assets/5558370/15986539/98c8a81a-303c-11e6-86f2-8d2bc7924ee9.png) This shows the pagination for failed jobs and the effect of jumping and setting page size: ![paginate failed jobs](https://cloud.githubusercontent.com/assets/5558370/15986556/d8c1323e-303c-11e6-8e4b-7bdb030ea42b.png) Author: Tao Lin <nblintao@gmail.com> Closes #13620 from nblintao/dev.
* [SPARK-16166][CORE] Also take off-heap memory usage into consideration in ↵jerryshao2016-07-257-9/+26
| | | | | | | | | | | | | | | | | | | | | | | log and webui display ## What changes were proposed in this pull request? Currently in the log and UI display, only on-heap storage memory is calculated and displayed, ``` 16/06/27 13:41:52 INFO MemoryStore: Block rdd_5_0 stored as values in memory (estimated size 17.8 KB, free 665.9 MB) ``` <img width="1232" alt="untitled" src="https://cloud.githubusercontent.com/assets/850797/16369960/53fb614e-3c6e-11e6-8fa3-7ffe65abcb49.png"> With [SPARK-13992](https://issues.apache.org/jira/browse/SPARK-13992) off-heap memory is supported for data persistence, so here change to also take off-heap storage memory into consideration. ## How was this patch tested? Unit test and local verification. Author: jerryshao <sshao@hortonworks.com> Closes #13920 from jerryshao/SPARK-16166.
* Revert "[SPARK-15271][MESOS] Allow force pulling executor docker images"Josh Rosen2016-07-256-91/+22
| | | | This reverts commit 978cd5f125eb5a410bad2e60bf8385b11cf1b978.
* [SPARK-15271][MESOS] Allow force pulling executor docker imagesPhilipp Hoffmann2016-07-256-22/+91
| | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Mesos agents by default will not pull docker images which are cached locally already. In order to run Spark executors from mutable tags like `:latest` this commit introduces a Spark setting `spark.mesos.executor.docker.forcePullImage`. Setting this flag to true will tell the Mesos agent to force pull the docker image (default is `false` which is consistent with the previous implementation and Mesos' default behaviour). ## How was this patch tested? I ran a sample application including this change on a Mesos cluster and verified the correct behaviour for both, with and without, force pulling the executor image. As expected the image is being force pulled if the flag is set. Author: Philipp Hoffmann <mail@philipphoffmann.de> Closes #13051 from philipphoffmann/force-pull-image.
* [SPARK-5581][CORE] When writing sorted map output file, avoid open / …Brian Cho2016-07-247-156/+192
| | | | | | | | | | | | | | | | | | | | | | …close between each partition ## What changes were proposed in this pull request? Replace commitAndClose with separate commit and close to avoid opening and closing the file between partitions. ## How was this patch tested? Run existing unit tests, add a few unit tests regarding reverts. Observed a ~20% reduction in total time in tasks on stages with shuffle writes to many partitions. JoshRosen Author: Brian Cho <bcho@fb.com> Closes #13382 from dafrista/separatecommit-master.
* [SPARK-16416][CORE] force eager creation of loggers to avoid shutdown hook ↵Mikael Ståldal2016-07-247-0/+9
| | | | | | | | | | | | | | | | conflicts ## What changes were proposed in this pull request? Force eager creation of loggers to avoid shutdown hook conflicts. ## How was this patch tested? Manually tested with a project using Log4j 2, verified that the shutdown hook conflict issue was solved. Author: Mikael Ståldal <mikael.staldal@magine.com> Closes #14320 from mikaelstaldal/shutdown-hook-logging.
* [SPARK-16194] Mesos Driver env varsMichael Gummelt2016-07-214-106/+191
| | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Added new configuration namespace: spark.mesos.env.* This allows a user submitting a job in cluster mode to set arbitrary environment variables on the driver. spark.mesos.driverEnv.KEY=VAL will result in the env var "KEY" being set to "VAL" I've also refactored the tests a bit so we can re-use code in MesosClusterScheduler. And I've refactored the command building logic in `buildDriverCommand`. Command builder values were very intertwined before, and now it's easier to determine exactly how each variable is set. ## How was this patch tested? unit tests Author: Michael Gummelt <mgummelt@mesosphere.io> Closes #14167 from mgummelt/driver-env-vars.
* [SPARK-16272][CORE] Allow config values to reference conf, env, system props.Marcelo Vanzin2016-07-204-32/+233
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This allows configuration to be more flexible, for example, when the cluster does not have a homogeneous configuration (e.g. packages are installed on different paths in different nodes). By allowing one to reference the environment from the conf, it becomes possible to work around those in certain cases. As part of the implementation, ConfigEntry now keeps track of all "known" configs (i.e. those created through the use of ConfigBuilder), since that list is used by the resolution code. This duplicates some code in SQLConf, which could potentially be merged with this now. It will also make it simpler to implement some missing features such as filtering which configs show up in the UI or in event logs - which are not part of this change. Another change is in the way ConfigEntry reads config data; it now takes a string map and a function that reads env variables, so that it can be called both from SparkConf and SQLConf. This makes it so both places follow the same read path, instead of having to replicate certain logic in SQLConf. There are still a couple of methods in SQLConf that peek into fields of ConfigEntry directly, though. Tested via unit tests, and by using the new variable expansion functionality in a shell session with a custom spark.sql.hive.metastore.jars value. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #14022 from vanzin/SPARK-16272.
* [SPARK-15951] Change Executors Page to use datatables to support sorting ↵Kishor Patil2016-07-209-301/+693
| | | | | | | | | | | | | | | | | | | columns and searching 1. Create the executorspage-template.html for displaying application information in datables. 2. Added REST API endpoint "allexecutors" to be able to see all executors created for particular job. 3. The executorspage.js uses jQuery to access the data from /api/v1/applications/appid/allexecutors REST API, and use DataTable to display executors for the application. It also, generates summary of dead/live and total executors created during life of the application. 4. Similar changes applicable to Executors Page on history server for a given application. Snapshots for how it looks like now: <img width="938" alt="screen shot 2016-06-14 at 2 45 44 pm" src="https://cloud.githubusercontent.com/assets/6090397/16060092/ad1de03a-324b-11e6-8469-9eaa3f2548b5.png"> New Executors Page screenshot looks like this: <img width="1436" alt="screen shot 2016-06-15 at 10 12 01 am" src="https://cloud.githubusercontent.com/assets/6090397/16085514/ee7004f0-32e1-11e6-9340-33d91e407f2b.png"> Author: Kishor Patil <kpatil@yahoo-inc.com> Closes #13670 from kishorvpatil/execTemplates.
* [SPARK-16613][CORE] RDD.pipe returns values for empty partitionsSean Owen2016-07-202-1/+15
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Document RDD.pipe semantics; don't execute process for empty input partitions. Note this includes the fix in https://github.com/apache/spark/pull/14256 because it's necessary to even test this. One or the other will merge the fix. ## How was this patch tested? Jenkins tests including new test. Author: Sean Owen <sowen@cloudera.com> Closes #14260 from srowen/SPARK-16613.