Commit graph

13574 commits

Author SHA1 Message Date
tedyu 3e0a6cf1e0 [SPARK-11572] Exit AsynchronousListenerBus thread when stop() is called
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.
2015-11-10 16:51:25 -08:00
Imran Rashid 33112f9c48 [SPARK-10192][CORE] simple test w/ failure involving a shared dependency
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.
2015-11-10 16:50:22 -08:00
Yu ISHIKAWA c0e48dfa61 [SPARK-11566] [MLLIB] [PYTHON] Refactoring GaussianMixtureModel.gaussians in Python
cc jkbradley

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #9534 from yu-iskw/SPARK-11566.
2015-11-10 16:42:28 -08:00
Bryan Cutler a3989058c0 [SPARK-10827][CORE] AppClient should not use askWithReply in receiveAndReply
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.
2015-11-10 16:32:32 -08:00
Herman van Hovell 21c562fa03 [SPARK-9241][SQL] Supporting multiple DISTINCT columns - follow-up (3)
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.
2015-11-10 16:28:21 -08:00
Yin Huai 3121e78168 [SPARK-9830][SPARK-11641][SQL][FOLLOW-UP] Remove AggregateExpression1 and 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.
2015-11-10 16:25:22 -08:00
Joseph K. Bradley e281b87398 [SPARK-5565][ML] LDA wrapper for Pipelines API
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.
2015-11-10 16:20:10 -08:00
Josh Rosen 1dde39d796 [SPARK-9818] Re-enable Docker tests for JDBC data source
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.
2015-11-10 15:58:30 -08:00
felixcheung 32790fe724 [SPARK-11567] [PYTHON] Add Python API for corr Aggregate function
like `df.agg(corr("col1", "col2")`

davies

Author: felixcheung <felixcheung_m@hotmail.com>

Closes #9536 from felixcheung/pyfunc.
2015-11-10 15:47:10 -08:00
Pravin Gadakh 638c51d938 [SPARK-11550][DOCS] Replace example code in mllib-optimization.md using include_example
Author: Pravin Gadakh <pravingadakh177@gmail.com>

Closes #9516 from pravingadakh/SPARK-11550.
2015-11-10 14:47:04 -08:00
Michael Armbrust 724cf7a38c [SPARK-11616][SQL] Improve toString for Dataset
Author: Michael Armbrust <michael@databricks.com>

Closes #9586 from marmbrus/dataset-toString.
2015-11-10 14:30:19 -08:00
unknown dba1a62cf1 [SPARK-7316][MLLIB] RDD sliding window with step
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.
2015-11-10 14:25:06 -08:00
Joseph K. Bradley 18350a5700 [SPARK-11618][ML] Minor refactoring of basic ML import/export
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.
2015-11-10 11:36:43 -08:00
Yanbo Liang f14e95115c [ML][R] SparkR::glm summary result to compare with native R
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.
2015-11-10 11:34:36 -08:00
Nong Li 87aedc48c0 [SPARK-10371][SQL] Implement subexpr elimination for UnsafeProjections
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.
2015-11-10 11:28:53 -08:00
Wenchen Fan 53600854c2 [SPARK-11590][SQL] use native json_tuple in lateral view
Author: Wenchen Fan <wenchen@databricks.com>

Closes #9562 from cloud-fan/json-tuple.
2015-11-10 11:21:31 -08:00
Wenchen Fan dfcfcbcc04 [SPARK-11578][SQL][FOLLOW-UP] complete the user facing api for typed aggregation
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.
2015-11-10 11:14:25 -08:00
Oscar D. Lara Yejas 47735cdc2a [SPARK-10863][SPARKR] Method coltypes() (New version)
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.
2015-11-10 11:07:57 -08:00
Yin Huai e0701c7560 [SPARK-9830][SQL] Remove AggregateExpression1 and Aggregate Operator used to 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.
2015-11-10 11:06:29 -08:00
Lianhui Wang 6e5fc37883 [SPARK-11252][NETWORK] ShuffleClient should release connection after 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.
2015-11-10 10:40:08 -08:00
Josh Rosen 689386b1c6 [SPARK-7841][BUILD] Stop using retrieveManaged to retrieve dependencies in SBT
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.
2015-11-10 10:14:19 -08:00
Xusen Yin a81f47ff74 [SPARK-11382] Replace example code in mllib-decision-tree.md using 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.
2015-11-10 10:05:53 -08:00
Paul Chandler 5507a9d093 Fix typo in driver page
"Comamnd property" => "Command property"

Author: Paul Chandler <pestilence669@users.noreply.github.com>

