-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-34415][ML] Randomization in hyperparameter optimization #31535
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
56 commits
Select commit
Hold shift + click to select a range
606b959
Random integers within a range.
PhillHenry c518e5f
Refactored.
PhillHenry 37f32c2
Random longs.
PhillHenry 77cf678
Better use of type classes.
PhillHenry 8bd0dd7
Checks distribution.
PhillHenry 6c9a918
Refactored.
PhillHenry 46540bc
Merge branch 'master' of https://github.com/apache/spark
PhillHenry 5489de6
Formatting.
PhillHenry 3d1f46c
Linear random doubles added.
PhillHenry 290c815
Refactored.
PhillHenry dadbd54
Added random floats.
PhillHenry f8339d7
Refactored.
PhillHenry 70cdf24
Even more refactoring.
PhillHenry 8148a69
RandomRange tests in a separate class.
PhillHenry 0d9fd66
Merge branch 'master' of https://github.com/apache/spark
PhillHenry ea945c5
Linear numerics supported.
PhillHenry cb4e4b8
Checkstyle.
PhillHenry f558060
Log space.
PhillHenry d0c8eaa
Log space.
PhillHenry affb9e4
Logarithm for any base.
PhillHenry 7808b7c
Logarithm for any base.
PhillHenry b433ab1
Still a problem with double conversion.
PhillHenry f12cf9a
Merge branch 'master' of https://github.com/apache/spark
PhillHenry 844f706
Extreme Long/Int ranges may cause trouble with being converted to a d…
PhillHenry 3d03565
Restored a tag that was making my IntelliJ upset.
PhillHenry 27b323e
Removed test println.
PhillHenry 5e870ea
Merge branch 'master' of https://github.com/apache/spark
PhillHenry ef7bfd7
Commit re. Hyperopt and its ilk.
PhillHenry 0de1ba5
@Since tags added.
PhillHenry 9e7b7bd
Code style.
PhillHenry fd8ac9f
Code style.
PhillHenry e402df5
Code style.
PhillHenry e54a58f
Code style.
PhillHenry b0455a1
Code style.
PhillHenry f641d51
Superfluous parentheses.
PhillHenry 6f6dabd
Merge branch 'master' of https://github.com/apache/spark into ParamRa…
PhillHenry 86a781d
[SPARK-34415][ML] Made private anything that wasn't and that was not …
PhillHenry b805ea5
[SPARK-34415][ML] Oops. The user needs Limits and added log methods.
PhillHenry b02f64e
[SPARK-34415][ML] Oops. The user needs Limits and added log methods.
PhillHenry 44061a4
[SPARK-34415][ML] Oops. Base 10.
PhillHenry 07a1e01
[SPARK-34415][ML] Added Java specific API and tests.
PhillHenry 232d359
Merge branch 'master' of https://github.com/apache/spark into ParamRa…
PhillHenry 2344495
[SPARK-34415][ML] Random Long generated removed as superfluous (per c…
PhillHenry 25737d7
[SPARK-34415][ML] Documentation and Scala example.
PhillHenry 308f1c3
[SPARK-34415][ML] Documentation, Scala and Java examples.
PhillHenry e88f907
[SPARK-34415][ML] Removed random log2 space, fixed error in documenta…
PhillHenry 4e48759
[SPARK-34415][ML] Everything that can be made private as srowen recom…
PhillHenry 4fab7ac
Merge branch 'master' of https://github.com/apache/spark into ParamRa…
PhillHenry 259edfe
[SPARK-34415][ML] ScalaStyle violation.
PhillHenry 62f305c
Merge branch 'master' of https://github.com/apache/spark into ParamRa…
PhillHenry a41c8f3
[SPARK-34415][ML] Very hacky first draft of the Python version of Par…
PhillHenry 73d077b
[SPARK-34415][ML] Added ParamRandomBuilder to the .pyi file. More tes…
PhillHenry 5d89774
[SPARK-34415][ML] Python log10 space.
PhillHenry 183c2cd
[SPARK-34415][ML] More tests.
PhillHenry 73e3b0c
Merge branch 'master' of https://github.com/apache/spark into ParamRa…
PhillHenry ddfe4a9
[SPARK-34415][ML] Python example.
PhillHenry File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
83 changes: 83 additions & 0 deletions
83
.../java/org/apache/spark/examples/ml/JavaModelSelectionViaRandomHyperparametersExample.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,83 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.examples.ml; | ||
|
|
||
| // $example on$ | ||
| import org.apache.spark.ml.evaluation.RegressionEvaluator; | ||
| import org.apache.spark.ml.param.ParamMap; | ||
| import org.apache.spark.ml.regression.LinearRegression; | ||
| import org.apache.spark.ml.tuning.*; | ||
| import org.apache.spark.sql.Dataset; | ||
| import org.apache.spark.sql.Row; | ||
| import org.apache.spark.sql.SparkSession; | ||
| // $example off$ | ||
|
|
||
| /** | ||
| * A simple example demonstrating model selection using ParamRandomBuilder. | ||
| * | ||
| * Run with | ||
| * {{{ | ||
| * bin/run-example ml.JavaModelSelectionViaRandomHyperparametersExample | ||
| * }}} | ||
| */ | ||
| public class JavaModelSelectionViaRandomHyperparametersExample { | ||
|
|
||
| public static void main(String[] args) { | ||
| SparkSession spark = SparkSession | ||
| .builder() | ||
| .appName("JavaModelSelectionViaTrainValidationSplitExample") | ||
| .getOrCreate(); | ||
|
|
||
| // $example on$ | ||
| Dataset<Row> data = spark.read().format("libsvm") | ||
| .load("data/mllib/sample_linear_regression_data.txt"); | ||
|
|
||
| LinearRegression lr = new LinearRegression(); | ||
|
|
||
| // We sample the regularization parameter logarithmically over the range [0.01, 1.0]. | ||
| // This means that values around 0.01, 0.1 and 1.0 are roughly equally likely. | ||
| // Note that both parameters must be greater than zero as otherwise we'll get an infinity. | ||
| // We sample the the ElasticNet mixing parameter uniformly over the range [0, 1] | ||
| // Note that in real life, you'd choose more than the 5 samples we see below. | ||
| ParamMap[] hyperparameters = new ParamRandomBuilder() | ||
| .addLog10Random(lr.regParam(), 0.01, 1.0, 5) | ||
| .addRandom(lr.elasticNetParam(), 0.0, 1.0, 5) | ||
| .addGrid(lr.fitIntercept()) | ||
| .build(); | ||
|
|
||
| System.out.println("hyperparameters:"); | ||
| for (ParamMap param : hyperparameters) { | ||
| System.out.println(param); | ||
| } | ||
|
|
||
| CrossValidator cv = new CrossValidator() | ||
| .setEstimator(lr) | ||
| .setEstimatorParamMaps(hyperparameters) | ||
| .setEvaluator(new RegressionEvaluator()) | ||
| .setNumFolds(3); | ||
| CrossValidatorModel cvModel = cv.fit(data); | ||
| LinearRegression parent = (LinearRegression)cvModel.bestModel().parent(); | ||
|
|
||
| System.out.println("Optimal model has\n" + lr.regParam() + " = " + parent.getRegParam() | ||
| + "\n" + lr.elasticNetParam() + " = "+ parent.getElasticNetParam() | ||
| + "\n" + lr.fitIntercept() + " = " + parent.getFitIntercept()); | ||
| // $example off$ | ||
|
|
||
| spark.stop(); | ||
| } | ||
| } |
79 changes: 79 additions & 0 deletions
79
...in/scala/org/apache/spark/examples/ml/ModelSelectionViaRandomHyperparametersExample.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,79 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.examples.ml | ||
|
|
||
| // $example on$ | ||
| import org.apache.spark.ml.evaluation.RegressionEvaluator | ||
| import org.apache.spark.ml.regression.LinearRegression | ||
| import org.apache.spark.ml.tuning.{CrossValidator, CrossValidatorModel, Limits, ParamRandomBuilder} | ||
| import org.apache.spark.ml.tuning.RandomRanges._ | ||
| // $example off$ | ||
| import org.apache.spark.sql.SparkSession | ||
|
|
||
| /** | ||
| * A simple example demonstrating model selection using ParamRandomBuilder. | ||
| * | ||
| * Run with | ||
| * {{{ | ||
| * bin/run-example ml.ModelSelectionViaRandomHyperparametersExample | ||
| * }}} | ||
| */ | ||
| object ModelSelectionViaRandomHyperparametersExample { | ||
| def main(args: Array[String]): Unit = { | ||
| val spark = SparkSession | ||
| .builder | ||
| .appName("ModelSelectionViaTrainValidationSplitExample") | ||
| .getOrCreate() | ||
| // scalastyle:off println | ||
| // $example on$ | ||
| // Prepare training and test data. | ||
| val data = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") | ||
|
|
||
| val lr = new LinearRegression().setMaxIter(10) | ||
|
|
||
| // We sample the regularization parameter logarithmically over the range [0.01, 1.0]. | ||
| // This means that values around 0.01, 0.1 and 1.0 are roughly equally likely. | ||
| // Note that both parameters must be greater than zero as otherwise we'll get an infinity. | ||
| // We sample the the ElasticNet mixing parameter uniformly over the range [0, 1] | ||
| // Note that in real life, you'd choose more than the 5 samples we see below. | ||
| val hyperparameters = new ParamRandomBuilder() | ||
| .addLog10Random(lr.regParam, Limits(0.01, 1.0), 5) | ||
| .addGrid(lr.fitIntercept) | ||
| .addRandom(lr.elasticNetParam, Limits(0.0, 1.0), 5) | ||
| .build() | ||
|
|
||
| println(s"hyperparameters:\n${hyperparameters.mkString("\n")}") | ||
|
|
||
| val cv: CrossValidator = new CrossValidator() | ||
| .setEstimator(lr) | ||
| .setEstimatorParamMaps(hyperparameters) | ||
| .setEvaluator(new RegressionEvaluator) | ||
| .setNumFolds(3) | ||
| val cvModel: CrossValidatorModel = cv.fit(data) | ||
| val parent: LinearRegression = cvModel.bestModel.parent.asInstanceOf[LinearRegression] | ||
|
|
||
| println(s"""Optimal model has: | ||
| |${lr.regParam} = ${parent.getRegParam} | ||
| |${lr.elasticNetParam} = ${parent.getElasticNetParam} | ||
| |${lr.fitIntercept} = ${parent.getFitIntercept}""".stripMargin) | ||
| // $example off$ | ||
|
|
||
| spark.stop() | ||
| } | ||
| // scalastyle:on println | ||
| } |
160 changes: 160 additions & 0 deletions
160
mllib/src/main/scala/org/apache/spark/ml/tuning/ParamRandomBuilder.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,160 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.ml.tuning | ||
|
|
||
| import org.apache.spark.annotation.Since | ||
| import org.apache.spark.ml.param._ | ||
| import org.apache.spark.ml.tuning.RandomRanges._ | ||
|
|
||
| case class Limits[T: Numeric](x: T, y: T) | ||
|
|
||
| private[ml] abstract class RandomT[T: Numeric] { | ||
| def randomT(): T | ||
| def randomTLog(n: Int): T | ||
| } | ||
|
|
||
| abstract class Generator[T: Numeric] { | ||
| def apply(lim: Limits[T]): RandomT[T] | ||
| } | ||
|
|
||
| object RandomRanges { | ||
|
|
||
| private val rnd = new scala.util.Random | ||
|
|
||
| private[tuning] def randomBigInt0To(x: BigInt): BigInt = { | ||
| var randVal = BigInt(x.bitLength, rnd) | ||
| while (randVal > x) { | ||
| randVal = BigInt(x.bitLength, rnd) | ||
| } | ||
| randVal | ||
| } | ||
|
|
||
| private[ml] def bigIntBetween(lower: BigInt, upper: BigInt): BigInt = { | ||
| val diff: BigInt = upper - lower | ||
| randomBigInt0To(diff) + lower | ||
| } | ||
|
|
||
| private def randomBigDecimalBetween(lower: BigDecimal, upper: BigDecimal): BigDecimal = { | ||
PhillHenry marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| val zeroCenteredRnd: BigDecimal = BigDecimal(rnd.nextDouble() - 0.5) | ||
| val range: BigDecimal = upper - lower | ||
| val halfWay: BigDecimal = lower + range / 2 | ||
| (zeroCenteredRnd * range) + halfWay | ||
| } | ||
|
|
||
| implicit object DoubleGenerator extends Generator[Double] { | ||
PhillHenry marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| def apply(limits: Limits[Double]): RandomT[Double] = new RandomT[Double] { | ||
| import limits._ | ||
| val lower: Double = math.min(x, y) | ||
| val upper: Double = math.max(x, y) | ||
|
|
||
| override def randomTLog(n: Int): Double = | ||
| RandomRanges.randomLog(lower, upper, n) | ||
|
|
||
| override def randomT(): Double = | ||
| randomBigDecimalBetween(BigDecimal(lower), BigDecimal(upper)).doubleValue | ||
| } | ||
| } | ||
|
|
||
| implicit object FloatGenerator extends Generator[Float] { | ||
| def apply(limits: Limits[Float]): RandomT[Float] = new RandomT[Float] { | ||
| import limits._ | ||
| val lower: Float = math.min(x, y) | ||
| val upper: Float = math.max(x, y) | ||
|
|
||
| override def randomTLog(n: Int): Float = | ||
| RandomRanges.randomLog(lower, upper, n).toFloat | ||
|
|
||
| override def randomT(): Float = | ||
| randomBigDecimalBetween(BigDecimal(lower), BigDecimal(upper)).floatValue | ||
| } | ||
| } | ||
|
|
||
| implicit object IntGenerator extends Generator[Int] { | ||
| def apply(limits: Limits[Int]): RandomT[Int] = new RandomT[Int] { | ||
| import limits._ | ||
| val lower: Int = math.min(x, y) | ||
| val upper: Int = math.max(x, y) | ||
|
|
||
| override def randomTLog(n: Int): Int = | ||
| RandomRanges.randomLog(lower, upper, n).toInt | ||
|
|
||
| override def randomT(): Int = | ||
| bigIntBetween(BigInt(lower), BigInt(upper)).intValue | ||
| } | ||
| } | ||
|
|
||
| private[ml] def logN(x: Double, base: Int): Double = math.log(x) / math.log(base) | ||
|
|
||
| private[ml] def randomLog(lower: Double, upper: Double, n: Int): Double = { | ||
| val logLower: Double = logN(lower, n) | ||
| val logUpper: Double = logN(upper, n) | ||
| val logLimits: Limits[Double] = Limits(logLower, logUpper) | ||
| val rndLogged: RandomT[Double] = RandomRanges(logLimits) | ||
| math.pow(n, rndLogged.randomT()) | ||
| } | ||
|
|
||
| private[ml] def apply[T: Generator](lim: Limits[T])(implicit t: Generator[T]): RandomT[T] = t(lim) | ||
|
|
||
| } | ||
|
|
||
| /** | ||
| * "For any distribution over a sample space with a finite maximum, the maximum of 60 random | ||
| * observations lies within the top 5% of the true maximum, with 95% probability" | ||
| * - Evaluating Machine Learning Models by Alice Zheng | ||
| * https://www.oreilly.com/library/view/evaluating-machine-learning/9781492048756/ch04.html | ||
| * | ||
| * Note: if you want more sophisticated hyperparameter tuning, consider Python libraries | ||
| * such as Hyperopt. | ||
| */ | ||
| @Since("3.2.0") | ||
| class ParamRandomBuilder extends ParamGridBuilder { | ||
| def addRandom[T: Generator](param: Param[T], lim: Limits[T], n: Int): this.type = { | ||
| val gen: RandomT[T] = RandomRanges(lim) | ||
| addGrid(param, (1 to n).map { _: Int => gen.randomT() }) | ||
| } | ||
|
|
||
| def addLog10Random[T: Generator](param: Param[T], lim: Limits[T], n: Int): this.type = | ||
| addLogRandom(param, lim, n, 10) | ||
|
|
||
| private def addLogRandom[T: Generator](param: Param[T], lim: Limits[T], | ||
| n: Int, base: Int): this.type = { | ||
| val gen: RandomT[T] = RandomRanges(lim) | ||
| addGrid(param, (1 to n).map { _: Int => gen.randomTLog(base) }) | ||
| } | ||
|
|
||
| // specialized versions for Java. | ||
|
|
||
| def addRandom(param: DoubleParam, x: Double, y: Double, n: Int): this.type = | ||
| addRandom(param, Limits(x, y), n)(DoubleGenerator) | ||
|
|
||
| def addLog10Random(param: DoubleParam, x: Double, y: Double, n: Int): this.type = | ||
| addLogRandom(param, Limits(x, y), n, 10)(DoubleGenerator) | ||
|
|
||
| def addRandom(param: FloatParam, x: Float, y: Float, n: Int): this.type = | ||
| addRandom(param, Limits(x, y), n)(FloatGenerator) | ||
|
|
||
| def addLog10Random(param: FloatParam, x: Float, y: Float, n: Int): this.type = | ||
| addLogRandom(param, Limits(x, y), n, 10)(FloatGenerator) | ||
|
|
||
| def addRandom(param: IntParam, x: Int, y: Int, n: Int): this.type = | ||
| addRandom(param, Limits(x, y), n)(IntGenerator) | ||
|
|
||
| def addLog10Random(param: IntParam, x: Int, y: Int, n: Int): this.type = | ||
| addLogRandom(param, Limits(x, y), n, 10)(IntGenerator) | ||
|
|
||
| } | ||
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.