- Jul 18, 2015
-
-
Paweł Kozikowski authored
Previous seed resulted in empty test data set. Author: Paweł Kozikowski <mupakoz@gmail.com> Closes #7477 from mupakoz/patch-1 and squashes the following commits: f5d41ee [Paweł Kozikowski] Mllib Naive Bayes example data set enlarged
-
- Jul 17, 2015
-
-
Rekha Joshi authored
Implement IntArrayParam in mllib Author: Rekha Joshi <rekhajoshm@gmail.com> Author: Joshi <rekhajoshm@gmail.com> Closes #7481 from rekhajoshm/SPARK-9118 and squashes the following commits: d3b1766 [Joshi] Implement IntArrayParam 0be142d [Rekha Joshi] Merge pull request #3 from apache/master 106fd8e [Rekha Joshi] Merge pull request #2 from apache/master e3677c9 [Rekha Joshi] Merge pull request #1 from apache/master
-
Yu ISHIKAWA authored
I Implemented the KMeans API for spark.ml Pipelines. But it doesn't include clustering abstractions for spark.ml (SPARK-7610). It would fit for another issues. And I'll try it later, since we are trying to add the hierarchical clustering algorithms in another issue. Thanks. [SPARK-7879] KMeans API for spark.ml Pipelines - ASF JIRA https://issues.apache.org/jira/browse/SPARK-7879 Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #6756 from yu-iskw/SPARK-7879 and squashes the following commits: be752de [Yu ISHIKAWA] Add assertions a14939b [Yu ISHIKAWA] Fix the dashed line's length in pyspark.ml.rst 4c61693 [Yu ISHIKAWA] Remove the test about whether "features" and "prediction" columns exist or not in Python fb2417c [Yu ISHIKAWA] Use getInt, instead of get f397be4 [Yu ISHIKAWA] Switch the comparisons. ca78b7d [Yu ISHIKAWA] Add the Scala docs about the constraints of each parameter. effc650 [Yu ISHIKAWA] Using expertSetParam and expertGetParam c8dc6e6 [Yu ISHIKAWA] Remove an unnecessary test 19a9d63 [Yu ISHIKAWA] Include spark.ml.clustering to python tests 1abb19c [Yu ISHIKAWA] Add the statements about spark.ml.clustering into pyspark.ml.rst f8338bc [Yu ISHIKAWA] Add the placeholders in Python 4a03003 [Yu ISHIKAWA] Test for contains in Python 6566c8b [Yu ISHIKAWA] Use `get`, instead of `apply` 288e8d5 [Yu ISHIKAWA] Using `contains` to check the column names 5a7d574 [Yu ISHIKAWA] Renamce `validateInitializationMode` to `validateInitMode` and remove throwing exception 97cfae3 [Yu ISHIKAWA] Fix the type of return value of `KMeans.copy` e933723 [Yu ISHIKAWA] Remove the default value of seed from the Model class 978ee2c [Yu ISHIKAWA] Modify the docs of KMeans, according to mllib's KMeans 2ec80bc [Yu ISHIKAWA] Fit on 1 line e186be1 [Yu ISHIKAWA] Make a few variables, setters and getters be expert ones b2c205c [Yu ISHIKAWA] Rename the method `getInitializationSteps` to `getInitSteps` and `setInitializationSteps` to `setInitSteps` in Scala and Python f43f5b4 [Yu ISHIKAWA] Rename the method `getInitializationMode` to `getInitMode` and `setInitializationMode` to `setInitMode` in Scala and Python 3cb5ba4 [Yu ISHIKAWA] Modify the description about epsilon and the validation 4fa409b [Yu ISHIKAWA] Add a comment about the default value of epsilon 2f392e1 [Yu ISHIKAWA] Make some variables `final` and Use `IntParam` and `DoubleParam` 19326f8 [Yu ISHIKAWA] Use `udf`, instead of callUDF 4d2ad1e [Yu ISHIKAWA] Modify the indentations 0ae422f [Yu ISHIKAWA] Add a test for `setParams` 4ff7913 [Yu ISHIKAWA] Add "ml.clustering" to `javacOptions` in SparkBuild.scala 11ffdf1 [Yu ISHIKAWA] Use `===` and the variable 220a176 [Yu ISHIKAWA] Set a random seed in the unit testing 92c3efc [Yu ISHIKAWA] Make the points for a test be fewer c758692 [Yu ISHIKAWA] Modify the parameters of KMeans in Python 6aca147 [Yu ISHIKAWA] Add some unit testings to validate the setter methods 687cacc [Yu ISHIKAWA] Alias mllib.KMeans as MLlibKMeans in KMeansSuite.scala a4dfbef [Yu ISHIKAWA] Modify the last brace and indentations 5bedc51 [Yu ISHIKAWA] Remve an extra new line 444c289 [Yu ISHIKAWA] Add the validation for `runs` e41989c [Yu ISHIKAWA] Modify how to validate `initStep` 7ea133a [Yu ISHIKAWA] Change how to validate `initMode` 7991e15 [Yu ISHIKAWA] Add a validation for `k` c2df35d [Yu ISHIKAWA] Make `predict` private 93aa2ff [Yu ISHIKAWA] Use `withColumn` in `transform` d3a79f7 [Yu ISHIKAWA] Remove the inhefited docs e9532e1 [Yu ISHIKAWA] make `parentModel` of KMeansModel private 8559772 [Yu ISHIKAWA] Remove the `paramMap` parameter of KMeans 6684850 [Yu ISHIKAWA] Rename `initializationSteps` to `initSteps` 99b1b96 [Yu ISHIKAWA] Rename `initializationMode` to `initMode` 79ea82b [Yu ISHIKAWA] Modify the parameters of KMeans docs 6569bcd [Yu ISHIKAWA] Change how to set the default values with `setDefault` 20a795a [Yu ISHIKAWA] Change how to set the default values with `setDefault` 11c2a12 [Yu ISHIKAWA] Limit the imports badb481 [Yu ISHIKAWA] Alias spark.mllib.{KMeans, KMeansModel} f80319a [Yu ISHIKAWA] Rebase mater branch and add copy methods 85d92b1 [Yu ISHIKAWA] Add `KMeans.setPredictionCol` aa9469d [Yu ISHIKAWA] Fix a python test suite error caused by python 3.x c2d6bcb [Yu ISHIKAWA] ADD Java test suites of the KMeans API for spark.ml Pipeline 598ed2e [Yu ISHIKAWA] Implement the KMeans API for spark.ml Pipelines in Python 63ad785 [Yu ISHIKAWA] Implement the KMeans API for spark.ml Pipelines in Scala
-
Yijie Shen authored
JIRA: https://issues.apache.org/jira/browse/SPARK-8280 https://issues.apache.org/jira/browse/SPARK-8281 Author: Yijie Shen <henry.yijieshen@gmail.com> Closes #7451 from yijieshen/nan_null2 and squashes the following commits: 47a529d [Yijie Shen] style fix 63dee44 [Yijie Shen] handle log expressions similar to Hive 188be51 [Yijie Shen] null to nan in Math Expression
-
Daoyuan Wang authored
When the `condition` extracted by `ExtractEquiJoinKeys` contain join Predicate for left semi join, we can not plan it as semiJoin. Such as SELECT * FROM testData2 x LEFT SEMI JOIN testData2 y ON x.b = y.b AND x.a >= y.a + 2 Condition `x.a >= y.a + 2` can not evaluate on table `x`, so it throw errors Author: Daoyuan Wang <daoyuan.wang@intel.com> Closes #5643 from adrian-wang/spark7026 and squashes the following commits: cc09809 [Daoyuan Wang] refactor semijoin and add plan test 575a7c8 [Daoyuan Wang] fix notserializable 27841de [Daoyuan Wang] fix rebase 10bf124 [Daoyuan Wang] fix style 72baa02 [Daoyuan Wang] fix style 8e0afca [Daoyuan Wang] merge commits for rebase
-
Tathagata Das authored
Current Kinesis unit tests do not test createStream by sending data. This PR is to add such unit test. Note that this unit will not run by default. It will only run when the relevant environment variables are set. Author: Tathagata Das <tathagata.das1565@gmail.com> Closes #7413 from tdas/kinesis-tests and squashes the following commits: 0e16db5 [Tathagata Das] Added more comments regarding testOrIgnore 1ea5ce0 [Tathagata Das] Added more comments c7caef7 [Tathagata Das] Address comments a297b59 [Tathagata Das] Reverted unnecessary change in KafkaStreamSuite 90c9bde [Tathagata Das] Removed scalatest.FunSuite deb7f4f [Tathagata Das] Removed scalatest.FunSuite 18c2208 [Tathagata Das] Changed how SparkFunSuite is inherited dbb33a5 [Tathagata Das] Added license 88f6dab [Tathagata Das] Added scala docs c6be0d7 [Tathagata Das] minor changes 24a992b [Tathagata Das] Moved KinesisTestUtils to src instead of test for future python usage 465b55d [Tathagata Das] Made unit tests optional in a nice way 4d70703 [Tathagata Das] Added license 129d436 [Tathagata Das] Minor updates cc36510 [Tathagata Das] Added KinesisStreamSuite
-
Wenchen Fan authored
Author: Wenchen Fan <cloud0fan@outlook.com> Closes #7452 from cloud-fan/boolean-simplify and squashes the following commits: 2a6e692 [Wenchen Fan] fix style d3cfd26 [Wenchen Fan] fix BooleanSimplification in case-insensitive
-
Wenchen Fan authored
The check was unreachable before, as `case operator: LogicalPlan` catches everything already. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #7449 from cloud-fan/tmp and squashes the following commits: 2bb6637 [Wenchen Fan] add test 5493aea [Wenchen Fan] add the check back 27221a7 [Wenchen Fan] remove unnecessary analysis check code for self join
-
Yijie Shen authored
JIRA: https://issues.apache.org/jira/browse/SPARK-9080 cc rxin Author: Yijie Shen <henry.yijieshen@gmail.com> Closes #7464 from yijieshen/isNaN and squashes the following commits: 11ae039 [Yijie Shen] add isNaN in functions 666718e [Yijie Shen] add isNaN predicate expression
-
Reynold Xin authored
Just a small change to add Product type to the base expression/plan abstract classes, based on suggestions on #7434 and offline discussions. Author: Reynold Xin <rxin@databricks.com> Closes #7479 from rxin/remove-self-types and squashes the following commits: e407ffd [Reynold Xin] [SPARK-9142][SQL] Removing unnecessary self types in Catalyst.
-
Joshi authored
This makes sure attempts are listed in the order they were executed, and that the app's state matches the state of the most current attempt. Author: Joshi <rekhajoshm@gmail.com> Author: Rekha Joshi <rekhajoshm@gmail.com> Closes #7253 from rekhajoshm/SPARK-8593 and squashes the following commits: 874dd80 [Joshi] History Server: updated order for multiple attempts(logcleaner) 716e0b1 [Joshi] History Server: updated order for multiple attempts(descending start time works everytime) 548c753 [Joshi] History Server: updated order for multiple attempts(descending start time works everytime) 83306a8 [Joshi] History Server: updated order for multiple attempts(descending start time) b0fc922 [Joshi] History Server: updated order for multiple attempts(updated comment) cc0fda7 [Joshi] History Server: updated order for multiple attempts(updated test) 304cb0b [Joshi] History Server: updated order for multiple attempts(reverted HistoryPage) 85024e8 [Joshi] History Server: updated order for multiple attempts a41ac4b [Joshi] History Server: updated order for multiple attempts ab65fa1 [Joshi] History Server: some attempt completed to work with showIncomplete 0be142d [Rekha Joshi] Merge pull request #3 from apache/master 106fd8e [Rekha Joshi] Merge pull request #2 from apache/master e3677c9 [Rekha Joshi] Merge pull request #1 from apache/master
-
Bryan Cutler authored
Broadcast of ensemble models in transformImpl before call to predict Author: Bryan Cutler <bjcutler@us.ibm.com> Closes #6300 from BryanCutler/bcast-ensemble-models-7127 and squashes the following commits: 86e73de [Bryan Cutler] [SPARK-7127] Replaced deprecated callUDF with udf 40a139d [Bryan Cutler] Merge branch 'master' into bcast-ensemble-models-7127 9afad56 [Bryan Cutler] [SPARK-7127] Simplified calls by overriding transformImpl and using broadcasted model in callUDF to make prediction 1f34be4 [Bryan Cutler] [SPARK-7127] Removed accidental newline 171a6ce [Bryan Cutler] [SPARK-7127] Used modelAccessor parameter in predictImpl to access broadcasted model 6fd153c [Bryan Cutler] [SPARK-7127] Applied broadcasting to remaining ensemble models aaad77b [Bryan Cutler] [SPARK-7127] Removed abstract class for broadcasting model, instead passing a prediction function as param to transform 83904bb [Bryan Cutler] [SPARK-7127] Adding broadcast of model before prediction in RandomForestClassifier
-
Yanbo Liang authored
Add Python API for PCA transformer Author: Yanbo Liang <ybliang8@gmail.com> Closes #7190 from yanboliang/spark-8792 and squashes the following commits: 8f4ac31 [Yanbo Liang] address comments 8a79cc0 [Yanbo Liang] Add Python API for PCA transformer
-
Feynman Liang authored
[SPARK-9090] [ML] Fix definition of residual in LinearRegressionSummary, EnsembleTestHelper, and SquaredError Make the definition of residuals in Spark consistent with literature. We have been using `prediction - label` for residuals, but literature usually defines `residual = label - prediction`. Author: Feynman Liang <fliang@databricks.com> Closes #7435 from feynmanliang/SPARK-9090-Fix-LinearRegressionSummary-Residuals and squashes the following commits: f4b39d8 [Feynman Liang] Fix doc bc12a92 [Feynman Liang] Tweak EnsembleTestHelper and SquaredError residuals 63f0d60 [Feynman Liang] Fix definition of residual
-
zsxwing authored
This is an alternative way to fix `SPARK-5681`. It minimizes the changes. Closes #4467 Author: zsxwing <zsxwing@gmail.com> Author: Liang-Chi Hsieh <viirya@gmail.com> Closes #6294 from zsxwing/pr4467 and squashes the following commits: 709ac1f [zsxwing] Fix the comment e103e8a [zsxwing] Move ReceiverTracker.stop into ReceiverTracker.stop f637142 [zsxwing] Address minor code style comments a178d37 [zsxwing] Move 'stopReceivers' to the event looop to resolve the race condition 51fb07e [zsxwing] Fix the code style 3cb19a3 [zsxwing] Merge branch 'master' into pr4467 b4c29e7 [zsxwing] Stop receiver only if we start it c41ee94 [zsxwing] Make stopReceivers private 7c73c1f [zsxwing] Use trackerStateLock to protect trackerState a8120c0 [zsxwing] Merge branch 'master' into pr4467 7b1d9af [zsxwing] "case Throwable" => "case NonFatal" 15ed4a1 [zsxwing] Register before starting the receiver fff63f9 [zsxwing] Use a lock to eliminate the race condition when stopping receivers and registering receivers happen at the same time. e0ef72a [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into tracker_status_timeout 19b76d9 [Liang-Chi Hsieh] Remove timeout. 34c18dc [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into tracker_status_timeout c419677 [Liang-Chi Hsieh] Fix style. 9e1a760 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into tracker_status_timeout 355f9ce [Liang-Chi Hsieh] Separate register and start events for receivers. 3d568e8 [Liang-Chi Hsieh] Let receivers get registered first before going started. ae0d9fd [Liang-Chi Hsieh] Merge branch 'master' into tracker_status_timeout 77983f3 [Liang-Chi Hsieh] Add tracker status and stop to receive messages when stopping tracker.
-
Wenchen Fan authored
a follow up of https://github.com/apache/spark/pull/7353 1. we should use `Calendar.HOUR_OF_DAY` instead of `Calendar.HOUR`(this is for AM, PM). 2. we should call `c.set(Calendar.MILLISECOND, 0)` after `Calendar.getInstance` I'm not sure why the tests didn't fail in jenkins, but I ran latest spark master branch locally and `DateTimeUtilsSuite` failed. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #7473 from cloud-fan/datetime and squashes the following commits: 66cdaf2 [Wenchen Fan] fix several bugs in DateTimeUtils.stringToTimestamp
-
Yanbo Liang authored
Naive Bayes API for spark.ml Pipelines Author: Yanbo Liang <ybliang8@gmail.com> Closes #7284 from yanboliang/spark-8600 and squashes the following commits: bc890f7 [Yanbo Liang] remove labels valid check c3de687 [Yanbo Liang] remove labels from ml.NaiveBayesModel a2b3088 [Yanbo Liang] address comments 3220b82 [Yanbo Liang] trigger jenkins 3018a41 [Yanbo Liang] address comments 208e166 [Yanbo Liang] Naive Bayes API for spark.ml Pipelines
-
Yuhao Yang authored
jira: https://issues.apache.org/jira/browse/SPARK-9062 Currently output type of Tokenizer is Array(String, false), which is not compatible with Word2Vec and Other transformers since their input type is Array(String, true). Seq[String] in udf will be treated as Array(String, true) by default. I'm not sure what's the recommended way for Tokenizer to handle the null value in the input. Any suggestion will be welcome. Author: Yuhao Yang <hhbyyh@gmail.com> Closes #7414 from hhbyyh/tokenizer and squashes the following commits: c01bd7a [Yuhao Yang] change output type of tokenizer
-
Davies Liu authored
Vectors.dense() should accept numbers directly, like the one in Scala. We already use it in doctests, it worked by luck. cc mengxr jkbradley Author: Davies Liu <davies@databricks.com> Closes #7476 from davies/fix_vectors_dense and squashes the following commits: e0fd292 [Davies Liu] fix Vectors.dense
-
tien-dungle authored
The change here is to keep the cached RDDs in the graph object so that when the graph.unpersist() is called these RDDs are correctly unpersisted. ```java import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD import org.slf4j.LoggerFactory import org.apache.spark.graphx.util.GraphGenerators // Create an RDD for the vertices val users: RDD[(VertexId, (String, String))] = sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")))) // Create an RDD for edges val relationships: RDD[Edge[String]] = sc.parallelize(Array(Edge(3L, 7L, "collab"), Edge(5L, 3L, "advisor"), Edge(2L, 5L, "colleague"), Edge(5L, 7L, "pi"))) // Define a default user in case there are relationship with missing user val defaultUser = ("John Doe", "Missing") // Build the initial Graph val graph = Graph(users, relationships, defaultUser) graph.cache().numEdges graph.unpersist() sc.getPersistentRDDs.foreach( r => println( r._2.toString)) ``` Author: tien-dungle <tien-dung.le@realimpactanalytics.com> Closes #7469 from tien-dungle/SPARK-9109_Graphx-unpersist and squashes the following commits: 8d87997 [tien-dungle] Keep the cached edge in the graph
-
Liang-Chi Hsieh authored
JIRA: https://issues.apache.org/jira/browse/SPARK-8945 Add add and subtract expressions for IntervalType. Author: Liang-Chi Hsieh <viirya@appier.com> This patch had conflicts when merged, resolved by Committer: Reynold Xin <rxin@databricks.com> Closes #7398 from viirya/interval_add_subtract and squashes the following commits: acd1f1e [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into interval_add_subtract 5abae28 [Liang-Chi Hsieh] For comments. 6f5b72e [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into interval_add_subtract dbe3906 [Liang-Chi Hsieh] For comments. 13a2fc5 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into interval_add_subtract 83ec129 [Liang-Chi Hsieh] Remove intervalMethod. acfe1ab [Liang-Chi Hsieh] Fix scala style. d3e9d0e [Liang-Chi Hsieh] Add add and subtract expressions for IntervalType.
-
zhichao.li authored
cc chenghao-intel adrian-wang Author: zhichao.li <zhichao.li@intel.com> Closes #6872 from zhichao-li/conv and squashes the following commits: 6ef3b37 [zhichao.li] add unittest and comments 78d9836 [zhichao.li] polish dataframe api and add unittest e2bace3 [zhichao.li] update to use ImplicitCastInputTypes cbcad3f [zhichao.li] add function conv
-
Wenchen Fan authored
instead of return false, throw exception when check equality between external and internal row is better. Author: Wenchen Fan <cloud0fan@outlook.com> Closes #7460 from cloud-fan/row-compare and squashes the following commits: 8a20911 [Wenchen Fan] improve equals 402daa8 [Wenchen Fan] throw exception when check equality between external and internal row
-
Yanbo Liang authored
fix wrong annotation of RFormula.formula Author: Yanbo Liang <ybliang8@gmail.com> Closes #7470 from yanboliang/RFormula and squashes the following commits: 61f1919 [Yanbo Liang] fix wrong annotation
-
Hari Shreedharan authored
In client side, the flow is SparkSubmit -> SparkContext -> yarn/Client. Since the yarn client only gets a cloned config and the staging dir is set here, it is not really possible to do re-logins in the SparkContext. So, do the initial logins in Spark Submit and do re-logins as we do now in the AM, but the Client behaves like an executor in this specific context and reads the credentials file to update the tokens. This way, even if the streaming context is started up from checkpoint - it is fine since we have logged in from SparkSubmit itself itself. Author: Hari Shreedharan <hshreedharan@apache.org> Closes #7394 from harishreedharan/yarn-client-login and squashes the following commits: 9a2166f [Hari Shreedharan] make it possible to use command line args and config parameters together. de08f57 [Hari Shreedharan] Fix import order. 5c4fa63 [Hari Shreedharan] Add a comment explaining what is being done in YarnClientSchedulerBackend. c872caa [Hari Shreedharan] Fix typo in log message. 2c80540 [Hari Shreedharan] Move token renewal to YarnClientSchedulerBackend. 0c48ac2 [Hari Shreedharan] Remove direct use of ExecutorDelegationTokenUpdater in Client. 26f8bfa [Hari Shreedharan] [SPARK-8851][YARN] In Client mode, make sure the client logs in and updates tokens. 58b1969 [Hari Shreedharan] Simple attempt 1.
-
Davies Liu authored
Added two projections: GenerateUnsafeProjection and FromUnsafeProjection, which could be used to convert UnsafeRow from/to GenericInternalRow. They will re-use the buffer during projection, similar to MutableProjection (without all the interface MutableProjection has). cc rxin JoshRosen Author: Davies Liu <davies@databricks.com> Closes #7437 from davies/unsafe_proj2 and squashes the following commits: dbf538e [Davies Liu] test with all the expression (only for supported types) dc737b2 [Davies Liu] address comment e424520 [Davies Liu] fix scala style 70e231c [Davies Liu] address comments 729138d [Davies Liu] Merge branch 'master' of github.com:apache/spark into unsafe_proj2 5a26373 [Davies Liu] unsafe projections
-
Yu ISHIKAWA authored
[[SPARK-9093] Fix single-quotes strings in SparkR - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-9093) This is the result of lintr at the rivision:01155162 [[SPARK-9093] The result of lintr at 01155162](https://gist.github.com/yu-iskw/8c47acf3202796da4d01) Author: Yu ISHIKAWA <yuu.ishikawa@gmail.com> Closes #7439 from yu-iskw/SPARK-9093 and squashes the following commits: 61c391e [Yu ISHIKAWA] [SPARK-9093][SparkR] Fix single-quotes strings in SparkR
-
Wenchen Fan authored
Currently we will stop project collapse when the lower projection has nondeterministic expressions. However it's overkill sometimes, we should be able to optimize `df.select(Rand(10)).select('a)` to `df.select('a)` Author: Wenchen Fan <cloud0fan@outlook.com> Closes #7445 from cloud-fan/non-deterministic and squashes the following commits: 0deaef6 [Wenchen Fan] Improve project collapse with nondeterministic expressions
-
Reynold Xin authored
-
Xiangrui Meng authored
Measure lower and upper bounds for task time and use them for validation. This PR also implements `Stopwatch.toString`. This suite should finish in less than 1 second. jkbradley pwendell Author: Xiangrui Meng <meng@databricks.com> Closes #7457 from mengxr/SPARK-9126 and squashes the following commits: 4b40faa [Xiangrui Meng] simplify tests 739f5bd [Xiangrui Meng] do not assert on time taken by Thread.sleep()
-
Joseph K. Bradley authored
This PR copies the RandomForest implementation from spark.mllib to spark.ml. Note that this includes the DecisionTree implementation, but not the GradientBoostedTrees one (which will come later). I essentially copied a minimal amount of code to spark.ml, removed the use of bins (and only used splits), and modified code only as much as necessary to get it to compile. The spark.ml implementation still uses some spark.mllib classes (privately), which can be moved in future PRs. This refactoring will be helpful in extending the node representation to include more information, such as class probabilities. Specifically: * Copied code from spark.mllib to spark.ml: * mllib.tree.DecisionTree, mllib.tree.RandomForest copied to ml.tree.impl.RandomForest (main implementation) * NodeIdCache (needed to use splits instead of bins) * TreePoint (use splits instead of bins) * Added ml.tree.LearningNode used in RandomForest training (needed vars) * Removed bins from implementation, and only used splits * Small fix in JavaDecisionTreeRegressorSuite CC: mengxr manishamde codedeft chouqin Author: Joseph K. Bradley <joseph@databricks.com> Closes #7294 from jkbradley/dt-move-impl and squashes the following commits: 48749be [Joseph K. Bradley] cleanups based on code review, mostly style bea9703 [Joseph K. Bradley] scala style fixes. added some scala doc 4e6d2a4 [Joseph K. Bradley] removed unnecessary use of copyValues, setParent for trees 9a4d721 [Joseph K. Bradley] cleanups. removed InfoGainStats from ml, using old one for now. 836e7d4 [Joseph K. Bradley] Fixed test suite failures bd5e063 [Joseph K. Bradley] fixed bucketizing issue 0df3759 [Joseph K. Bradley] Need to remove use of Bucketizer d5224a9 [Joseph K. Bradley] modified tree and forest to use moved impl cc01823 [Joseph K. Bradley] still editing RF to get it to work 19143fb [Joseph K. Bradley] More progress, but not done yet. Rebased with master after 1.4 release.
-
- Jul 16, 2015
-
-
Wenchen Fan authored
Author: Wenchen Fan <cloud0fan@outlook.com> Closes #7291 from cloud-fan/row and squashes the following commits: a11addf [Wenchen Fan] move hashCode back to internal row 2de6180 [Wenchen Fan] making apply() call to get() fbe1b24 [Wenchen Fan] add null check ebdf148 [Wenchen Fan] address comments 25ef087 [Wenchen Fan] remove duplicated equals method for Row
-
zsxwing authored
[SPARK-8857][SPARK-8859][Core]Add an internal flag to Accumulable and send internal accumulator updates to the driver via heartbeats This PR includes the following changes: 1. Remove the thread local `Accumulators.localAccums`. Instead, all Accumulators in the executors will register with its TaskContext. 2. Add an internal flag to Accumulable. For internal Accumulators, their updates will be sent to the driver via heartbeats. Author: zsxwing <zsxwing@gmail.com> Closes #7448 from zsxwing/accumulators and squashes the following commits: c24bc5b [zsxwing] Add comments bd7dcf1 [zsxwing] Add an internal flag to Accumulable and send internal accumulator updates to the driver via heartbeats
-
Andrew Or authored
**Symptom.** If an executor in an application times out, `HeartbeatReceiver` attempts to kill it. After this happens, however, the application never gets an executor back even when there are cluster resources available. **Cause.** The issue is that `sc.killExecutor` automatically assumes that the application wishes to adjust its resource requirements permanently downwards. This is not the intention in `HeartbeatReceiver`, however, which simply wants a replacement for the expired executor. **Fix.** Differentiate between the intention to kill and the intention to replace an executor with a fresh one. More details can be found in the commit message. Author: Andrew Or <andrew@databricks.com> Closes #7107 from andrewor14/heartbeat-no-kill and squashes the following commits: 1cd2cd7 [Andrew Or] Add regression test for SPARK-8119 25a347d [Andrew Or] Reuse more code in scheduler backend 31ebd40 [Andrew Or] Differentiate between kill and replace
-
Timothy Chen authored
Mesos supports framework authentication and role to be set per framework, which the role is used to identify the framework's role which impacts the sharing weight of resource allocation and optional authentication information to allow the framework to be connected to the master. Author: Timothy Chen <tnachen@gmail.com> Closes #4960 from tnachen/mesos_fw_auth and squashes the following commits: 0f9f03e [Timothy Chen] Fix review comments. 8f9488a [Timothy Chen] Fix rebase f7fc2a9 [Timothy Chen] Add mesos role, auth and secret.
-
Lianhui Wang authored
andrewor14 davies vanzin can you take a look at this? thanks Author: Lianhui Wang <lianhuiwang09@gmail.com> Closes #7438 from lianhuiwang/SPARK-8646 and squashes the following commits: cb3f12d [Lianhui Wang] add whitespace 6d874a6 [Lianhui Wang] support pyspark for yarn-client
-
Aaron Davidson authored
Example exception (new part at bottom, clearly demarcated): ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 0.0 failed 1 times, most recent failure: Lost task 0.0 in stage 0.0 (TID 0, localhost): java.lang.RuntimeException: uh-oh! at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38$$anonfun$apply$mcJ$sp$2.apply(DAGSchedulerSuite.scala:880) at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38$$anonfun$apply$mcJ$sp$2.apply(DAGSchedulerSuite.scala:880) at scala.collection.Iterator$$anon$11.next(Iterator.scala:328) at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1640) at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1099) at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1099) at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1777) at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1777) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:63) at org.apache.spark.scheduler.Task.run(Task.scala:70) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:213) 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:744) Driver stacktrace: at org.apache.spark.scheduler.DAGScheduler.org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages(DAGScheduler.scala:1298) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1289) at org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1.apply(DAGScheduler.scala:1288) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:47) at org.apache.spark.scheduler.DAGScheduler.abortStage(DAGScheduler.scala:1288) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:755) at org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1.apply(DAGScheduler.scala:755) at scala.Option.foreach(Option.scala:236) at org.apache.spark.scheduler.DAGScheduler.handleTaskSetFailed(DAGScheduler.scala:755) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:1509) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1470) at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:1459) at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:48) at org.apache.spark.scheduler.DAGScheduler.runJob(DAGScheduler.scala:560) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1744) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1762) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1777) at org.apache.spark.SparkContext.runJob(SparkContext.scala:1791) at org.apache.spark.rdd.RDD.count(RDD.scala:1099) at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply$mcJ$sp(DAGSchedulerSuite.scala:880) at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply(DAGSchedulerSuite.scala:880) at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37$$anonfun$38.apply(DAGSchedulerSuite.scala:880) at org.scalatest.Assertions$class.intercept(Assertions.scala:997) at org.scalatest.FunSuite.intercept(FunSuite.scala:1555) at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply$mcV$sp(DAGSchedulerSuite.scala:879) at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply(DAGSchedulerSuite.scala:878) at org.apache.spark.scheduler.DAGSchedulerSuite$$anonfun$37.apply(DAGSchedulerSuite.scala:878) at org.scalatest.Transformer$$anonfun$apply$1.apply$mcV$sp(Transformer.scala:22) at org.scalatest.OutcomeOf$class.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.FunSuiteLike$$anon$1.apply(FunSuiteLike.scala:166) at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:42) at org.scalatest.FunSuiteLike$class.invokeWithFixture$1(FunSuiteLike.scala:163) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.FunSuiteLike$$anonfun$runTest$1.apply(FunSuiteLike.scala:175) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.FunSuiteLike$class.runTest(FunSuiteLike.scala:175) at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfter$$super$runTest(DAGSchedulerSuite.scala:70) at org.scalatest.BeforeAndAfter$class.runTest(BeforeAndAfter.scala:200) at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(DAGSchedulerSuite.scala:70) at org.scalatest.BeforeAndAfterEach$class.runTest(BeforeAndAfterEach.scala:255) at org.apache.spark.scheduler.DAGSchedulerSuite.runTest(DAGSchedulerSuite.scala:70) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.FunSuiteLike$$anonfun$runTests$1.apply(FunSuiteLike.scala:208) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:413) at org.scalatest.SuperEngine$$anonfun$traverseSubNodes$1$1.apply(Engine.scala:401) at scala.collection.immutable.List.foreach(List.scala:318) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.org$scalatest$SuperEngine$$runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:483) at org.scalatest.FunSuiteLike$class.runTests(FunSuiteLike.scala:208) at org.scalatest.FunSuite.runTests(FunSuite.scala:1555) at org.scalatest.Suite$class.run(Suite.scala:1424) at org.scalatest.FunSuite.org$scalatest$FunSuiteLike$$super$run(FunSuite.scala:1555) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.FunSuiteLike$$anonfun$run$1.apply(FunSuiteLike.scala:212) at org.scalatest.SuperEngine.runImpl(Engine.scala:545) at org.scalatest.FunSuiteLike$class.run(FunSuiteLike.scala:212) at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfter$$super$run(DAGSchedulerSuite.scala:70) at org.scalatest.BeforeAndAfter$class.run(BeforeAndAfter.scala:241) at org.apache.spark.scheduler.DAGSchedulerSuite.org$scalatest$BeforeAndAfterAll$$super$run(DAGSchedulerSuite.scala:70) at org.scalatest.BeforeAndAfterAll$class.liftedTree1$1(BeforeAndAfterAll.scala:257) at org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:256) at org.apache.spark.scheduler.DAGSchedulerSuite.run(DAGSchedulerSuite.scala:70) at org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:462) at org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:671) at sbt.ForkMain$Run$2.call(ForkMain.java:294) at sbt.ForkMain$Run$2.call(ForkMain.java:284) at java.util.concurrent.FutureTask.run(FutureTask.java:262) 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:744) ``` Author: Aaron Davidson <aaron@databricks.com> Closes #7028 from aarondav/stack-trace and squashes the following commits: 4714664 [Aaron Davidson] [SPARK-8644] Include call site in SparkException stack traces thrown by job failures
-
jerryshao authored
Author: jerryshao <saisai.shao@intel.com> Author: Saisai Shao <saisai.shao@intel.com> Closes #5060 from jerryshao/SPARK-6304 and squashes the following commits: 89b01f5 [jerryshao] Update the unit test to add more cases 275d252 [jerryshao] Address the comments 7cc146d [jerryshao] Address the comments 2624723 [jerryshao] Fix rebase conflict 45befaa [Saisai Shao] Update the unit test bbc1c9c [Saisai Shao] Fix checkpointing doesn't retain driver port issue
-
Reynold Xin authored
This builds on #7433 but also removes LeafNode/UnaryNode. These are slightly more complicated to remove. I had to change some abstract classes to traits in order for it to work. The problem with LeafNode/UnaryNode is that they are often mixed in at the end of an Expression, and then the toString function actually gets resolved to the ones defined in TreeNode, rather than in Expression. Author: Reynold Xin <rxin@databricks.com> Closes #7434 from rxin/remove-binary-unary-leaf-node and squashes the following commits: 9e8a4de [Reynold Xin] Generator should not be foldable. 3135a8b [Reynold Xin] SortOrder should not be foldable. 9c589cf [Reynold Xin] Fixed one more test case... 2225331 [Reynold Xin] Aggregate expressions should not be foldable. 16b5c90 [Reynold Xin] [SPARK-9085][SQL] Remove LeafNode, UnaryNode, BinaryNode from TreeNode.
-
Yijie Shen authored
JIRA: https://issues.apache.org/jira/browse/SPARK-6941 Author: Yijie Shen <henry.yijieshen@gmail.com> Closes #7342 from yijieshen/SPARK-6941 and squashes the following commits: f82cbe7 [Yijie Shen] reorder import dd67e40 [Yijie Shen] resolve comments 09518af [Yijie Shen] fix import order in DataframeSuite 0c635d4 [Yijie Shen] make match more specific 9df388d [Yijie Shen] move check into PreWriteCheck 847ab20 [Yijie Shen] Detect insertion error in DataSourceStrategy
-