Closes #9578 from pestilence669/fix_spelling.
2015-11-10 12:59:53 +01:00
Davies Liu 521b3cae11 [SPARK-11598] [SQL] enable tests for ShuffledHashOuterJoin
Author: Davies Liu <davies@databricks.com>

Closes #9573 from davies/join_condition.
2015-11-09 23:28:32 -08:00
Davies Liu d6cd3a18e7 [SPARK-11599] [SQL] fix NPE when resolve Hive UDF in SQLParser
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.
2015-11-09 23:27:36 -08:00
Shivaram Venkataraman c4e19b3819 [SPARK-11587][SPARKR] Fix the summary generic to match base R
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.
2015-11-09 21:06:01 -08:00
Burak Yavuz 1431319e5b Add mockito as an explicit test dependency to spark-streaming
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.
2015-11-09 18:53:57 -08:00
Shixiong Zhu 6502944f39 [SPARK-11333][STREAMING] Add executorId to ReceiverInfo and display it in UI
Expose executorId to `ReceiverInfo` and UI since it's helpful when there are multiple executors running in the same host. Screenshot:

<img width="1058" alt="screen shot 2015-11-02 at 10 52 19 am" src="https://cloud.githubusercontent.com/assets/1000778/10890968/2e2f5512-8150-11e5-8d9d-746e826b69e8.png">

Author: Shixiong Zhu <shixiong@databricks.com>
Author: zsxwing <zsxwing@gmail.com>

Closes #9418 from zsxwing/SPARK-11333.
2015-11-09 18:13:37 -08:00
zsxwing 1f0f14efe3 [SPARK-11462][STREAMING] Add JavaStreamingListener
Currently, StreamingListener is not Java friendly because it exposes some Scala collections to Java users directly, such as Option, Map.

This PR added a Java version of StreamingListener and a bunch of Java friendly classes for Java users.

Author: zsxwing <zsxwing@gmail.com>
Author: Shixiong Zhu <shixiong@databricks.com>

Closes #9420 from zsxwing/java-streaming-listener.
2015-11-09 17:38:19 -08:00
Burak Yavuz 0ce6f9b2d2 [SPARK-11141][STREAMING] Batch ReceivedBlockTrackerLogEvents for WAL writes
When using S3 as a directory for WALs, the writes take too long. The driver gets very easily bottlenecked when multiple receivers send AddBlock events to the ReceiverTracker. This PR adds batching of events in the ReceivedBlockTracker so that receivers don't get blocked by the driver for too long.

cc zsxwing tdas

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #9143 from brkyvz/batch-wal-writes.
2015-11-09 17:35:12 -08:00
Burak Yavuz 26062d2260 [SPARK-11198][STREAMING][KINESIS] Support de-aggregation of records during recovery
While the KCL handles de-aggregation during the regular operation, during recovery we use the lower level api, and therefore need to de-aggregate the records.

tdas Testing is an issue, we need protobuf magic to do the aggregated records. Maybe we could depend on KPL for tests?

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #9403 from brkyvz/kinesis-deaggregation.
2015-11-09 17:18:49 -08:00
Yuhao Yang 61f9c8711c [SPARK-11069][ML] Add RegexTokenizer option to convert to lowercase
jira: https://issues.apache.org/jira/browse/SPARK-11069
quotes from jira:
Tokenizer converts strings to lowercase automatically, but RegexTokenizer does not. It would be nice to add an option to RegexTokenizer to convert to lowercase. Proposal:
call the Boolean Param "toLowercase"
set default to false (so behavior does not change)

Actually sklearn converts to lowercase before tokenizing too

Author: Yuhao Yang <hhbyyh@gmail.com>

Closes #9092 from hhbyyh/tokenLower.
2015-11-09 16:55:23 -08:00
Yu ISHIKAWA 7dc9d8dba6 [SPARK-11610][MLLIB][PYTHON][DOCS] Make the docs of LDAModel.describeTopics in Python more specific
cc jkbradley

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #9577 from yu-iskw/SPARK-11610.
2015-11-09 16:25:29 -08:00
Reynold Xin 675c7e723c [SPARK-11564][SQL] Fix documentation for DataFrame.take/collect
Author: Reynold Xin <rxin@databricks.com>

Closes #9557 from rxin/SPARK-11564-1.
2015-11-09 16:22:15 -08:00
Michael Armbrust 9c740a9ddf [SPARK-11578][SQL] User API for Typed Aggregation
This PR adds a new interface for user-defined aggregations, that can be used in `DataFrame` and `Dataset` operations to take all of the elements of a group and reduce them to a single value.

For example, the following aggregator extracts an `int` from a specific class and adds them up:

