aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [MINOR][ML] Use coefficients replace weightsYanbo Liang2015-12-032-2/+2
| | | | | | | | | Use ```coefficients``` replace ```weights```, I wish they are the last two. mengxr Author: Yanbo Liang <ybliang8@gmail.com> Closes #10065 from yanboliang/coefficients.
* [SPARK-12108] Make event logs smallerAndrew Or2015-12-034-40/+20
| | | | | | | | | | | | **Problem.** Event logs in 1.6 were much bigger than 1.5. I ran page rank and the event log size in 1.6 was almost 5x that in 1.5. I did a bisect to find that the RDD callsite added in #9398 is largely responsible for this. **Solution.** This patch removes the long form of the callsite (which is not used!) from the event log. This reduces the size of the event log significantly. *Note on compatibility*: if this patch is to be merged into 1.6.0, then it won't break any compatibility. Otherwise, if it is merged into 1.6.1, then we might need to add more backward compatibility handling logic (currently does not exist yet). Author: Andrew Or <andrew@databricks.com> Closes #10115 from andrewor14/smaller-event-logs.
* [SPARK-12101][CORE] Fix thread pools that cannot cache tasks in Worker and ↵Shixiong Zhu2015-12-033-22/+12
| | | | | | | | | | AppClient `SynchronousQueue` cannot cache any task. This issue is similar to #9978. It's an easy fix. Just use the fixed `ThreadUtils.newDaemonCachedThreadPool`. Author: Shixiong Zhu <shixiong@databricks.com> Closes #10108 from zsxwing/fix-threadpool.
* [SPARK-12059][CORE] Avoid assertion error when unexpected state transition ↵jerryshao2015-12-031-2/+3
| | | | | | | | | | | | met in Master Downgrade to warning log for unexpected state transition. andrewor14 please review, thanks a lot. Author: jerryshao <sshao@hortonworks.com> Closes #10091 from jerryshao/SPARK-12059.
* [SPARK-11314][YARN] add service API and test service for Yarn Cluster schedulersSteve Loughran2015-12-039-38/+431
| | | | | | | | | | | | | | | | This is purely the yarn/src/main and yarn/src/test bits of the YARN ATS integration: the extension model to load and run implementations of `SchedulerExtensionService` in the yarn cluster scheduler process —and to stop them afterwards. There's duplication between the two schedulers, yarn-client and yarn-cluster, at least in terms of setting everything up, because the common superclass, `YarnSchedulerBackend` is in spark-core, and the extension services need the YARN app/attempt IDs. If you look at how the the extension services are loaded, the case class `SchedulerExtensionServiceBinding` is used to pass in config info -currently just the spark context and the yarn IDs, of which one, the attemptID, will be null when running client-side. I'm passing in a case class to ensure that it would be possible in future to add extra arguments to the binding class, yet, as the method signature will not have changed, still be able to load existing services. There's no functional extension service here, just one for testing. The real tests come in the bigger pull requests. At the same time, there's no restriction of this extension service purely to the ATS history publisher. Anything else that wants to listen to the spark context and publish events could use this, and I'd also consider writing one for the YARN-913 registry service, so that the URLs of the web UI would be locatable through that (low priority; would make more sense if integrated with a REST client). There's no minicluster test. Given the test execution overhead of setting up minicluster tests, it'd probably be better to add an extension service into one of the existing tests. Author: Steve Loughran <stevel@hortonworks.com> Closes #9182 from steveloughran/stevel/feature/SPARK-1537-service.
* [SPARK-12116][SPARKR][DOCS] document how to workaround function name ↵felixcheung2015-12-031-1/+2
| | | | | | | | | | conflicts with dplyr shivaram Author: felixcheung <felixcheung_m@hotmail.com> Closes #10119 from felixcheung/rdocdplyrmasked.
* [DOCUMENTATION][KAFKA] fix typo in kafka/OffsetRange.scalamicrowishing2015-12-032-2/+2
| | | | | | | | this is to fix some typo in external/kafka/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala Author: microwishing <wei.zhu@kaiyuandao.com> Closes #10121 from microwishing/master.
* [DOCUMENTATION][MLLIB] typo in mllib docJeff Zhang2015-12-031-1/+1
| | | | | | | | \cc mengxr Author: Jeff Zhang <zjffdu@apache.org> Closes #10093 from zjffdu/mllib_typo.
* [SPARK-12088][SQL] check connection.isClosed before calling connection…Huaxin Gao2015-12-031-1/+1
| | | | | | | | | | | | | In Java Spec java.sql.Connection, it has boolean getAutoCommit() throws SQLException Throws: SQLException - if a database access error occurs or this method is called on a closed connection So if conn.getAutoCommit is called on a closed connection, a SQLException will be thrown. Even though the code catch the SQLException and program can continue, I think we should check conn.isClosed before calling conn.getAutoCommit to avoid the unnecessary SQLException. Author: Huaxin Gao <huaxing@oc0558782468.ibm.com> Closes #10095 from huaxingao/spark-12088.
* [SPARK-12109][SQL] Expressions's simpleString should delegate to its toString.Yin Huai2015-12-033-5/+3
| | | | | | | | | | | | | | | | https://issues.apache.org/jira/browse/SPARK-12109 The change of https://issues.apache.org/jira/browse/SPARK-11596 exposed the problem. In the sql plan viz, the filter shows ![image](https://cloud.githubusercontent.com/assets/2072857/11547075/1a285230-9906-11e5-8481-2bb451e35ef1.png) After changes in this PR, the viz is back to normal. ![image](https://cloud.githubusercontent.com/assets/2072857/11547080/2bc570f4-9906-11e5-8897-3b3bff173276.png) Author: Yin Huai <yhuai@databricks.com> Closes #10111 from yhuai/SPARK-12109.
* [SPARK-12082][FLAKY-TEST] Increase timeouts in NettyBlockTransferSecuritySuiteJosh Rosen2015-12-031-1/+1
| | | | | | | | We should try increasing a timeout in NettyBlockTransferSecuritySuite in order to reduce that suite's flakiness in Jenkins. Author: Josh Rosen <joshrosen@databricks.com> Closes #10113 from JoshRosen/SPARK-12082.
* [SPARK-12000] do not specify arg types when reference a method in ScalaDocXiangrui Meng2015-12-022-3/+3
| | | | | | | | | | This fixes SPARK-12000, verified on my local with JDK 7. It seems that `scaladoc` try to match method names and messed up with annotations. cc: JoshRosen jkbradley Author: Xiangrui Meng <meng@databricks.com> Closes #10114 from mengxr/SPARK-12000.2.
* [SPARK-12093][SQL] Fix the error of comment in DDLParserYadong Qi2015-12-031-3/+3
| | | | | | Author: Yadong Qi <qiyadong2010@gmail.com> Closes #10096 from watermen/patch-1.
* [SPARK-10266][DOCUMENTATION, ML] Fixed @Since annotation for ml.tunningYu ISHIKAWA2015-12-023-16/+58
| | | | | | | | | | | | cc mengxr noel-smith I worked on this issues based on https://github.com/apache/spark/pull/8729. ehsanmok thank you for your contricution! Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Author: Ehsan M.Kermani <ehsanmo1367@gmail.com> Closes #9338 from yu-iskw/JIRA-10266.
* [SPARK-12001] Allow partially-stopped StreamingContext to be completely stoppedJosh Rosen2015-12-021-22/+27
| | | | | | | | | | | | If `StreamingContext.stop()` is interrupted midway through the call, the context will be marked as stopped but certain state will have not been cleaned up. Because `state = STOPPED` will be set, subsequent `stop()` calls will be unable to finish stopping the context, preventing any new StreamingContexts from being created. This patch addresses this issue by only marking the context as `STOPPED` once the `stop()` has successfully completed which allows `stop()` to be called a second time in order to finish stopping the context in case the original `stop()` call was interrupted. I discovered this issue by examining logs from a failed Jenkins run in which this race condition occurred in `FailureSuite`, leaking an unstoppable context and causing all subsequent tests to fail. Author: Josh Rosen <joshrosen@databricks.com> Closes #9982 from JoshRosen/SPARK-12001.
* [SPARK-12094][SQL] Prettier tree string for TreeNodeCheng Lian2015-12-021-5/+26
| | | | | | | | When examining plans of complex queries with multiple joins, a pain point of mine is that, it's hard to immediately see the sibling node of a specific query plan node. This PR adds tree lines for the tree string of a `TreeNode`, so that the result can be visually more intuitive. Author: Cheng Lian <lian@databricks.com> Closes #10099 from liancheng/prettier-tree-string.
* [SPARK-3580][CORE] Add Consistent Method To Get Number of RDD Partitions ↵Jeroen Schot2015-12-025-1/+30
| | | | | | | | | | | | Across Different Languages I have tried to address all the comments in pull request https://github.com/apache/spark/pull/2447. Note that the second commit (using the new method in all internal code of all components) is quite intrusive and could be omitted. Author: Jeroen Schot <jeroen.schot@surfsara.nl> Closes #9767 from schot/master.
* [SPARK-12090] [PYSPARK] consider shuffle in coalesce()Davies Liu2015-12-011-1/+1
| | | | | | Author: Davies Liu <davies@databricks.com> Closes #10090 from davies/fix_coalesce.
* [SPARK-11949][SQL] Check bitmasks to set nullable propertyLiang-Chi Hsieh2015-12-011-4/+9
| | | | | | | | | | | | Following up #10038. We can use bitmasks to determine which grouping expressions need to be set as nullable. cc yhuai Author: Liang-Chi Hsieh <viirya@appier.com> Closes #10067 from viirya/fix-cube-following.
* [SPARK-12087][STREAMING] Create new JobConf for every batch in saveAsHadoopFilesTathagata Das2015-12-011-1/+2
| | | | | | | | | | | | | | | The JobConf object created in `DStream.saveAsHadoopFiles` is used concurrently in multiple places: * The JobConf is updated by `RDD.saveAsHadoopFile()` before the job is launched * The JobConf is serialized as part of the DStream checkpoints. These concurrent accesses (updating in one thread, while the another thread is serializing it) can lead to concurrentModidicationException in the underlying Java hashmap using in the internal Hadoop Configuration object. The solution is to create a new JobConf in every batch, that is updated by `RDD.saveAsHadoopFile()`, while the checkpointing serializes the original JobConf. Tests to be added in #9988 will fail reliably without this patch. Keeping this patch really small to make sure that it can be added to previous branches. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #10088 from tdas/SPARK-12087.
* [SPARK-12077][SQL] change the default plan for single distinctDavies Liu2015-12-012-3/+3
| | | | | | | | | | Use try to match the behavior for single distinct aggregation with Spark 1.5, but that's not scalable, we should be robust by default, have a flag to address performance regression for low cardinality aggregation. cc yhuai nongli Author: Davies Liu <davies@databricks.com> Closes #10075 from davies/agg_15.
* [SPARK-12081] Make unified memory manager work with small heapsAndrew Or2015-12-014-7/+41
| | | | | | | | | | The existing `spark.memory.fraction` (default 0.75) gives the system 25% of the space to work with. For small heaps, this is not enough: e.g. default 1GB leaves only 250MB system memory. This is especially a problem in local mode, where the driver and executor are crammed in the same JVM. Members of the community have reported driver OOM's in such cases. **New proposal.** We now reserve 300MB before taking the 75%. For 1GB JVMs, this leaves `(1024 - 300) * 0.75 = 543MB` for execution and storage. This is proposal (1) listed in the [JIRA](https://issues.apache.org/jira/browse/SPARK-12081). Author: Andrew Or <andrew@databricks.com> Closes #10081 from andrewor14/unified-memory-small-heaps.
* [SPARK-8414] Ensure context cleaner periodic cleanupsAndrew Or2015-12-011-1/+20
| | | | | | | | Garbage collection triggers cleanups. If the driver JVM is huge and there is little memory pressure, we may never clean up shuffle files on executors. This is a problem for long-running applications (e.g. streaming). Author: Andrew Or <andrew@databricks.com> Closes #10070 from andrewor14/periodic-gc.
* [SPARK-11596][SQL] In TreeNode's argString, if a TreeNode is not a child of ↵Yin Huai2015-12-011-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | the current TreeNode, we should only return the simpleString. In TreeNode's argString, if a TreeNode is not a child of the current TreeNode, we will only return the simpleString. I tested the [following case provided by Cristian](https://issues.apache.org/jira/browse/SPARK-11596?focusedCommentId=15019241&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15019241). ``` val c = (1 to 20).foldLeft[Option[DataFrame]] (None) { (curr, idx) => println(s"PROCESSING >>>>>>>>>>> $idx") val df = sqlContext.sparkContext.parallelize((0 to 10).zipWithIndex).toDF("A", "B") val union = curr.map(_.unionAll(df)).getOrElse(df) union.cache() Some(union) } c.get.explain(true) ``` Without the change, `c.get.explain(true)` took 100s. With the change, `c.get.explain(true)` took 26ms. https://issues.apache.org/jira/browse/SPARK-11596 Author: Yin Huai <yhuai@databricks.com> Closes #10079 from yhuai/SPARK-11596.
* [SPARK-11352][SQL] Escape */ in the generated comments.Yin Huai2015-12-013-3/+18
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-11352 Author: Yin Huai <yhuai@databricks.com> Closes #10072 from yhuai/SPARK-11352.
* [SPARK-11788][SQL] surround timestamp/date value with quotes in JDBC data sourceHuaxin Gao2015-12-012-1/+14
| | | | | | | | | | | When query the Timestamp or Date column like the following val filtered = jdbcdf.where($"TIMESTAMP_COLUMN" >= beg && $"TIMESTAMP_COLUMN" < end) The generated SQL query is "TIMESTAMP_COLUMN >= 2015-01-01 00:00:00.0" It should have quote around the Timestamp/Date value such as "TIMESTAMP_COLUMN >= '2015-01-01 00:00:00.0'" Author: Huaxin Gao <huaxing@oc0558782468.ibm.com> Closes #9872 from huaxingao/spark-11788.
* [SPARK-11328][SQL] Improve error message when hitting this issueNong Li2015-12-012-3/+22
| | | | | | | | | | | | | | | | The issue is that the output commiter is not idempotent and retry attempts will fail because the output file already exists. It is not safe to clean up the file as this output committer is by design not retryable. Currently, the job fails with a confusing file exists error. This patch is a stop gap to tell the user to look at the top of the error log for the proper message. This is difficult to test locally as Spark is hardcoded not to retry. Manually verified by upping the retry attempts. Author: Nong Li <nong@databricks.com> Author: Nong Li <nongli@gmail.com> Closes #10080 from nongli/spark-11328.
* [SPARK-12075][SQL] Speed up HiveComparisionTest by avoiding / speeding up ↵Josh Rosen2015-12-023-14/+62
| | | | | | | | | | | | | | | | | | | TestHive.reset() When profiling HiveCompatibilitySuite, I noticed that most of the time seems to be spent in expensive `TestHive.reset()` calls. This patch speeds up suites based on HiveComparisionTest, such as HiveCompatibilitySuite, with the following changes: - Avoid `TestHive.reset()` whenever possible: - Use a simple set of heuristics to guess whether we need to call `reset()` in between tests. - As a safety-net, automatically re-run failed tests by calling `reset()` before the re-attempt. - Speed up the expensive parts of `TestHive.reset()`: loading the `src` and `srcpart` tables took roughly 600ms per test, so we now avoid this by using a simple heuristic which only loads those tables by tests that reference them. This is based on simple string matching over the test queries which errs on the side of loading in more situations than might be strictly necessary. After these changes, HiveCompatibilitySuite seems to run in about 10 minutes. This PR is a revival of #6663, an earlier experimental PR from June, where I played around with several possible speedups for this suite. Author: Josh Rosen <joshrosen@databricks.com> Closes #10055 from JoshRosen/speculative-testhive-reset.
* [SPARK-12002][STREAMING][PYSPARK] Fix python direct stream checkpoint ↵jerryshao2015-12-012-6/+56
| | | | | | | | | | | | | recovery issue Fixed a minor race condition in #10017 Closes #10017 Author: jerryshao <sshao@hortonworks.com> Author: Shixiong Zhu <shixiong@databricks.com> Closes #10074 from zsxwing/review-pr10017.
* [SPARK-11961][DOC] Add docs of ChiSqSelectorXusen Yin2015-12-013-0/+178
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-11961 Author: Xusen Yin <yinxusen@gmail.com> Closes #9965 from yinxusen/SPARK-11961.
* Revert "[SPARK-12060][CORE] Avoid memory copy in ↵Shixiong Zhu2015-12-012-34/+4
| | | | | | JavaSerializerInstance.serialize" This reverts commit 1401166576c7018c5f9c31e0a6703d5fb16ea339.
* [SPARK-12004] Preserve the RDD partitioner through RDD checkpointingTathagata Das2015-12-013-31/+173
| | | | | | | | The solution is the save the RDD partitioner in a separate file in the RDD checkpoint directory. That is, `<checkpoint dir>/_partitioner`. In most cases, whether the RDD partitioner was recovered or not, does not affect the correctness, only reduces performance. So this solution makes a best-effort attempt to save and recover the partitioner. If either fails, the checkpointing is not affected. This makes this patch safe and backward compatible. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #9983 from tdas/SPARK-12004.
* [SPARK-12030] Fix Platform.copyMemory to handle overlapping regions.Nong Li2015-12-012-6/+82
| | | | | | | | | | This bug was exposed as memory corruption in Timsort which uses copyMemory to copy large regions that can overlap. The prior implementation did not handle this case half the time and always copied forward, resulting in the data being corrupt. Author: Nong Li <nong@databricks.com> Closes #10068 from nongli/spark-12030.
* [SPARK-12065] Upgrade Tachyon from 0.8.1 to 0.8.2Josh Rosen2015-12-012-2/+2
| | | | | | | | This commit upgrades the Tachyon dependency from 0.8.1 to 0.8.2. Author: Josh Rosen <joshrosen@databricks.com> Closes #10054 from JoshRosen/upgrade-to-tachyon-0.8.2.
* [SPARK-11821] Propagate Kerberos keytab for all environmentswoj-i2015-12-013-5/+10
| | | | | | | | | andrewor14 the same PR as in branch 1.5 harishreedharan Author: woj-i <wojciechindyk@gmail.com> Closes #9859 from woj-i/master.
* [SPARK-11905][SQL] Support Persist/Cache and Unpersist in Dataset APIsgatorsmile2015-12-016-18/+162
| | | | | | | | | | | | | | Persist and Unpersist exist in both RDD and Dataframe APIs. I think they are still very critical in Dataset APIs. Not sure if my understanding is correct? If so, could you help me check if the implementation is acceptable? Please provide your opinions. marmbrus rxin cloud-fan Thank you very much! Author: gatorsmile <gatorsmile@gmail.com> Author: xiaoli <lixiao1983@gmail.com> Author: Xiao Li <xiaoli@Xiaos-MacBook-Pro.local> Closes #9889 from gatorsmile/persistDS.
* [SPARK-11954][SQL] Encoder for JavaBeansWenchen Fan2015-12-019-20/+608
| | | | | | | | | | | create java version of `constructorFor` and `extractorFor` in `JavaTypeInference` Author: Wenchen Fan <wenchen@databricks.com> This patch had conflicts when merged, resolved by Committer: Michael Armbrust <michael@databricks.com> Closes #9937 from cloud-fan/pojo.
* [SPARK-11856][SQL] add type cast if the real type is different but ↵Wenchen Fan2015-12-0110-32/+335
| | | | | | | | | | | compatible with encoder schema When we build the `fromRowExpression` for an encoder, we set up a lot of "unresolved" stuff and lost the required data type, which may lead to runtime error if the real type doesn't match the encoder's schema. For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type is `[a: int, b: long]`, then we will hit runtime error and say that we can't construct class `Data` with int and long, because we lost the information that `b` should be a string. Author: Wenchen Fan <wenchen@databricks.com> Closes #9840 from cloud-fan/err-msg.
* [SPARK-12068][SQL] use a single column in Dataset.groupBy and count will failWenchen Fan2015-12-014-7/+27
| | | | | | | | The reason is that, for a single culumn `RowEncoder`(or a single field product encoder), when we use it as the encoder for grouping key, we should also combine the grouping attributes, although there is only one grouping attribute. Author: Wenchen Fan <wenchen@databricks.com> Closes #10059 from cloud-fan/bug.
* [SPARK-12046][DOC] Fixes various ScalaDoc/JavaDoc issuesCheng Lian2015-12-0125-133/+152
| | | | | | | | This PR backports PR #10039 to master Author: Cheng Lian <lian@databricks.com> Closes #10063 from liancheng/spark-12046.doc-fix.master.
* [SPARK-12060][CORE] Avoid memory copy in JavaSerializerInstance.serializeShixiong Zhu2015-12-012-4/+34
| | | | | | | | | | `JavaSerializerInstance.serialize` uses `ByteArrayOutputStream.toByteArray` to get the serialized data. `ByteArrayOutputStream.toByteArray` needs to copy the content in the internal array to a new array. However, since the array will be converted to `ByteBuffer` at once, we can avoid the memory copy. This PR added `ByteBufferOutputStream` to access the protected `buf` and convert it to a `ByteBuffer` directly. Author: Shixiong Zhu <shixiong@databricks.com> Closes #10051 from zsxwing/SPARK-12060.
* [SPARK-11949][SQL] Set field nullable property for GroupingSets to get ↵Liang-Chi Hsieh2015-12-012-2/+18
| | | | | | | | | | | | correct results for null values JIRA: https://issues.apache.org/jira/browse/SPARK-11949 The result of cube plan uses incorrect schema. The schema of cube result should set nullable property to true because the grouping expressions will have null values. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #10038 from viirya/fix-cube.
* [SPARK-11898][MLLIB] Use broadcast for the global tables in Word2VecYuhao Yang2015-12-011-1/+6
| | | | | | | | | | | | | | | | jira: https://issues.apache.org/jira/browse/SPARK-11898 syn0Global and sync1Global in word2vec are quite large objects with size (vocab * vectorSize * 8), yet they are passed to worker using basic task serialization. Use broadcast can greatly improve the performance. My benchmark shows that, for 1M vocabulary and default vectorSize 100, changing to broadcast can help, 1. decrease the worker memory consumption by 45%. 2. decrease running time by 40%. This will also help extend the upper limit for Word2Vec. Author: Yuhao Yang <hhbyyh@gmail.com> Closes #9878 from hhbyyh/w2vBC.
* [SPARK-12018][SQL] Refactor common subexpression elimination codeLiang-Chi Hsieh2015-11-303-34/+14
| | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-12018 The code of common subexpression elimination can be factored and simplified. Some unnecessary variables can be removed. Author: Liang-Chi Hsieh <viirya@appier.com> Closes #10009 from viirya/refactor-subexpr-eliminate.
* [HOTFIX][SPARK-12000] Add missing quotes in Jekyll API docs plugin.Josh Rosen2015-11-301-1/+1
| | | | I accidentally omitted these as part of #10049.
* [SPARK-12049][CORE] User JVM shutdown hook can cause deadlock at shutdownSean Owen2015-11-301-17/+16
| | | | | | | | Avoid potential deadlock with a user app's shutdown hook thread by more narrowly synchronizing access to 'hooks' Author: Sean Owen <sowen@cloudera.com> Closes #10042 from srowen/SPARK-12049.
* [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.