aboutsummaryrefslogtreecommitdiff
path: root/core
Commit message (Collapse)AuthorAgeFilesLines
* [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.
* [SPARK-3249][DOC] Fix links in ScalaDoc that cause warning messages in ↵hyukjinkwon2017-01-175-10/+9
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | `sbt/sbt unidoc` ## What changes were proposed in this pull request? This PR proposes to fix ambiguous link warnings by simply making them as code blocks for both javadoc and scaladoc. ``` [warn] .../spark/core/src/main/scala/org/apache/spark/Accumulator.scala:20: The link target "SparkContext#accumulator" is ambiguous. Several members fit the target: [warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala:281: The link target "runMiniBatchSGD" is ambiguous. Several members fit the target: [warn] .../spark/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala:83: The link target "run" is ambiguous. Several members fit the target: ... ``` This PR also fixes javadoc8 break as below: ``` [error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found [error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product} [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found [error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product} [error] ^ [error] .../spark/sql/core/target/java/org/apache/spark/sql/LowPrioritySQLImplicits.java:7: error: reference not found [error] * newProductEncoder - to disambiguate for {link List}s which are both {link Seq} and {link Product} [error] ^ [info] 3 errors ``` ## How was this patch tested? Manually via `sbt unidoc > output.txt` and the checked it via `cat output.txt | grep ambiguous` and `sbt unidoc | grep error`. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16604 from HyukjinKwon/SPARK-3249.
* [SPARK-19219][SQL] Fix Parquet log output defaultsNick Lavers2017-01-171-0/+4
| | | | | | | | | | | | | | ## What changes were proposed in this pull request? Changing the default parquet logging levels to reflect the changes made in PR [#15538](https://github.com/apache/spark/pull/15538), in order to prevent the flood of log messages by default. ## How was this patch tested? Default log output when reading from parquet 1.6 files was compared with and without this change. The change eliminates the extraneous logging and makes the output readable. Author: Nick Lavers <nick.lavers@videoamp.com> Closes #16580 from nicklavers/spark-19219-set_default_parquet_log_level.
* [SPARK-19082][SQL] Make ignoreCorruptFiles work for ParquetLiang-Chi Hsieh2017-01-161-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? We have a config `spark.sql.files.ignoreCorruptFiles` which can be used to ignore corrupt files when reading files in SQL. Currently the `ignoreCorruptFiles` config has two issues and can't work for Parquet: 1. We only ignore corrupt files in `FileScanRDD` . Actually, we begin to read those files as early as inferring data schema from the files. For corrupt files, we can't read the schema and fail the program. A related issue reported at http://apache-spark-developers-list.1001551.n3.nabble.com/Skip-Corrupted-Parquet-blocks-footer-tc20418.html 2. In `FileScanRDD`, we assume that we only begin to read the files when starting to consume the iterator. However, it is possibly the files are read before that. In this case, `ignoreCorruptFiles` config doesn't work too. This patch targets Parquet datasource. If this direction is ok, we can address the same issue for other datasources like Orc. Two main changes in this patch: 1. Replace `ParquetFileReader.readAllFootersInParallel` by implementing the logic to read footers in multi-threaded manner We can't ignore corrupt files if we use `ParquetFileReader.readAllFootersInParallel`. So this patch implements the logic to do the similar thing in `readParquetFootersInParallel`. 2. In `FileScanRDD`, we need to ignore corrupt file too when we call `readFunction` to return iterator. One thing to notice is: We read schema from Parquet file's footer. The method to read footer `ParquetFileReader.readFooter` throws `RuntimeException`, instead of `IOException`, if it can't successfully read the footer. Please check out https://github.com/apache/parquet-mr/blob/df9d8e415436292ae33e1ca0b8da256640de9710/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L470. So this patch catches `RuntimeException`. One concern is that it might also shadow other runtime exceptions other than reading corrupt files. ## 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 #16474 from viirya/fix-ignorecorrupted-parquet-files.
* [SPARK-19042] spark executor can't download the jars when uber jar's http ↵xiaojian.fxj2017-01-151-2/+2
| | | | | | | | | | url contains any query strings If the uber jars' https contains any query strings, the Executor.updateDependencies method can't can't download the jars correctly. This is because the "localName = name.split("/").last" won't get the expected jar's url. The bug fix is the same as [SPARK-17855] Author: xiaojian.fxj <xiaojian.fxj@alibaba-inc.com> Closes #16509 from hustfxj/bug.
* [SPARK-12757][CORE] lower "block locks were not released" log to info levelFelix Cheung2017-01-121-1/+1
| | | | | | | | | | ## What changes were proposed in this pull request? lower "block locks were not released" log to info level, as it is generating a lot of warnings in running ML, graph calls, as pointed out in the JIRA. Author: Felix Cheung <felixcheung_m@hotmail.com> Closes #16513 from felixcheung/blocklockswarn.
* [SPARK-19183][SQL] Add deleteWithJob hook to internal commit protocol APIEric Liang2017-01-121-0/+9
| | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Currently in SQL we implement overwrites by calling fs.delete() directly on the original data. This is not ideal since we the original files end up deleted even if the job aborts. We should extend the commit protocol to allow file overwrites to be managed as well. ## How was this patch tested? Existing tests. I also fixed a bunch of tests that were depending on the commit protocol implementation being set to the legacy mapreduce one. cc rxin cloud-fan Author: Eric Liang <ekl@databricks.com> Author: Eric Liang <ekhliang@gmail.com> Closes #16554 from ericl/add-delete-protocol.
* [SPARK-17568][CORE][DEPLOY] Add spark-submit option to override ivy settings ↵Bryan Cutler2017-01-112-66/+183
| | | | | | | | | | | | | | | | | | used to resolve packages/artifacts ## What changes were proposed in this pull request? Adding option in spark-submit to allow overriding the default IvySettings used to resolve artifacts as part of the Spark Packages functionality. This will allow all artifact resolution to go through a central managed repository, such as Nexus or Artifactory, where site admins can better approve and control what is used with Spark apps. This change restructures the creation of the IvySettings object in two distinct ways. First, if the `spark.ivy.settings` option is not defined then `buildIvySettings` will create a default settings instance, as before, with defined repositories (Maven Central) included. Second, if the option is defined, the ivy settings file will be loaded from the given path and only repositories defined within will be used for artifact resolution. ## How was this patch tested? Existing tests for default behaviour, Manual tests that load a ivysettings.xml file with local and Nexus repositories defined. Added new test to load a simple Ivy settings file with a local filesystem resolver. Author: Bryan Cutler <cutlerb@gmail.com> Author: Ian Hummel <ian@themodernlife.net> Closes #15119 from BryanCutler/spark-custom-IvySettings.
* [SPARK-19117][TESTS] Skip the tests using script transformation on Windowshyukjinkwon2017-01-102-17/+19
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? This PR proposes to skip the tests for script transformation failed on Windows due to fixed bash location. ``` SQLQuerySuite: - script *** FAILED *** (553 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 56.0 failed 1 times, most recent failure: Lost task 0.0 in stage 56.0 (TID 54, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - Star Expansion - script transform *** FAILED *** (2 seconds, 375 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 389.0 failed 1 times, most recent failure: Lost task 0.0 in stage 389.0 (TID 725, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - test script transform for stdout *** FAILED *** (2 seconds, 813 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 391.0 failed 1 times, most recent failure: Lost task 0.0 in stage 391.0 (TID 726, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - test script transform for stderr *** FAILED *** (2 seconds, 407 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 393.0 failed 1 times, most recent failure: Lost task 0.0 in stage 393.0 (TID 727, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - test script transform data type *** FAILED *** (171 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 395.0 failed 1 times, most recent failure: Lost task 0.0 in stage 395.0 (TID 728, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ``` HiveQuerySuite: - transform *** FAILED *** (359 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1347.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1347.0 (TID 2395, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - schema-less transform *** FAILED *** (344 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1348.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1348.0 (TID 2396, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with custom field delimiter *** FAILED *** (296 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1349.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1349.0 (TID 2397, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with custom field delimiter2 *** FAILED *** (297 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1350.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1350.0 (TID 2398, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with custom field delimiter3 *** FAILED *** (312 milliseconds) Failed to execute query using catalyst: Error: Job aborted due to stage failure: Task 0 in stage 1351.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1351.0 (TID 2399, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - transform with SerDe2 *** FAILED *** (437 milliseconds) org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1355.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1355.0 (TID 2403, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ``` LogicalPlanToSQLSuite: - script transformation - schemaless *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1968.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1968.0 (TID 3932, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - alias list *** FAILED *** (94 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1969.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1969.0 (TID 3933, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - alias list with type *** FAILED *** (93 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1970.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1970.0 (TID 3934, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format delimited clause with only one format property *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1971.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1971.0 (TID 3935, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format delimited clause with multiple format properties *** FAILED *** (94 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1972.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1972.0 (TID 3936, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format serde clauses with SERDEPROPERTIES *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1973.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1973.0 (TID 3937, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation - row format serde clauses without SERDEPROPERTIES *** FAILED *** (78 milliseconds) ... Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 1974.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1974.0 (TID 3938, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ``` ScriptTransformationSuite: - cat without SerDe *** FAILED *** (156 milliseconds) ... Caused by: java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - cat with LazySimpleSerDe *** FAILED *** (63 milliseconds) ... org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2383.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2383.0 (TID 4819, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation should not swallow errors from upstream operators (no serde) *** FAILED *** (78 milliseconds) ... org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2384.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2384.0 (TID 4820, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - script transformation should not swallow errors from upstream operators (with serde) *** FAILED *** (47 milliseconds) ... org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 2385.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2385.0 (TID 4821, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified - SPARK-14400 script transformation should fail for bad script command *** FAILED *** (47 milliseconds) "Job aborted due to stage failure: Task 0 in stage 2386.0 failed 1 times, most recent failure: Lost task 0.0 in stage 2386.0 (TID 4822, localhost, executor driver): java.io.IOException: Cannot run program "/bin/bash": CreateProcess error=2, The system cannot find the file specified ``` ## How was this patch tested? AppVeyor as below: ``` SQLQuerySuite: - script !!! CANCELED !!! (63 milliseconds) - Star Expansion - script transform !!! CANCELED !!! (0 milliseconds) - test script transform for stdout !!! CANCELED !!! (0 milliseconds) - test script transform for stderr !!! CANCELED !!! (0 milliseconds) - test script transform data type !!! CANCELED !!! (0 milliseconds) ``` ``` HiveQuerySuite: - transform !!! CANCELED !!! (31 milliseconds) - schema-less transform !!! CANCELED !!! (0 milliseconds) - transform with custom field delimiter !!! CANCELED !!! (0 milliseconds) - transform with custom field delimiter2 !!! CANCELED !!! (0 milliseconds) - transform with custom field delimiter3 !!! CANCELED !!! (0 milliseconds) - transform with SerDe2 !!! CANCELED !!! (0 milliseconds) ``` ``` LogicalPlanToSQLSuite: - script transformation - schemaless !!! CANCELED !!! (78 milliseconds) - script transformation - alias list !!! CANCELED !!! (0 milliseconds) - script transformation - alias list with type !!! CANCELED !!! (0 milliseconds) - script transformation - row format delimited clause with only one format property !!! CANCELED !!! (15 milliseconds) - script transformation - row format delimited clause with multiple format properties !!! CANCELED !!! (0 milliseconds) - script transformation - row format serde clauses with SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds) - script transformation - row format serde clauses without SERDEPROPERTIES !!! CANCELED !!! (0 milliseconds) ``` ``` ScriptTransformationSuite: - cat without SerDe !!! CANCELED !!! (62 milliseconds) - cat with LazySimpleSerDe !!! CANCELED !!! (0 milliseconds) - script transformation should not swallow errors from upstream operators (no serde) !!! CANCELED !!! (0 milliseconds) - script transformation should not swallow errors from upstream operators (with serde) !!! CANCELED !!! (0 milliseconds) - SPARK-14400 script transformation should fail for bad script command !!! CANCELED !!! (0 milliseconds) ``` Jenkins tests Author: hyukjinkwon <gurwls223@gmail.com> Closes #16501 from HyukjinKwon/windows-bash.
* [SPARK-18922][SQL][CORE][STREAMING][TESTS] Fix all identified tests failed ↵hyukjinkwon2017-01-102-2/+7
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | due to path and resource-not-closed problems on Windows ## What changes were proposed in this pull request? This PR proposes to fix all the test failures identified by testing with AppVeyor. **Scala - aborted tests** ``` WindowQuerySuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.execution.WindowQuerySuite *** ABORTED *** (156 milliseconds) org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: C:projectssparksqlhive argetscala-2.11 est-classesdatafilespart_tiny.txt; OrcSourceSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.orc.OrcSourceSuite *** ABORTED *** (62 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ParquetMetastoreSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetMetastoreSuite *** ABORTED *** (4 seconds, 703 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ParquetSourceSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.sql.hive.ParquetSourceSuite *** ABORTED *** (3 seconds, 907 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-581a6575-454f-4f21-a516-a07f95266143; KafkaRDDSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaRDDSuite *** ABORTED *** (5 seconds, 212 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-4722304d-213e-4296-b556-951df1a46807 DirectKafkaStreamSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.DirectKafkaStreamSuite *** ABORTED *** (7 seconds, 127 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d0d3eba7-4215-4e10-b40e-bb797e89338e at org.apache.spark.util.Utils$.deleteRecursively(Utils.scala:1010) ReliableKafkaStreamSuite Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.ReliableKafkaStreamSuite *** ABORTED *** (5 seconds, 498 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-d33e45a0-287e-4bed-acae-ca809a89d888 KafkaStreamSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaStreamSuite *** ABORTED *** (2 seconds, 892 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-59c9d169-5a56-4519-9ef0-cefdbd3f2e6c KafkaClusterSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka.KafkaClusterSuite *** ABORTED *** (1 second, 690 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-3ef402b0-8689-4a60-85ae-e41e274f179d DirectKafkaStreamSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.DirectKafkaStreamSuite *** ABORTED *** (59 seconds, 626 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-426107da-68cf-4d94-b0d6-1f428f1c53f6 KafkaRDDSuite: Exception encountered when attempting to run a suite with class name: org.apache.spark.streaming.kafka010.KafkaRDDSuite *** ABORTED *** (2 minutes, 6 seconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-b9ce7929-5dae-46ab-a0c4-9ef6f58fbc2 ``` **Java - failed tests** ``` Test org.apache.spark.streaming.kafka.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-1cee32f4-4390-4321-82c9-e8616b3f0fb0, took 9.61 sec Test org.apache.spark.streaming.kafka.JavaKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-f42695dd-242e-4b07-847c-f299b8e4676e, took 11.797 sec Test org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-85c0d062-78cf-459c-a2dd-7973572101ce, took 1.581 sec Test org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite.testKafkaRDD failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-49eb6b5c-8366-47a6-83f2-80c443c48280, took 17.895 sec org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite.testKafkaStream failed: java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-898cf826-d636-4b1c-a61a-c12a364c02e7, took 8.858 sec ``` **Scala - failed tests** ``` PartitionProviderCompatibilitySuite: - insert overwrite partition of new datasource table overwrites just partition *** FAILED *** (828 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-bb6337b9-4f99-45ab-ad2c-a787ab965c09 - SPARK-18635 special chars in partition values - partition management true *** FAILED *** (5 seconds, 360 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - SPARK-18635 special chars in partition values - partition management false *** FAILED *** (141 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` UtilsSuite: - reading offset bytes of a file (compressed) *** FAILED *** (0 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-ecb2b7d5-db8b-43a7-b268-1bf242b5a491 - reading offset bytes across multiple files (compressed) *** FAILED *** (0 milliseconds) java.io.IOException: Failed to delete: C:\projects\spark\target\tmp\spark-25cc47a8-1faa-4da5-8862-cf174df63ce0 ``` ``` StatisticsSuite: - MetastoreRelations fallback to HDFS for size estimation *** FAILED *** (110 milliseconds) org.apache.spark.sql.catalyst.analysis.NoSuchTableException: Table or view 'csv_table' not found in database 'default'; ``` ``` SQLQuerySuite: - permanent UDTF *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count_temp'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 24 - describe functions - user defined functions *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Undefined function: 'udtf_count'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 7 - CTAS without serde with location *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-ed673d73-edfc-404e-829e-2e2b9725d94e/c1 - derived from Hive query file: drop_database_removes_partition_dirs.q *** FAILED *** (47 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_database_removes_partition_dirs_table - derived from Hive query file: drop_table_removes_partition_dirs.q *** FAILED *** (0 milliseconds) java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: file:C:projectsspark%09arget%09mpspark-d2ddf08e-699e-45be-9ebd-3dfe619680fe/drop_table_removes_partition_dirs_table2 - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH *** FAILED *** (109 milliseconds) java.nio.file.InvalidPathException: Illegal char <:> at index 2: /C:/projects/spark/sql/hive/projectsspark arget mpspark-1a122f8c-dfb3-46c4-bab1-f30764baee0e/*part-r* ``` ``` HiveDDLSuite: - drop external tables in default database *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - add/drop partitions - external table *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - create/drop database - location without pre-created directory *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - create/drop database - location with pre-created directory *** FAILED *** (32 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - drop database containing tables - CASCADE *** FAILED *** (94 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - drop an empty database - CASCADE *** FAILED *** (63 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - drop database containing tables - RESTRICT *** FAILED *** (47 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - drop an empty database - RESTRICT *** FAILED *** (47 milliseconds) CatalogDatabase(db1,,file:/C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be/db1.db,Map()) did not equal CatalogDatabase(db1,,file:C:/projects/spark/target/tmp/warehouse-d0665ee0-1e39-4805-b471-0b764f7838be\db1.db,Map()) (HiveDDLSuite.scala:675) - CREATE TABLE LIKE an external data source table *** FAILED *** (140 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-c5eba16d-07ae-4186-95bb-21c5811cf888; - CREATE TABLE LIKE an external Hive serde table *** FAILED *** (16 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - desc table for data source table - no user-defined schema *** FAILED *** (125 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-e8bf5bf5-721a-4cbe-9d6 at scala.collection.immutable.List.foreach(List.scala:381)d-5543a8301c1d; ``` ``` MetastoreDataSourcesSuite - CTAS: persisted bucketed data source table *** FAILED *** (16 milliseconds) java.lang.IllegalArgumentException: Can not create a Path from an empty string ``` ``` ShowCreateTableSuite: - simple external hive table *** FAILED *** (0 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` PartitionedTablePerfStatsSuite: - hive table: partitioned pruned table reports only selected files *** FAILED *** (313 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - datasource table: partitioned pruned table reports only selected files *** FAILED *** (219 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-311f45f8-d064-4023-a4bb-e28235bff64d; - hive table: lazy partition pruning reads only necessary partition data *** FAILED *** (203 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - datasource table: lazy partition pruning reads only necessary partition data *** FAILED *** (187 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-fde874ca-66bd-4d0b-a40f-a043b65bf957; - hive table: lazy partition pruning with file status caching enabled *** FAILED *** (188 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - datasource table: lazy partition pruning with file status caching enabled *** FAILED *** (187 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-e6d20183-dd68-4145-acbe-4a509849accd; - hive table: file status caching respects refresh table and refreshByPath *** FAILED *** (172 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - datasource table: file status caching respects refresh table and refreshByPath *** FAILED *** (203 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-8b2c9651-2adf-4d58-874f-659007e21463; - hive table: file status cache respects size limit *** FAILED *** (219 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - datasource table: file status cache respects size limit *** FAILED *** (171 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-7835ab57-cb48-4d2c-bb1d-b46d5a4c47e4; - datasource table: table setup does not scan filesystem *** FAILED *** (266 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-20598d76-c004-42a7-8061-6c56f0eda5e2; - hive table: table setup does not scan filesystem *** FAILED *** (266 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - hive table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 281 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - datasource table: num hive client calls does not scale with partition count *** FAILED *** (2 seconds, 422 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4cfed321-4d1d-4b48-8d34-5c169afff383; - hive table: files read and cached when filesource partition management is off *** FAILED *** (234 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); - datasource table: all partition data cached in memory when partition management is off *** FAILED *** (203 milliseconds) org.apache.spark.sql.AnalysisException: Path does not exist: file:/C:projectsspark arget mpspark-4bcc0398-15c9-4f6a-811e-12d40f3eec12; - SPARK-18700: table loaded only once even when resolved concurrently *** FAILED *** (1 second, 266 milliseconds) org.apache.spark.sql.AnalysisException: org.apache.hadoop.hive.ql.metadata.HiveException: MetaException(message:java.lang.IllegalArgumentException: Can not create a Path from an empty string); ``` ``` HiveSparkSubmitSuite: - temporary Hive UDF: define a UDF and use it *** FAILED *** (2 seconds, 94 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - permanent Hive UDF: define a UDF and use it *** FAILED *** (281 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - permanent Hive UDF: use a already defined permanent function *** FAILED *** (718 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-8368: includes jars passed in through --jars *** FAILED *** (3 seconds, 521 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-8020: set sql conf in spark conf *** FAILED *** (0 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-8489: MissingRequirementError during reflection *** FAILED *** (94 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-9757 Persist Parquet relation with decimal column *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-11009 fix wrong result of Window function in cluster mode *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-14244 fix window partition size attribute binding failure *** FAILED *** (78 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - set spark.sql.warehouse.dir *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - set hive.metastore.warehouse.dir *** FAILED *** (15 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-16901: set javax.jdo.option.ConnectionURL *** FAILED *** (16 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified - SPARK-18360: default table path of tables in default database should depend on the location of default database *** FAILED *** (15 milliseconds) java.io.IOException: Cannot run program "./bin/spark-submit" (in directory "C:\projects\spark"): CreateProcess error=2, The system cannot find the file specified ``` ``` UtilsSuite: - resolveURIs with multiple paths *** FAILED *** (0 milliseconds) ".../jar3,file:/C:/pi.py[%23]py.pi,file:/C:/path%..." did not equal ".../jar3,file:/C:/pi.py[#]py.pi,file:/C:/path%..." (UtilsSuite.scala:468) ``` ``` CheckpointSuite: - recovery with file input stream *** FAILED *** (10 seconds, 205 milliseconds) The code passed to eventually never returned normally. Attempted 660 times over 10.014272499999999 seconds. Last failure message: Unexpected internal error near index 1 \ ^. (CheckpointSuite.scala:680) ``` ## How was this patch tested? Manually via AppVeyor as below: **Scala - aborted tests** ``` WindowQuerySuite - all passed OrcSourceSuite: - SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds) org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625) ParquetMetastoreSuite - all passed ParquetSourceSuite - all passed KafkaRDDSuite - all passed DirectKafkaStreamSuite - all passed ReliableKafkaStreamSuite - all passed KafkaStreamSuite - all passed KafkaClusterSuite - all passed DirectKafkaStreamSuite - all passed KafkaRDDSuite - all passed ``` **Java - failed tests** ``` org.apache.spark.streaming.kafka.JavaKafkaRDDSuite - all passed org.apache.spark.streaming.kafka.JavaDirectKafkaStreamSuite - all passed org.apache.spark.streaming.kafka.JavaKafkaStreamSuite - all passed org.apache.spark.streaming.kafka010.JavaDirectKafkaStreamSuite - all passed org.apache.spark.streaming.kafka010.JavaKafkaRDDSuite - all passed ``` **Scala - failed tests** ``` PartitionProviderCompatibilitySuite: - insert overwrite partition of new datasource table overwrites just partition (1 second, 953 milliseconds) - SPARK-18635 special chars in partition values - partition management true (6 seconds, 31 milliseconds) - SPARK-18635 special chars in partition values - partition management false (4 seconds, 578 milliseconds) ``` ``` UtilsSuite: - reading offset bytes of a file (compressed) (203 milliseconds) - reading offset bytes across multiple files (compressed) (0 milliseconds) ``` ``` StatisticsSuite: - MetastoreRelations fallback to HDFS for size estimation (94 milliseconds) ``` ``` SQLQuerySuite: - permanent UDTF (407 milliseconds) - describe functions - user defined functions (441 milliseconds) - CTAS without serde with location (2 seconds, 831 milliseconds) - derived from Hive query file: drop_database_removes_partition_dirs.q (734 milliseconds) - derived from Hive query file: drop_table_removes_partition_dirs.q (563 milliseconds) - SPARK-17796 Support wildcard character in filename for LOAD DATA LOCAL INPATH (453 milliseconds) ``` ``` HiveDDLSuite: - drop external tables in default database (3 seconds, 5 milliseconds) - add/drop partitions - external table (2 seconds, 750 milliseconds) - create/drop database - location without pre-created directory (500 milliseconds) - create/drop database - location with pre-created directory (407 milliseconds) - drop database containing tables - CASCADE (453 milliseconds) - drop an empty database - CASCADE (375 milliseconds) - drop database containing tables - RESTRICT (328 milliseconds) - drop an empty database - RESTRICT (391 milliseconds) - CREATE TABLE LIKE an external data source table (953 milliseconds) - CREATE TABLE LIKE an external Hive serde table (3 seconds, 782 milliseconds) - desc table for data source table - no user-defined schema (1 second, 150 milliseconds) ``` ``` MetastoreDataSourcesSuite - CTAS: persisted bucketed data source table (875 milliseconds) ``` ``` ShowCreateTableSuite: - simple external hive table (78 milliseconds) ``` ``` PartitionedTablePerfStatsSuite: - hive table: partitioned pruned table reports only selected files (1 second, 109 milliseconds) - datasource table: partitioned pruned table reports only selected files (860 milliseconds) - hive table: lazy partition pruning reads only necessary partition data (859 milliseconds) - datasource table: lazy partition pruning reads only necessary partition data (1 second, 219 milliseconds) - hive table: lazy partition pruning with file status caching enabled (875 milliseconds) - datasource table: lazy partition pruning with file status caching enabled (890 milliseconds) - hive table: file status caching respects refresh table and refreshByPath (922 milliseconds) - datasource table: file status caching respects refresh table and refreshByPath (640 milliseconds) - hive table: file status cache respects size limit (469 milliseconds) - datasource table: file status cache respects size limit (453 milliseconds) - datasource table: table setup does not scan filesystem (328 milliseconds) - hive table: table setup does not scan filesystem (313 milliseconds) - hive table: num hive client calls does not scale with partition count (5 seconds, 431 milliseconds) - datasource table: num hive client calls does not scale with partition count (4 seconds, 79 milliseconds) - hive table: files read and cached when filesource partition management is off (656 milliseconds) - datasource table: all partition data cached in memory when partition management is off (484 milliseconds) - SPARK-18700: table loaded only once even when resolved concurrently (2 seconds, 578 milliseconds) ``` ``` HiveSparkSubmitSuite: - temporary Hive UDF: define a UDF and use it (1 second, 745 milliseconds) - permanent Hive UDF: define a UDF and use it (406 milliseconds) - permanent Hive UDF: use a already defined permanent function (375 milliseconds) - SPARK-8368: includes jars passed in through --jars (391 milliseconds) - SPARK-8020: set sql conf in spark conf (156 milliseconds) - SPARK-8489: MissingRequirementError during reflection (187 milliseconds) - SPARK-9757 Persist Parquet relation with decimal column (157 milliseconds) - SPARK-11009 fix wrong result of Window function in cluster mode (156 milliseconds) - SPARK-14244 fix window partition size attribute binding failure (156 milliseconds) - set spark.sql.warehouse.dir (172 milliseconds) - set hive.metastore.warehouse.dir (156 milliseconds) - SPARK-16901: set javax.jdo.option.ConnectionURL (157 milliseconds) - SPARK-18360: default table path of tables in default database should depend on the location of default database (172 milliseconds) ``` ``` UtilsSuite: - resolveURIs with multiple paths (0 milliseconds) ``` ``` CheckpointSuite: - recovery with file input stream (4 seconds, 452 milliseconds) ``` Note: after resolving the aborted tests, there is a test failure identified as below: ``` OrcSourceSuite: - SPARK-18220: read Hive orc table with varchar column *** FAILED *** (4 seconds, 417 milliseconds) org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.mr.MapRedTask. org.apache.hadoop.io.nativeio.NativeIO$Windows.access0(Ljava/lang/String;I)Z at org.apache.spark.sql.hive.client.HiveClientImpl$$anonfun$runHive$1.apply(HiveClientImpl.scala:625) ``` This does not look due to this problem so this PR does not fix it here. Author: hyukjinkwon <gurwls223@gmail.com> Closes #16451 from HyukjinKwon/all-path-resource-fixes.
* [SPARK-19026] SPARK_LOCAL_DIRS(multiple directories on different disks) ↵zuotingbing2017-01-081-6/+19
| | | | | | | | | | | | | cannot be deleted JIRA Issue: https://issues.apache.org/jira/browse/SPARK-19026 SPARK_LOCAL_DIRS (Standalone) can be a comma-separated list of multiple directories on different disks, e.g. SPARK_LOCAL_DIRS=/dir1,/dir2,/dir3, if there is a IOExecption when create sub directory on dir3 , the sub directory which have been created successfully on dir1 and dir2 cannot be deleted anymore when the application finishes. So we should catch the IOExecption at Utils.createDirectory , otherwise the variable "appDirectories(appId)" which the function maybeCleanupApplication calls will not be set then dir1 and dir2 will not be cleaned up . Author: zuotingbing <zuo.tingbing9@zte.com.cn> Closes #16439 from zuotingbing/master.
* [SPARK-17931] Eliminate unnecessary task (de) serializationKay Ousterhout2017-01-069-142/+215
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | In the existing code, there are three layers of serialization involved in sending a task from the scheduler to an executor: - A Task object is serialized - The Task object is copied to a byte buffer that also contains serialized information about any additional JARs, files, and Properties needed for the task to execute. This byte buffer is stored as the member variable serializedTask in the TaskDescription class. - The TaskDescription is serialized (in addition to the serialized task + JARs, the TaskDescription class contains the task ID and other metadata) and sent in a LaunchTask message. While it *is* necessary to have two layers of serialization, so that the JAR, file, and Property info can be deserialized prior to deserializing the Task object, the third layer of deserialization is unnecessary. This commit eliminates a layer of serialization by moving the JARs, files, and Properties into the TaskDescription class. This commit also serializes the Properties manually (by traversing the map), as is done with the JARs and files, which reduces the final serialized size. Unit tests This is a simpler alternative to the approach proposed in #15505. shivaram and I did some benchmarking of this and #15505 on a 20-machine m2.4xlarge EC2 machines (160 cores). We ran ~30 trials of code [1] (a very simple job with 10K tasks per stage) and measured the average time per stage: Before this change: 2490ms With this change: 2345 ms (so ~6% improvement over the baseline) With witgo's approach in #15505: 2046 ms (~18% improvement over baseline) The reason that #15505 has a more significant improvement is that it also moves the serialization from the TaskSchedulerImpl thread to the CoarseGrainedSchedulerBackend thread. I added that functionality on top of this change, and got almost the same improvement [1] as #15505 (average of 2103ms). I think we should decouple these two changes, both so we have some record of the improvement form each individual improvement, and because this change is more about simplifying the code base (the improvement is negligible) while the other is about performance improvement. The plan, currently, is to merge this PR and then merge the remaining part of #15505 that moves serialization. [1] The reason the improvement wasn't quite as good as with #15505 when we ran the benchmarks is almost certainly because, at the point when we ran the benchmarks, I hadn't updated the code to manually serialize the Properties (instead the code was using Java's default serialization for the Properties object, whereas #15505 manually serialized the Properties). This PR has since been updated to manually serialize the Properties, just like the other maps. Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #16053 from kayousterhout/SPARK-17931.
* [SPARK-19033][CORE] Add admin acls for history serverjerryshao2017-01-062-7/+124
| | | | | | | | | | | | | | | | ## What changes were proposed in this pull request? Current HistoryServer's ACLs is derived from application event-log, which means the newly changed ACLs cannot be applied to the old data, this will become a problem where newly added admin cannot access the old application history UI, only the new application can be affected. So here propose to add admin ACLs for history server, any configured user/group could have the view access to all the applications, while the view ACLs derived from application run-time still take effect. ## How was this patch tested? Unit test added. Author: jerryshao <sshao@hortonworks.com> Closes #16470 from jerryshao/SPARK-19033.
* [SPARK-14958][CORE] Failed task not handled when there's error deserializing ↵Rui Li2017-01-052-2/+25
| | | | | | | | | | | | | | | | | | failure reason ## What changes were proposed in this pull request? TaskResultGetter tries to deserialize the TaskEndReason before handling the failed task. If an error is thrown during deserialization, the failed task won't be handled, which leaves the job hanging. The PR proposes to handle the failed task in a finally block. ## How was this patch tested? In my case I hit a NoClassDefFoundError and the job hangs. Manually verified the patch can fix it. Author: Rui Li <rui.li@intel.com> Author: Rui Li <lirui@apache.org> Author: Rui Li <shlr@cn.ibm.com> Closes #12775 from lirui-intel/SPARK-14958.
* [SPARK-19062] Utils.writeByteBuffer bug fixKay Ousterhout2017-01-042-1/+28
| | | | | | | | | | | | This commit changes Utils.writeByteBuffer so that it does not change the position of the ByteBuffer that it writes out, and adds a unit test for this functionality. cc mridulm Author: Kay Ousterhout <kayousterhout@gmail.com> Closes #16462 from kayousterhout/SPARK-19062.
* [MINOR][DOCS] Remove consecutive duplicated words/typo in Spark RepoNiranjan Padmanabhan2017-01-0412-13/+13
| | | | | | | | | | | | ## What changes were proposed in this pull request? There are many locations in the Spark repo where the same word occurs consecutively. Sometimes they are appropriately placed, but many times they are not. This PR removes the inappropriately duplicated words. ## How was this patch tested? N/A since only docs or comments were updated. Author: Niranjan Padmanabhan <niranjan.padmanabhan@gmail.com> Closes #16455 from neurons/np.structure_streaming_doc.
* [MINOR] Add missing sc.stop() to end of examplesWeiqing Yang2017-01-031-11/+9
| | | | | | | | | | | | | ## What changes were proposed in this pull request? Add `finally` clause for `sc.stop()` in the `test("register and deregister Spark listener from SparkContext")`. ## How was this patch tested? Pass the build and unit tests. Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #16426 from weiqingy/testIssue.