aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-11647] Attempt to reduce time/flakiness of Thriftserver CLI and ↵Josh Rosen2015-11-114-18/+38
| | | | | | | | | | | | | | | | SparkSubmit tests This patch aims to reduce the test time and flakiness of HiveSparkSubmitSuite, SparkSubmitSuite, and CliSuite. Key changes: - Disable IO synchronization calls for Derby writes, since durability doesn't matter for tests. This was done for HiveCompatibilitySuite in #6651 and resulted in huge test speedups. - Add a few missing `--conf`s to disable various Spark UIs. The CliSuite, in particular, never disabled these UIs, leaving it prone to port-contention-related flakiness. - Fix two instances where tests defined `beforeAll()` methods which were never called because the appropriate traits were not mixed in. I updated these tests suites to extend `BeforeAndAfterEach` so that they play nicely with our `ResetSystemProperties` trait. Author: Josh Rosen <joshrosen@databricks.com> Closes #9623 from JoshRosen/SPARK-11647.
* [SPARK-11335][STREAMING] update kafka direct python docs on how to get the ↵Nick Evans2015-11-111-1/+14
| | | | | | | | | | | | offset ranges for a KafkaRDD tdas koeninger This updates the Spark Streaming + Kafka Integration Guide doc with a working method to access the offsets of a `KafkaRDD` through Python. Author: Nick Evans <me@nicolasevans.org> Closes #9289 from manygrams/update_kafka_direct_python_docs.
* [SPARK-11645][SQL] Remove OpenHashSet for the old aggregate.Reynold Xin2015-11-115-316/+5
| | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #9621 from rxin/SPARK-11645.
* [SPARK-11644][SQL] Remove the option to turn off unsafe and codegen.Reynold Xin2015-11-1127-494/+257
| | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #9618 from rxin/SPARK-11644.
* [SPARK-11639][STREAMING][FLAKY-TEST] Implement BlockingWriteAheadLog for ↵Burak Yavuz2015-11-112-47/+80
| | | | | | | | | | | | testing the BatchedWriteAheadLog Several elements could be drained if the main thread is not fast enough. zsxwing warned me about a similar problem, but missed it here :( Submitting the fix using a waiter. cc tdas Author: Burak Yavuz <brkyvz@gmail.com> Closes #9605 from brkyvz/fix-flaky-test.
* [SPARK-6152] Use shaded ASM5 to support closure cleaning of Java 8 compiled ↵Josh Rosen2015-11-1112-57/+125
| | | | | | | | | | | | | | | | classes This patch modifies Spark's closure cleaner (and a few other places) to use ASM 5, which is necessary in order to support cleaning of closures that were compiled by Java 8. In order to avoid ASM dependency conflicts, Spark excludes ASM from all of its dependencies and uses a shaded version of ASM 4 that comes from `reflectasm` (see [SPARK-782](https://issues.apache.org/jira/browse/SPARK-782) and #232). This patch updates Spark to use a shaded version of ASM 5.0.4 that was published by the Apache XBean project; the POM used to create the shaded artifact can be found at https://github.com/apache/geronimo-xbean/blob/xbean-4.4/xbean-asm5-shaded/pom.xml. http://movingfulcrum.tumblr.com/post/80826553604/asm-framework-50-the-missing-migration-guide was a useful resource while upgrading the code to use the new ASM5 opcodes. I also added a new regression tests in the `java8-tests` subproject; the existing tests were insufficient to catch this bug, which only affected Scala 2.11 user code which was compiled targeting Java 8. Author: Josh Rosen <joshrosen@databricks.com> Closes #9512 from JoshRosen/SPARK-6152.
* [SQL][MINOR] remove newLongEncoder in functionsWenchen Fan2015-11-111-4/+2
| | | | | | | | it may shadows the one from implicits in some case. Author: Wenchen Fan <wenchen@databricks.com> Closes #9629 from cloud-fan/minor.
* [SPARK-11564][SQL][FOLLOW-UP] clean up java tuple encoderWenchen Fan2015-11-1114-113/+65
| | | | | | | | | | | We need to support custom classes like java beans and combine them into tuple, and it's very hard to do it with the TypeTag-based approach. We should keep only the compose-based way to create tuple encoder. This PR also move `Encoder` to `org.apache.spark.sql` Author: Wenchen Fan <wenchen@databricks.com> Closes #9567 from cloud-fan/java.
* [SPARK-11656][SQL] support typed aggregate in project listWenchen Fan2015-11-112-4/+27
| | | | | | | | insert `aEncoder` like we do in `agg` Author: Wenchen Fan <wenchen@databricks.com> Closes #9630 from cloud-fan/select.
* [SQL][MINOR] rename present to finish in AggregatorWenchen Fan2015-11-113-5/+5
| | | | | | Author: Wenchen Fan <wenchen@databricks.com> Closes #9617 from cloud-fan/tmp.
* [SPARK-11646] WholeTextFileRDD should return Text rather than StringReynold Xin2015-11-115-44/+69
| | | | | | | | If it returns Text, we can reuse this in Spark SQL to provide a WholeTextFile data source and directly convert the Text into UTF8String without extra string decoding and encoding. Author: Reynold Xin <rxin@databricks.com> Closes #9622 from rxin/SPARK-11646.
* [SPARK-11626][ML] ml.feature.Word2Vec.transform() function very slowYuming Wang2015-11-111-18/+16
| | | | | | | | | | org.apache.spark.ml.feature.Word2Vec.transform() very slow. we should not read broadcast every sentence. Author: Yuming Wang <q79969786@gmail.com> Author: yuming.wang <q79969786@gmail.com> Author: Xiangrui Meng <meng@databricks.com> Closes #9592 from 979969786/master.
* [SPARK-10371][SQL][FOLLOW-UP] fix code styleWenchen Fan2015-11-113-33/+30
| | | | | | Author: Wenchen Fan <wenchen@databricks.com> Closes #9627 from cloud-fan/follow.
* [SPARK-11500][SQL] Not deterministic order of columns when using merging ↵hyukjinkwon2015-11-113-17/+55
| | | | | | | | | | | | | | | | | | | | | | schemas. https://issues.apache.org/jira/browse/SPARK-11500 As filed in SPARK-11500, if merging schemas is enabled, the order of files to touch is a matter which might affect the ordering of the output columns. This was mostly because of the use of `Set` and `Map` so I replaced them to `LinkedHashSet` and `LinkedHashMap` to keep the insertion order. Also, I changed `reduceOption` to `reduceLeftOption`, and replaced the order of `filesToTouch` from `metadataStatuses ++ commonMetadataStatuses ++ needMerged` to `needMerged ++ metadataStatuses ++ commonMetadataStatuses` in order to touch the part-files first which always have the schema in footers whereas the others might not exist. One nit is, If merging schemas is not enabled, but when multiple files are given, there is no guarantee of the output order, since there might not be a summary file for the first file, which ends up putting ahead the columns of the other files. However, I thought this should be okay since disabling merging schemas means (assumes) all the files have the same schemas. In addition, in the test code for this, I only checked the names of fields. Author: hyukjinkwon <gurwls223@gmail.com> Closes #9517 from HyukjinKwon/SPARK-11500.
* [SPARK-11290][STREAMING] Basic implementation of trackStateByKeyTathagata Das2015-11-1010-19/+2125
| | | | | | | | | | | | | | | | | | | | | | | | | | | Current updateStateByKey provides stateful processing in Spark Streaming. It allows the user to maintain per-key state and manage that state using an updateFunction. The updateFunction is called for each key, and it uses new data and existing state of the key, to generate an updated state. However, based on community feedback, we have learnt the following lessons. * Need for more optimized state management that does not scan every key * Need to make it easier to implement common use cases - (a) timeout of idle data, (b) returning items other than state The high level idea that of this PR * Introduce a new API trackStateByKey that, allows the user to update per-key state, and emit arbitrary records. The new API is necessary as this will have significantly different semantics than the existing updateStateByKey API. This API will have direct support for timeouts. * Internally, the system will keep the state data as a map/list within the partitions of the state RDDs. The new data RDDs will be partitioned appropriately, and for all the key-value data, it will lookup the map/list in the state RDD partition and create a new list/map of updated state data. The new state RDD partition will be created based on the update data and if necessary, with old data. Here is the detailed design doc. Please take a look and provide feedback as comments. https://docs.google.com/document/d/1NoALLyd83zGs1hNGMm0Pc5YOVgiPpMHugGMk6COqxxE/edit#heading=h.ph3w0clkd4em This is still WIP. Major things left to be done. - [x] Implement basic functionality of state tracking, with initial RDD and timeouts - [x] Unit tests for state tracking - [x] Unit tests for initial RDD and timeout - [ ] Unit tests for TrackStateRDD - [x] state creating, updating, removing - [ ] emitting - [ ] checkpointing - [x] Misc unit tests for State, TrackStateSpec, etc. - [x] Update docs and experimental tags Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #9256 from tdas/trackStateByKey.
* [SPARK-11463] [PYSPARK] only install signal in main threadDavies Liu2015-11-101-1/+4
| | | | | | | | Only install signal in main thread, or it will fail to create context in not-main thread. Author: Davies Liu <davies@databricks.com> Closes #9574 from davies/python_signal.
* [SPARK-11468] [SPARKR] add stddev/variance agg functions for Columnfelixcheung2015-11-105-30/+297
| | | | | | | | | | Checked names, none of them should conflict with anything in base shivaram davies rxin Author: felixcheung <felixcheung_m@hotmail.com> Closes #9489 from felixcheung/rstddev.
* [SPARK-10192][HOTFIX] Fix NPE in test that was added in #8402Josh Rosen2015-11-101-1/+1
| | | | | | | | This fixes an NPE introduced in SPARK-10192 / #8402. Author: Josh Rosen <joshrosen@databricks.com> Closes #9620 from JoshRosen/SPARK-10192-hotfix.
* [SPARK-6726][ML] Import/export for spark.ml LogisticRegressionModelJoseph K. Bradley2015-11-104-11/+152
| | | | | | | | | | This PR adds model save/load for spark.ml's LogisticRegressionModel. It also does minor refactoring of the default save/load classes to reuse code. CC: mengxr Author: Joseph K. Bradley <joseph@databricks.com> Closes #9606 from jkbradley/logreg-io2.
* [MINOR] License header formatting fixMarc Prud'hommeaux2015-11-101-15/+15
| | | | | | | | The header wasn't indented properly. Author: Marc Prud'hommeaux <mwp1@cornell.edu> Closes #9312 from mprudhom/patch-1.
* [MINOR] Fix typo in AggregationQuerySuite.scalaForest Fang2015-11-101-2/+2
| | | | | | Author: Forest Fang <saurfang@users.noreply.github.com> Closes #9357 from saurfang/patch-1.
* [SPARK-11361][STREAMING] Show scopes of RDD operations inside ↵Tathagata Das2015-11-107-33/+147
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | DStream.foreachRDD and DStream.transform in DAG viz Currently, when a DStream sets the scope for RDD generated by it, that scope is not allowed to be overridden by the RDD operations. So in case of `DStream.foreachRDD`, all the RDDs generated inside the foreachRDD get the same scope - `foreachRDD <time>`, as set by the `ForeachDStream`. So it is hard to debug generated RDDs in the RDD DAG viz in the Spark UI. This patch allows the RDD operations inside `DStream.transform` and `DStream.foreachRDD` to append their own scopes to the earlier DStream scope. I have also slightly tweaked how callsites are set such that the short callsite reflects the RDD operation name and line number. This tweak is necessary as callsites are not managed through scopes (which support nesting and overriding) and I didnt want to add another local property to control nesting and overriding of callsites. ## Before: ![image](https://cloud.githubusercontent.com/assets/663212/10808548/fa71c0c4-7da9-11e5-9af0-5737793a146f.png) ## After: ![image](https://cloud.githubusercontent.com/assets/663212/10808659/37bc45b6-7dab-11e5-8041-c20be6a9bc26.png) The code that was used to generate this is: ``` val lines = ssc.socketTextStream(args(0), args(1).toInt, StorageLevel.MEMORY_AND_DISK_SER) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) wordCounts.foreachRDD { rdd => val temp = rdd.map { _ -> 1 }.reduceByKey( _ + _) val temp2 = temp.map { _ -> 1}.reduceByKey(_ + _) val count = temp2.count println(count) } ``` Note - The inner scopes of the RDD operations map/reduceByKey inside foreachRDD is visible - The short callsites of stages refers to the line number of the RDD ops rather than the same line number of foreachRDD in all three cases. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #9315 from tdas/SPARK-11361.
* [SPARK-11615] Drop @VisibleForTesting annotationtedyu2015-11-107-15/+14
| | | | | | | | | | See http://search-hadoop.com/m/q3RTtjpe8r1iRbTj2 for discussion. Summary: addition of VisibleForTesting annotation resulted in spark-shell malfunctioning. Author: tedyu <yuzhihong@gmail.com> Closes #9585 from tedyu/master.
* [SPARK-11572] Exit AsynchronousListenerBus thread when stop() is calledtedyu2015-11-101-6/+3
| | | | | | | | | | | As vonnagy reported in the following thread: http://search-hadoop.com/m/q3RTtk982kvIow22 Attempts to join the thread in AsynchronousListenerBus resulted in lock up because AsynchronousListenerBus thread was still getting messages `SparkListenerExecutorMetricsUpdate` from the DAGScheduler Author: tedyu <yuzhihong@gmail.com> Closes #9546 from ted-yu/master.
* [SPARK-10192][CORE] simple test w/ failure involving a shared dependencyImran Rashid2015-11-101-2/+49
| | | | | | | | | | just trying to increase test coverage in the scheduler, this already works. It includes a regression test for SPARK-9809 copied some test utils from https://github.com/apache/spark/pull/5636, we can wait till that is merged first Author: Imran Rashid <irashid@cloudera.com> Closes #8402 from squito/test_retry_in_shared_shuffle_dep.
* [SPARK-11566] [MLLIB] [PYTHON] Refactoring GaussianMixtureModel.gaussians in ↵Yu ISHIKAWA2015-11-102-16/+7
| | | | | | | | | | Python cc jkbradley Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #9534 from yu-iskw/SPARK-11566.
* [SPARK-10827][CORE] AppClient should not use `askWithReply` in `receiveAndReply`Bryan Cutler2015-11-102-4/+238
| | | | | | | | Changed AppClient to be non-blocking in `receiveAndReply` by using a separate thread to wait for response and reply to the context. The threads are managed by a thread pool. Also added unit tests for the AppClient interface. Author: Bryan Cutler <bjcutler@us.ibm.com> Closes #9317 from BryanCutler/appClient-receiveAndReply-SPARK-10827.
* [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)Herman van Hovell2015-11-102-8/+42
| | | | | | | | | | | | | This PR is a 2nd follow-up for [SPARK-9241](https://issues.apache.org/jira/browse/SPARK-9241). It contains the following improvements: * Fix for a potential bug in distinct child expression and attribute alignment. * Improved handling of duplicate distinct child expressions. * Added test for distinct UDAF with multiple children. cc yhuai Author: Herman van Hovell <hvanhovell@questtec.nl> Closes #9566 from hvanhovell/SPARK-9241-followup-2.
* [SPARK-9830][SPARK-11641][SQL][FOLLOW-UP] Remove AggregateExpression1 and ↵Yin Huai2015-11-1010-54/+160
| | | | | | | | | | | | update toString of Exchange https://issues.apache.org/jira/browse/SPARK-9830 This is the follow-up pr for https://github.com/apache/spark/pull/9556 to address davies' comments. Author: Yin Huai <yhuai@databricks.com> Closes #9607 from yhuai/removeAgg1-followup.
* [SPARK-5565][ML] LDA wrapper for Pipelines APIJoseph K. Bradley2015-11-103-5/+946
| | | | | | | | | | | | | | | This adds LDA to spark.ml, the Pipelines API. It follows the design doc in the JIRA: [https://issues.apache.org/jira/browse/SPARK-5565], with one major change: * I eliminated doc IDs. These are not necessary with DataFrames since the user can add an ID column as needed. Note: This will conflict with [https://github.com/apache/spark/pull/9484], but I'll try to merge [https://github.com/apache/spark/pull/9484] first and then rebase this PR. CC: hhbyyh feynmanliang If you have a chance to make a pass, that'd be really helpful--thanks! Now that I'm done traveling & this PR is almost ready, I'll see about reviewing other PRs critical for 1.6. CC: mengxr Author: Joseph K. Bradley <joseph@databricks.com> Closes #9513 from jkbradley/lda-pipelines.
* [SPARK-9818] Re-enable Docker tests for JDBC data sourceJosh Rosen2015-11-108-2/+664
| | | | | | | | | | | | | This patch re-enables tests for the Docker JDBC data source. These tests were reverted in #4872 due to transitive dependency conflicts introduced by the `docker-client` library. This patch should avoid those problems by using a version of `docker-client` which shades its transitive dependencies and by performing some build-magic to work around problems with that shaded JAR. In addition, I significantly refactored the tests to simplify the setup and teardown code and to fix several Docker networking issues which caused problems when running in `boot2docker`. Closes #8101. Author: Josh Rosen <joshrosen@databricks.com> Author: Yijie Shen <henry.yijieshen@gmail.com> Closes #9503 from JoshRosen/docker-jdbc-tests.
* [SPARK-11567] [PYTHON] Add Python API for corr Aggregate functionfelixcheung2015-11-101-0/+16
| | | | | | | | | | like `df.agg(corr("col1", "col2")` davies Author: felixcheung <felixcheung_m@hotmail.com> Closes #9536 from felixcheung/pyfunc.
* [SPARK-11550][DOCS] Replace example code in mllib-optimization.md using ↵Pravin Gadakh2015-11-103-143/+200
| | | | | | | | include_example Author: Pravin Gadakh <pravingadakh177@gmail.com> Closes #9516 from pravingadakh/SPARK-11550.
* [SPARK-11616][SQL] Improve toString for DatasetMichael Armbrust2015-11-104-13/+47
| | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #9586 from marmbrus/dataset-toString.
* [SPARK-7316][MLLIB] RDD sliding window with stepunknown2015-11-103-39/+54
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | Implementation of step capability for sliding window function in MLlib's RDD. Though one can use current sliding window with step 1 and then filter every Nth window, it will take more time and space (N*data.count times more than needed). For example, below are the results for various windows and steps on 10M data points: Window | Step | Time | Windows produced ------------ | ------------- | ---------- | ---------- 128 | 1 | 6.38 | 9999873 128 | 10 | 0.9 | 999988 128 | 100 | 0.41 | 99999 1024 | 1 | 44.67 | 9998977 1024 | 10 | 4.74 | 999898 1024 | 100 | 0.78 | 99990 ``` import org.apache.spark.mllib.rdd.RDDFunctions._ val rdd = sc.parallelize(1 to 10000000, 10) rdd.count val window = 1024 val step = 1 val t = System.nanoTime(); val windows = rdd.sliding(window, step); println(windows.count); println((System.nanoTime() - t) / 1e9) ``` Author: unknown <ulanov@ULANOV3.americas.hpqcorp.net> Author: Alexander Ulanov <nashb@yandex.ru> Author: Xiangrui Meng <meng@databricks.com> Closes #5855 from avulanov/SPARK-7316-sliding.
* [SPARK-11618][ML] Minor refactoring of basic ML import/exportJoseph K. Bradley2015-11-101-27/+30
| | | | | | | | | | | | Refactoring * separated overwrite and param save logic in DefaultParamsWriter * added sparkVersion to DefaultParamsWriter CC: mengxr Author: Joseph K. Bradley <joseph@databricks.com> Closes #9587 from jkbradley/logreg-io.
* [ML][R] SparkR::glm summary result to compare with native RYanbo Liang2015-11-102-22/+11
| | | | | | | | Follow up #9561. Due to [SPARK-11587](https://issues.apache.org/jira/browse/SPARK-11587) has been fixed, we should compare SparkR::glm summary result with native R output rather than hard-code one. mengxr Author: Yanbo Liang <ybliang8@gmail.com> Closes #9590 from yanboliang/glm-r-test.
* [SPARK-10371][SQL] Implement subexpr elimination for UnsafeProjectionsNong Li2015-11-1011-16/+523
| | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch adds the building blocks for codegening subexpr elimination and implements it end to end for UnsafeProjection. The building blocks can be used to do the same thing for other operators. It introduces some utilities to compute common sub expressions. Expressions can be added to this data structure. The expr and its children will be recursively matched against existing expressions (ones previously added) and grouped into common groups. This is built using the existing `semanticEquals`. It does not understand things like commutative or associative expressions. This can be done as future work. After building this data structure, the codegen process takes advantage of it by: 1. Generating a helper function in the generated class that computes the common subexpression. This is done for all common subexpressions that have at least two occurrences and the expression tree is sufficiently complex. 2. When generating the apply() function, if the helper function exists, call that instead of regenerating the expression tree. Repeated calls to the helper function shortcircuit the evaluation logic. Author: Nong Li <nong@databricks.com> Author: Nong Li <nongli@gmail.com> This patch had conflicts when merged, resolved by Committer: Michael Armbrust <michael@databricks.com> Closes #9480 from nongli/spark-10371.
* [SPARK-11590][SQL] use native json_tuple in lateral viewWenchen Fan2015-11-108-40/+104
| | | | | | Author: Wenchen Fan <wenchen@databricks.com> Closes #9562 from cloud-fan/json-tuple.
* [SPARK-11578][SQL][FOLLOW-UP] complete the user facing api for typed aggregationWenchen Fan2015-11-104-14/+99
| | | | | | | | | | | | | | Currently the user facing api for typed aggregation has some limitations: * the customized typed aggregation must be the first of aggregation list * the customized typed aggregation can only use long as buffer type * the customized typed aggregation can only use flat type as result type This PR tries to remove these limitations. Author: Wenchen Fan <wenchen@databricks.com> Closes #9599 from cloud-fan/agg.
* [SPARK-10863][SPARKR] Method coltypes() (New version)Oscar D. Lara Yejas2015-11-107-18/+124
| | | | | | | | This is a follow up on PR #8984, as the corresponding branch for such PR was damaged. Author: Oscar D. Lara Yejas <olarayej@mail.usf.edu> Closes #9579 from olarayej/SPARK-10863_NEW14.
* [SPARK-9830][SQL] Remove AggregateExpression1 and Aggregate Operator used to ↵Yin Huai2015-11-1064-2260/+743
| | | | | | | | | | | | | | | | | | | evaluate AggregateExpression1s https://issues.apache.org/jira/browse/SPARK-9830 This PR contains the following main changes. * Removing `AggregateExpression1`. * Removing `Aggregate` operator, which is used to evaluate `AggregateExpression1`. * Removing planner rule used to plan `Aggregate`. * Linking `MultipleDistinctRewriter` to analyzer. * Renaming `AggregateExpression2` to `AggregateExpression` and `AggregateFunction2` to `AggregateFunction`. * Updating places where we create aggregate expression. The way to create aggregate expressions is `AggregateExpression(aggregateFunction, mode, isDistinct)`. * Changing `val`s in `DeclarativeAggregate`s that touch children of this function to `lazy val`s (when we create aggregate expression in DataFrame API, children of an aggregate function can be unresolved). Author: Yin Huai <yhuai@databricks.com> Closes #9556 from yhuai/removeAgg1.
* [SPARK-11252][NETWORK] ShuffleClient should release connection after ↵Lianhui Wang2015-11-106-15/+81
| | | | | | | | | | | fetching blocks had been completed for external shuffle with yarn's external shuffle, ExternalShuffleClient of executors reserve its connections for yarn's NodeManager until application has been completed. so it will make NodeManager and executors have many socket connections. in order to reduce network pressure of NodeManager's shuffleService, after registerWithShuffleServer or fetchBlocks have been completed in ExternalShuffleClient, connection for NM's shuffleService needs to be closed.andrewor14 rxin vanzin Author: Lianhui Wang <lianhuiwang09@gmail.com> Closes #9227 from lianhuiwang/spark-11252.
* [SPARK-7841][BUILD] Stop using retrieveManaged to retrieve dependencies in SBTJosh Rosen2015-11-102-6/+18
| | | | | | | | | | | | | | This patch modifies Spark's SBT build so that it no longer uses `retrieveManaged` / `lib_managed` to store its dependencies. The motivations for this change are nicely described on the JIRA ticket ([SPARK-7841](https://issues.apache.org/jira/browse/SPARK-7841)); my personal interest in doing this stems from the fact that `lib_managed` has caused me some pain while debugging dependency issues in another PR of mine. Removing our use of `lib_managed` would be trivial except for one snag: the Datanucleus JARs, required by Spark SQL's Hive integration, cannot be included in assembly JARs due to problems with merging OSGI `plugin.xml` files. As a result, several places in the packaging and deployment pipeline assume that these Datanucleus JARs are copied to `lib_managed/jars`. In the interest of maintaining compatibility, I have chosen to retain the `lib_managed/jars` directory _only_ for these Datanucleus JARs and have added custom code to `SparkBuild.scala` to automatically copy those JARs to that folder as part of the `assembly` task. `dev/mima` also depended on `lib_managed` in a hacky way in order to set classpaths when generating MiMa excludes; I've updated this to obtain the classpaths directly from SBT instead. /cc dragos marmbrus pwendell srowen Author: Josh Rosen <joshrosen@databricks.com> Closes #9575 from JoshRosen/SPARK-7841.
* [SPARK-11382] Replace example code in mllib-decision-tree.md using ↵Xusen Yin2015-11-108-247/+438
| | | | | | | | | | | | include_example https://issues.apache.org/jira/browse/SPARK-11382 B.T.W. I fix an error in naive_bayes_example.py. Author: Xusen Yin <yinxusen@gmail.com> Closes #9596 from yinxusen/SPARK-11382.
* Fix typo in driver pagePaul Chandler2015-11-101-1/+1
| | | | | | | | "Comamnd property" => "Command property" Author: Paul Chandler <pestilence669@users.noreply.github.com> Closes #9578 from pestilence669/fix_spelling.
* [SPARK-11598] [SQL] enable tests for ShuffledHashOuterJoinDavies Liu2015-11-091-204/+231
| | | | | | Author: Davies Liu <davies@databricks.com> Closes #9573 from davies/join_condition.
* [SPARK-11599] [SQL] fix NPE when resolve Hive UDF in SQLParserDavies Liu2015-11-092-9/+34
| | | | | | | | | | The DataFrame APIs that takes a SQL expression always use SQLParser, then the HiveFunctionRegistry will called outside of Hive state, cause NPE if there is not a active Session State for current thread (in PySpark). cc rxin yhuai Author: Davies Liu <davies@databricks.com> Closes #9576 from davies/hive_udf.
* [SPARK-11587][SPARKR] Fix the summary generic to match base RShivaram Venkataraman2015-11-094-10/+16
| | | | | | | | | The signature is summary(object, ...) as defined in https://stat.ethz.ch/R-manual/R-devel/library/base/html/summary.html Author: Shivaram Venkataraman <shivaram@cs.berkeley.edu> Closes #9582 from shivaram/summary-fix.
* Add mockito as an explicit test dependency to spark-streamingBurak Yavuz2015-11-091-0/+5
| | | | | | | | | While sbt successfully compiles as it properly pulls the mockito dependency, maven builds have broken. We need this in ASAP. tdas Author: Burak Yavuz <brkyvz@gmail.com> Closes #9584 from brkyvz/fix-master.