Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,30 @@

package org.apache.spark.mllib.stat.test

import scala.beans.BeanInfo

import org.apache.spark.Logging
import org.apache.spark.annotation.{Experimental, Since}
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.api.java.JavaDStream
import org.apache.spark.streaming.dstream.DStream
import org.apache.spark.util.StatCounter

/**
* Class that represents the group and value of a sample.
*
* @param isExperiment if the sample is of the experiment group.
* @param value numeric value of the observation.
*/
@Since("1.6.0")
@BeanInfo
case class BinarySample @Since("1.6.0") (
@Since("1.6.0") isExperiment: Boolean,
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This can be Int to support multiClass sample data, right?

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This was not part of the design, but I agree it would be nice someday. I'll ping @mengxr since he reviewed the original PRs, but I think we'll keep it as is for now.

@Since("1.6.0") value: Double) {
override def toString: String = {
s"($isExperiment, $value)"
}
}

/**
* :: Experimental ::
* Performs online 2-sample significance testing for a stream of (Boolean, Double) pairs. The
Expand Down Expand Up @@ -97,6 +115,20 @@ class StreamingTest @Since("1.6.0") () extends Logging with Serializable {
testMethod.doTest(pairedSummaries)
}

/**
* Register a [[JavaDStream]] of values for significance testing.
*
* @param data stream of BinarySample(isExperiment,value) pairs where the isExperiment denotes
* group (true = experiment, false = control) and the value is the numerical metric
* to test for significance
* @return stream of significance testing results
*/
@Since("1.6.0")
def registerStream(data: JavaDStream[BinarySample]): JavaDStream[StreamingTestResult] = {
JavaDStream.fromDStream(registerStream(data.dstream.map(sample =>
(sample.isExperiment, sample.value))))
}

/** Drop all batches inside the peace period. */
private[stat] def dropPeacePeriod(
data: DStream[(Boolean, Double)]): DStream[(Boolean, Double)] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods._

import org.apache.spark.{Logging, SparkContext}
import org.apache.spark.annotation.Since
import org.apache.spark.annotation.{DeveloperApi, Since}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.regression.LabeledPoint
Expand Down Expand Up @@ -186,6 +186,7 @@ class GradientBoostedTreesModel @Since("1.2.0") (
object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] {

/**
* :: DeveloperApi ::
* Compute the initial predictions and errors for a dataset for the first
* iteration of gradient boosting.
* @param data: training data.
Expand All @@ -196,6 +197,7 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] {
* corresponding to every sample.
*/
@Since("1.4.0")
@DeveloperApi
def computeInitialPredictionAndError(
data: RDD[LabeledPoint],
initTreeWeight: Double,
Expand All @@ -209,6 +211,7 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] {
}

/**
* :: DeveloperApi ::
* Update a zipped predictionError RDD
* (as obtained with computeInitialPredictionAndError)
* @param data: training data.
Expand All @@ -220,6 +223,7 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] {
* corresponding to each sample.
*/
@Since("1.4.0")
@DeveloperApi
def updatePredictionError(
data: RDD[LabeledPoint],
predictionAndError: RDD[(Double, Double)],
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,9 @@
import java.io.Serializable;

import java.util.Arrays;
import java.util.List;

import org.apache.spark.SparkConf;
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

organize imports

import org.junit.After;
import org.junit.Before;
import org.junit.Test;
Expand All @@ -32,20 +34,36 @@
import org.apache.spark.api.java.JavaSparkContext;
import org.apache.spark.mllib.linalg.Vectors;
import org.apache.spark.mllib.regression.LabeledPoint;
import org.apache.spark.mllib.stat.test.BinarySample;
import org.apache.spark.mllib.stat.test.ChiSqTestResult;
import org.apache.spark.mllib.stat.test.KolmogorovSmirnovTestResult;
import org.apache.spark.mllib.stat.test.StreamingTest;
import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
import static org.apache.spark.streaming.JavaTestUtils.attachTestInputStream;
import static org.apache.spark.streaming.JavaTestUtils.attachTestOutputStream;
import static org.apache.spark.streaming.JavaTestUtils.runStreams;

public class JavaStatisticsSuite implements Serializable {
private transient JavaSparkContext sc;
private transient JavaStreamingContext ssc;

@Before
public void setUp() {
sc = new JavaSparkContext("local", "JavaStatistics");
SparkConf conf = new SparkConf()
.setMaster("local[2]")
.setAppName("JavaStatistics")
.set("spark.streaming.clock", "org.apache.spark.util.ManualClock");
sc = new JavaSparkContext(conf);
ssc = new JavaStreamingContext(sc, new Duration(1000));
ssc.checkpoint("checkpoint");
}

@After
public void tearDown() {
sc.stop();
ssc.stop();
ssc = null;
sc = null;
}

Expand Down Expand Up @@ -76,4 +94,21 @@ public void chiSqTest() {
new LabeledPoint(0.0, Vectors.dense(2.4, 8.1))));
ChiSqTestResult[] testResults = Statistics.chiSqTest(data);
}

@Test
public void StreamingTest() {
List<BinarySample> trainingBatch = Arrays.asList(
new BinarySample(true, 1.0),
new BinarySample(false, 2.0));
JavaDStream<BinarySample> training =
attachTestInputStream(ssc, Arrays.asList(trainingBatch, trainingBatch), 2);
int numBatches = 2;
StreamingTest model = new StreamingTest()
.setWindowSize(0)
.setPeacePeriod(0)
.setTestMethod("welch");
model.registerStream(training);
attachTestOutputStream(training);
runStreams(ssc, numBatches, numBatches);
}
}