Commit graph

13023 commits

Author SHA1 Message Date
Andrew Or fd61b00487 [Minor] style fix for previous commit f24316e 2015-09-22 00:05:30 -07:00
Madhusudanan Kandasamy f24316e6d9 [SPARK-10458] [SPARK CORE] Added isStopped() method in SparkContext
Added isStopped() method in SparkContext

Author: Madhusudanan Kandasamy <madhusudanan@in.ibm.com>

Closes #8749 from kmadhugit/SPARK-10458.
2015-09-22 00:03:48 -07:00
Liang-Chi Hsieh 1fcefef069 [SPARK-10446][SQL] Support to specify join type when calling join with usingColumns
JIRA: https://issues.apache.org/jira/browse/SPARK-10446

Currently the method `join(right: DataFrame, usingColumns: Seq[String])` only supports inner join. It is more convenient to have it support other join types.

Author: Liang-Chi Hsieh <viirya@appier.com>

Closes #8600 from viirya/usingcolumns_df.
2015-09-21 23:46:00 -07:00
Ewan Leith 781b21ba2a [SPARK-10419] [SQL] Adding SQLServer support for datetimeoffset types to JdbcDialects
Reading from Microsoft SQL Server over jdbc fails when the table contains datetimeoffset types.

This patch registers a SQLServer JDBC Dialect that maps datetimeoffset to a String, as Microsoft suggest.

Author: Ewan Leith <ewan.leith@realitymine.com>

Closes #8575 from realitymine-coordinator/sqlserver.
2015-09-21 23:43:20 -07:00
Jian Feng 0180b849db [SPARK-10577] [PYSPARK] DataFrame hint for broadcast join
https://issues.apache.org/jira/browse/SPARK-10577

Author: Jian Feng <jzhang.chs@gmail.com>

Closes #8801 from Jianfeng-chs/master.
2015-09-21 23:36:41 -07:00
Sean Owen bf20d6c9f9 [SPARK-10716] [BUILD] spark-1.5.0-bin-hadoop2.6.tgz file doesn't uncompress on OS X due to hidden file
Remove ._SUCCESS.crc hidden file that may cause problems in distribution tar archive, and is not used

Author: Sean Owen <sowen@cloudera.com>

Closes #8846 from srowen/SPARK-10716.
2015-09-21 23:29:59 -07:00
Holden Karau 1cd6741572 [SPARK-9821] [PYSPARK] pyspark-reduceByKey-should-take-a-custom-partitioner
from the issue:

In Scala, I can supply a custom partitioner to reduceByKey (and other aggregation/repartitioning methods like aggregateByKey and combinedByKey), but as far as I can tell from the Pyspark API, there's no way to do the same in Python.
Here's an example of my code in Scala:
weblogs.map(s => (getFileType(s), 1)).reduceByKey(new FileTypePartitioner(),_+_)
But I can't figure out how to do the same in Python. The closest I can get is to call repartition before reduceByKey like so:
weblogs.map(lambda s: (getFileType(s), 1)).partitionBy(3,hash_filetype).reduceByKey(lambda v1,v2: v1+v2).collect()
But that defeats the purpose, because I'm shuffling twice instead of once, so my performance is worse instead of better.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8569 from holdenk/SPARK-9821-pyspark-reduceByKey-should-take-a-custom-partitioner.
2015-09-21 23:21:24 -07:00
Hossein c986e933a9 [SPARK-10711] [SPARKR] Do not assume spark.submit.deployMode is always set
In ```RUtils.sparkRPackagePath()``` we
1. Call ``` sys.props("spark.submit.deployMode")``` which returns null if ```spark.submit.deployMode``` is not suet
2. Call ``` sparkConf.get("spark.submit.deployMode")``` which throws ```NoSuchElementException``` if ```spark.submit.deployMode``` is not set. This patch simply passes a default value ("cluster") for ```spark.submit.deployMode```.

cc rxin