```scala
  case class Data(i: Int)

  val customSummer =  new Aggregator[Data, Int, Int] {
    def prepare(d: Data) = d.i
    def reduce(l: Int, r: Int) = l + r
    def present(r: Int) = r
  }.toColumn()

  val ds: Dataset[Data] = ...
  val aggregated = ds.select(customSummer)
```

By using helper functions, users can make a generic `Aggregator` that works on any input type:

```scala
/** An `Aggregator` that adds up any numeric type returned by the given function. */
class SumOf[I, N : Numeric](f: I => N) extends Aggregator[I, N, N] with Serializable {
  val numeric = implicitly[Numeric[N]]
  override def zero: N = numeric.zero
  override def reduce(b: N, a: I): N = numeric.plus(b, f(a))
  override def present(reduction: N): N = reduction
}

def sum[I, N : Numeric : Encoder](f: I => N): TypedColumn[I, N] = new SumOf(f).toColumn
```

These aggregators can then be used alongside other built-in SQL aggregations.

```scala
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()
ds
  .groupBy(_._1)
  .agg(
    sum(_._2),                // The aggregator defined above.
    expr("sum(_2)").as[Int],  // A built-in dynatically typed aggregation.
    count("*"))               // A built-in statically typed aggregation.
  .collect()

res0: ("a", 30, 30, 2L), ("b", 3, 3, 2L), ("c", 1, 1, 1L)
```

The current implementation focuses on integrating this into the typed API, but currently only supports running aggregations that return a single long value as explained in `TypedAggregateExpression`.  This will be improved in a followup PR.

Author: Michael Armbrust <michael@databricks.com>

Closes #9555 from marmbrus/dataset-useragg.
2015-11-09 16:11:00 -08:00
gatorsmile 2f38378856 [SPARK-11360][DOC] Loss of nullability when writing parquet files
This fix is to add one line to explain the current behavior of Spark SQL when writing Parquet files. All columns are forced to be nullable for compatibility reasons.

Author: gatorsmile <gatorsmile@gmail.com>

Closes #9314 from gatorsmile/lossNull.
2015-11-09 16:06:48 -08:00
hyukjinkwon 9565c246ea [SPARK-9557][SQL] Refactor ParquetFilterSuite and remove old ParquetFilters code
Actually this was resolved by https://github.com/apache/spark/pull/8275.

But I found the JIRA issue for this is not marked as resolved since the PR above was made for another issue but the PR above resolved both.

I commented that this is resolved by the PR above; however, I opened this PR as I would like to just add
a little bit of corrections.

In the previous PR, I refactored the test by not reducing just collecting filters; however, this would not test  properly `And` filter (which is not given to the tests). I unintentionally changed this from the original way (before being refactored).

In this PR, I just followed the original way to collect filters by reducing.

I would like to close this if this PR is inappropriate and somebody would like this deal with it in the separate PR related with this.

Author: hyukjinkwon <gurwls223@gmail.com>

Closes #9554 from HyukjinKwon/SPARK-9557.
2015-11-09 15:20:50 -08:00
Wenchen Fan fcb57e9c73 [SPARK-11564][SQL][FOLLOW-UP] improve java api for GroupedDataset
created `MapGroupFunction`, `FlatMapGroupFunction`, `CoGroupFunction`

Author: Wenchen Fan <wenchen@databricks.com>

Closes #9564 from cloud-fan/map.
2015-11-09 15:16:47 -08:00
Yu ISHIKAWA 8a2336893a [SPARK-6517][MLLIB] Implement the Algorithm of Hierarchical Clustering
I implemented a hierarchical clustering algorithm again.  This PR doesn't include examples, documentation and spark.ml APIs. I am going to send another PRs later.
https://issues.apache.org/jira/browse/SPARK-6517

- This implementation based on a bi-sectiong K-means clustering.
    - It derives from the freeman-lab 's implementation
