Skip to content

Commit

Permalink
[SPARK-34415][ML] Randomization in hyperparameter optimization
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

Code in the PR generates random parameters for hyperparameter tuning. A discussion with Sean Owen can be found on the dev mailing list here:

http://apache-spark-developers-list.1001551.n3.nabble.com/Hyperparameter-Optimization-via-Randomization-td30629.html

All code is entirely my own work and I license the work to the project under the project’s open source license.

### Why are the changes needed?

Randomization can be a more effective techinique than a grid search since min/max points can fall between the grid and never be found. Randomisation is not so restricted although the probability of finding minima/maxima is dependent on the number of attempts.

Alice Zheng has an accessible description on how this technique works at https://www.oreilly.com/library/view/evaluating-machine-learning/9781492048756/ch04.html

Although there are Python libraries with more sophisticated techniques, not every Spark developer is using Python.

### Does this PR introduce _any_ user-facing change?

A new class (`ParamRandomBuilder.scala`) and its tests have been created but there is no change to existing code. This class offers an alternative to `ParamGridBuilder` and can be dropped into the code wherever `ParamGridBuilder` appears. Indeed, it extends `ParamGridBuilder` and is completely compatible with  its interface. It merely adds one method that provides a range over which a hyperparameter will be randomly defined.

### How was this patch tested?

Tests `ParamRandomBuilderSuite.scala` and `RandomRangesSuite.scala` were added.

`ParamRandomBuilderSuite` is the analogue of the already existing `ParamGridBuilderSuite` which tests the user-facing interface.

`RandomRangesSuite` uses ScalaCheck to test the random ranges over which hyperparameters are distributed.

Closes apache#31535 from PhillHenry/ParamRandomBuilder.

Authored-by: Phillip Henry <[email protected]>
Signed-off-by: Sean Owen <[email protected]>
  • Loading branch information
PhillHenry authored and srowen committed Feb 27, 2021
1 parent 1aeafb4 commit 397b843
Show file tree
Hide file tree
Showing 10 changed files with 806 additions and 3 deletions.
36 changes: 35 additions & 1 deletion docs/ml-tuning.md
Original file line number Diff line number Diff line change
Expand Up @@ -71,10 +71,44 @@ for multiclass problems, a [`MultilabelClassificationEvaluator`](api/scala/org/a
[`RankingEvaluator`](api/scala/org/apache/spark/ml/evaluation/RankingEvaluator.html) for ranking problems. The default metric used to
choose the best `ParamMap` can be overridden by the `setMetricName` method in each of these evaluators.

To help construct the parameter grid, users can use the [`ParamGridBuilder`](api/scala/org/apache/spark/ml/tuning/ParamGridBuilder.html) utility.
To help construct the parameter grid, users can use the [`ParamGridBuilder`](api/scala/org/apache/spark/ml/tuning/ParamGridBuilder.html) utility (see the *Cross-Validation* section below for an example).
By default, sets of parameters from the parameter grid are evaluated in serial. Parameter evaluation can be done in parallel by setting `parallelism` with a value of 2 or more (a value of 1 will be serial) before running model selection with `CrossValidator` or `TrainValidationSplit`.
The value of `parallelism` should be chosen carefully to maximize parallelism without exceeding cluster resources, and larger values may not always lead to improved performance. Generally speaking, a value up to 10 should be sufficient for most clusters.

Alternatively, users can use the [`ParamRandomBuilder`](api/scala/org/apache/spark/ml/tuning/ParamRandomBuilder.html) utility.
This has the same properties of `ParamGridBuilder` mentioned above, but hyperparameters are chosen at random within a user-defined range.
The mathematical principle behind this is that given enough samples, the probability of at least one sample *not* being near the optimum within a range tends to zero.
Irrespective of machine learning model, the expected number of samples needed to have at least one within 5% of the optimum is about 60.
If this 5% volume lies between the parameters defined in a grid search, it will *never* be found by `ParamGridBuilder`.

<div class="codetabs">

<div data-lang="scala" markdown="1">

Refer to the [`ParamRandomBuilder` Scala docs](api/scala/org/apache/spark/ml/tuning/ParamRandomBuilder.html) for details on the API.

{% include_example scala/org/apache/spark/examples/ml/ModelSelectionViaRandomHyperparametersExample.scala %}
</div>

<div data-lang="java" markdown="1">

Refer to the [`ParamRandomBuilder` Java docs](api/java/org/apache/spark/ml/tuning/ParamRandomBuilder.html) for details on the API.

{% include_example java/org/apache/spark/examples/ml/JavaModelSelectionViaRandomHyperparametersExample.java %}
</div>

<div data-lang="python" markdown="1">

Python users are recommended to look at Python libraries that are specifically for hyperparameter tuning such as Hyperopt.

Refer to the [`ParamRandomBuilder` Java docs](api/python/reference/api/pyspark.ml.tuning.ParamRandomBuilder.html) for details on the API.

{% include_example python/ml/model_selection_random_hyperparameters_example.py %}

</div>

</div>

# Cross-Validation

`CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets. E.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. To evaluate a particular `ParamMap`, `CrossValidator` computes the average evaluation metric for the 3 `Model`s produced by fitting the `Estimator` on the 3 different (training, test) dataset pairs.
Expand Down
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();
}
}
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
}
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 = {
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] {
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)

}
Loading

0 comments on commit 397b843

Please sign in to comment.