Author: Hossein <hossein@databricks.com>

Closes #8832 from falaki/SPARK-10711.
2015-09-21 21:09:59 -07:00
Yin Huai 0494c80ef5 [SPARK-10495] [SQL] Read date values in JSON data stored by Spark 1.5.0.
https://issues.apache.org/jira/browse/SPARK-10681

Author: Yin Huai <yhuai@databricks.com>

Closes #8806 from yhuai/SPARK-10495.
2015-09-21 18:06:45 -07:00
Tathagata Das 72869883f1 [SPARK-10649] [STREAMING] Prevent inheriting job group and irrelevant job description in streaming jobs
The job group, and job descriptions information is passed through thread local properties, and get inherited by child threads. In case of spark streaming, the streaming jobs inherit these properties from the thread that called streamingContext.start(). This may not make sense.

1. Job group: This is mainly used for cancelling a group of jobs together. It does not make sense to cancel streaming jobs like this, as the effect will be unpredictable. And its not a valid usecase any way, to cancel a streaming context, call streamingContext.stop()

2. Job description: This is used to pass on nice text descriptions for jobs to show up in the UI. The job description of the thread that calls streamingContext.start() is not useful for all the streaming jobs, as it does not make sense for all of the streaming jobs to have the same description, and the description may or may not be related to streaming.

The solution in this PR is meant for the Spark master branch, where local properties are inherited by cloning the properties. The job group and job description in the thread that starts the streaming scheduler are explicitly removed, so that all the subsequent child threads does not inherit them. Also, the starting is done in a new child thread, so that setting the job group and description for streaming, does not change those properties in the thread that called streamingContext.start().

Author: Tathagata Das <tathagata.das1565@gmail.com>

Closes #8781 from tdas/SPARK-10649.
2015-09-21 16:47:52 -07:00
noelsmith 7c4f852bfc [DOC] [PYSPARK] [MLLIB] Added newlines to docstrings to fix parameter formatting
Added newlines before `:param ...:` and `:return:` markup. Without these, parameter lists aren't formatted correctly in the API docs. I.e:

