aboutsummaryrefslogtreecommitdiff
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-11655][CORE] Fix deadlock in handling of launcher stop().Marcelo Vanzin2015-11-124-13/+39
| | | | | | | | | | | | | | | | The stop() callback was trying to close the launcher connection in the same thread that handles connection data, which ended up causing a deadlock. So avoid that by dispatching the stop() request in its own thread. On top of that, add some exception safety to a few parts of the code, and use "destroyForcibly" from Java 8 if it's available, to force kill the child process. The flip side is that "kill()" may not actually work if running Java 7. Author: Marcelo Vanzin <vanzin@cloudera.com> Closes #9633 from vanzin/SPARK-11655.
* [SPARK-11420] Updating Stddev support via Imperative AggregateJihongMa2015-11-1210-115/+52
| | | | | | | | switched stddev support from DeclarativeAggregate to ImperativeAggregate. Author: JihongMa <linlin200605@gmail.com> Closes #9380 from JihongMA/SPARK-11420.
* [SPARK-10113][SQL] Explicit error message for unsigned Parquet logical typeshyukjinkwon2015-11-122-0/+31
| | | | | | | | Parquet supports some unsigned datatypes. However, Since Spark does not support unsigned datatypes, it needs to emit an exception with a clear message rather then with the one saying illegal datatype. Author: hyukjinkwon <gurwls223@gmail.com> Closes #9646 from HyukjinKwon/SPARK-10113.
* [SPARK-11191][SQL] Looks up temporary function using execution Hive clientCheng Lian2015-11-123-5/+56
| | | | | | | | When looking up Hive temporary functions, we should always use the `SessionState` within the execution Hive client, since temporary functions are registered there. Author: Cheng Lian <lian@databricks.com> Closes #9664 from liancheng/spark-11191.fix-temp-function.
* Fixed error in scaladoc of convertToCanonicalEdgesGaurav Kumar2015-11-121-1/+1
| | | | | | | | The code convertToCanonicalEdges is such that srcIds are smaller than dstIds but the scaladoc suggested otherwise. Have fixed the same. Author: Gaurav Kumar <gauravkumar37@gmail.com> Closes #9666 from gauravkumar37/patch-1.
* [BUILD][MINOR] Remove non-exist yarnStable module in Sbt projectjerryshao2015-11-121-4/+2
| | | | | | | | Remove some old yarn related building codes, please review, thanks a lot. Author: jerryshao <sshao@hortonworks.com> Closes #9625 from jerryshao/remove-old-module.
* [SPARK-11673][SQL] Remove the normal Project physical operator (and keep ↵Reynold Xin2015-11-1227-287/+80
| | | | | | | | | | TungstenProject) Also make full outer join being able to produce UnsafeRows. Author: Reynold Xin <rxin@databricks.com> Closes #9643 from rxin/SPARK-11673.
* [SPARK-11661][SQL] Still pushdown filters returned by unhandledFilters.Yin Huai2015-11-125-24/+71
| | | | | | | | https://issues.apache.org/jira/browse/SPARK-11661 Author: Yin Huai <yhuai@databricks.com> Closes #9634 from yhuai/unhandledFilters.
* [SPARK-11674][ML] add private val after @transient in Word2VecModelXiangrui Meng2015-11-111-1/+1
| | | | | | | | This causes compile failure with Scala 2.11. See https://issues.scala-lang.org/browse/SI-8813. (Jenkins won't test Scala 2.11. I tested compile locally.) JoshRosen Author: Xiangrui Meng <meng@databricks.com> Closes #9644 from mengxr/SPARK-11674.
* [SPARK-11396] [SQL] add native implementation of datetime function ↵Daoyuan Wang2015-11-114-5/+77
| | | | | | | | | | | | to_unix_timestamp `to_unix_timestamp` is the deterministic version of `unix_timestamp`, as it accepts at least one parameters. Since the behavior here is quite similar to `unix_timestamp`, I think the dataframe API is not necessary here. Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #9347 from adrian-wang/to_unix_timestamp.
* [SPARK-11675][SQL] Remove shuffle hash joins.Reynold Xin2015-11-1112-717/+357
| | | | | | Author: Reynold Xin <rxin@databricks.com> Closes #9645 from rxin/SPARK-11675.
* [SPARK-8992][SQL] Add pivot to dataframe apiAndrew Ray2015-11-116-10/+255
| | | | | | | | | | | | | | | | | | | | | | | This adds a pivot method to the dataframe api. Following the lead of cube and rollup this adds a Pivot operator that is translated into an Aggregate by the analyzer. Currently the syntax is like: ~~courseSales.pivot(Seq($"year"), $"course", Seq("dotNET", "Java"), sum($"earnings"))~~ ~~Would we be interested in the following syntax also/alternatively? and~~ courseSales.groupBy($"year").pivot($"course", "dotNET", "Java").agg(sum($"earnings")) //or courseSales.groupBy($"year").pivot($"course").agg(sum($"earnings")) Later we can add it to `SQLParser`, but as Hive doesn't support it we cant add it there, right? ~~Also what would be the suggested Java friendly method signature for this?~~ Author: Andrew Ray <ray.andrew@gmail.com> Closes #7841 from aray/sql-pivot.
* [SPARK-11672][ML] disable spark.ml read/write testsXiangrui Meng2015-11-114-5/+5
| | | | | | | | | | | | Saw several failures on Jenkins, e.g., https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2040/testReport/org.apache.spark.ml.util/JavaDefaultReadWriteSuite/testDefaultReadWrite/. This is the first failure in master build: https://amplab.cs.berkeley.edu/jenkins/job/Spark-Master-SBT/3982/ I cannot reproduce it on local. So temporarily disable the tests and I will look into the issue under the same JIRA. I'm going to merge the PR after Jenkins passes compile. Author: Xiangrui Meng <meng@databricks.com> Closes #9641 from mengxr/SPARK-11672.
* [SPARK-10827] replace volatile with Atomic* in AppClient.scala.Reynold Xin2015-11-111-33/+35
| | | | | | | | This is a followup for #9317 to replace volatile fields with AtomicBoolean and AtomicReference. Author: Reynold Xin <rxin@databricks.com> Closes #9611 from rxin/SPARK-10827.
* [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.