- The basic idea is not changed from the previous version. (#2906)
    - However, It is 1000x faster than the previous version through parallel processing.

Thank you for your great cooperation, RJ Nowling(rnowling), Jeremy Freeman(freeman-lab), Xiangrui Meng(mengxr) and Sean Owen(srowen).

Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>
Author: Xiangrui Meng <meng@databricks.com>
Author: Yu ISHIKAWA <yu-iskw@users.noreply.github.com>

Closes #5267 from yu-iskw/new-hierarchical-clustering.
2015-11-09 14:56:36 -08:00
Burak Yavuz a3a7c9103e [SPARK-11359][STREAMING][KINESIS] Checkpoint to DynamoDB even when new data doesn't come in
Currently, the checkpoints to DynamoDB occur only when new data comes in, as we update the clock for the checkpointState. This PR makes the checkpoint a scheduled execution based on the `checkpointInterval`.

Author: Burak Yavuz <brkyvz@gmail.com>

Closes #9421 from brkyvz/kinesis-checkpoint.
2015-11-09 14:39:18 -08:00
Cheng Lian 150f6a89b7 [SPARK-11595] [SQL] Fixes ADD JAR when the input path contains URL scheme
Author: Cheng Lian <lian@databricks.com>

Closes #9569 from liancheng/spark-11595.fix-add-jar.
2015-11-09 14:32:52 -08:00
Nick Buroojy f138cb8733 [SPARK-9301][SQL] Add collect_set and collect_list aggregate functions
For now they are thin wrappers around the corresponding Hive UDAFs.

One limitation with these in Hive 0.13.0 is they only support aggregating primitive types.

I chose snake_case here instead of camelCase because it seems to be used in the majority of the multi-word fns.

Do we also want to add these to `functions.py`?

This approach was recommended here: https://github.com/apache/spark/pull/8592#issuecomment-154247089

marmbrus rxin

Author: Nick Buroojy <nick.buroojy@civitaslearning.com>

Closes #9526 from nburoojy/nick/udaf-alias.

(cherry picked from commit a6ee4f989d)
Signed-off-by: Michael Armbrust <michael@databricks.com>
2015-11-09 14:30:52 -08:00
Rishabh Bhardwaj b7720fa455 [SPARK-11548][DOCS] Replaced example code in mllib-collaborative-filtering.md using include_example
Kindly review the changes.

Author: Rishabh Bhardwaj <rbnext29@gmail.com>

Closes #9519 from rishabhbhardwaj/SPARK-11337.
2015-11-09 14:27:36 -08:00
sachin aggarwal 51d41e4b1a [SPARK-11552][DOCS][Replaced example code in ml-decision-tree.md using include_example]
I have tested it on my local, it is working fine, please review

Author: sachin aggarwal <different.sachin@gmail.com>

Closes #9539 from agsachin/SPARK-11552-real.
2015-11-09 14:25:42 -08:00
Felix Bechstein 5039a49b63 [SPARK-10471][CORE][MESOS] prevent getting offers for unmet constraints
this change rejects offers for slaves with unmet constraints for 120s to mitigate offer starvation.
this prevents mesos to send us these offers again and again.
in return, we get more offers for slaves which might meet our constraints.
and it enables mesos to send the rejected offers to other frameworks.

Author: Felix Bechstein <felix.bechstein@otto.de>

Closes #8639 from felixb/decline_offers_constraint_mismatch.
2015-11-09 13:36:14 -08:00
Yu ISHIKAWA 88a3fdcc78 [SPARK-10280][MLLIB][PYSPARK][DOCS] Add @since annotation to pyspark.ml.classification
Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com>

Closes #8690 from yu-iskw/SPARK-10280.
2015-11-09 13:16:04 -08:00
Bharat Lal 860ea0d386 [SPARK-11581][DOCS] Example mllib code in documentation incorrectly computes MSE
Author: Bharat Lal <bharat.iisc@gmail.com>

Closes #9560 from bharatl/SPARK-11581.
2015-11-09 11:33:01 -08:00
chriskang90 874cd66d4b [DOCS] Fix typo for Python section on unifying Kafka streams
1) kafkaStreams is a list.  The list should be unpacked when passing it into the streaming context union method, which accepts a variable number of streams.
2) print() should be pprint() for pyspark.

This contribution is my original work, and I license the work to the project under the project's open source license.

Author: chriskang90 <jckang@uchicago.edu>

Closes #9545 from c-kang/streaming_python_typo.
2015-11-09 19:39:22 +01:00
felixcheung cd174882a5 [SPARK-9865][SPARKR] Flaky SparkR test: test_sparkSQL.R: sample on a DataFrame
Make sample test less flaky by setting the seed

Tested with
```
repeat {  if (count(sample(df, FALSE, 0.1)) == 3) { break } }
```

Author: felixcheung <felixcheung_m@hotmail.com>

Closes #9549 from felixcheung/rsample.
2015-11-09 10:26:09 -08:00
tedyu 404a28f4ed [SPARK-11112] Fix Scala 2.11 compilation error in RDDInfo.scala
As shown in https://amplab.cs.berkeley.edu/jenkins/view/Spark-QA-Compile/job/Spark-Master-Scala211-Compile/1946/console , compilation fails with:
```
[error] /home/jenkins/workspace/Spark-Master-Scala211-Compile/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala:25: in class RDDInfo, multiple overloaded alternatives of constructor RDDInfo define default arguments.
[error] class RDDInfo(
[error]
```
This PR tries to fix the compilation error

Author: tedyu <yuzhihong@gmail.com>

Closes #9538 from tedyu/master.
2015-11-09 10:07:58 -08:00