![screen shot 2015-09-21 at 21 49 26](https://cloud.githubusercontent.com/assets/11915197/10004686/de3c41d4-60aa-11e5-9c50-a46dcb51243f.png)

.. looks like this once newline is added:

![screen shot 2015-09-21 at 21 50 14](https://cloud.githubusercontent.com/assets/11915197/10004706/f86bfb08-60aa-11e5-8524-ae4436713502.png)

Author: noelsmith <mail@noelsmith.com>

Closes #8851 from noel-smith/docstring-missing-newline-fix.
2015-09-21 14:24:19 -07:00
Holden Karau 362539f8d9 [SPARK-10630] [SQL] Add a createDataFrame API that takes in a java list
It would be nice to support creating a DataFrame directly from a Java List of Row.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8779 from holdenk/SPARK-10630-create-DataFrame-from-Java-List.
2015-09-21 13:33:10 -07:00
Marcelo Vanzin 97a99dde6e [SPARK-10676] [DOCS] Add documentation for SASL encryption options.
Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8803 from vanzin/SPARK-10676.
2015-09-21 13:15:44 -07:00
Feynman Liang aeef44a3e3 [SPARK-3147] [MLLIB] [STREAMING] Streaming 2-sample statistical significance testing
Implementation of significance testing using Streaming API.

Author: Feynman Liang <fliang@databricks.com>
Author: Feynman Liang <feynman.liang@gmail.com>

Closes #4716 from feynmanliang/ab_testing.
2015-09-21 13:11:28 -07:00
Holden Karau ba882db6f4 [SPARK-9769] [ML] [PY] add python api for countvectorizermodel
From JIRA: Add Python API, user guide and example for ml.feature.CountVectorizerModel

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8561 from holdenk/SPARK-9769-add-python-api-for-countvectorizermodel.
2015-09-21 13:06:23 -07:00
hushan[胡珊] b78c65b03a [SPARK-5259] [CORE] don't submit stage until its dependencies map outputs are registered
Track pending tasks by partition ID instead of Task objects.

Before this change, failure & retry could result in a case where a stage got submitted before the map output from its dependencies get registered.  This was due to an error in the condition for registering map outputs.

Author: hushan[胡珊] <hushan@xiaomi.com>
Author: Imran Rashid <irashid@cloudera.com>

Closes #7699 from squito/SPARK-5259.
2015-09-21 14:26:15 -05:00
Meihua Wu 331f0b10f7 [SPARK-9642] [ML] LinearRegression should supported weighted data
In many modeling application, data points are not necessarily sampled with equal probabilities. Linear regression should support weighting which account the over or under sampling.

work in progress.

Author: Meihua Wu <meihuawu@umich.edu>

Closes #8631 from rotationsymmetry/SPARK-9642.
2015-09-21 12:09:00 -07:00
Jacek Laskowski ca9fe540fe [SPARK-10662] [DOCS] Code snippets are not properly formatted in tables
* Backticks are processed properly in Spark Properties table
* Removed unnecessary spaces
* See http://people.apache.org/~pwendell/spark-nightly/spark-master-docs/latest/running-on-yarn.html

Author: Jacek Laskowski <jacek.laskowski@deepsense.io>

Closes #8795 from jaceklaskowski/docs-yarn-formatting.
2015-09-21 19:46:39 +01:00
zsxwing ebbf85f07b [SPARK-7989] [SPARK-10651] [CORE] [TESTS] Increase timeout to fix flaky tests
I noticed only one block manager registered with master in an unsuccessful build (https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.2,label=spark-test/3534/)
```
15/09/16 13:02:30.981 pool-1-thread-1-ScalaTest-running-BroadcastSuite INFO SparkContext: Running Spark version 1.6.0-SNAPSHOT
...
15/09/16 13:02:38.133 sparkDriver-akka.actor.default-dispatcher-19 INFO BlockManagerMasterEndpoint: Registering block manager localhost:48196 with 530.3 MB RAM, BlockManagerId(0, localhost, 48196)
```
In addition, the first block manager needed 7+ seconds to start. But the test expected 2 block managers so it failed.

However, there was no exception in this log file. So I checked a successful build (https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/3536/AMPLAB_JENKINS_BUILD_PROFILE=hadoop2.2,label=spark-test/) and it needed 4-5 seconds to set up the local cluster:
```
15/09/16 18:11:27.738 sparkWorker1-akka.actor.default-dispatcher-5 INFO Worker: Running Spark version 1.6.0-SNAPSHOT
...
15/09/16 18:11:30.838 sparkDriver-akka.actor.default-dispatcher-20 INFO BlockManagerMasterEndpoint: Registering block manager localhost:54202 with 530.3 MB RAM, BlockManagerId(1, localhost, 54202)
15/09/16 18:11:32.112 sparkDriver-akka.actor.default-dispatcher-20 INFO BlockManagerMasterEndpoint: Registering block manager localhost:32955 with 530.3 MB RAM, BlockManagerId(0, localhost, 32955)
```
In this build, the first block manager needed only 3+ seconds to start.

Comparing these two builds, I guess it's possible that the local cluster in `BroadcastSuite` cannot be ready in 10 seconds if the Jenkins worker is busy. So I just increased the timeout to 60 seconds to see if this can fix the issue.

Author: zsxwing <zsxwing@gmail.com>

Closes #8813 from zsxwing/fix-BroadcastSuite.
2015-09-21 11:39:04 -07:00
Holden Karau 20a61dbd9b [SPARK-10626] [MLLIB] create java friendly method for random rdd
SPARK-3136 added a large number of functions for creating Java RandomRDDs, but for people that want to use custom RandomDataGenerators we should make a Java friendly method.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8782 from holdenk/SPARK-10626-create-java-friendly-method-for-randomRDD.
2015-09-21 18:53:28 +01:00
vinodkc 0144039517 [SPARK-10631] [DOCUMENTATION, MLLIB, PYSPARK] Added documentation for few APIs
There are some missing API docs in pyspark.mllib.linalg.Vector (including DenseVector and SparseVector). We should add them based on their Scala counterparts.

Author: vinodkc <vinod.kc.in@gmail.com>

Closes #8834 from vinodkc/fix_SPARK-10631.
2015-09-20 22:55:24 -07:00
lewuathe 0c498717ba [SPARK-10715] [ML] Duplicate initialization flag in WeightedLeastSquare
There are duplicate set of initialization flag in `WeightedLeastSquares#add`.
`initialized` is already set in `init(Int)`.

Author: lewuathe <lewuathe@me.com>

Closes #8837 from Lewuathe/duplicate-initialization-flag.
2015-09-20 16:16:31 -07:00
Sean Owen 1aa9e50256 [SPARK-5905] [MLLIB] Note requirements for certain RowMatrix methods in docs
Note methods that fail for cols > 65535; note that SVD does not require n >= m
CC mengxr

Author: Sean Owen <sowen@cloudera.com>

Closes #8839 from srowen/SPARK-5905.
2015-09-20 16:05:12 -07:00
Josh Rosen 2117eea71e [SPARK-10710] Remove ability to disable spilling in core and SQL
It does not make much sense to set `spark.shuffle.spill` or `spark.sql.planner.externalSort` to false: I believe that these configurations were initially added as "escape hatches" to guard against bugs in the external operators, but these operators are now mature and well-tested. In addition, these configurations are not handled in a consistent way anymore: SQL's Tungsten codepath ignores these configurations and will continue to use spilling operators. Similarly, Spark Core's `tungsten-sort` shuffle manager does not respect `spark.shuffle.spill=false`.

This pull request removes these configurations, adds warnings at the appropriate places, and deletes a large amount of code which was only used in code paths that did not support spilling.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8831 from JoshRosen/remove-ability-to-disable-spilling.
2015-09-19 21:40:21 -07:00
zsxwing e789000b88 [SPARK-10155] [SQL] Change SqlParser to object to avoid memory leak
Since `scala.util.parsing.combinator.Parsers` is thread-safe since Scala 2.10 (See [SI-4929](https://issues.scala-lang.org/browse/SI-4929)), we can change SqlParser to object to avoid memory leak.

I didn't change other subclasses of `scala.util.parsing.combinator.Parsers` because there is only one instance in one SQLContext, which should not be an issue.

Author: zsxwing <zsxwing@gmail.com>

Closes #8357 from zsxwing/sql-memory-leak.
2015-09-19 18:22:43 -07:00
Alexis Seigneurin d83b6aae8b Fixed links to the API
Submitting this change on the master branch as requested in https://github.com/apache/spark/pull/8819#issuecomment-141505941

Author: Alexis Seigneurin <alexis.seigneurin@gmail.com>

Closes #8838 from aseigneurin/patch-2.
2015-09-19 12:01:22 +01:00
Kousuke Saruta d507f9c0b7 [SPARK-10584] [SQL] [DOC] Documentation about the compatible Hive version is wrong.
In Spark 1.5.0, Spark SQL is compatible with Hive 0.12.0 through 1.2.1 but the documentation is wrong.

/CC yhuai

Author: Kousuke Saruta <sarutak@oss.nttdata.co.jp>

Closes #8776 from sarutak/SPARK-10584-2.
2015-09-19 01:59:36 -07:00
Andrew Or 7ff8d68cc1 [SPARK-10474] [SQL] Aggregation fails to allocate memory for pointer array
When `TungstenAggregation` hits memory pressure, it switches from hash-based to sort-based aggregation in-place. However, in the process we try to allocate the pointer array for writing to the new `UnsafeExternalSorter` *before* actually freeing the memory from the hash map. This lead to the following exception:
```
 java.io.IOException: Could not acquire 65536 bytes of memory
        at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.initializeForWriting(UnsafeExternalSorter.java:169)
        at org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter.spill(UnsafeExternalSorter.java:220)
        at org.apache.spark.sql.execution.UnsafeKVExternalSorter.<init>(UnsafeKVExternalSorter.java:126)
        at org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap.destructAndCreateExternalSorter(UnsafeFixedWidthAggregationMap.java:257)
        at org.apache.spark.sql.execution.aggregate.TungstenAggregationIterator.switchToSortBasedAggregation(TungstenAggregationIterator.scala:435)
```

Author: Andrew Or <andrew@databricks.com>

Closes #8827 from andrewor14/allocate-pointer-array.
2015-09-18 23:58:25 -07:00
Cheng Lian 22be2ae147 [SPARK-10623] [SQL] Fixes ORC predicate push-down
When pushing down a leaf predicate, ORC `SearchArgument` builder requires an extra "parent" predicate (any one among `AND`/`OR`/`NOT`) to wrap the leaf predicate. E.g., to push down `a < 1`, we must build `AND(a < 1)` instead. Fortunately, when actually constructing the `SearchArgument`, the builder will eliminate all those unnecessary wrappers.

This PR is based on #8783 authored by zhzhan. I also took the chance to simply `OrcFilters` a little bit to improve readability.

Author: Cheng Lian <lian@databricks.com>

Closes #8799 from liancheng/spark-10623/fix-orc-ppd.
2015-09-18 18:42:20 -07:00
Eric Liang c8149ef2c5 [MINOR] [ML] override toString of AttributeGroup
This makes equality test failures much more readable.

mengxr

Author: Eric Liang <ekl@databricks.com>
Author: Eric Liang <ekhliang@gmail.com>

Closes #8826 from ericl/attrgroupstr.
2015-09-18 16:23:05 -07:00
Mingyu Kim 8074208fa4 [SPARK-10611] Clone Configuration for each task for NewHadoopRDD
This patch attempts to fix the Hadoop Configuration thread safety issue for NewHadoopRDD in the same way SPARK-2546 fixed the issue for HadoopRDD.

Author: Mingyu Kim <mkim@palantir.com>

Closes #8763 from mingyukim/mkim/SPARK-10611.
2015-09-18 15:40:58 -07:00
Reynold Xin 348d7c9a93 [SPARK-9808] Remove hash shuffle file consolidation.
Author: Reynold Xin <rxin@databricks.com>

Closes #8812 from rxin/SPARK-9808-1.
2015-09-18 13:48:41 -07:00
Holden Karau 3a22b1004f [SPARK-10449] [SQL] Don't merge decimal types with incompatable precision or scales
From JIRA: Schema merging should only handle struct fields. But currently we also reconcile decimal precision and scale information.

Author: Holden Karau <holden@pigscanfly.ca>

Closes #8634 from holdenk/SPARK-10449-dont-merge-different-precision.
2015-09-18 13:47:14 -07:00
Yijie Shen c6f8135ee5 [SPARK-10539] [SQL] Project should not be pushed down through Intersect or Except #8742
Intersect and Except are both set operators and they use the all the columns to compare equality between rows. When pushing their Project parent down, the relations they based on would change, therefore not an equivalent transformation.

JIRA: https://issues.apache.org/jira/browse/SPARK-10539

I added some comments based on the fix of https://github.com/apache/spark/pull/8742.

Author: Yijie Shen <henry.yijieshen@gmail.com>
Author: Yin Huai <yhuai@databricks.com>

Closes #8823 from yhuai/fix_set_optimization.
2015-09-18 13:20:13 -07:00
Cheng Lian 00a2911c5b [SPARK-10540] Fixes flaky all-data-type test
This PR breaks the original test case into multiple ones (one test case for each data type). In this way, test failure output can be much more readable.

Within each test case, we build a table with two columns, one of them is for the data type to test, the other is an "index" column, which is used to sort the DataFrame and workaround [SPARK-10591] [1]

[1]: https://issues.apache.org/jira/browse/SPARK-10591

Author: Cheng Lian <lian@databricks.com>

Closes #8768 from liancheng/spark-10540/test-all-data-types.
2015-09-18 12:19:08 -07:00
Yanbo Liang 35e8ab9390 [SPARK-10615] [PYSPARK] change assertEquals to assertEqual
As ```assertEquals``` is deprecated, so we need to change ```assertEquals``` to ```assertEqual``` for existing python unit tests.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #8814 from yanboliang/spark-10615.
2015-09-18 09:53:52 -07:00
Yash Datta 20fd35dfd1 [SPARK-10451] [SQL] Prevent unnecessary serializations in InMemoryColumnarTableScan
Many of the fields in InMemoryColumnar scan and InMemoryRelation can be made transient.

This  reduces my 1000ms job to abt 700 ms . The task size reduces from 2.8 mb to ~1300kb

Author: Yash Datta <Yash.Datta@guavus.com>

Closes #8604 from saucam/serde.
2015-09-18 08:22:38 -07:00
navis.ryu e3b5d6cb29 [SPARK-10684] [SQL] StructType.interpretedOrdering need not to be serialized
Kryo fails with buffer overflow even with max value (2G).

{noformat}
org.apache.spark.SparkException: Kryo serialization failed: Buffer overflow. Available: 0, required: 1
Serialization trace:
containsChild (org.apache.spark.sql.catalyst.expressions.BoundReference)
child (org.apache.spark.sql.catalyst.expressions.SortOrder)
array (scala.collection.mutable.ArraySeq)
ordering (org.apache.spark.sql.catalyst.expressions.InterpretedOrdering)
interpretedOrdering (org.apache.spark.sql.types.StructType)
schema (org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema). To avoid this, increase spark.kryoserializer.buffer.max value.
        at org.apache.spark.serializer.KryoSerializerInstance.serialize(KryoSerializer.scala:263)
        at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:240)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
        at java.lang.Thread.run(Thread.java:745)
{noformat}

Author: navis.ryu <navis@apache.org>

Closes #8808 from navis/SPARK-10684.
2015-09-18 00:43:02 -07:00
Reynold Xin 74d8f7dda8 Added <code> tag to documentation. 2015-09-17 22:46:13 -07:00
Felix Bechstein 9a56dcdf7f docs/running-on-mesos.md: state default values in default column
This PR simply uses the default value column for defaults.

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

Closes #8810 from felixb/fix_mesos_doc.
2015-09-17 22:42:46 -07:00
linweizhong 93c7650ab6 [SPARK-9522] [SQL] SparkSubmit process can not exit if kill application when HiveThriftServer was starting
When we start HiveThriftServer, we will start SparkContext first, then start HiveServer2, if we kill application while HiveServer2 is starting then SparkContext will stop successfully, but SparkSubmit process can not exit.

Author: linweizhong <linweizhong@huawei.com>

Closes #7853 from Sephiroth-Lin/SPARK-9522.
2015-09-17 22:25:24 -07:00
Reynold Xin d009da2f5c [SPARK-10682] [GRAPHX] Remove Bagel test suites.
Bagel has been deprecated and we haven't done any changes to it. There is no need to run those tests.

This should speed up tests by 1 min.

Author: Reynold Xin <rxin@databricks.com>

Closes #8807 from rxin/SPARK-10682.
2015-09-17 22:05:20 -07:00
Yanbo Liang 98f1ea67da [SPARK-8518] [ML] Log-linear models for survival analysis
[Accelerated Failure Time (AFT) model](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) is the most commonly used and easy to parallel method of survival analysis for censored survival data. It is the log-linear model based on the Weibull distribution of the survival time.
Users can refer to the R function [```survreg```](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/survreg.html) to compare the model and [```predict```](https://stat.ethz.ch/R-manual/R-devel/library/survival/html/predict.survreg.html) to compare the prediction. There are different kinds of model prediction, I have just select the type ```response``` which is default used for R.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #8611 from yanboliang/spark-8518.
2015-09-17 21:37:10 -07:00
Marcelo Vanzin 0f5ef6dfa6 [SPARK-10674] [TESTS] Increase timeouts in SaslIntegrationSuite.
1s seems to trigger too many times on the jenkins build boxes, so
increase the timeout and cross fingers.

Author: Marcelo Vanzin <vanzin@cloudera.com>

Closes #8802 from vanzin/SPARK-10674 and squashes the following commits:

3c93117 [Marcelo Vanzin] Use java 7 syntax.
d667d1b [Marcelo Vanzin] [SPARK-10674] [tests] Increase timeouts in SaslIntegrationSuite.
2015-09-17 19:16:34 -07:00
Eric Liang 4fbf332869 [SPARK-9698] [ML] Add RInteraction transformer for supporting R-style feature interactions
This is a pre-req for supporting the ":" operator in the RFormula feature transformer.

Design doc from umbrella task: https://docs.google.com/document/d/10NZNSEurN2EdWM31uFYsgayIPfCFHiuIu3pCWrUmP_c/edit

mengxr

Author: Eric Liang <ekl@databricks.com>

Closes #7987 from ericl/interaction.
2015-09-17 14:09:06 -07:00
Josh Rosen f1c911552c [SPARK-10657] Remove SCP-based Jenkins log archiving
As of https://issues.apache.org/jira/browse/SPARK-7561, we no longer need to use our custom SCP-based mechanism for archiving Jenkins logs on the master machine; this has been superseded by the use of a Jenkins plugin which archives the logs and provides public links to view them.

Per shaneknapp, we should remove this log syncing mechanism if it is no longer necessary; removing the need to SCP from the Jenkins workers to the masters is a desired step as part of some larger Jenkins infra refactoring.

Author: Josh Rosen <joshrosen@databricks.com>

Closes #8793 from JoshRosen/remove-jenkins-ssh-to-master.
2015-09-17 11:40:24 -07:00
Yanbo Liang 64743870f2 [SPARK-10394] [ML] Make GBTParams use shared stepSize
```GBTParams``` has ```stepSize``` as learning rate currently.
ML has shared param class ```HasStepSize```, ```GBTParams``` can extend from it rather than duplicated implementation.

Author: Yanbo Liang <ybliang8@gmail.com>

Closes #8552 from yanboliang/spark-10394.
2015-09-17 11:24:38 -07:00
Yin Huai aad644fbe2 [SPARK-10639] [SQL] Need to convert UDAF's result from scala to sql type
https://issues.apache.org/jira/browse/SPARK-10639

Author: Yin Huai <yhuai@databricks.com>

Closes #8788 from yhuai/udafConversion.
2015-09-17 11:14:52 -07:00
Michael Armbrust e0dc2bc232 [SPARK-10650] Clean before building docs
The [published docs for 1.5.0](http://spark.apache.org/docs/1.5.0/api/java/org/apache/spark/streaming/) have a bunch of test classes in them.  The only way I can reproduce this is to `test:compile` before running `unidoc`.  To prevent this from happening again, I've added a clean before doc generation.

Author: Michael Armbrust <michael@databricks.com>

Closes #8787 from marmbrus/testsInDocs.
2015-09-17 11:05:30 -07:00
Jeff Zhang 36d8b278d8 [SPARK-10531] [CORE] AppId is set as AppName in status rest api
Verify it manually.

Author: Jeff Zhang <zjffdu@apache.org>

Closes #8688 from zjffdu/SPARK-10531.
2015-09-17 10:25:18 -07:00