aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-12007][NETWORK] Avoid copies in the network lib's RPC layer.Marcelo Vanzin2015-11-3050-307/+589
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | This change seems large, but most of it is just replacing `byte[]` with `ByteBuffer` and `new byte[]` with `ByteBuffer.allocate()`, since it changes the network library's API. The following are parts of the code that actually have meaningful changes: - The Message implementations were changed to inherit from a new AbstractMessage that can optionally hold a reference to a body (in the form of a ManagedBuffer); this is similar to how ResponseWithBody worked before, except now it's not restricted to just responses. - The TransportFrameDecoder was pretty much rewritten to avoid copies as much as possible; it doesn't rely on CompositeByteBuf to accumulate incoming data anymore, since CompositeByteBuf has issues when slices are retained. The code now is able to create frames without having to resort to copying bytes except for a few bytes (containing the frame length) in very rare cases. - Some minor changes in the SASL layer to convert things back to `byte[]` since the JDK SASL API operates on those. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9987 from vanzin/SPARK-12007.
* [SPARK-12037][CORE] initialize heartbeatReceiverRef before calling ↵CodingCat2015-11-301-3/+4
| | | | | | | | | | | | startDriverHeartbeat https://issues.apache.org/jira/browse/SPARK-12037 a simple fix by changing the order of the statements Author: CodingCat <zhunansjtu@gmail.com> Closes #10032 from CodingCat/SPARK-12037.
* [SPARK-12035] Add more debug information in include_example tag of JekyllXusen Yin2015-11-301-4/+6
| | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-12035 When we debuging lots of example code files, like in https://github.com/apache/spark/pull/10002, it's hard to know which file causes errors due to limited information in `include_example.rb`. With their filenames, we can locate bugs easily. Author: Xusen Yin <yinxusen@gmail.com> Closes #10026 from yinxusen/SPARK-12035.
* [SPARK-12000] Fix API doc generation issuesJosh Rosen2015-11-304-9/+14
| | | | | | | | | | | | | | | This pull request fixes multiple issues with API doc generation. - Modify the Jekyll plugin so that the entire doc build fails if API docs cannot be generated. This will make it easy to detect when the doc build breaks, since this will now trigger Jenkins failures. - Change how we handle the `-target` compiler option flag in order to fix `javadoc` generation. - Incorporate doc changes from thunterdb (in #10048). Closes #10048. Author: Josh Rosen <joshrosen@databricks.com> Author: Timothy Hunter <timhunter@databricks.com> Closes #10049 from JoshRosen/fix-doc-build.
* [SPARK-12058][HOTFIX] Disable KinesisStreamTestsShixiong Zhu2015-11-301-0/+1
| | | | | | | | | | | | KinesisStreamTests in test.py is broken because of #9403. See https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/46896/testReport/(root)/KinesisStreamTests/test_kinesis_stream/ Because Streaming Python didn’t work when merging https://github.com/apache/spark/pull/9403, the PR build didn’t report the Python test failure actually. This PR just disabled the test to unblock #10039 Author: Shixiong Zhu <shixiong@databricks.com> Closes #10047 from zsxwing/disable-python-kinesis-test.
* fix Maven buildDavies Liu2015-11-301-1/+1
|
* [SPARK-11960][MLLIB][DOC] User guide for streaming testsFeynman Liang2015-11-303-0/+28
| | | | | | | | CC jkbradley mengxr josepablocam Author: Feynman Liang <feynman.liang@gmail.com> Closes #10005 from feynmanliang/streaming-test-user-guide.
* [SPARK-11975][ML] Remove duplicate mllib example (DT/RF/GBT in Java/Python)Yanbo Liang2015-11-306-692/+0
| | | | | | | | | | Remove duplicate mllib example (DT/RF/GBT in Java/Python). Since we have tutorial code for DT/RF/GBT classification/regression in Scala/Java/Python and example applications for DT/RF/GBT in Scala, so we mark these as duplicated and remove them. mengxr Author: Yanbo Liang <ybliang8@gmail.com> Closes #9954 from yanboliang/SPARK-11975.
* [SPARK-11689][ML] Add user guide and example code for LDA under spark.mlYuhao Yang2015-11-305-1/+208
| | | | | | | | | | | | | | jira: https://issues.apache.org/jira/browse/SPARK-11689 Add simple user guide for LDA under spark.ml and example code under examples/. Use include_example to include example code in the user guide markdown. Check SPARK-11606 for instructions. Original PR is reverted due to document build error. https://github.com/apache/spark/pull/9722 mengxr feynmanliang yinxusen Sorry for the troubling. Author: Yuhao Yang <hhbyyh@gmail.com> Closes #9974 from hhbyyh/ldaMLExample.
* [SPARK-12053][CORE] EventLoggingListener.getLogPath needs 4 parametersTeng Qiu2015-12-011-1/+1
| | | | | | | | | | | ```EventLoggingListener.getLogPath``` needs 4 input arguments: https://github.com/apache/spark/blob/v1.6.0-preview2/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala#L276-L280 the 3rd parameter should be appAttemptId, 4th parameter is codec... Author: Teng Qiu <teng.qiu@gmail.com> Closes #10044 from chutium/SPARK-12053.
* Revert "[SPARK-11206] Support SQL UI on the history server"Josh Rosen2015-11-3021-327/+135
| | | | | | | | This reverts commit cc243a079b1c039d6e7f0b410d1654d94a090e14 / PR #9297 I'm reverting this because it broke SQLListenerMemoryLeakSuite in the master Maven builds. See #9991 for a discussion of why this broke the tests.
* [MINOR][DOCS] fixed list display in ml-ensemblesBenFradet2015-11-301-0/+1
| | | | | | | | | | | | The list in ml-ensembles.md wasn't properly formatted and, as a result, was looking like this: ![old](http://i.imgur.com/2ZhELLR.png) This PR aims to make it look like this: ![new](http://i.imgur.com/0Xriwd2.png) Author: BenFradet <benjamin.fradet@gmail.com> Closes #10025 from BenFradet/ml-ensembles-doc.
* [SPARK-11982] [SQL] improve performance of cartesian productDavies Liu2015-11-304-9/+139
| | | | | | | | | | | | This PR improve the performance of CartesianProduct by caching the result of right plan. After this patch, the query time of TPC-DS Q65 go down to 4 seconds from 28 minutes (420X faster). cc nongli Author: Davies Liu <davies@databricks.com> Closes #9969 from davies/improve_cartesian.
* [SPARK-11700] [SQL] Remove thread local SQLContext in SparkPlanDavies Liu2015-11-306-21/+14
| | | | | | | | In 1.6, we introduce a public API to have a SQLContext for current thread, SparkPlan should use that. Author: Davies Liu <davies@databricks.com> Closes #9990 from davies/leak_context.
* [SPARK-11989][SQL] Only use commit in JDBC data source if the underlying ↵CK502015-11-301-3/+19
| | | | | | | | | | | | | | database supports transactions Fixes [SPARK-11989](https://issues.apache.org/jira/browse/SPARK-11989) Author: CK50 <christian.kurz@oracle.com> Author: Christian Kurz <christian.kurz@oracle.com> Closes #9973 from CK50/branch-1.6_non-transactional. (cherry picked from commit a589736a1b237ef2f3bd59fbaeefe143ddcc8f4e) Signed-off-by: Reynold Xin <rxin@databricks.com>
* [SPARK-12023][BUILD] Fix warnings while packaging spark with maven.Prashant Sharma2015-11-302-5/+5
| | | | | | | | this is a trivial fix, discussed [here](http://stackoverflow.com/questions/28500401/maven-assembly-plugin-warning-the-assembly-descriptor-contains-a-filesystem-roo/). Author: Prashant Sharma <scrapcodes@gmail.com> Closes #10014 from ScrapCodes/assembly-warning.
* [DOC] Explicitly state that top maintains the order of elementsWieland Hoffmann2015-11-302-3/+4
| | | | | | | | | Top is implemented in terms of takeOrdered, which already maintains the order, so top should, too. Author: Wieland Hoffmann <themineo@gmail.com> Closes #10013 from mineo/top-order.
* [MINOR][BUILD] Changed the comment to reflect the plugin project is there to ↵Prashant Sharma2015-11-301-3/+2
| | | | | | | | support SBT pom reader only. Author: Prashant Sharma <scrapcodes@gmail.com> Closes #10012 from ScrapCodes/minor-build-comment.
* [SPARK-11859][MESOS] SparkContext accepts invalid Master URLs in the form ↵toddwan2015-11-302-6/+15
| | | | | | | | | | | | | | | | | | | | | zk://host:port for a multi-master Mesos cluster using ZooKeeper * According to below doc and validation logic in [SparkSubmit.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala#L231), master URL for a mesos cluster should always start with `mesos://` http://spark.apache.org/docs/latest/running-on-mesos.html `The Master URLs for Mesos are in the form mesos://host:5050 for a single-master Mesos cluster, or mesos://zk://host:2181 for a multi-master Mesos cluster using ZooKeeper.` * However, [SparkContext.scala](https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/SparkContext.scala#L2749) fails the validation and can receive master URL in the form `zk://host:port` * For the master URLs in the form `zk:host:port`, the valid form should be `mesos://zk://host:port` * This PR restrict the validation in `SparkContext.scala`, and now only mesos master URLs prefixed with `mesos://` can be accepted. * This PR also updated corresponding unit test. Author: toddwan <tawan0109@outlook.com> Closes #9886 from toddwan/S11859.
* [SPARK-12039] [SQL] Ignore HiveSparkSubmitSuite's "SPARK-9757 Persist ↵Yin Huai2015-11-291-1/+1
| | | | | | | | | | | | Parquet relation with decimal column". https://issues.apache.org/jira/browse/SPARK-12039 Since it is pretty flaky in hadoop 1 tests, we can disable it while we are investigating the cause. Author: Yin Huai <yhuai@databricks.com> Closes #10035 from yhuai/SPARK-12039-ignore.
* [SPARK-12024][SQL] More efficient multi-column counting.Herman van Hovell2015-11-296-86/+33
| | | | | | | | | | | | In https://github.com/apache/spark/pull/9409 we enabled multi-column counting. The approach taken in that PR introduces a bit of overhead by first creating a row only to check if all of the columns are non-null. This PR fixes that technical debt. Count now takes multiple columns as its input. In order to make this work I have also added support for multiple columns in the single distinct code path. cc yhuai Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #10015 from hvanhovell/SPARK-12024.
* [SPARK-11781][SPARKR] SparkR has problem in inferring type of raw type.Sun Rui2015-11-294-32/+47
| | | | | | Author: Sun Rui <rui.sun@intel.com> Closes #9769 from sun-rui/SPARK-11781.
* [SPARK-9319][SPARKR] Add support for setting column names, typesfelixcheung2015-11-285-55/+185
| | | | | | | | | | | | | Add support for for colnames, colnames<-, coltypes<- Also added tests for names, names<- which have no test previously. I merged with PR 8984 (coltypes). Clicked the wrong thing, crewed up the PR. Recreated it here. Was #9218 shivaram sun-rui Author: felixcheung <felixcheung_m@hotmail.com> Closes #9654 from felixcheung/colnamescoltypes.
* [SPARK-12029][SPARKR] Improve column functions signature, param check, ↵felixcheung2015-11-282-34/+96
| | | | | | | | | | tests, fix doc and add examples shivaram sun-rui Author: felixcheung <felixcheung_m@hotmail.com> Closes #10019 from felixcheung/rfunctionsdoc.
* [SPARK-12028] [SQL] get_json_object returns an incorrect result when the ↵gatorsmile2015-11-272-2/+25
| | | | | | | | | | | | | | | | | | | | | | | value is null literals When calling `get_json_object` for the following two cases, both results are `"null"`: ```scala val tuple: Seq[(String, String)] = ("5", """{"f1": null}""") :: Nil val df: DataFrame = tuple.toDF("key", "jstring") val res = df.select(functions.get_json_object($"jstring", "$.f1")).collect() ``` ```scala val tuple2: Seq[(String, String)] = ("5", """{"f1": "null"}""") :: Nil val df2: DataFrame = tuple2.toDF("key", "jstring") val res3 = df2.select(functions.get_json_object($"jstring", "$.f1")).collect() ``` Fixed the problem and also added a test case. Author: gatorsmile <gatorsmile@gmail.com> Closes #10018 from gatorsmile/get_json_object.
* [SPARK-12020][TESTS][TEST-HADOOP2.0] PR builder cannot trigger hadoop 2.0 testYin Huai2015-11-271-1/+1
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-12020 Author: Yin Huai <yhuai@databricks.com> Closes #10010 from yhuai/SPARK-12020.
* [SPARK-12021][STREAMING][TESTS] Fix the potential dead-lock in ↵Shixiong Zhu2015-11-271-6/+19
| | | | | | | | | | StreamingListenerSuite In StreamingListenerSuite."don't call ssc.stop in listener", after the main thread calls `ssc.stop()`, `StreamingContextStoppingCollector` may call `ssc.stop()` in the listener bus thread, which is a dead-lock. This PR updated `StreamingContextStoppingCollector` to only call `ssc.stop()` in the first batch to avoid the dead-lock. Author: Shixiong Zhu <shixiong@databricks.com> Closes #10011 from zsxwing/fix-test-deadlock.
* [SPARK-12025][SPARKR] Rename some window rank function names for SparkRYanbo Liang2015-11-274-41/+41
| | | | | | | | | | | | | | Change ```cumeDist -> cume_dist, denseRank -> dense_rank, percentRank -> percent_rank, rowNumber -> row_number``` at SparkR side. There are two reasons that we should make this change: * We should follow the [naming convention rule of R](http://www.inside-r.org/node/230645) * Spark DataFrame has deprecated the old convention (such as ```cumeDist```) and will remove it in Spark 2.0. It's better to fix this issue before 1.6 release, otherwise we will make breaking API change. cc shivaram sun-rui Author: Yanbo Liang <ybliang8@gmail.com> Closes #10016 from yanboliang/SPARK-12025.
* [SPARK-11997] [SQL] NPE when save a DataFrame as parquet and partitioned by ↵Dilip Biswal2015-11-262-1/+14
| | | | | | | | | | long column Check for partition column null-ability while building the partition spec. Author: Dilip Biswal <dbiswal@us.ibm.com> Closes #10001 from dilipbiswal/spark-11997.
* Fix style violation for b63938a8b04Reynold Xin2015-11-261-1/+3
|
* [SPARK-11991] fixesJeremy Derr2015-11-261-0/+4
| | | | | | | | | | If `--private-ips` is required but not provided, spark_ec2.py may behave inappropriately, including attempting to ssh to localhost in attempts to verify ssh connectivity to the cluster. This fixes that behavior by raising a `UsageError` exception if `get_dns_name` is unable to determine a hostname as a result. Author: Jeremy Derr <jcderr@radius.com> Closes #9975 from jcderr/SPARK-11991/ec_spark.py_hostname_check.
* [SPARK-11778][SQL] add regression testHuaxin Gao2015-11-262-10/+32
| | | | | | | | | | | Fix regression test for SPARK-11778. marmbrus Could you please take a look? Thank you very much!! Author: Huaxin Gao <huaxing@oc0558782468.ibm.com> Closes #9890 from huaxingao/spark-11778-regression-test.
* [SPARK-11917][PYSPARK] Add SQLContext#dropTempTable to PySparkJeff Zhang2015-11-261-0/+9
| | | | | | Author: Jeff Zhang <zjffdu@apache.org> Closes #9903 from zjffdu/SPARK-11917.
* [SPARK-11881][SQL] Fix for postgresql fetchsize > 0mariusvniekerk2015-11-263-1/+39
| | | | | | | | | | | Reference: https://jdbc.postgresql.org/documentation/head/query.html#query-with-cursor In order for PostgreSQL to honor the fetchSize non-zero setting, its Connection.autoCommit needs to be set to false. Otherwise, it will just quietly ignore the fetchSize setting. This adds a new side-effecting dialect specific beforeFetch method that will fire before a select query is ran. Author: mariusvniekerk <marius.v.niekerk@gmail.com> Closes #9861 from mariusvniekerk/SPARK-11881.
* [SPARK-12011][SQL] Stddev/Variance etc should support columnName as argumentsYanbo Liang2015-11-262-0/+89
| | | | | | | | | | | | | | | | | | | | | Spark SQL aggregate function: ```Java stddev stddev_pop stddev_samp variance var_pop var_samp skewness kurtosis collect_list collect_set ``` should support ```columnName``` as arguments like other aggregate function(max/min/count/sum). Author: Yanbo Liang <ybliang8@gmail.com> Closes #9994 from yanboliang/SPARK-12011.
* [SPARK-11996][CORE] Make the executor thread dump work againShixiong Zhu2015-11-268-67/+29
| | | | | | | | | | In the previous implementation, the driver needs to know the executor listening address to send the thread dump request. However, in Netty RPC, the executor doesn't listen to any port, so the executor thread dump feature is broken. This patch makes the driver use the endpointRef stored in BlockManagerMasterEndpoint to send the thread dump request to fix it. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9976 from zsxwing/executor-thread-dump.
* doc typo: "classificaion" -> "classification"muxator2015-11-261-1/+1
| | | | | | Author: muxator <muxator@users.noreply.github.com> Closes #10008 from muxator/patch-1.
* [SPARK-11973][SQL] Improve optimizer code readability.Reynold Xin2015-11-262-26/+26
| | | | | | | | | | This is a followup for https://github.com/apache/spark/pull/9959. I added more documentation and rewrote some monadic code into simpler ifs. Author: Reynold Xin <rxin@databricks.com> Closes #9995 from rxin/SPARK-11973.
* [SPARK-11998][SQL][TEST-HADOOP2.0] When downloading Hadoop artifacts from ↵Yin Huai2015-11-263-17/+72
| | | | | | | | | | | | | | | | | | maven, we need to try to download the version that is used by Spark If we need to download Hive/Hadoop artifacts, try to download a Hadoop that matches the Hadoop used by Spark. If the Hadoop artifact cannot be resolved (e.g. Hadoop version is a vendor specific version like 2.0.0-cdh4.1.1), we will use Hadoop 2.4.0 (we used to hard code this version as the hadoop that we will download from maven) and we will not share Hadoop classes. I tested this match in my laptop with the following confs (these confs are used by our builds). All tests are good. ``` build/sbt -Phadoop-1 -Dhadoop.version=1.2.1 -Pkinesis-asl -Phive-thriftserver -Phive build/sbt -Phadoop-1 -Dhadoop.version=2.0.0-mr1-cdh4.1.1 -Pkinesis-asl -Phive-thriftserver -Phive build/sbt -Pyarn -Phadoop-2.2 -Pkinesis-asl -Phive-thriftserver -Phive build/sbt -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Pkinesis-asl -Phive-thriftserver -Phive ``` Author: Yin Huai <yhuai@databricks.com> Closes #9979 from yhuai/versionsSuite.
* [SPARK-11863][SQL] Unable to resolve order by if it contains mixture of ↵Dilip Biswal2015-11-262-3/+28
| | | | | | | | | | | | | | | | aliases and real columns this is based on https://github.com/apache/spark/pull/9844, with some bug fix and clean up. The problems is that, normal operator should be resolved based on its child, but `Sort` operator can also be resolved based on its grandchild. So we have 3 rules that can resolve `Sort`: `ResolveReferences`, `ResolveSortReferences`(if grandchild is `Project`) and `ResolveAggregateFunctions`(if grandchild is `Aggregate`). For example, `select c1 as a , c2 as b from tab group by c1, c2 order by a, c2`, we need to resolve `a` and `c2` for `Sort`. Firstly `a` will be resolved in `ResolveReferences` based on its child, and when we reach `ResolveAggregateFunctions`, we will try to resolve both `a` and `c2` based on its grandchild, but failed because `a` is not a legal aggregate expression. whoever merge this PR, please give the credit to dilipbiswal Author: Dilip Biswal <dbiswal@us.ibm.com> Author: Wenchen Fan <wenchen@databricks.com> Closes #9961 from cloud-fan/sort.
* [SPARK-12005][SQL] Work around VerifyError in HyperLogLogPlusPlus.Marcelo Vanzin2015-11-261-5/+8
| | | | | | | | Just move the code around a bit; that seems to make the JVM happy. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9985 from vanzin/SPARK-12005.
* [SPARK-11973] [SQL] push filter through aggregation with alias and literalsDavies Liu2015-11-263-11/+79
| | | | | | | | | | | | Currently, filter can't be pushed through aggregation with alias or literals, this patch fix that. After this patch, the time of TPC-DS query 4 go down to 13 seconds from 141 seconds (10x improvements). cc nongli yhuai Author: Davies Liu <davies@databricks.com> Closes #9959 from davies/push_filter2.
* [SPARK-11999][CORE] Fix the issue that ThreadUtils.newDaemonCachedThreadPool ↵Shixiong Zhu2015-11-252-3/+56
| | | | | | | | | | doesn't cache any task In the previous codes, `newDaemonCachedThreadPool` uses `SynchronousQueue`, which is wrong. `SynchronousQueue` is an empty queue that cannot cache any task. This patch uses `LinkedBlockingQueue` to fix it along with other fixes to make sure `newDaemonCachedThreadPool` can use at most `maxThreadNumber` threads, and after that, cache tasks to `LinkedBlockingQueue`. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9978 from zsxwing/cached-threadpool.
* [SPARK-11980][SPARK-10621][SQL] Fix json_tuple and add test cases forgatorsmile2015-11-251-10/+34
| | | | | | | | | | | | | | Added Python test cases for the function `isnan`, `isnull`, `nanvl` and `json_tuple`. Fixed a bug in the function `json_tuple` rxin , could you help me review my changes? Please let me know anything is missing. Thank you! Have a good Thanksgiving day! Author: gatorsmile <gatorsmile@gmail.com> Closes #9977 from gatorsmile/json_tuple.
* [SPARK-12003] [SQL] remove the prefix for name after expanded starDavies Liu2015-11-251-1/+1
| | | | | | | | Right now, the expended start will include the name of expression as prefix for column, that's not better than without expending, we should not have the prefix. Author: Davies Liu <davies@databricks.com> Closes #9984 from davies/expand_star.
* [SPARK-11206] Support SQL UI on the history serverCarson Wang2015-11-2521-135/+327
| | | | | | | | | | | | | | On the live web UI, there is a SQL tab which provides valuable information for the SQL query. But once the workload is finished, we won't see the SQL tab on the history server. It will be helpful if we support SQL UI on the history server so we can analyze it even after its execution. To support SQL UI on the history server: 1. I added an `onOtherEvent` method to the `SparkListener` trait and post all SQL related events to the same event bus. 2. Two SQL events `SparkListenerSQLExecutionStart` and `SparkListenerSQLExecutionEnd` are defined in the sql module. 3. The new SQL events are written to event log using Jackson. 4. A new trait `SparkHistoryListenerFactory` is added to allow the history server to feed events to the SQL history listener. The SQL implementation is loaded at runtime using `java.util.ServiceLoader`. Author: Carson Wang <carson.wang@intel.com> Closes #9297 from carsonwang/SqlHistoryUI.
* [SPARK-11983][SQL] remove all unused codegen fallback traitDaoyuan Wang2015-11-253-6/+4
| | | | | | Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #9966 from adrian-wang/removeFallback.
* Fix Aggregator documentation (rename present to finish).Reynold Xin2015-11-251-1/+1
|
* [SPARK-11866][NETWORK][CORE] Make sure timed out RPCs are cleaned up.Marcelo Vanzin2015-11-2518-192/+374
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This change does a couple of different things to make sure that the RpcEnv-level code and the network library agree about the status of outstanding RPCs. For RPCs that do not expect a reply ("RpcEnv.send"), support for one way messages (hello CORBA!) was added to the network layer. This is a "fire and forget" message that does not require any state to be kept by the TransportClient; as a result, the RpcEnv 'Ack' message is not needed anymore. For RPCs that do expect a reply ("RpcEnv.ask"), the network library now returns the internal RPC id; if the RpcEnv layer decides to time out the RPC before the network layer does, it now asks the TransportClient to forget about the RPC, so that if the network-level timeout occurs, the client is not killed. As part of implementing the above, I cleaned up some of the code in the netty rpc backend, removing types that were not necessary and factoring out some common code. Of interest is a slight change in the exceptions when posting messages to a stopped RpcEnv; that's mostly to avoid nasty error messages from the local-cluster backend when shutting down, which pollutes the terminal output. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9917 from vanzin/SPARK-11866.
* [SPARK-11935][PYSPARK] Send the Python exceptions in TransformFunction and ↵Shixiong Zhu2015-11-253-19/+144
| | | | | | | | | | | | TransformFunctionSerializer to Java The Python exception track in TransformFunction and TransformFunctionSerializer is not sent back to Java. Py4j just throws a very general exception, which is hard to debug. This PRs adds `getFailure` method to get the failure message in Java side. Author: Shixiong Zhu <shixiong@databricks.com> Closes #9922 from zsxwing/SPARK-11935.