From e7d31363f5ce1db717d19cb34f2e3f832cc7c8ec Mon Sep 17 00:00:00 2001 From: Erik Erlandson Date: Tue, 15 Sep 2020 09:46:40 -0700 Subject: [PATCH] Toward release 0.5.0 - fast TDigest and Spark-3 Aggregator API (#20) * update isarn-sketches dep to 0.2.2 * test design with thin shim class for new fast TDigest to clean up the API * update initial commands * scope t digest shim class, TDigestAggregator companion obj * bump isarn-sketches to 0.3.0 * example of a java/python binding * modify python tdigest UDT and a test UDF * ScalarNumeric, data type functions, python renaming, commenting out old code * spark 3.0 supports scala 2.12 only * http -> https * TDigestArrayAggregator * array function overloadings * add instructions for cleaning out ivy on local publish * spark vector aggregations * no longer need UDT for tdigest array * old tdigest UDTs are obsolete * remove package object * sketches.spark.tdigest._ * tdigest.scala * TDigestReduceAggregator * TDigestArrayReduceAggregator * TDigestUDAF.scala is obsolete * TDigestArrayReduceAggregator inherit from TDigestArrayAggregatorBase * factor out compression and maxdiscrete from TDigestArrayAggregatorBase * /udaf/ -> /spark/ * /udaf/ -> /spark/ * move python TDigestUDT into spark/tdigest.py * update sbt build mappings for python refactor * update readme python for new organization * copyright * unused imports * more unused imports * switch to fast java TDigest * explicit import of JavaPredictionModel * /pipelines/ -> /pipelines/spark/ * python/isarnproject/pipelines/__init__.py * update build mappings for new python organization * update package paths for new organization * fix package object path * update copyright * update pyspark tdigest to be cleaner and analogous to java implementation * spark pipeline param delta -> compression * fi.scala * update assembly dep and move it into plugins * add scaladoc * move ScalarNumeric out of tdigest specific package * update README examples for scala * spark.sparkContext * update python tdigest examples * update feature importance examples * isarn-sketches-java * utest harness for spark testing * TDigestAggregator test * TDigestAggregator test * KSD cumulative distribution divergence measure for unit testing * test counts * BigDecimal range * local build against spark-3.0.1-SNAPSHOT * test TDigestArrayAggregator * tests for spark ML vector types * cache test data sets * test tdigest reducing aggregators * epsD * move approx to base class * disable parallel test execution to prevent spark cluster teardown race conditions * feature importance unit test * build against spark 3.0.1 * xsbt -> sbt * 0.5.0 --- README.md | 514 ++++++-------- build.sbt | 47 +- project/assembly.sbt | 1 - project/plugins.sbt | 6 +- .../udaf => pipelines/spark}/__init__.py | 0 .../isarnproject/pipelines/{ => spark}/fi.py | 20 +- .../sketches/{udt => spark}/__init__.py | 0 python/isarnproject/sketches/spark/tdigest.py | 373 +++++++++++ python/isarnproject/sketches/udaf/tdigest.py | 176 ----- python/isarnproject/sketches/udt/tdigest.py | 324 --------- .../sketches/udt/TDigestUDT.scala | 188 ------ .../{TDigestFI.scala => spark/fi.scala} | 46 +- .../pipelines/{ => spark}/package.scala | 6 +- .../isarnproject/sketches/spark/infra.scala | 53 ++ .../isarnproject/sketches/spark/tdigest.scala | 631 ++++++++++++++++++ .../sketches/udaf/TDigestUDAF.scala | 347 ---------- .../isarnproject/sketches/udaf/package.scala | 176 ----- .../org/isarnproject/pipelines/spark/fi.scala | 61 ++ .../isarnproject/sketches/spark/tdigest.scala | 197 ++++++ .../org/isarnproject/testing/spark.scala | 49 ++ 20 files changed, 1642 insertions(+), 1573 deletions(-) delete mode 100644 project/assembly.sbt rename python/isarnproject/{sketches/udaf => pipelines/spark}/__init__.py (100%) rename python/isarnproject/pipelines/{ => spark}/fi.py (87%) rename python/isarnproject/sketches/{udt => spark}/__init__.py (100%) create mode 100644 python/isarnproject/sketches/spark/tdigest.py delete mode 100644 python/isarnproject/sketches/udaf/tdigest.py delete mode 100644 python/isarnproject/sketches/udt/tdigest.py delete mode 100644 src/main/scala/org/apache/spark/isarnproject/sketches/udt/TDigestUDT.scala rename src/main/scala/org/isarnproject/pipelines/{TDigestFI.scala => spark/fi.scala} (90%) rename src/main/scala/org/isarnproject/pipelines/{ => spark}/package.scala (90%) create mode 100644 src/main/scala/org/isarnproject/sketches/spark/infra.scala create mode 100644 src/main/scala/org/isarnproject/sketches/spark/tdigest.scala delete mode 100644 src/main/scala/org/isarnproject/sketches/udaf/TDigestUDAF.scala delete mode 100644 src/main/scala/org/isarnproject/sketches/udaf/package.scala create mode 100644 src/test/scala/org/isarnproject/pipelines/spark/fi.scala create mode 100644 src/test/scala/org/isarnproject/sketches/spark/tdigest.scala create mode 100644 src/test/scala/org/isarnproject/testing/spark.scala diff --git a/README.md b/README.md index c7ec933..6e4fdf7 100644 --- a/README.md +++ b/README.md @@ -2,404 +2,316 @@ Routines and data structures for using isarn-sketches idiomatically in Apache Spark ## API documentation -https://isarn.github.io/isarn-sketches-spark/latest/api/#org.isarnproject.sketches.udaf.package +https://isarn.github.io/isarn-sketches-spark/latest/api/#org.isarnproject.sketches.spark ## How to use in your project ``` scala // Note that the version of spark is part of the release name. -// This example is for spark 2.4: -libraryDependencies += "org.isarnproject" %% "isarn-sketches-spark" % "0.4.0-sp2.4" +// This example is for spark 3.0: +libraryDependencies += "org.isarnproject" %% "isarn-sketches-spark" % "0.5.0-sp3.0" + +// This package builds against some `% Provided` Apache Spark dependencies: +libraryDependencies += "org.apache.spark" %% "spark-core" % sparkVersion +libraryDependencies += "org.apache.spark" %% "spark-sql" % sparkVersion +libraryDependencies += "org.apache.spark" %% "spark-mllib" % sparkVersion ``` -Currently supported: +Python code for supporting `pyspark` is also packaged with all of the artifacts above. +Spark will automatically extract and compile Python components for use with PySpark, +as illustrated in pyspark examples below. +Python 2 is no longer supported, but may work. -- spark 2.2, scala 2.11 -- spark 2.3, scala 2.11 -- spark 2.4, scala 2.11 and 2.12 -- spark 3.0, scala 2.12 +Beginning with isarn-sketches-spark 0.5.0, only spark >= 3.0 is supported, +due to substantial changes to Spark's API for user defined aggregation. +Some context on these changes is available in this Spark Summmit +[talk](https://databricks.com/session_na20/user-defined-aggregation-in-apache-spark-a-love-story). -If you are interested in a python/spark/scala build that is not listed above, please contact me and/or file an issue! +Versions of Spark and Scala that are currently supported: -Python code is also packaged with all of the artifacts above. -Spark will automatically extract and compile Python components for use with PySpark. -Python 2 and 3 are supported. Note that Python 2 is EOL as of January 2020. +- spark 3.0, scala 2.12 -This package builds against some `% Provided` Apache Spark dependencies: -```scala -libraryDependencies += "org.apache.spark" %% "spark-core" % sparkVersion -libraryDependencies += "org.apache.spark" %% "spark-sql" % sparkVersion -libraryDependencies += "org.apache.spark" %% "spark-mllib" % sparkVersion -``` +If you are interested in a spark/scala build that is not listed above, please contact me and/or file an issue! ## How to use from the Spark CLI -Several Spark CLI tools accept the `--packages` argument, as with this `spark-shell` example: +Several Spark CLI tools accept the `--packages` argument, including spark-shell and pyspark. +Following is an example using `spark-shell`: ```bash -$ spark-shell --packages "org.isarnproject:isarn-sketches-spark_2.12:0.4.0-sp2.4" +$ spark-shell --packages "org.isarnproject:isarn-sketches-spark_2.12:0.5.0-sp3.0" ``` Note that you need to explicitly include the scala version as part of the package name. ## Examples -### Sketch a numeric column +### Sketch a numeric column (scala) ```scala -scala> import org.isarnproject.sketches._, org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ -import org.isarnproject.sketches._ -import org.isarnproject.sketches.udaf._ -import org.apache.spark.isarnproject.sketches.udt._ - -scala> import scala.util.Random.nextGaussian -import scala.util.Random.nextGaussian +scala> import org.isarnproject.sketches.java.TDigest, org.isarnproject.sketches.spark.tdigest._, scala.util.Random._ -scala> val data = sc.parallelize(Vector.fill(1000){(nextGaussian, nextGaussian)}).toDF.as[(Double, Double)] -data: org.apache.spark.sql.Dataset[(Double, Double)] = [_1: double, _2: double] +scala> val data = spark.createDataFrame(Vector.fill(1000){(nextInt(10), nextGaussian)}) +data: org.apache.spark.sql.DataFrame = [_1: int, _2: double] -scala> val udaf = tdigestUDAF[Double].delta(0.2).maxDiscrete(25) -udaf: org.isarnproject.sketches.udaf.TDigestUDAF[Double] = TDigestUDAF(0.2,25) +scala> val udf = TDigestAggregator.udf[Double](compression = 0.2, maxDiscrete = 25) +udf: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... -scala> val agg = data.agg(udaf($"_1"), udaf($"_2")) -agg: org.apache.spark.sql.DataFrame = [tdigestudaf(_1): tdigest, tdigestudaf(_2): tdigest] +scala> val agg = data.agg(udf($"_1"), udf($"_2")).first +agg: org.apache.spark.sql.Row = [TDigest(0.0 -> (105.0, 105.0), ... -scala> val (td1, td2) = (agg.first.getAs[TDigestSQL](0).tdigest, agg.first.getAs[TDigestSQL](1).tdigest) -td1: org.isarnproject.sketches.TDigest = TDigest(0.2,25,151,TDigestMap(-3.1241237514093707 -> (1.0, 1.0), ... +scala> val (td1, td2) = (agg.getAs[TDigest](0), agg.getAs[TDigest](1)) +td1: org.isarnproject.sketches.java.TDigest = TDigest(0.0 -> (105.0, 105.0), ... -scala> td1.cdf(0) -res1: Double = 0.5159531867457404 +scala> (td1.cdf(2), td2.cdf(2)) +res0: (Double, Double) = (0.2365,0.9682691795524728) -scala> td2.cdf(0) -res2: Double = 0.504233763693618 +scala> (td1.samplePMF, td2.samplePDF) +res1: (Double, Double) = (8.0,-0.6712314520185372) ``` -### Sketch a numeric array column +### Sketch a numeric array column (scala) ```scala -scala> import org.isarnproject.sketches._, org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ -import org.isarnproject.sketches._ -import org.isarnproject.sketches.udaf._ -import org.apache.spark.isarnproject.sketches.udt._ - -scala> import scala.util.Random._ -import scala.util.Random._ +scala> import org.isarnproject.sketches.java.TDigest, org.isarnproject.sketches.spark.tdigest._, scala.util.Random._ scala> val data = spark.createDataFrame(Vector.fill(1000){(nextInt(10), Vector.fill(5){nextGaussian})}) data: org.apache.spark.sql.DataFrame = [_1: int, _2: array] -scala> val udaf1 = tdigestUDAF[Int].maxDiscrete(20) -udaf1: org.isarnproject.sketches.udaf.TDigestUDAF[Int] = TDigestUDAF(0.5,20) +scala> val udf1 = TDigestAggregator.udf[Int](maxDiscrete = 25) +udf1: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... -scala> val udafA = tdigestArrayUDAF[Double] -udafA: org.isarnproject.sketches.udaf.TDigestArrayUDAF[Double] = TDigestArrayUDAF(0.5,0) +scala> val udf2 = TDigestArrayAggregator.udf[Double](compression = 0.5) +udf2: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... -scala> val (first1, firstA) = (data.agg(udaf1($"_1")).first, data.agg(udafA($"_2")).first) -first1: org.apache.spark.sql.Row = [TDigestSQL(TDigest(0.5,20,19,TDigestMap(-9.0 -> (51.0, 51.0),... -firstA: org.apache.spark.sql.Row = [TDigestArraySQL([Lorg.isarnproject.sketches.TDigest;@782b0d37)] +scala> val agg = data.agg(udf1($"_1"), udf2($"_2")).first +agg: org.apache.spark.sql.Row = [TDigest(0.0 -> (104.0, 104.0), ... -scala> val sample1 = Vector.fill(10) { first1.getAs[TDigestSQL](0).tdigest.sample } -sample1: scala.collection.immutable.Vector[Double] = Vector(0.0, 7.0, 9.0, 6.0, 1.0, 3.0, 4.0, 0.0, 9.0, 0.0) +scala> agg.getAs[TDigest](0).samplePMF +res0: Double = 9.0 -scala> val sampleA = firstA.getAs[TDigestArraySQL](0).tdigests.map(_.sample) -sampleA: Array[Double] = Array(0.5079398036724695, 0.7518583956493221, -0.054376728126603546, 0.7141623682043323, 0.4788564991204228) +scala> agg.getAs[Seq[TDigest]](1).map(_.samplePDF) +res1: Seq[Double] = ArrayBuffer(-0.40804832001013713, -0.5952280168647848, -0.4973297020191356, -0.9404857531406351, 2.347194542873795) ``` -### Sketch a column of ML Vector +### Sketch a column of ML Vector (scala) ```scala -scala> import org.isarnproject.sketches._, org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ -import org.isarnproject.sketches._ -import org.isarnproject.sketches.udaf._ -import org.apache.spark.isarnproject.sketches.udt._ - -scala> import org.apache.spark.ml.linalg.Vectors -import org.apache.spark.ml.linalg.Vectors - -scala> import scala.util.Random._ -import scala.util.Random._ +scala> import org.isarnproject.sketches.java.TDigest, org.isarnproject.sketches.spark.tdigest._, scala.util.Random._, org.apache.spark.ml.linalg.Vectors scala> val data = spark.createDataFrame(Vector.fill(1000){(nextInt(10), Vectors.dense(nextGaussian,nextGaussian,nextGaussian))}) data: org.apache.spark.sql.DataFrame = [_1: int, _2: vector] -scala> val udafV = tdigestMLVecUDAF -udafV: org.isarnproject.sketches.udaf.TDigestMLVecUDAF = TDigestMLVecUDAF(0.5,0) +scala> val udf = TDigestMLVecAggregator.udf(compression = 0.5, maxDiscrete = 0) +udf: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... -scala> val firstV = data.agg(udafV($"_2")).first -firstV: org.apache.spark.sql.Row = [TDigestArraySQL([Lorg.isarnproject.sketches.TDigest;@42b579cd)] +scala> val agg = data.agg(udf($"_2")).first +agg: org.apache.spark.sql.Row = [WrappedArray(TDigest(-3.881918499979969 -> (1.0, 1.0), ... -scala> val sampleV = firstV.getAs[TDigestArraySQL](0).tdigests.map(_.sample) -sampleV: Array[Double] = Array(1.815862652134914, 0.24668895676164276, 0.09236479932949887) +scala> val samples = agg.getAs[Seq[TDigest]](0).map(_.samplePDF) +samples: Seq[Double] = ArrayBuffer(0.28973374164214244, 0.4981749043377094, -0.8945453848202787) -scala> val medianV = firstV.getAs[TDigestArraySQL](0).tdigests.map(_.cdfInverse(0.5)) -medianV: Array[Double] = Array(-0.049806905959001196, -0.08528817932077674, -0.05291800642695017) +scala> val medians = agg.getAs[Seq[TDigest]](0).map(_.cdfInverse(0.5)) +medians: Seq[Double] = ArrayBuffer(0.03123637037282659, -0.07172360154570709, -0.04260955558310061) ``` -### Sketch a column of MLLib Vector +### Sketch a column of MLLib Vector (scala) ```scala -scala> import org.isarnproject.sketches._, org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ -import org.isarnproject.sketches._ -import org.isarnproject.sketches.udaf._ -import org.apache.spark.isarnproject.sketches.udt._ - -scala> import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.linalg.Vectors - -scala> import scala.util.Random._ -import scala.util.Random._ +scala> import org.isarnproject.sketches.java.TDigest, org.isarnproject.sketches.spark.tdigest._, scala.util.Random._, org.apache.spark.mllib.linalg.Vectors scala> val data = spark.createDataFrame(Vector.fill(1000){(nextInt(10), Vectors.dense(nextGaussian,nextGaussian,nextGaussian))}) data: org.apache.spark.sql.DataFrame = [_1: int, _2: vector] -scala> val udafV = tdigestMLLibVecUDAF -udafV: org.isarnproject.sketches.udaf.TDigestMLLibVecUDAF = TDigestMLLibVecUDAF(0.5,0) +scala> val udf = TDigestMLLibVecAggregator.udf(compression = 0.5, maxDiscrete = 0) +udf: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... -scala> val firstV = data.agg(udafV($"_2")).first -firstV: org.apache.spark.sql.Row = [TDigestArraySQL([Lorg.isarnproject.sketches.TDigest;@6bffea90)] +scala> val agg = data.agg(udf($"_2")).first +agg: org.apache.spark.sql.Row = [WrappedArray(TDigest(-3.7272857909819344 -> (1.0, 1.0), ... -scala> val sampleV = firstV.getAs[TDigestArraySQL](0).tdigests.map(_.sample) -sampleV: Array[Double] = Array(0.10298190759496548, -0.1968752746464183, -1.0139250851274562) +scala> val samples = agg.getAs[Seq[TDigest]](0).map(_.samplePDF) +samples: Seq[Double] = ArrayBuffer(0.8780228679691738, -0.7636457587390327, 2.213341479782084) -scala> val medianV = firstV.getAs[TDigestArraySQL](0).tdigests.map(_.cdfInverse(0.5)) -medianV: Array[Double] = Array(0.025820266848484798, 0.01951778217339037, 0.09701138847692858) +scala> val medians = agg.getAs[Seq[TDigest]](0).map(_.cdfInverse(0.5)) +medians: Seq[Double] = ArrayBuffer(-0.01676307618586101, 0.03846529110807051, -0.029124197911563777) ``` -### Reduce a column (or grouping) of T-Digests +### Reduce a column (or grouping) of T-Digests (scala) ```scala -scala> import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._, org.isarnproject.sketches._, scala.util.Random._ -import org.isarnproject.sketches.udaf._ -import org.apache.spark.isarnproject.sketches.udt._ -import org.isarnproject.sketches._ -import scala.util.Random._ - -scala> val x = sc.parallelize(Vector.fill(1000) { nextGaussian }).toDF("x") -x: org.apache.spark.sql.DataFrame = [x: double] +scala> import org.isarnproject.sketches.java.TDigest, org.isarnproject.sketches.spark.tdigest._, scala.util.Random._ -scala> val g = sc.parallelize(Seq(1,2,3,4,5)).toDF("g") -g: org.apache.spark.sql.DataFrame = [g: int] - -scala> val data = g.crossJoin(x) +scala> val data = spark.createDataFrame(Vector.fill(5000){(nextInt(5), nextGaussian)}).toDF("g", "x") data: org.apache.spark.sql.DataFrame = [g: int, x: double] -scala> val udaf = tdigestUDAF[Double] -udaf: org.isarnproject.sketches.udaf.TDigestUDAF[Double] = TDigestUDAF(0.5,0) +scala> val udf = TDigestAggregator.udf[Double]() +udf: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... -scala> val tds = data.groupBy("g").agg(udaf($"x").alias("tdigests")) -tds: org.apache.spark.sql.DataFrame = [g: int, tdigests: tdigest] +scala> val grp = data.groupBy("g").agg(udf($"x").alias("tdigests")) +grp: org.apache.spark.sql.DataFrame = [g: int, tdigests: tdigest] -scala> tds.show() -+---+--------------------+ +scala> grp.show() ++---+--------------------+ | g| tdigests| +---+--------------------+ -| 1|TDigestSQL(TDiges...| -| 3|TDigestSQL(TDiges...| -| 5|TDigestSQL(TDiges...| -| 4|TDigestSQL(TDiges...| -| 2|TDigestSQL(TDiges...| +| 1|TDigest(-3.054140...| +| 3|TDigest(-3.368392...| +| 4|TDigest(-3.439268...| +| 2|TDigest(-3.927057...| +| 0|TDigest(-3.169235...| +---+--------------------+ -scala> val td = tds.agg(tdigestReduceUDAF($"tdigests").alias("tdigest")) -td: org.apache.spark.sql.DataFrame = [tdigest: tdigest] +scala> val udfred = TDigestReduceAggregator.udf(compression = 0.7) +udfred: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... + +scala> val agg = grp.agg(udfred($"tdigests")).first +agg: org.apache.spark.sql.Row = [TDigest(-3.9270575018252663 -> (1.0, 1.0), ... -scala> td.show() -+--------------------+ -| tdigest| -+--------------------+ -|TDigestSQL(TDiges...| -+--------------------+ +scala> val sample = agg.getAs[TDigest](0).sample +sample: Double = 0.6633542575218054 ``` -### Reduce a column (or grouping) of T-Digest Arrays +### Reduce a column (or grouping) of T-Digest Arrays (scala) ```scala -scala> import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._, org.isarnproject.sketches._, scala.util.Random._ -import org.isarnproject.sketches.udaf._ -import org.apache.spark.isarnproject.sketches.udt._ -import org.isarnproject.sketches._ -import scala.util.Random._ - -scala> val x = sc.parallelize(Vector.fill(1000) { Vector.fill(3) { nextGaussian } }).toDF("x") -x: org.apache.spark.sql.DataFrame = [x: array] +scala> import org.isarnproject.sketches.java.TDigest, org.isarnproject.sketches.spark.tdigest._, scala.util.Random._ -scala> val g = sc.parallelize(Seq(1,2,3,4,5)).toDF("g") -g: org.apache.spark.sql.DataFrame = [g: int] - -scala> val data = g.crossJoin(x) +scala> val data = spark.createDataFrame(Vector.fill(5000){(nextInt(5), Vector.fill(3) { nextGaussian })}).toDF("g", "x") data: org.apache.spark.sql.DataFrame = [g: int, x: array] -scala> val udaf = tdigestArrayUDAF[Double] -udaf: org.isarnproject.sketches.udaf.TDigestArrayUDAF[Double] = TDigestArrayUDAF(0.5,0) +scala> val udf = TDigestArrayAggregator.udf[Double]() +udf: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... -scala> val tds = data.groupBy("g").agg(udaf($"x").alias("tdigests")) -tds: org.apache.spark.sql.DataFrame = [g: int, tdigests: tdigestarray] +scala> val grp = data.groupBy("g").agg(udf($"x").alias("tdigests")) +grp: org.apache.spark.sql.DataFrame = [g: int, tdigests: array] -scala> tds.show() -+---+--------------------+ +scala> grp.show() ++---+--------------------+ | g| tdigests| +---+--------------------+ -| 1|TDigestArraySQL([...| -| 3|TDigestArraySQL([...| -| 5|TDigestArraySQL([...| -| 4|TDigestArraySQL([...| -| 2|TDigestArraySQL([...| +| 1|[TDigest(-3.25266...| +| 3|[TDigest(-3.05690...| +| 4|[TDigest(-3.66651...| +| 2|[TDigest(-3.46231...| +| 0|[TDigest(-2.94039...| +---+--------------------+ -scala> val td = tds.agg(tdigestArrayReduceUDAF($"tdigests")) -td: org.apache.spark.sql.DataFrame = [tdigestarrayreduceudaf(tdigests): tdigestarray] +scala> val udfred = TDigestArrayReduceAggregator.udf(compression = 0.7) +udfred: org.apache.spark.sql.expressions.UserDefinedFunction = UserDefinedAggregator( ... + +scala> val agg = grp.agg(udfred($"tdigests")).first +agg: org.apache.spark.sql.Row = [WrappedArray(TDigest(-3.6665115390677423 -> (1.0, 1.0), ... -scala> td.show() -+--------------------------------+ -|tdigestarrayreduceudaf(tdigests)| -+--------------------------------+ -| TDigestArraySQL([...| -+--------------------------------+ +scala> val samples = agg.getAs[Seq[TDigest]](0).map(_.sample) +samples: Seq[Double] = ArrayBuffer(-0.741335878221013, 0.981730493526761, -0.6359834079354106) ``` ### Sketch a numeric column (python) ```python ->>> from isarnproject.sketches.udaf.tdigest import * ->>> from random import gauss ->>> from pyspark.sql.types import * ->>> data = sc.parallelize([[gauss(0,1)] for x in range(1000)]).toDF(StructType([StructField("x", DoubleType())])) ->>> agg = data.agg(tdigestDoubleUDAF("x")) ->>> td = agg.first()[0] ->>> td.cdfInverse(0.5) -0.046805581998797419 ->>> +>>> from random import gauss, randint +>>> from isarnproject.sketches.spark.tdigest import * +>>> data = spark.createDataFrame([[randint(1,10),gauss(0,1)] for x in range(1000)]) +>>> udf1 = tdigestIntUDF("_1", maxDiscrete = 25) +>>> udf2 = tdigestDoubleUDF("_2", compression = 0.5) +>>> agg = data.agg(udf1, udf2).first() +>>> agg[0].samplePMF() +2.0 +>>> agg[1].samplePDF() +-0.8707704090068431 ``` ### Sketch a numeric array column (python) ```python ->>> from isarnproject.sketches.udaf.tdigest import * ->>> from random import gauss ->>> from pyspark.sql.types import * ->>> data = sc.parallelize([[[gauss(0,1),gauss(0,1),gauss(0,1)]] for x in range(1000)]).toDF(StructType([StructField("x", ArrayType(DoubleType()))])) ->>> agg = data.agg(tdigestDoubleArrayUDAF("x")) ->>> tds = agg.first()[0] ->>> [t.cdfInverse(0.5) for t in tds] -[0.046116924117141189, -0.011071666930287466, -0.019006033872431105] ->>> +>>> from random import gauss, randint +>>> from isarnproject.sketches.spark.tdigest import * +>>> data = spark.createDataFrame([[[gauss(0,1),gauss(0,1),gauss(0,1)]] for x in range(1000)]) +>>> udf = tdigestDoubleArrayUDF("_1", compression = 0.7) +>>> agg = data.agg(udf).first() +>>> [td.samplePDF() for td in agg[0]] +[0.6802628605487977, -0.649936837383734, -0.84228662547744] ``` ### Sketch a column of ML Vectors (python) ```python ->>> from isarnproject.sketches.udaf.tdigest import * ->>> from random import gauss ->>> from pyspark.ml.linalg import VectorUDT, Vectors ->>> from pyspark.sql.types import * ->>> data = sc.parallelize([[Vectors.dense([gauss(0,1),gauss(0,1),gauss(0,1)])] for x in range(1000)]).toDF(StructType([StructField("x", VectorUDT())])) ->>> agg = data.agg(tdigestMLVecUDAF("x")) ->>> tds = agg.first()[0] ->>> [t.cdfInverse(0.5) for t in tds] -[0.02859498787770634, -0.0027338622700039117, 0.041590980872883487] ->>> +>>> from random import gauss, randint +>>> from isarnproject.sketches.spark.tdigest import * +>>> from pyspark.ml.linalg import Vectors +>>> data = spark.createDataFrame([[Vectors.dense([gauss(0,1),gauss(0,1),gauss(0,1)])] for x in range(1000)]) +>>> udf = tdigestMLVecUDF("_1", compression = 0.7) +>>> agg = data.agg(udf).first() +>>> [td.cdfInverse(0.5) for td in agg[0]] +[-0.03088430803668949, -0.002903353148573491, 0.01640559766046329] ``` ### Sketch a column of MLLib Vectors (python) ```python ->>> from isarnproject.sketches.udaf.tdigest import * ->>> from random import gauss ->>> from pyspark.mllib.linalg import VectorUDT, Vectors ->>> from pyspark.sql.types import * ->>> data = sc.parallelize([[Vectors.dense([gauss(0,1),gauss(0,1),gauss(0,1)])] for x in range(1000)]).toDF(StructType([StructField("x", VectorUDT())])) ->>> agg = data.agg(tdigestMLLibVecUDAF("x")) ->>> tds = agg.first()[0] ->>> [t.cdfInverse(0.5) for t in tds] -[0.02859498787770634, -0.0027338622700039117, 0.041590980872883487] ->>> +>>> from random import gauss, randint +>>> from isarnproject.sketches.spark.tdigest import * +>>> from pyspark.mllib.linalg import Vectors +>>> data = spark.createDataFrame([[Vectors.dense([gauss(0,1),gauss(0,1),gauss(0,1)])] for x in range(1000)]) +>>> udf = tdigestMLLibVecUDF("_1", compression = 0.7) +>>> agg = data.agg(udf).first() +>>> [td.cdfInverse(0.5) for td in agg[0]] +[-0.03390700592837903, -0.0479047778031452, -0.02260427238692181] ``` ### Reduce a column (or grouping) of T-Digests (python) ```python ->>> from isarnproject.sketches.udaf.tdigest import * ->>> from random import gauss ->>> from pyspark.sql.types import * ->>> x = sc.parallelize([[gauss(0,1)] for x in range(1000)]).toDF(StructType([StructField("x", DoubleType())])) ->>> g = sc.parallelize([[1+x] for x in range(5)]).toDF(StructType([StructField("g", IntegerType())])) ->>> data = g.crossJoin(x) ->>> tds = data.groupBy("g").agg(tdigestDoubleUDAF("x").alias("tdigests")) ->>> tds.show() -+---+--------------------+ +>>> from random import gauss, randint +>>> from isarnproject.sketches.spark.tdigest import * +>>> data = spark.createDataFrame([[randint(1,5), gauss(0,1)] for x in range(5000)]).toDF("g","x") +>>> grp = data.groupBy("g").agg(tdigestDoubleUDF("x").alias("tdigests")) +>>> grp.show() ++---+--------------------+ | g| tdigests| +---+--------------------+ -| 1|TDigestSQL(TDiges...| -| 3|TDigestSQL(TDiges...| -| 5|TDigestSQL(TDiges...| -| 4|TDigestSQL(TDiges...| -| 2|TDigestSQL(TDiges...| +| 5|TDigest(-2.907724...| +| 1|TDigest(-2.914628...| +| 3|TDigest(-3.288239...| +| 2|TDigest(-3.389084...| +| 4|TDigest(-3.507597...| +---+--------------------+ ->>> td = tds.agg(tdigestReduceUDAF("tdigests").alias("tdigest")) ->>> td.show() -+--------------------+ -| tdigest| -+--------------------+ -|TDigestSQL(TDiges...| -+--------------------+ - ->>> +>>> udf = tdigestReduceUDF("tdigests") +>>> agg = grp.agg(udf).first() +>>> agg[0].sample() +-0.14793866496592997 ``` ### Reduce a column (or grouping) of T-Digest Arrays (python) ```python ->>> from isarnproject.sketches.udaf.tdigest import * ->>> from random import gauss ->>> from pyspark.ml.linalg import VectorUDT, Vectors ->>> from pyspark.sql.types import * ->>> x = sc.parallelize([[Vectors.dense([gauss(0,1),gauss(0,1),gauss(0,1)])] for x in range(1000)]).toDF(StructType([StructField("x", VectorUDT())])) ->>> g = sc.parallelize([[1+x] for x in range(5)]).toDF(StructType([StructField("g", IntegerType())])) ->>> data = g.crossJoin(x) ->>> tds = data.groupBy("g").agg(tdigestMLVecUDAF("x").alias("tdigests")) ->>> tds.show() +>>> from random import gauss, randint +>>> from isarnproject.sketches.spark.tdigest import * +>>> data = spark.createDataFrame([[randint(1,5), [gauss(0,1),gauss(0,1),gauss(0,1)]] for x in range(5000)]).toDF("g","x") +>>> grp = data.groupBy("g").agg(tdigestDoubleArrayUDF("x").alias("tdigests")) +>>> grp.show() +---+--------------------+ | g| tdigests| +---+--------------------+ -| 1|TDigestArraySQL([...| -| 3|TDigestArraySQL([...| -| 5|TDigestArraySQL([...| -| 4|TDigestArraySQL([...| -| 2|TDigestArraySQL([...| +| 5|[TDigest(-3.38098...| +| 1|[TDigest(-2.88380...| +| 3|[TDigest(-3.40987...| +| 2|[TDigest(-3.75224...| +| 4|[TDigest(-2.66571...| +---+--------------------+ ->>> td = tds.agg(tdigestArrayReduceUDAF("tdigests").alias("tdigest")) ->>> td.show() -+--------------------+ -| tdigest| -+--------------------+ -|TDigestArraySQL([...| -+--------------------+ +>>> udf = tdigestArrayReduceUDF("tdigests") +>>> agg = grp.agg(udf).first() +>>> [td.cdfInverse(0.5) for td in agg[0]] +[-0.04635615835441749, -0.025723034166600753, -0.025168480174964893] ``` ### Compute feature importance with respect to a predictive model ```scala -scala> :paste -// Entering paste mode (ctrl-D to finish) +scala> import org.isarnproject.pipelines.spark.fi.{TDigestFI,TDigestFIModel}, org.apache.spark.ml.regression.LinearRegression -import org.apache.spark.ml.regression.LinearRegression +scala> val training = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") +training: org.apache.spark.sql.DataFrame = [label: double, features: vector] -val training = spark.read.format("libsvm") - .load("data/mllib/sample_linear_regression_data.txt") +scala> val lr = new LinearRegression().setMaxIter(10).setRegParam(0.3).setElasticNetParam(0.8) +lr: org.apache.spark.ml.regression.LinearRegression = linReg_5d7a1cf3dafa -val lr = new LinearRegression() - .setMaxIter(10) - .setRegParam(0.3) - .setElasticNetParam(0.8) +scala> val lrModel = lr.fit(training) +lrModel: org.apache.spark.ml.regression.LinearRegressionModel = LinearRegressionModel: uid=linReg_5d7a1cf3dafa, numFeatures=10 -val lrModel = lr.fit(training) +scala> val fi = new TDigestFI().setCompression(0.3).setMaxDiscrete(10) +fi: org.isarnproject.pipelines.spark.fi.TDigestFI = TDigestFI_6837561844f2 -import org.isarnproject.pipelines.{TDigestFI,TDigestFIModel} +scala> val fiMod = fi.fit(training).setTargetModel(lrModel).setDeviationMeasure("rms-dev").setFeatureNames(Array.tabulate(10){j=>s"x$j"}) +fiMod: org.isarnproject.pipelines.spark.fi.TDigestFIModel = TDigestFI_6837561844f2 -val fi = new TDigestFI().setDelta(0.3).setMaxDiscrete(10) - -val fiMod = fi.fit(training) - .setTargetModel(lrModel) - .setDeviationMeasure("rms-dev") - .setFeatureNames(Array.tabulate(10){j=>s"x$j"}) - -val imp = fiMod.transform(training) - -// Exiting paste mode, now interpreting. - -import org.apache.spark.ml.regression.LinearRegression -training: org.apache.spark.sql.DataFrame = [label: double, features: vector] -lr: org.apache.spark.ml.regression.LinearRegression = linReg_ad8ebef9cfe8 -lrModel: org.apache.spark.ml.regression.LinearRegressionModel = linReg_ad8ebef9cfe8 -import org.isarnproject.pipelines.{TDigestFI, TDigestFIModel} -fi: org.isarnproject.pipelines.TDigestFI = TDigestFI_67b1cff93349 -fiMod: org.isarnproject.pipelines.TDigestFIModel = TDigestFI_67b1cff93349 +scala> val imp = fiMod.transform(training) imp: org.apache.spark.sql.DataFrame = [name: string, importance: double] scala> imp.show @@ -407,45 +319,41 @@ scala> imp.show |name| importance| +----+-------------------+ | x0| 0.0| -| x1|0.27093413867331134| -| x2|0.27512986364699304| -| x3| 1.4284480425303374| -| x4|0.04472982597939822| -| x5| 0.5981079647203551| +| x1| 0.2642731504552658| +| x2| 0.2775267570310568| +| x3| 1.48027354456237| +| x4| 0.0442095774509019| +| x5| 0.620636336433091| | x6| 0.0| -| x7|0.11970670592684969| -| x8| 0.1668815037423663| -| x9|0.17970574939101025| +| x7|0.12650113005096197| +| x8| 0.1644528333598182| +| x9| 0.1883875750326046| +----+-------------------+ ``` ### Compute feature importance with respect to a predictive model (python) ```python +>>> from isarnproject.pipelines.spark.fi import * >>> from pyspark.ml.regression import LinearRegression ->>> training = spark.read.format("libsvm") \ -... .load("data/mllib/sample_linear_regression_data.txt") ->>> lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) +>>> training = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") +>>> lr = LinearRegression(maxIter=10, regParam=0.3, elasticNetParam=0.8) >>> lrModel = lr.fit(training) ->>> from isarnproject.pipelines.fi import * ->>> fi = TDigestFI().setDelta(0.3).setMaxDiscrete(10) ->>> fiMod = fi.fit(training) \ -... .setTargetModel(lrModel) \ -... .setDeviationMeasure("rms-dev") \ -... .setFeatureNames(["x%d" % (j) for j in range(10)]) +>>> fi = TDigestFI(compression = 0.3, maxDiscrete = 10) +>>> fiMod = fi.fit(training).setTargetModel(lrModel).setDeviationMeasure("rms-dev").setFeatureNames(["x%d" % (j) for j in range(10)]) >>> imp = fiMod.transform(training) >>> imp.show() -+----+-------------------+ -|name| importance| -+----+-------------------+ -| x0| 0.0| -| x1| 0.2513147892886899| -| x2|0.28992477834838837| -| x3| 1.4906022974248356| -| x4|0.04197189119745892| -| x5| 0.6213459845972947| -| x6| 0.0| -| x7|0.12463038543257152| -| x8|0.17144699470039335| -| x9|0.18428188512840307| -+----+-------------------+ ++----+--------------------+ +|name| importance| ++----+--------------------+ +| x0| 0.0| +| x1| 0.2617304778862077| +| x2| 0.26451433792352613| +| x3| 1.5244246022297059| +| x4|0.043227915487816015| +| x5| 0.6195605571925815| +| x6| 0.0| +| x7| 0.11735009989902982| +| x8| 0.17250227692634765| +| x9| 0.18251143533748138| ++----+--------------------+ ``` diff --git a/build.sbt b/build.sbt index be2e74e..d256f0b 100644 --- a/build.sbt +++ b/build.sbt @@ -1,8 +1,10 @@ -// xsbt clean unidoc previewSite -// xsbt clean unidoc ghpagesPushSite -// xsbt +publish +// sbt clean unidoc previewSite +// sbt clean unidoc ghpagesPushSite +// sbt +publish // https://oss.sonatype.org // make sure sparkVersion is set as you want prior to +publish +// when doing localPublish, also do: +// rm -rf /home/eje/.ivy2/local/org.isarnproject /home/eje/.ivy2/cache/org.isarnproject import scala.sys.process._ @@ -10,9 +12,9 @@ name := "isarn-sketches-spark" organization := "org.isarnproject" -val packageVersion = "0.4.1-SNAPSHOT" +val packageVersion = "0.5.0" -val sparkVersion = "3.0.0" +val sparkVersion = "3.0.1" val sparkSuffix = s"""sp${sparkVersion.split('.').take(2).mkString(".")}""" @@ -20,7 +22,7 @@ version := s"${packageVersion}-${sparkSuffix}" scalaVersion := "2.12.11" -crossScalaVersions := Seq("2.12.11") // scala 2.12.11 when spark supports it +crossScalaVersions := Seq("2.12.11") pomIncludeRepository := { _ => false } @@ -54,14 +56,22 @@ developers := List( ) ) +resolvers += Resolver.mavenLocal + libraryDependencies ++= Seq( - "org.isarnproject" %% "isarn-sketches" % "0.1.2", + "org.isarnproject" % "isarn-sketches-java" % "0.3.0", "org.apache.spark" %% "spark-core" % sparkVersion % Provided, "org.apache.spark" %% "spark-sql" % sparkVersion % Provided, "org.apache.spark" %% "spark-mllib" % sparkVersion % Provided, - "org.isarnproject" %% "isarn-scalatest" % "0.0.3" % Test, - "org.scalatest" %% "scalatest" % "3.0.5" % Test, - "org.apache.commons" % "commons-math3" % "3.6.1" % Test) + "com.lihaoyi" %% "utest" % "0.7.4" % Test) + +// tell sbt about utest +testFrameworks += new TestFramework("utest.runner.Framework") + +// default is to run tests in parallel, asynchronously, but +// that breaks both spark-cluster setup and teardown, and also breaks +// repeatability of the random data generation +parallelExecution in Test := false initialCommands in console := """ |import org.apache.spark.SparkConf @@ -70,10 +80,10 @@ initialCommands in console := """ |import org.apache.spark.SparkContext._ |import org.apache.spark.rdd.RDD |import org.apache.spark.ml.linalg.Vectors - |import org.isarnproject.sketches.TDigest - |import org.isarnproject.sketches.udaf._ - |import org.apache.spark.isarnproject.sketches.udt._ - |val initialConf = new SparkConf().setAppName("repl").set("spark.serializer", "org.apache.spark.serializer.KryoSerializer").set("spark.kryoserializer.buffer", "16mb") + |import org.apache.spark.sql.functions._ + |import org.isarnproject.sketches.java.TDigest + |import org.isarnproject.sketches.spark._ + |val initialConf = new SparkConf().setAppName("repl") |val spark = SparkSession.builder.config(initialConf).master("local[2]").getOrCreate() |import spark._, spark.implicits._ |val sc = spark.sparkContext @@ -90,12 +100,11 @@ scalacOptions ++= Seq("-unchecked", "-deprecation", "-feature") mappings in (Compile, packageBin) ++= Seq( (baseDirectory.value / "python" / "isarnproject" / "__init__.py") -> "isarnproject/__init__.py", (baseDirectory.value / "python" / "isarnproject" / "pipelines" / "__init__.py") -> "isarnproject/pipelines/__init__.py", - (baseDirectory.value / "python" / "isarnproject" / "pipelines" / "fi.py") -> "isarnproject/pipelines/fi.py", + (baseDirectory.value / "python" / "isarnproject" / "pipelines" / "spark" / "__init__.py") -> "isarnproject/pipelines/spark/__init__.py", + (baseDirectory.value / "python" / "isarnproject" / "pipelines" / "spark" / "fi.py") -> "isarnproject/pipelines/spark/fi.py", (baseDirectory.value / "python" / "isarnproject" / "sketches" / "__init__.py") -> "isarnproject/sketches/__init__.py", - (baseDirectory.value / "python" / "isarnproject" / "sketches" / "udaf" / "__init__.py") -> "isarnproject/sketches/udaf/__init__.py", - (baseDirectory.value / "python" / "isarnproject" / "sketches" / "udaf" / "tdigest.py") -> "isarnproject/sketches/udaf/tdigest.py", - (baseDirectory.value / "python" / "isarnproject" / "sketches" / "udt" / "__init__.py") -> "isarnproject/sketches/udt/__init__.py", - (baseDirectory.value / "python" / "isarnproject" / "sketches" / "udt" / "tdigest.py") -> "isarnproject/sketches/udt/tdigest.py" + (baseDirectory.value / "python" / "isarnproject" / "sketches" / "spark" / "__init__.py") -> "isarnproject/sketches/spark/__init__.py", + (baseDirectory.value / "python" / "isarnproject" / "sketches" / "spark" / "tdigest.py") -> "isarnproject/sketches/spark/tdigest.py", ) test in assembly := {} diff --git a/project/assembly.sbt b/project/assembly.sbt deleted file mode 100644 index d95475f..0000000 --- a/project/assembly.sbt +++ /dev/null @@ -1 +0,0 @@ -addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.14.7") diff --git a/project/plugins.sbt b/project/plugins.sbt index 4f35abb..7b7c7a1 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,11 +1,11 @@ resolvers += Resolver.url( "bintray-sbt-plugin-releases", - url("http://dl.bintray.com/content/sbt/sbt-plugin-releases"))( + url("https://dl.bintray.com/content/sbt/sbt-plugin-releases"))( Resolver.ivyStylePatterns) resolvers += "sonatype-releases" at "https://oss.sonatype.org/content/repositories/releases/" -resolvers += "jgit-repo" at "http://download.eclipse.org/jgit/maven" +resolvers += "jgit-repo" at "https://download.eclipse.org/jgit/maven" addSbtPlugin("com.typesafe.sbt" % "sbt-ghpages" % "0.6.3") @@ -15,6 +15,8 @@ addSbtPlugin("io.crashbox" % "sbt-gpg" % "0.2.1") addSbtPlugin("org.xerial.sbt" % "sbt-sonatype" % "3.9.2") +addSbtPlugin("com.eed3si9n" % "sbt-assembly" % "0.15.0") + // scoverage and coveralls deps are at old versions to avoid a bug in the current versions // update these when this fix is released: https://github.com/scoverage/sbt-coveralls/issues/73 //addSbtPlugin("org.scoverage" % "sbt-scoverage" % "1.0.4") diff --git a/python/isarnproject/sketches/udaf/__init__.py b/python/isarnproject/pipelines/spark/__init__.py similarity index 100% rename from python/isarnproject/sketches/udaf/__init__.py rename to python/isarnproject/pipelines/spark/__init__.py diff --git a/python/isarnproject/pipelines/fi.py b/python/isarnproject/pipelines/spark/fi.py similarity index 87% rename from python/isarnproject/pipelines/fi.py rename to python/isarnproject/pipelines/spark/fi.py index 628d847..12d0c27 100644 --- a/python/isarnproject/pipelines/fi.py +++ b/python/isarnproject/pipelines/spark/fi.py @@ -1,7 +1,7 @@ from pyspark import since, keyword_only from pyspark.ml.param.shared import * from pyspark.ml.util import * -from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaWrapper +from pyspark.ml.wrapper import JavaEstimator, JavaModel, JavaWrapper, JavaPredictionModel from pyspark.ml.common import inherit_doc from pyspark.sql import DataFrame @@ -14,7 +14,7 @@ def toPredictionModel(value): raise TypeError("object %s was not a JavaPredictionModel" % (value)) class TDigestParams(Params): - delta = Param(Params._dummy(), "delta", "tdigest compression parameter", + compression = Param(Params._dummy(), "compression", "tdigest compression parameter", typeConverter=TypeConverters.toFloat) maxDiscrete = Param(Params._dummy(), "maxDiscrete", "maximum discrete values", typeConverter=TypeConverters.toInt) @@ -22,11 +22,11 @@ class TDigestParams(Params): def __init__(self): super(TDigestParams, self).__init__() - def setDelta(self, value): - return self._set(delta=value) + def setCompression(self, value): + return self._set(compression=value) - def getDelta(self): - return self.getOrDefault(self.delta) + def getCompression(self): + return self.getOrDefault(self.compression) def setMaxDiscrete(self, value): return self._set(maxDiscrete=value) @@ -90,15 +90,15 @@ class TDigestFI(JavaEstimator, TDigestFIParams, JavaMLWritable, JavaMLReadable): """ @keyword_only - def __init__(self, delta = 0.5, maxDiscrete = 0, featuresCol = "features"): + def __init__(self, compression = 0.5, maxDiscrete = 0, featuresCol = "features"): super(TDigestFI, self).__init__() - self._java_obj = self._new_java_obj("org.isarnproject.pipelines.TDigestFI", self.uid) - self._setDefault(delta = 0.5, maxDiscrete = 0, featuresCol = "features") + self._java_obj = self._new_java_obj("org.isarnproject.pipelines.spark.fi.TDigestFI", self.uid) + self._setDefault(compression = 0.5, maxDiscrete = 0, featuresCol = "features") kwargs = self._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, delta = 0.5, maxDiscrete = 0, featuresCol = "features"): + def setParams(self, compression = 0.5, maxDiscrete = 0, featuresCol = "features"): kwargs = self._input_kwargs return self._set(**kwargs) diff --git a/python/isarnproject/sketches/udt/__init__.py b/python/isarnproject/sketches/spark/__init__.py similarity index 100% rename from python/isarnproject/sketches/udt/__init__.py rename to python/isarnproject/sketches/spark/__init__.py diff --git a/python/isarnproject/sketches/spark/tdigest.py b/python/isarnproject/sketches/spark/tdigest.py new file mode 100644 index 0000000..c105852 --- /dev/null +++ b/python/isarnproject/sketches/spark/tdigest.py @@ -0,0 +1,373 @@ +import sys +import random +import itertools as it +from bisect import bisect_left, bisect_right + +from pyspark.sql.types import UserDefinedType, StructField, StructType, \ + ArrayType, DoubleType, IntegerType +from pyspark.sql.column import Column, _to_java_column, _to_seq +from pyspark.context import SparkContext + +__all__ = ['tdigestIntUDF', 'tdigestLongUDF', 'tdigestFloatUDF', 'tdigestDoubleUDF', \ + 'tdigestMLVecUDF', 'tdigestMLLibVecUDF', \ + 'tdigestIntArrayUDF', 'tdigestLongArrayUDF', 'tdigestFloatArrayUDF', 'tdigestDoubleArrayUDF', \ + 'tdigestReduceUDF', 'tdigestArrayReduceUDF', \ + 'TDigest'] + +def tdigestIntUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of integer data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestIntUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestLongUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of long integer data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestLongUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestFloatUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of (single precision) float data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestFloatUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestDoubleUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of double float data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestDoubleUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestMLVecUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of ML Vector data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestMLVecUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestMLLibVecUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of MLLib Vector data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestMLLibVecUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestIntArrayUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of integer-array data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestIntArrayUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestLongArrayUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of long-integer array data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestLongArrayUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestFloatArrayUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of (single-precision) float array data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestFloatArrayUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestDoubleArrayUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of double array data. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestDoubleArrayUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestReduceUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of t-digests. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestReduceUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +def tdigestArrayReduceUDF(col, compression=0.5, maxDiscrete=0): + """ + Return a UDF for aggregating a column of t-digest vectors. + + :param col: name of the column to aggregate + :param compression: T-Digest compression parameter (default 0.5) + :param maxDiscrete: maximum unique discrete values to store before reverting to + continuous (default 0) + """ + sc = SparkContext._active_spark_context + tdapply = sc._jvm.org.isarnproject.sketches.spark.tdigest.functions.tdigestArrayReduceUDF( \ + compression, maxDiscrete).apply + return Column(tdapply(_to_seq(sc, [col], _to_java_column))) + +class TDigestUDT(UserDefinedType): + @classmethod + def sqlType(cls): + return StructType([ + StructField("compression", DoubleType(), False), + StructField("maxDiscrete", IntegerType(), False), + StructField("cent", ArrayType(DoubleType(), False), False), + StructField("mass", ArrayType(DoubleType(), False), False)]) + + @classmethod + def module(cls): + return "isarnproject.sketches.udt.tdigest" + + @classmethod + def scalaUDT(cls): + return "org.apache.spark.isarnproject.sketches.udtdev.TDigestUDT" + + def simpleString(self): + return "tdigest" + + def serialize(self, obj): + if isinstance(obj, TDigest): + return (obj.compression, obj.maxDiscrete, obj._cent, obj._mass) + else: + raise TypeError("cannot serialize %r of type %r" % (obj, type(obj))) + + def deserialize(self, datum): + return TDigest(datum[0], datum[1], datum[2], datum[3]) + +class TDigest(object): + """ + A T-Digest sketch of a cumulative numeric distribution. + This is a "read-only" python mirror of org.isarnproject.sketches.java.TDigest which supports + all cdf and sampling methods, but does not currently support update with new data. It is + assumed to have been produced with a t-digest aggregating UDF, also exposed in this package. + """ + + # Because this is a value and not a function, TDigestUDT has to be defined above, + # and in the same file. + __UDT__ = TDigestUDT() + + def __init__(self, compression, maxDiscrete, cent, mass): + self.compression = float(compression) + self.maxDiscrete = int(maxDiscrete) + assert self.compression > 0.0, "compression must be > 0" + assert self.maxDiscrete >= 0, "maxDiscrete must be >= 0" + self._cent = [float(v) for v in cent] + self._mass = [float(v) for v in mass] + assert len(self._mass) == len(self._cent), "cluster mass and cent must have same dimension" + self.nclusters = len(self._cent) + # Current implementation is "read only" so we can just store cumulative sum here. + # To support updating, 'csum' would need to become a Fenwick tree array + self._csum = list(it.accumulate(self._mass)) + + def __repr__(self): + return "TDigest(%s, %s, %s, %s, %s)" % \ + (repr(self.compression), repr(self.maxDiscrete), repr(self.nclusters), repr(self._cent), repr(self._mass)) + + def mass(self): + """ + Total mass accumulated by this TDigest + """ + if len(self._csum) == 0: return 0.0 + return self._csum[-1] + + def size(self): + """ + Number of clusters in this TDigest + """ + return len(self._cent) + + def isEmpty(self): + """ + Returns True if this TDigest is empty, False otherwise + """ + return len(self._cent) == 0 + + def __reduce__(self): + return (self.__class__, (self.compression, self.maxDiscrete, self.nclusters, self._cent, self._mass, )) + + def _lmcovj(self, m): + assert self.nclusters >= 2 + assert (m >= 0.0) and (m <= self.mass()) + return bisect_left(self._csum, m) + + def _rmcovj(self, m): + assert self.nclusters >= 2 + assert (m >= 0.0) and (m <= self.mass()) + return bisect_right(self._csum, m) - 1 + + def _rcovj(self, x): + return bisect_right(self._cent, x) - 1 + + # emulates behavior from isarn java TDigest, which computes + # cumulative sum via a Fenwick tree + def _ftSum(self, j): + if (j < 0): return 0.0 + if (j >= self.nclusters): return self.mass() + return self._csum[j] + + def cdf(self, xx): + """ + Return CDF(x) of a numeric value x, with respect to this TDigest CDF sketch. + """ + x = float(xx) + j1 = self._rcovj(x) + if (j1 < 0): return 0.0 + if (j1 >= self.nclusters - 1): return 1.0 + j2 = j1 + 1 + c1 = self._cent[j1] + c2 = self._cent[j2] + tm1 = self._mass[j1] + tm2 = self._mass[j2] + s = self._ftSum(j1 - 1) + d1 = 0.0 if (j1 == 0) else tm1 / 2.0 + m1 = s + d1 + m2 = m1 + (tm1 - d1) + (tm2 if (j2 == self.nclusters - 1) else tm2 / 2.0) + m = m1 + (x - c1) * (m2 - m1) / (c2 - c1) + return min(m2, max(m1, m)) / self.mass() + + def cdfInverse(self, qq): + """ + Given a value q on [0,1], return the value x such that CDF(x) = q. + Returns NaN for any q > 1 or < 0, or if this TDigest is empty. + """ + q = float(qq) + if (q < 0.0) or (q > 1.0): return float('nan') + if (self.nclusters == 0): return float('nan') + if (self.nclusters == 1): return self._cent[0] + m = q * self.mass() + j1 = self._rmcovj(m) + j2 = j1 + 1 + c1 = self._cent[j1] + c2 = self._cent[j2] + tm1 = self._mass[j1] + tm2 = self._mass[j2] + s = self._ftSum(j1 - 1) + d1 = 0.0 if (j1 == 0) else tm1 / 2.0 + m1 = s + d1 + m2 = m1 + (tm1 - d1) + (tm2 if (j2 == self.nclusters - 1) else tm2 / 2.0) + x = c1 + (m - m1) * (c2 - c1) / (m2 - m1) + return min(c2, max(c1, x)) + + def cdfDiscrete(self, xx): + """ + return CDF(x) for a numeric value x, assuming the sketch is representing a + discrete distribution. + """ + x = float(xx) + j = self._rcovj(x) + return self._ftSum(j) / self.mass() + + def cdfDiscreteInverse(self, qq): + """ + Given a value q on [0,1], return the value x such that CDF(x) = q, assuming + the sketch is represenging a discrete distribution. + Returns NaN for any q > 1 or < 0, or if this TDigest is empty. + """ + q = float(qq) + if (q < 0.0) or (q > 1.0): return float('nan') + if self.nclusters == 0: return float('nan') + if self.nclusters == 1: return self._cent[0] + m = q * self.mass() + j = self._lmcovj(m) + return self._cent[j] + + def samplePDF(self): + """ + Return a random sampling from the sketched distribution, using inverse + transform sampling, assuming a continuous distribution. + """ + return self.cdfInverse(random.random()) + + def samplePMF(self): + """ + Return a random sampling from the sketched distribution, using inverse + transform sampling, assuming a discrete distribution. + """ + return self.cdfDiscreteInverse(random.random()) + + def sample(self): + """ + Return a random sampling from the sketched distribution, using inverse + transform sampling, assuming a discrete distribution if the number of + TDigest clusters is <= maxDiscrete, and a continuous distribution otherwise. + """ + if self.maxDiscrete <= self.nclusters: + return self.cdfDiscreteInverse(random.random()) + return self.cdfInverse(random.random()) diff --git a/python/isarnproject/sketches/udaf/tdigest.py b/python/isarnproject/sketches/udaf/tdigest.py deleted file mode 100644 index 11b69e3..0000000 --- a/python/isarnproject/sketches/udaf/tdigest.py +++ /dev/null @@ -1,176 +0,0 @@ -from pyspark.sql.column import Column, _to_java_column, _to_seq -from pyspark.context import SparkContext - -__all__ = ['tdigestIntUDAF', 'tdigestLongUDAF', 'tdigestFloatUDAF', 'tdigestDoubleUDAF', \ - 'tdigestMLVecUDAF', 'tdigestMLLibVecUDAF', \ - 'tdigestIntArrayUDAF', 'tdigestLongArrayUDAF', \ - 'tdigestFloatArrayUDAF', 'tdigestDoubleArrayUDAF', \ - 'tdigestReduceUDAF', 'tdigestArrayReduceUDAF'] - -def tdigestIntUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of integer data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestIntUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestLongUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of long integer data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestLongUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestFloatUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of (single precision) float data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestFloatUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestDoubleUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of double float data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestDoubleUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestMLVecUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of ML Vector data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestMLVecUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestMLLibVecUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of MLLib Vector data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestMLLibVecUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestIntArrayUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of integer-array data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestIntArrayUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestLongArrayUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of long-integer array data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestLongArrayUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestFloatArrayUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of (single-precision) float array data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestFloatArrayUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestDoubleArrayUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of double array data. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestDoubleArrayUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestReduceUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of t-digests. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestReduceUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) - -def tdigestArrayReduceUDAF(col, delta=0.5, maxDiscrete=0): - """ - Return a UDAF for aggregating a column of t-digest vectors. - - :param col: name of the column to aggregate - :param delta: T-Digest compression parameter (default 0.5) - :param maxDiscrete: maximum unique discrete values to store before reverting to - continuous (default 0) - """ - sc = SparkContext._active_spark_context - tdapply = sc._jvm.org.isarnproject.sketches.udaf.pythonBindings.tdigestArrayReduceUDAF( \ - delta, maxDiscrete).apply - return Column(tdapply(_to_seq(sc, [col], _to_java_column))) diff --git a/python/isarnproject/sketches/udt/tdigest.py b/python/isarnproject/sketches/udt/tdigest.py deleted file mode 100644 index 1b89dc1..0000000 --- a/python/isarnproject/sketches/udt/tdigest.py +++ /dev/null @@ -1,324 +0,0 @@ - -import sys -import random - -import numpy as np - -from pyspark.sql.types import UserDefinedType, StructField, StructType, \ - ArrayType, DoubleType, IntegerType - -__all__ = ['TDigest'] - -class TDigestUDT(UserDefinedType): - @classmethod - def sqlType(cls): - return StructType([ - StructField("delta", DoubleType(), False), - StructField("maxDiscrete", IntegerType(), False), - StructField("nclusters", IntegerType(), False), - StructField("clustX", ArrayType(DoubleType(), False), False), - StructField("clustM", ArrayType(DoubleType(), False), False)]) - - @classmethod - def module(cls): - return "isarnproject.sketches.udt.tdigest" - - @classmethod - def scalaUDT(cls): - return "org.apache.spark.isarnproject.sketches.udt.TDigestUDT" - - def simpleString(self): - return "tdigest" - - def serialize(self, obj): - if isinstance(obj, TDigest): - return (obj.delta, obj.maxDiscrete, obj.nclusters, \ - [float(v) for v in obj.clustX], \ - [float(v) for v in obj.clustM]) - else: - raise TypeError("cannot serialize %r of type %r" % (obj, type(obj))) - - def deserialize(self, datum): - return TDigest(datum[0], datum[1], datum[2], datum[3], datum[4]) - -class TDigestArrayUDT(UserDefinedType): - @classmethod - def sqlType(cls): - return StructType([ - StructField("delta", DoubleType(), False), - StructField("maxDiscrete", IntegerType(), False), - StructField("clusterS", ArrayType(IntegerType(), False), False), - StructField("clusterX", ArrayType(DoubleType(), False), False), - StructField("clusterM", ArrayType(DoubleType(), False), False)]) - - @classmethod - def module(cls): - return "isarnproject.sketches.udt.tdigest" - - @classmethod - def scalaUDT(cls): - return "org.apache.spark.isarnproject.sketches.udt.TDigestArrayUDT" - - def simpleString(self): - return "tdigestarray" - - def serialize(self, obj): - if isinstance(obj, TDigestList): - clustS = [] - clustX = [] - clustM = [] - for td in obj: - clustS.append(td.nclusters) - clustX += [float(v) for v in td.clustX] - clustM += [float(v) for v in td.clustM] - delta = obj[0].delta if len(obj) > 0 else 0.5 - maxDiscrete = obj[0].maxDiscrete if len(obj) > 0 else 0 - return (delta, maxDiscrete, clustS, clustX, clustM) - else: - raise TypeError("cannot serialize %r of type %r" % (obj, type(obj))) - - def deserialize(self, datum): - delta = datum[0] - maxDiscrete = datum[1] - clusterS = datum[2] - clusterX = datum[3] - clusterM = datum[4] - tdlist = TDigestList() - b = 0 - for s in clusterS: - clustX = clusterX[b:b+s] - clustM = clusterM[b:b+s] - tdlist.append(TDigest(delta, maxDiscrete, s, clustX, clustM)) - b += s - return tdlist - -class TDigestList(list): - """ - A subclass of a list of TDigest objects, deserialized from a Dataset. - This subclass has a __UDT__ element - """ - - __UDT__ = TDigestArrayUDT() - -class TDigest(object): - """ - A T-Digest sketch of a cumulative numeric distribution. - This is a "read-only" python mirror of org.isarnproject.sketches.TDigest which supports - all cdf and sampling methods, but does not currently support update with new data. It is - assumed to have been produced with a t-digest UDAF, also exposed in this package. - """ - - # Because this is a value and not a function, TDigestUDT has to be defined above, - # and in the same file. - __UDT__ = TDigestUDT() - - def __init__(self, delta, maxDiscrete, nclusters, clustX, clustM): - self.delta = float(delta) - self.maxDiscrete = int(maxDiscrete) - self.nclusters = int(nclusters) - self.clustX = np.array(clustX, dtype=np.float64) - self.clustM = np.array(clustM, dtype=np.float64) - self.clustP = np.cumsum(clustM) - assert len(self.clustX) == self.nclusters, "nclusters does not match cluster mass array" - assert len(self.clustX) == len(self.clustM), "cluster mass array does not match cluster center array" - - def __repr__(self): - return "TDigest(%s, %s, %s, %s, %s)" % \ - (repr(self.delta), repr(self.maxDiscrete), repr(self.nclusters), repr(self.clustX), repr(self.clustM)) - - def mass(self): - """ - Total mass accumulated by this TDigest - """ - if len(self.clustP) == 0: - return 0.0 - return self.clustP[-1] - - def isEmpty(self): - """ - Returns True if this TDigest is empty, False otherwise - """ - return len(self.clustX) == 0 - - def __reduce__(self): - return (self.__class__, (self.delta, self.maxDiscrete, self.nclusters, self.clustX, self.clustM, )) - - # The "right cover" of a value x, w.r.t. the clusters in this TDigest - def __covR__(self, x): - n = len(self.clustX) - if n == 0: - return Cover(None, None) - j = np.searchsorted(self.clustX, x, side='right') - if j == n: - return Cover(n - 1, None) - if x < self.clustX[0]: - return Cover(None, 0) - return Cover(j - 1, j) - - # The "mass cover" of a mass m, w.r.t. the clusters in this TDigest - def __covM__(self, m): - n = len(self.clustP) - if n == 0: - return Cover(None, None) - j = np.searchsorted(self.clustP, m, side='right') - if j == n: - return Cover(n - 1, None) - if m < self.clustP[0]: - return Cover(None, 0) - return Cover(j - 1, j) - - # Get a "corrected mass" for two clusters. - # Returns "centered" mass for interior clusters, and uncentered for edge clusters. - def __m1m2__(self, j, c1, tm1, c2, tm2): - assert len(self.clustX) > 0, "unexpected empty TDigest" - # s is the "open" prefix sum, for x < c1 - s = self.clustP[j] - tm1 - d1 = 0.0 if c1 == self.clustX[0] else tm1 / 2.0 - d2 = tm2 if c2 == self.clustX[-1] else tm2 / 2.0 - m1 = s + d1 - m2 = m1 + (tm1 - d1) + d2 - return (m1, m2) - - # Get the inverse CDF, using the "corrected mass" - def __cdfI__(self, j, m, c1, tm1, c2, tm2): - m1, m2 = self.__m1m2__(j, c1, tm1, c2, tm2) - x = c1 + (m - m1) * (c2 - c1) / (m2 - m1) - return min(c2, max(c1, x)) - - def cdf(self, xx): - """ - Return CDF(x) of a numeric value x, with respect to this TDigest CDF sketch. - """ - x = float(xx) - jcov = self.__covR__(x) - cov = jcov.map(lambda j: (self.clustX[j], self.clustM[j])) - if (not cov.l.isEmpty()) and (not cov.r.isEmpty()): - c1, tm1 = cov.l.get() - c2, tm2 = cov.r.get() - m1, m2 = self.__m1m2__(jcov.l.get(), c1, tm1, c2, tm2) - m = m1 + (x - c1) * (m2 - m1) / (c2 - c1) - return min(m2, max(m1, m)) / self.mass() - if cov.r.isEmpty(): - return 1.0 - return 0.0 - - def cdfInverse(self, qq): - """ - Given a value q on [0,1], return the value x such that CDF(x) = q. - Returns NaN for any q > 1 or < 0, or if this TDigest is empty. - """ - q = float(qq) - if (q < 0.0) or (q > 1.0): - return float('nan') - if self.isEmpty(): - return float('nan') - jcov = self.__covM__(q * self.mass()) - cov = jcov.map(lambda j: (self.clustX[j], self.clustM[j])) - m = q * self.mass() - if (not cov.l.isEmpty()) and (not cov.r.isEmpty()): - c1, tm1 = cov.l.get() - c2, tm2 = cov.r.get() - return self.__cdfI__(jcov.l.get(), m, c1, tm1, c2, tm2) - if not cov.r.isEmpty(): - c = cov.r.get()[0] - jcovR = self.__covR__(c) - covR = jcovR.map(lambda j: (self.clustX[j], self.clustM[j])) - if (not covR.l.isEmpty()) and (not covR.r.isEmpty()): - c1, tm1 = covR.l.get() - c2, tm2 = covR.r.get() - return self.__cdfI__(jcovR.l.get(), m, c1, tm1, c2, tm2) - return float('nan') - if not cov.l.isEmpty(): - c = cov.l.get()[0] - return c - return float('nan') - - def cdfDiscrete(self, xx): - """ - return CDF(x) for a numeric value x, assuming the sketch is representing a - discrete distribution. - """ - if self.isEmpty(): - return 0.0 - j = np.searchsorted(self.clustX, float(xx), side='right') - if (j == 0): - return 0.0 - return self.clustP[j - 1] / self.mass() - - def cdfDiscreteInverse(self, qq): - """ - Given a value q on [0,1], return the value x such that CDF(x) = q, assuming - the sketch is represenging a discrete distribution. - Returns NaN for any q > 1 or < 0, or if this TDigest is empty. - """ - q = float(qq) - if (q < 0.0) or (q > 1.0): - return float('nan') - if self.isEmpty(): - return float('nan') - j = np.searchsorted(self.clustP, q * self.mass(), side='left') - return self.clustX[j] - - def samplePDF(self): - """ - Return a random sampling from the sketched distribution, using inverse - transform sampling, assuming a continuous distribution. - """ - return self.cdfInverse(random.random()) - - def samplePMF(self): - """ - Return a random sampling from the sketched distribution, using inverse - transform sampling, assuming a discrete distribution. - """ - return self.cdfDiscreteInverse(random.random()) - - def sample(self): - """ - Return a random sampling from the sketched distribution, using inverse - transform sampling, assuming a discrete distribution if the number of - TDigest clusters is <= maxDiscrete, and a continuous distribution otherwise. - """ - if self.maxDiscrete <= self.nclusters: - return self.cdfDiscreteInverse(random.random()) - return self.cdfInverse(random.random()) - -class Cover(object): - """ - Analog of org.isarnproject.collections.mixmaps.nearest.Cover[Any]. - Not intended for public consumption. - """ - def __repr__(self): - return "Cover(%s, %s)" % (repr(self.l), repr(self.r)) - - def __init__(self, l, r): - self.l = Option(l) - self.r = Option(r) - - def map(self, f): - assert hasattr(f, '__call__'), "f must be callable" - return Cover(self.l.map(f).value, self.r.map(f).value) - -class Option(object): - """ - Analog of Scala Option[Any]. I only implemented the methods I need. - Not intented for public consumption. - """ - def __repr__(self): - return "Option(%s)" % (repr(self.value)) - - def __init__(self, v): - self.value = v - - def get(self): - assert self.value is not None, "Opt value was None" - return self.value - - def map(self, f): - assert hasattr(f, '__call__'), "f must be callable" - if self.value is None: - return Option(None) - return Option(f(self.value)) - - def isEmpty(self): - return self.value is None diff --git a/src/main/scala/org/apache/spark/isarnproject/sketches/udt/TDigestUDT.scala b/src/main/scala/org/apache/spark/isarnproject/sketches/udt/TDigestUDT.scala deleted file mode 100644 index de1e1ce..0000000 --- a/src/main/scala/org/apache/spark/isarnproject/sketches/udt/TDigestUDT.scala +++ /dev/null @@ -1,188 +0,0 @@ -/* -Copyright 2017 Erik Erlandson -Licensed 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.isarnproject.sketches.udt - -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} -import org.isarnproject.sketches.TDigest -import org.isarnproject.sketches.tdmap.TDigestMap - -/** A type for receiving the results of deserializing [[TDigestUDT]]. - * The payload is the tdigest member field, holding a TDigest object. - * This is necessary because (a) I define the TDigest type is defined in the isarn-sketches - * package and I do not - * want any Spark dependencies on that package, and (b) Spark has privatized UserDefinedType under - * org.apache.spark scope, and so I have to have a paired result type in the same scope. - * @param tdigest The TDigest payload, which does the actual sketching. - */ -@SQLUserDefinedType(udt = classOf[TDigestUDT]) -case class TDigestSQL(tdigest: TDigest) - -/** A UserDefinedType for serializing and deserializing [[TDigestSQL]] structures during UDAF - * aggregations. - */ -class TDigestUDT extends UserDefinedType[TDigestSQL] { - def userClass: Class[TDigestSQL] = classOf[TDigestSQL] - - override def pyUDT: String = "isarnproject.sketches.udt.tdigest.TDigestUDT" - - override def typeName: String = "tdigest" - - override def equals(obj: Any): Boolean = { - obj match { - case _: TDigestUDT => true - case _ => false - } - } - - override def hashCode(): Int = classOf[TDigestUDT].getName.hashCode() - - private[spark] override def asNullable: TDigestUDT = this - - def sqlType: DataType = StructType( - StructField("delta", DoubleType, false) :: - StructField("maxDiscrete", IntegerType, false) :: - StructField("nclusters", IntegerType, false) :: - StructField("clustX", ArrayType(DoubleType, false), false) :: - StructField("clustM", ArrayType(DoubleType, false), false) :: - Nil) - - def serialize(tdsql: TDigestSQL): Any = serializeTD(tdsql.tdigest) - - def deserialize(datum: Any): TDigestSQL = TDigestSQL(deserializeTD(datum)) - - private[sketches] def serializeTD(td: TDigest): InternalRow = { - val TDigest(delta, maxDiscrete, nclusters, clusters) = td - val row = new GenericInternalRow(5) - row.setDouble(0, delta) - row.setInt(1, maxDiscrete) - row.setInt(2, nclusters) - val clustX = clusters.keys.toArray - val clustM = clusters.values.toArray - row.update(3, UnsafeArrayData.fromPrimitiveArray(clustX)) - row.update(4, UnsafeArrayData.fromPrimitiveArray(clustM)) - row - } - - private[sketches] def deserializeTD(datum: Any): TDigest = datum match { - case row: InternalRow => - require(row.numFields == 5, s"expected row length 5, got ${row.numFields}") - val delta = row.getDouble(0) - val maxDiscrete = row.getInt(1) - val nclusters = row.getInt(2) - val clustX = row.getArray(3).toDoubleArray() - val clustM = row.getArray(4).toDoubleArray() - val clusters = clustX.zip(clustM) - .foldLeft(TDigestMap.empty) { case (td, e) => td + e } - TDigest(delta, maxDiscrete, nclusters, clusters) - case u => throw new Exception(s"failed to deserialize: $u") - } -} - -/** Instantiated instance of [[TDigestUDT]] for use by UDAF objects */ -case object TDigestUDT extends TDigestUDT - -/** A type for receiving the results of deserializing [[TDigestArrayUDT]]. - * The payload is the tdigests member field, holding an Array of TDigest objects. - * @param tdigests An array of TDigest objects, which do the actual sketching. - * @see [[TDigestSQL]] for additional context - */ -@SQLUserDefinedType(udt = classOf[TDigestArrayUDT]) -case class TDigestArraySQL(tdigests: Array[TDigest]) - -/** - * A UserDefinedType for serializing and deserializing [[TDigestArraySQL]] objects - * during UDAF aggregations - */ -class TDigestArrayUDT extends UserDefinedType[TDigestArraySQL] { - def userClass: Class[TDigestArraySQL] = classOf[TDigestArraySQL] - - override def pyUDT: String = "isarnproject.sketches.udt.tdigest.TDigestArrayUDT" - - override def typeName: String = "tdigestarray" - - override def equals(obj: Any): Boolean = { - obj match { - case _: TDigestArrayUDT => true - case _ => false - } - } - - override def hashCode(): Int = classOf[TDigestArrayUDT].getName.hashCode() - - private[spark] override def asNullable: TDigestArrayUDT = this - - // Spark seems to have trouble with ArrayType data that isn't - // serialized using UnsafeArrayData (SPARK-21277), so my workaround - // is to store all the cluster information flattened into single Unsafe arrays. - // To deserialize, I unpack the slices. - def sqlType: DataType = StructType( - StructField("delta", DoubleType, false) :: - StructField("maxDiscrete", IntegerType, false) :: - StructField("clusterS", ArrayType(IntegerType, false), false) :: - StructField("clusterX", ArrayType(DoubleType, false), false) :: - StructField("ClusterM", ArrayType(DoubleType, false), false) :: - Nil) - - def serialize(tdasql: TDigestArraySQL): Any = { - val row = new GenericInternalRow(5) - val tda: Array[TDigest] = tdasql.tdigests - val delta = if (tda.isEmpty) 0.0 else tda.head.delta - val maxDiscrete = if (tda.isEmpty) 0 else tda.head.maxDiscrete - val clustS = tda.map(_.nclusters) - val clustX = tda.flatMap(_.clusters.keys) - val clustM = tda.flatMap(_.clusters.values) - row.setDouble(0, delta) - row.setInt(1, maxDiscrete) - row.update(2, UnsafeArrayData.fromPrimitiveArray(clustS)) - row.update(3, UnsafeArrayData.fromPrimitiveArray(clustX)) - row.update(4, UnsafeArrayData.fromPrimitiveArray(clustM)) - row - } - - def deserialize(datum: Any): TDigestArraySQL = datum match { - case row: InternalRow => - require(row.numFields == 5, s"expected row length 5, got ${row.numFields}") - val delta = row.getDouble(0) - val maxDiscrete = row.getInt(1) - val clustS = row.getArray(2).toIntArray() - val clustX = row.getArray(3).toDoubleArray() - val clustM = row.getArray(4).toDoubleArray() - var beg = 0 - val tda = clustS.map { nclusters => - val x = clustX.slice(beg, beg + nclusters) - val m = clustM.slice(beg, beg + nclusters) - val clusters = x.zip(m).foldLeft(TDigestMap.empty) { case (td, e) => td + e } - val td = TDigest(delta, maxDiscrete, nclusters, clusters) - beg += nclusters - td - } - TDigestArraySQL(tda) - case u => throw new Exception(s"failed to deserialize: $u") - } -} - -/** A [[TDigestArrayUDT]] instance for use in declaring UDAF objects */ -case object TDigestArrayUDT extends TDigestArrayUDT - -/** Shims for exposing Spark's VectorUDT objects outside of org.apache.spark scope */ -object TDigestUDTInfra { - private val udtML = new org.apache.spark.ml.linalg.VectorUDT - def udtVectorML: DataType = udtML - - private val udtMLLib = new org.apache.spark.mllib.linalg.VectorUDT - def udtVectorMLLib: DataType = udtMLLib -} diff --git a/src/main/scala/org/isarnproject/pipelines/TDigestFI.scala b/src/main/scala/org/isarnproject/pipelines/spark/fi.scala similarity index 90% rename from src/main/scala/org/isarnproject/pipelines/TDigestFI.scala rename to src/main/scala/org/isarnproject/pipelines/spark/fi.scala index eab1a4b..1b74ab7 100644 --- a/src/main/scala/org/isarnproject/pipelines/TDigestFI.scala +++ b/src/main/scala/org/isarnproject/pipelines/spark/fi.scala @@ -1,5 +1,5 @@ /* -Copyright 2017 Erik Erlandson +Copyright 2017-2020 Erik Erlandson Licensed 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 @@ -11,10 +11,7 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.isarnproject.pipelines - -import scala.reflect.ClassTag -import scala.collection.mutable.WrappedArray +package org.isarnproject.pipelines.spark import org.apache.spark.ml.{Estimator, Model, PredictionModel} import org.apache.spark.ml.classification.ClassificationModel @@ -28,13 +25,10 @@ import org.apache.spark.sql.{Dataset, DataFrame} import org.apache.spark.broadcast.Broadcast import org.apache.spark.ml.linalg.{Vector => MLVector, DenseVector => MLDense, SparseVector => MLSparse} -import org.apache.spark.sql.expressions.MutableAggregationBuffer -import org.apache.spark.sql.expressions.UserDefinedAggregateFunction import org.apache.spark.sql.Row -import org.isarnproject.sketches.TDigest -import org.apache.spark.isarnproject.sketches.udt._ -import org.isarnproject.sketches.udaf._ +import org.isarnproject.sketches.java.TDigest +import org.apache.spark.isarnproject.sketches.tdigest.udt.infra.udtVectorML // Defining these in a subpackage so the package can have other // param definitions added to it elsewhere. I'm keeping them visible @@ -60,11 +54,11 @@ package params { * Defaults to 0.5 * @group param */ - final val delta: DoubleParam = - new DoubleParam(this, "delta", "t-digest compression (> 0)", ParamValidators.gt(0.0)) - setDefault(delta, org.isarnproject.sketches.TDigest.deltaDefault) - final def getDelta: Double = $(delta) - final def setDelta(value: Double): this.type = set(delta, value) + final val compression: DoubleParam = + new DoubleParam(this, "compression", "t-digest compression (> 0)", ParamValidators.gt(0.0)) + setDefault(compression, 0.5) + final def getCompression: Double = $(compression) + final def setCompression(value: Double): this.type = set(compression, value) /** * Maximum number of discrete values to sketch before transitioning to continuous mode @@ -149,6 +143,8 @@ package params { import params._ +package fi { + /** * Model/Transformer for transforming input feature data into a DataFrame containing * "name" and "importance" columns, mapping feature name to its computed importance. @@ -183,7 +179,7 @@ class TDigestFIModel( def transformSchema(schema: StructType): StructType = { require(schema.fieldNames.contains($(featuresCol))) schema($(featuresCol)) match { - case sf: StructField => require(sf.dataType.equals(TDigestUDTInfra.udtVectorML)) + case sf: StructField => require(sf.dataType.equals(udtVectorML)) } // Output is two columns: feature names and corresponding importances @@ -217,7 +213,7 @@ class TDigestFIModel( val refpred = predictMethod.invoke(model, fvec).asInstanceOf[Double] for { j <- 0 until m } { val t = farr(j) - farr(j) = ftd(j).sample + farr(j) = ftd(j).sample() val pred = predictMethod.invoke(model, fvec).asInstanceOf[Double] farr(j) = t dev(j) += fdev(refpred, pred) @@ -262,7 +258,7 @@ class TDigestFI(override val uid: String) extends Estimator[TDigestFIModel] with def transformSchema(schema: StructType): StructType = { require(schema.fieldNames.contains($(featuresCol))) schema($(featuresCol)) match { - case sf: StructField => require(sf.dataType.equals(TDigestUDTInfra.udtVectorML)) + case sf: StructField => require(sf.dataType.equals(udtVectorML)) } // I can't figure out the purpose for outputting a modified schema here. // Until further notice I'm going to output an empty one. @@ -274,19 +270,19 @@ class TDigestFI(override val uid: String) extends Estimator[TDigestFIModel] with .treeAggregate(Array.empty[TDigest])({ case (ttd, Row(fv: MLVector)) => val m = fv.size val td = - if (ttd.length > 0) ttd else Array.fill(m)(TDigest.empty($(delta), $(maxDiscrete))) + if (ttd.length > 0) ttd else Array.fill(m)(TDigest.empty($(compression), $(maxDiscrete))) require(td.length == m, "Inconsistent feature vector size $m") fv match { case v: MLSparse => var jBeg = 0 v.foreachActive((j, x) => { - for { k <- jBeg until j } { td(k) += 0.0 } - td(j) += x + for { k <- jBeg until j } { td(k).update(0.0) } + td(j).update(x) jBeg = j + 1 }) - for { k <- jBeg until v.size } { td(k) += 0.0 } + for { k <- jBeg until v.size } { td(k).update(0.0) } case _ => - for { j <- 0 until fv.size } { td(j) += fv(j) } + for { j <- 0 until fv.size } { td(j).update(fv(j)) } } td }, @@ -298,7 +294,7 @@ class TDigestFI(override val uid: String) extends Estimator[TDigestFIModel] with } else { require(td1.length == td2.length, "mismatched t-digest arrays") for { j <- 0 until td1.length } { - td1(j) ++= td2(j) + td1(j).merge(td2(j)) } td1 }) @@ -307,3 +303,5 @@ class TDigestFI(override val uid: String) extends Estimator[TDigestFIModel] with model } } + +} // package fi diff --git a/src/main/scala/org/isarnproject/pipelines/package.scala b/src/main/scala/org/isarnproject/pipelines/spark/package.scala similarity index 90% rename from src/main/scala/org/isarnproject/pipelines/package.scala rename to src/main/scala/org/isarnproject/pipelines/spark/package.scala index fcacf0b..cb9c3e4 100644 --- a/src/main/scala/org/isarnproject/pipelines/package.scala +++ b/src/main/scala/org/isarnproject/pipelines/spark/package.scala @@ -1,5 +1,5 @@ /* -Copyright 2017 Erik Erlandson +Copyright 2017-2020 Erik Erlandson Licensed 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 @@ -11,11 +11,11 @@ See the License for the specific language governing permissions and limitations under the License. */ -package org.isarnproject +package org.isarnproject.pipelines import scala.language.existentials -package object pipelines { +package object spark { private[pipelines] def inheritances(obj: Any): Seq[String] = { var ih: List[Class[_]] = Nil var sc = obj.getClass.getSuperclass diff --git a/src/main/scala/org/isarnproject/sketches/spark/infra.scala b/src/main/scala/org/isarnproject/sketches/spark/infra.scala new file mode 100644 index 0000000..21457e3 --- /dev/null +++ b/src/main/scala/org/isarnproject/sketches/spark/infra.scala @@ -0,0 +1,53 @@ +/* +Copyright 2017-2020 Erik Erlandson +Licensed 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.isarnproject.sketches.spark.infra + +// scala's standard Numeric doesn't support java.lang.xxx +trait ScalarNumeric[N] extends Serializable { + def toDouble(v: N): Double +} +object ScalarNumeric { + implicit val javaIntIsSN: ScalarNumeric[java.lang.Integer] = + new ScalarNumeric[java.lang.Integer] { + @inline def toDouble(v: java.lang.Integer): Double = v.toDouble + } + implicit val javaLongIsSN: ScalarNumeric[java.lang.Long] = + new ScalarNumeric[java.lang.Long] { + @inline def toDouble(v: java.lang.Long): Double = v.toDouble + } + implicit val javaFloatIsSN: ScalarNumeric[java.lang.Float] = + new ScalarNumeric[java.lang.Float] { + @inline def toDouble(v: java.lang.Float): Double = v.toDouble + } + implicit val javaDoubleIsSN: ScalarNumeric[java.lang.Double] = + new ScalarNumeric[java.lang.Double] { + @inline def toDouble(v: java.lang.Double): Double = v + } + implicit val scalaIntIsSN: ScalarNumeric[Int] = + new ScalarNumeric[Int] { + @inline def toDouble(v: Int): Double = v.toDouble + } + implicit val scalaLongIsSN: ScalarNumeric[Long] = + new ScalarNumeric[Long] { + @inline def toDouble(v: Long): Double = v.toDouble + } + implicit val scalaFloatIsSN: ScalarNumeric[Float] = + new ScalarNumeric[Float] { + @inline def toDouble(v: Float): Double = v.toDouble + } + implicit val scalaDoubleIsSN: ScalarNumeric[Double] = + new ScalarNumeric[Double] { + @inline def toDouble(v: Double): Double = v + } +} diff --git a/src/main/scala/org/isarnproject/sketches/spark/tdigest.scala b/src/main/scala/org/isarnproject/sketches/spark/tdigest.scala new file mode 100644 index 0000000..cff2d9f --- /dev/null +++ b/src/main/scala/org/isarnproject/sketches/spark/tdigest.scala @@ -0,0 +1,631 @@ +/* +Copyright 2017-2020 Erik Erlandson +Licensed 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. +*/ + +/** + * Aggregators and supporting definitions for using T-Digest sketches with Spark + * DataFrame and Dataset. + */ +package org.isarnproject.sketches.spark.tdigest { + +import org.apache.spark.sql.types.SQLUserDefinedType +import org.apache.spark.sql.expressions.Aggregator +import org.apache.spark.sql.{ Encoder, Encoders } +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + +// Every time I do this I die a little inside +import org.apache.spark.mllib.linalg.{ Vector => MLLibVec } +import org.apache.spark.ml.linalg.{ Vector => MLVec } + +import org.isarnproject.sketches.spark.infra.ScalarNumeric + +import infra.TDigest + +/** + * A spark Aggregator for sketching a numeric column with a t-digest + * @tparam V the numeric value type + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ +class TDigestAggregator[V](compression: Double, maxDiscrete: Int)( + implicit + vnum: ScalarNumeric[V]) + extends + Aggregator[V, TDigest, TDigest] +{ + def zero: TDigest = new TDigest(compression, maxDiscrete) + def reduce(td: TDigest, v: V): TDigest = { + if (v != null) td.update(vnum.toDouble(v)) + td + } + def merge(td1: TDigest, td2: TDigest): TDigest = { + td1.merge(td2) + td1 + } + def finish(td: TDigest): TDigest = td + def bufferEncoder: Encoder[TDigest] = ExpressionEncoder[TDigest]() + def outputEncoder: Encoder[TDigest] = ExpressionEncoder[TDigest]() +} + +/** Companion object and definitions for TDigestAggregator */ +object TDigestAggregator { + import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.sql.functions.udaf + import org.apache.spark.sql.expressions.UserDefinedFunction + + /** + * Construct a TDigestAggregator + * @tparam V the numeric value type expected from input column + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def apply[V]( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault)( + implicit + vnum: ScalarNumeric[V]): TDigestAggregator[V] = + new TDigestAggregator[V](compression, maxDiscrete) + + /** + * Return a spark UDF, usable with DataFrames, + * that aggregates a numeric column using a t-digest. + * @tparam V the numeric value type expected from input column + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def udf[V]( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault)( + implicit + vnum: ScalarNumeric[V], + ttV: TypeTag[V]): UserDefinedFunction = + udaf(apply[V](compression, maxDiscrete)) +} + +/** + * A base class for aggregating over array-like collections of values. + * Supplies all Aggregator methods except `reduce`, which is class specific. + * Maintains and returns an Array[TDigest], regardless of specific input type. + * @tparam V the specific array-like column type expected from input column + */ +abstract class TDigestArrayAggregatorBase[V] + extends + Aggregator[V, Array[TDigest], Array[TDigest]] { + def zero: Array[TDigest] = Array.empty[TDigest] + def merge(td1: Array[TDigest], td2: Array[TDigest]): Array[TDigest] = { + if (td1.isEmpty) td2 + else if (td2.isEmpty) td1 + else { + require(td1.length == td2.length) + for { j <- 0 until td1.length } { td1(j).merge(td2(j)) } + td1 + } + } + def finish(td: Array[TDigest]): Array[TDigest] = td + def bufferEncoder: Encoder[Array[TDigest]] = ExpressionEncoder[Array[TDigest]]() + def outputEncoder: Encoder[Array[TDigest]] = ExpressionEncoder[Array[TDigest]]() +} + +/** + * A spark Aggregator for sketching an array of numeric values with an array of t-digests + * @tparam V the numeric value type contained in the array + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ +class TDigestArrayAggregator[V]( + compression: Double, + maxDiscrete: Int)( + implicit + vnum: ScalarNumeric[V]) + extends + TDigestArrayAggregatorBase[Array[V]] { + + def reduce(tdai: Array[TDigest], data: Array[V]): Array[TDigest] = { + if (data == null) tdai else { + val tda = if (!tdai.isEmpty || data.isEmpty) tdai else + Array.fill(data.length) { new TDigest(compression, maxDiscrete) } + require(tda.length == data.length) + for { j <- 0 until tda.length } { tda(j).update(vnum.toDouble(data(j))) } + tda + } + } +} + +/** Companion object and definitions */ +object TDigestArrayAggregator { + import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.sql.functions.udaf + import org.apache.spark.sql.expressions.UserDefinedFunction + + /** + * Construct a TDigestArrayAggregator + * @tparam V the numeric value type contained in the array column + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def apply[V]( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault)( + implicit + vnum: ScalarNumeric[V]): TDigestArrayAggregator[V] = + new TDigestArrayAggregator[V](compression, maxDiscrete) + + /** + * Return a spark UDF, usable with DataFrames, + * that aggregates a numeric array column using an array of t-digests. + * @tparam V the numeric value type expected from input column + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def udf[V]( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault)( + implicit + vnum: ScalarNumeric[V], + ttV: TypeTag[V]): UserDefinedFunction = + udaf(apply[V](compression, maxDiscrete)) +} + +/** + * A spark Aggregator for sketching the values in an MLLib Vector with an array of t-digests + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ +class TDigestMLLibVecAggregator( + compression: Double, + maxDiscrete: Int) + extends + TDigestArrayAggregatorBase[MLLibVec] { + + def reduce(tdai: Array[TDigest], data: MLLibVec): Array[TDigest] = { + if (data == null) tdai else { + val tda = if (!tdai.isEmpty || (data.size == 0)) tdai else + Array.fill(data.size) { new TDigest(compression, maxDiscrete) } + require(tda.length == data.size) + data match { + case v: org.apache.spark.mllib.linalg.SparseVector => + var jBeg = 0 + v.foreachActive((j, x) => { + for { k <- jBeg until j } { tda(k).update(0.0) } + tda(j).update(x) + jBeg = j + 1 + }) + for { k <- jBeg until data.size } { tda(k).update(0.0) } + case _ => + for { j <- 0 until data.size } { tda(j).update(data(j)) } + } + tda + } + } +} + +/** Companion object and definitions */ +object TDigestMLLibVecAggregator { + import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.sql.functions.udaf + import org.apache.spark.sql.expressions.UserDefinedFunction + + /** + * Construct a TDigestMLLibVecAggregator + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def apply( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): TDigestMLLibVecAggregator = + new TDigestMLLibVecAggregator(compression, maxDiscrete) + + /** + * Return a spark UDF, usable with DataFrames, + * that aggregates an MLLib Vector column using an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def udf( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): UserDefinedFunction = + udaf(apply(compression, maxDiscrete)) +} + +/** + * A spark Aggregator for sketching the values in an ML Vector with an array of t-digests + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ +class TDigestMLVecAggregator( + compression: Double, + maxDiscrete: Int) + extends + TDigestArrayAggregatorBase[MLVec] { + + def reduce(tdai: Array[TDigest], data: MLVec): Array[TDigest] = { + if (data == null) tdai else { + val tda = if (!tdai.isEmpty || (data.size == 0)) tdai else + Array.fill(data.size) { new TDigest(compression, maxDiscrete) } + require(tda.length == data.size) + data match { + case v: org.apache.spark.ml.linalg.SparseVector => + var jBeg = 0 + v.foreachActive((j, x) => { + for { k <- jBeg until j } { tda(k).update(0.0) } + tda(j).update(x) + jBeg = j + 1 + }) + for { k <- jBeg until data.size } { tda(k).update(0.0) } + case _ => + for { j <- 0 until data.size } { tda(j).update(data(j)) } + } + tda + } + } +} + +/** Companion object and definitions */ +object TDigestMLVecAggregator { + import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.sql.functions.udaf + import org.apache.spark.sql.expressions.UserDefinedFunction + + /** + * Construct a TDigestMLVecAggregator + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def apply( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): TDigestMLVecAggregator = + new TDigestMLVecAggregator(compression, maxDiscrete) + + /** + * Return a spark UDF, usable with DataFrames, + * that aggregates an ML Vector column using an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def udf( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): UserDefinedFunction = + udaf(apply(compression, maxDiscrete)) +} + +/** + * A spark Aggregator for reducing a column of t-digests down to a single t-digest. + * Therefore, in this Aggregator, 'reduce' and 'merge' both perform a t-digest merge operation. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ +class TDigestReduceAggregator( + compression: Double, + maxDiscrete: Int) + extends + Aggregator[TDigest, TDigest, TDigest] +{ + def zero: TDigest = new TDigest(compression, maxDiscrete) + def reduce(td: TDigest, tdi: TDigest): TDigest = { + if (tdi != null) td.merge(tdi) + td + } + def merge(td1: TDigest, td2: TDigest): TDigest = { + td1.merge(td2) + td1 + } + def finish(td: TDigest): TDigest = td + def bufferEncoder: Encoder[TDigest] = ExpressionEncoder[TDigest]() + def outputEncoder: Encoder[TDigest] = ExpressionEncoder[TDigest]() +} + +/** Companion object and definitions */ +object TDigestReduceAggregator { + import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.sql.functions.udaf + import org.apache.spark.sql.expressions.UserDefinedFunction + + /** + * Construct a TDigestReduceAggregator + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def apply( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): TDigestReduceAggregator = + new TDigestReduceAggregator(compression, maxDiscrete) + + /** + * Return a spark UDF, usable with DataFrames, + * that aggregates a column of t-digests down to a single t-digest. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def udf( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): UserDefinedFunction = + udaf(apply(compression, maxDiscrete)) +} + +/** + * A spark Aggregator for reducing a column containing arrays of t-digests + * down to a single array of t-digests. + * Therefore, in this Aggregator, 'reduce' and 'merge' both perform t-digest merge operations. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ +class TDigestArrayReduceAggregator( + compression: Double, + maxDiscrete: Int) + extends + TDigestArrayAggregatorBase[Array[TDigest]] { + + def reduce(tdai: Array[TDigest], data: Array[TDigest]): Array[TDigest] = { + if (data == null) tdai else { + val tda = if (!tdai.isEmpty || data.isEmpty) tdai else + Array.fill(data.length) { new TDigest(compression, maxDiscrete) } + require(tda.length == data.length) + for { j <- 0 until tda.length } { tda(j).merge(data(j)) } + tda + } + } +} + +/** Companion object and definitions */ +object TDigestArrayReduceAggregator { + import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.sql.functions.udaf + import org.apache.spark.sql.expressions.UserDefinedFunction + + /** + * Construct a TDigestArrayReduceAggregator + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def apply( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): TDigestArrayReduceAggregator = + new TDigestArrayReduceAggregator(compression, maxDiscrete) + + /** + * Return a spark UDF, usable with DataFrames, + * that aggregates a column of t-digest arrays down to a single t-digest array. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + */ + def udf( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault): UserDefinedFunction = + udaf(apply(compression, maxDiscrete)) +} + +/** + * Convenience functions that do not require type parameters or typeclasses to invoke. + * Use cases include calling from java or supporting pyspark bindings. + */ +object functions { + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of integer values with a t-digest. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestIntUDF(compression: Double, maxDiscrete: Int) = + TDigestAggregator.udf[Int](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of long-integer values with a t-digest. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestLongUDF(compression: Double, maxDiscrete: Int) = + TDigestAggregator.udf[Long](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of float values with a t-digest. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestFloatUDF(compression: Double, maxDiscrete: Int) = + TDigestAggregator.udf[Float](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of double values with a t-digest. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestDoubleUDF(compression: Double, maxDiscrete: Int) = + TDigestAggregator.udf[Double](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of integer arrays with an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestIntArrayUDF(compression: Double, maxDiscrete: Int) = + TDigestArrayAggregator.udf[Int](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of long-integer arrays with an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestLongArrayUDF(compression: Double, maxDiscrete: Int) = + TDigestArrayAggregator.udf[Long](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of float arrays with an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestFloatArrayUDF(compression: Double, maxDiscrete: Int) = + TDigestArrayAggregator.udf[Float](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of double arrays with an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestDoubleArrayUDF(compression: Double, maxDiscrete: Int) = + TDigestArrayAggregator.udf[Double](compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of MLLib Vectors with an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestMLLibVecUDF(compression: Double, maxDiscrete: Int) = + TDigestMLLibVecAggregator.udf(compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of ML Vectors with an array of t-digests. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestMLVecUDF(compression: Double, maxDiscrete: Int) = + TDigestMLVecAggregator.udf(compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of t-digests into a single t-digest. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestReduceUDF(compression: Double, maxDiscrete: Int) = + TDigestReduceAggregator.udf(compression, maxDiscrete) + + /** + * Obtain a UDF, usable with DataFrames, + * for aggregating a column of t-digest arrays into a single t-digest array. + * @param compression the t-digest compression parameter. + * @param maxDiscrete maximum number of discrete values to track in PMF mode. + * @return the new aggregating UDF + */ + def tdigestArrayReduceUDF(compression: Double, maxDiscrete: Int) = + TDigestArrayReduceAggregator.udf(compression, maxDiscrete) +} + +package infra { + import org.isarnproject.sketches.java.{ TDigest => BaseTD } + import org.apache.spark.isarnproject.sketches.tdigest.udt.TDigestUDT + + // the only reason for this shim class is to link it to TDigestUDT + // the user does not need to see this shim, and can do: + // resultRow.getAs[org.isarnproject.sketches.java.TDigest](0) + @SQLUserDefinedType(udt = classOf[TDigestUDT]) + class TDigest( + compression: Double, + maxDiscrete: Int, + cent: Array[Double], + mass: Array[Double]) + extends + BaseTD(compression, maxDiscrete, cent, mass) { + + def this( + compression: Double = TDigest.compressionDefault, + maxDiscrete: Int = TDigest.maxDiscreteDefault) { + this(compression, maxDiscrete, null, null) + } + } + + object TDigest { + val compressionDefault: Double = 0.5 + val maxDiscreteDefault: Int = 0 + } +} + +} // package org.isarnproject.sketches.spark.tdigest + +// I need to accept that Spark is never going to fix this. +package org.apache.spark.isarnproject.sketches.tdigest.udt { + +import java.util.Arrays + +import org.apache.spark.sql.types._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} + +import org.isarnproject.sketches.spark.tdigest.infra.TDigest + +class TDigestUDT extends UserDefinedType[TDigest] { + def userClass: Class[TDigest] = classOf[TDigest] + + override def pyUDT: String = "isarnproject.sketches.spark.tdigest.TDigestUDT" + + override def typeName: String = "tdigest" + + override def equals(obj: Any): Boolean = { + obj match { + case _: TDigestUDT => true + case _ => false + } + } + + override def hashCode(): Int = classOf[TDigestUDT].getName.hashCode() + + private[spark] override def asNullable: TDigestUDT = this + + def sqlType: DataType = StructType( + StructField("compression", DoubleType, false) :: + StructField("maxDiscrete", IntegerType, false) :: + StructField("cent", ArrayType(DoubleType, false), false) :: + StructField("mass", ArrayType(DoubleType, false), false) :: + Nil) + + def serialize(td: TDigest): Any = { + val row = new GenericInternalRow(4) + row.setDouble(0, td.getCompression()) + row.setInt(1, td.getMaxDiscrete()) + val clustX = java.util.Arrays.copyOf(td.getCentUnsafe(), td.size()) + row.update(2, UnsafeArrayData.fromPrimitiveArray(clustX)) + val clustM = java.util.Arrays.copyOf(td.getMassUnsafe(), td.size()) + row.update(3, UnsafeArrayData.fromPrimitiveArray(clustM)) + row + } + + def deserialize(datum: Any): TDigest = datum match { + case row: InternalRow => + require(row.numFields == 4, s"expected row length 4, got ${row.numFields}") + val compression = row.getDouble(0) + val maxDiscrete = row.getInt(1) + val clustX = row.getArray(2).toDoubleArray() + val clustM = row.getArray(3).toDoubleArray() + new TDigest(compression, maxDiscrete, clustX, clustM) + case u => throw new Exception(s"failed to deserialize: $u") + } +} + +/** Shims for exposing Spark's VectorUDT objects outside of org.apache.spark scope */ +object infra { + private object udtML extends org.apache.spark.ml.linalg.VectorUDT + def udtVectorML: DataType = udtML + + private object udtMLLib extends org.apache.spark.mllib.linalg.VectorUDT + def udtVectorMLLib: DataType = udtMLLib +} + +} // package diff --git a/src/main/scala/org/isarnproject/sketches/udaf/TDigestUDAF.scala b/src/main/scala/org/isarnproject/sketches/udaf/TDigestUDAF.scala deleted file mode 100644 index 84019ae..0000000 --- a/src/main/scala/org/isarnproject/sketches/udaf/TDigestUDAF.scala +++ /dev/null @@ -1,347 +0,0 @@ -/* -Copyright 2017 Erik Erlandson -Licensed 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.isarnproject.sketches.udaf - -import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.expressions.MutableAggregationBuffer -import org.apache.spark.sql.expressions.UserDefinedAggregateFunction -import org.apache.spark.sql.Row - -import org.isarnproject.sketches.TDigest - -import org.apache.spark.isarnproject.sketches.udt._ - -/** - * A UDAF for sketching numeric data with a TDigest. - * Expected to be created using [[tdigestUDAF]]. - * @tparam N the expected numeric type of the data; Double, Int, etc - * @param deltaV The delta value to be used by the TDigest object - * @param maxDiscreteV The maxDiscrete value to be used by the TDigest object - */ -case class TDigestUDAF[N](deltaV: Double, maxDiscreteV: Int)(implicit - num: Numeric[N], - dataTpe: TDigestUDAFDataType[N]) extends UserDefinedAggregateFunction { - - /** customize the delta value to be used by the TDigest object */ - def delta(deltaP: Double) = this.copy(deltaV = deltaP) - - /** customize the maxDiscrete value to be used by the TDigest object */ - def maxDiscrete(maxDiscreteP: Int) = this.copy(maxDiscreteV = maxDiscreteP) - - // A t-digest is deterministic, but it is only statistically associative or commutative - // and spark will merge partition results in nondeterministic order. That makes - // the result of the aggregation statistically "deterministic" but not strictly so. - def deterministic: Boolean = false - - def inputSchema: StructType = StructType(StructField("x", dataTpe.tpe) :: Nil) - - def bufferSchema: StructType = StructType(StructField("tdigest", TDigestUDT) :: Nil) - - def dataType: DataType = TDigestUDT - - def initialize(buf: MutableAggregationBuffer): Unit = { - buf(0) = TDigestSQL(TDigest.empty(deltaV, maxDiscreteV)) - } - - def update(buf: MutableAggregationBuffer, input: Row): Unit = { - if (!input.isNullAt(0)) { - buf(0) = TDigestSQL(buf.getAs[TDigestSQL](0).tdigest + num.toDouble(input.getAs[N](0))) - } - } - - def merge(buf1: MutableAggregationBuffer, buf2: Row): Unit = { - buf1(0) = TDigestSQL(buf1.getAs[TDigestSQL](0).tdigest ++ buf2.getAs[TDigestSQL](0).tdigest) - } - - def evaluate(buf: Row): Any = buf.getAs[TDigestSQL](0) -} - -/** A base class that defines the common functionality for array sketching UDAFs */ -abstract class TDigestMultiUDAF extends UserDefinedAggregateFunction { - def deltaV: Double - def maxDiscreteV: Int - - def deterministic: Boolean = false - - def bufferSchema: StructType = StructType(StructField("tdigests", TDigestArrayUDT) :: Nil) - - def dataType: DataType = TDigestArrayUDT - - def initialize(buf: MutableAggregationBuffer): Unit = { - // we don't know vector size yet - buf(0) = TDigestArraySQL(Array.empty[TDigest]) - } - - def merge(buf1: MutableAggregationBuffer, buf2: Row): Unit = { - val tds2 = buf2.getAs[TDigestArraySQL](0).tdigests - if (!tds2.isEmpty) { - val tdt = buf1.getAs[TDigestArraySQL](0).tdigests - val tds1 = if (!tdt.isEmpty) tdt else { - Array.fill(tds2.length) { TDigest.empty(deltaV, maxDiscreteV) } - } - require(tds1.length == tds2.length) - for { j <- 0 until tds1.length } { tds1(j) ++= tds2(j) } - buf1(0) = TDigestArraySQL(tds1) - } - } - - def evaluate(buf: Row): Any = buf.getAs[TDigestArraySQL](0) -} - -/** - * A UDAF for sketching a column of ML Vectors with an array of TDigest objects. - * Expected to be created using [[tdigestMLVecUDAF]]. - * @param deltaV The delta value to be used by the TDigest object - * @param maxDiscreteV The maxDiscrete value to be used by the TDigest object - */ -case class TDigestMLVecUDAF(deltaV: Double, maxDiscreteV: Int) extends TDigestMultiUDAF { - import org.apache.spark.ml.linalg.{ Vector => Vec } - - /** customize the delta value to be used by the TDigest object */ - def delta(deltaP: Double) = this.copy(deltaV = deltaP) - - /** customize the maxDiscrete value to be used by the TDigest object */ - def maxDiscrete(maxDiscreteP: Int) = this.copy(maxDiscreteV = maxDiscreteP) - - def inputSchema: StructType = StructType(StructField("vector", TDigestUDTInfra.udtVectorML) :: Nil) - - def update(buf: MutableAggregationBuffer, input: Row): Unit = { - if (!input.isNullAt(0)) { - val vec = input.getAs[Vec](0) - val tdt = buf.getAs[TDigestArraySQL](0).tdigests - val tdigests = if (!tdt.isEmpty) tdt else { - Array.fill(vec.size) { TDigest.empty(deltaV, maxDiscreteV) } - } - require(tdigests.length == vec.size) - vec match { - case v: org.apache.spark.ml.linalg.SparseVector => - var jBeg = 0 - v.foreachActive((j, x) => { - for { k <- jBeg until j } { tdigests(k) += 0.0 } - tdigests(j) += x - jBeg = j + 1 - }) - for { k <- jBeg until vec.size } { tdigests(k) += 0.0 } - case _ => - for { j <- 0 until vec.size } { tdigests(j) += vec(j) } - } - buf(0) = TDigestArraySQL(tdigests) - } - } -} - -/** - * A UDAF for sketching a column of MLLib Vectors with an array of TDigest objects. - * Expected to be created using [[tdigestMLLibVecUDAF]]. - * @param deltaV The delta value to be used by the TDigest object - * @param maxDiscreteV The maxDiscrete value to be used by the TDigest object - */ -case class TDigestMLLibVecUDAF(deltaV: Double, maxDiscreteV: Int) extends TDigestMultiUDAF { - import org.apache.spark.mllib.linalg.{ Vector => Vec } - - /** customize the delta value to be used by the TDigest object */ - def delta(deltaP: Double) = this.copy(deltaV = deltaP) - - /** customize the maxDiscrete value to be used by the TDigest object */ - def maxDiscrete(maxDiscreteP: Int) = this.copy(maxDiscreteV = maxDiscreteP) - - def inputSchema: StructType = - StructType(StructField("vector", TDigestUDTInfra.udtVectorMLLib) :: Nil) - - def update(buf: MutableAggregationBuffer, input: Row): Unit = { - if (!input.isNullAt(0)) { - val vec = input.getAs[Vec](0) - val tdt = buf.getAs[TDigestArraySQL](0).tdigests - val tdigests = if (!tdt.isEmpty) tdt else { - Array.fill(vec.size) { TDigest.empty(deltaV, maxDiscreteV) } - } - require(tdigests.length == vec.size) - vec match { - case v: org.apache.spark.mllib.linalg.SparseVector => - var jBeg = 0 - v.foreachActive((j, x) => { - for { k <- jBeg until j } { tdigests(k) += 0.0 } - tdigests(j) += x - jBeg = j + 1 - }) - for { k <- jBeg until vec.size } { tdigests(k) += 0.0 } - case _ => - for { j <- 0 until vec.size } { tdigests(j) += vec(j) } - } - buf(0) = TDigestArraySQL(tdigests) - } - } -} - -/** - * A UDAF for sketching a column of numeric ArrayData with an array of TDigest objects. - * Expected to be created using [[tdigestArrayUDAF]]. - * @tparam N the expected numeric type of the data; Double, Int, etc - * @param deltaV The delta value to be used by the TDigest objects - * @param maxDiscreteV The maxDiscrete value to be used by the TDigest objects - */ -case class TDigestArrayUDAF[N](deltaV: Double, maxDiscreteV: Int)(implicit - num: Numeric[N], - dataTpe: TDigestUDAFDataType[N]) extends TDigestMultiUDAF { - - /** customize the delta value to be used by the TDigest object */ - def delta(deltaP: Double) = this.copy(deltaV = deltaP) - - /** customize the maxDiscrete value to be used by the TDigest object */ - def maxDiscrete(maxDiscreteP: Int) = this.copy(maxDiscreteV = maxDiscreteP) - - def inputSchema: StructType = - StructType(StructField("array", ArrayType(dataTpe.tpe, true)) :: Nil) - - def update(buf: MutableAggregationBuffer, input: Row): Unit = { - if (!input.isNullAt(0)) { - val data = input.getSeq[N](0) - val tdt = buf.getAs[TDigestArraySQL](0).tdigests - val tdigests = if (!tdt.isEmpty) tdt else { - Array.fill(data.length) { TDigest.empty(deltaV, maxDiscreteV) } - } - require(tdigests.length == data.length) - var j = 0 - for { x <- data } { - if (x != null) tdigests(j) += num.toDouble(x) - j += 1 - } - buf(0) = TDigestArraySQL(tdigests) - } - } -} - -/** - * A UDAF for aggregating (reducing) a column of t-digests. - * Expected to be created using [[tdigestReduceUDAF]]. - * @param deltaV The delta value to be used by the TDigest objects - * @param maxDiscreteV The maxDiscrete value to be used by the TDigest objects - */ -case class TDigestReduceUDAF(deltaV: Double, maxDiscreteV: Int) extends - UserDefinedAggregateFunction { - - /** customize the delta value to be used by the TDigest object */ - def delta(deltaP: Double) = this.copy(deltaV = deltaP) - - /** customize the maxDiscrete value to be used by the TDigest object */ - def maxDiscrete(maxDiscreteP: Int) = this.copy(maxDiscreteV = maxDiscreteP) - - def deterministic: Boolean = false - - def inputSchema: StructType = StructType(StructField("tdigest", TDigestUDT) :: Nil) - - def bufferSchema: StructType = StructType(StructField("tdigest", TDigestUDT) :: Nil) - - def dataType: DataType = TDigestUDT - - def initialize(buf: MutableAggregationBuffer): Unit = { - buf(0) = TDigestSQL(TDigest.empty(deltaV, maxDiscreteV)) - } - - def update(buf: MutableAggregationBuffer, input: Row): Unit = this.merge(buf, input) - - def merge(buf1: MutableAggregationBuffer, buf2: Row): Unit = { - if (!buf2.isNullAt(0)) { - buf1(0) = TDigestSQL(buf1.getAs[TDigestSQL](0).tdigest ++ buf2.getAs[TDigestSQL](0).tdigest) - } - } - - def evaluate(buf: Row): Any = buf.getAs[TDigestSQL](0) -} - -/** - * A UDAF for aggregating (reducing) a column of t-digest vectors. - * Expected to be created using [[tdigestArrayReduceUDAF]]. - * @param deltaV The delta value to be used by the TDigest objects - * @param maxDiscreteV The maxDiscrete value to be used by the TDigest objects - */ -case class TDigestArrayReduceUDAF(deltaV: Double, maxDiscreteV: Int) extends - UserDefinedAggregateFunction { - - /** customize the delta value to be used by the TDigest object */ - def delta(deltaP: Double) = this.copy(deltaV = deltaP) - - /** customize the maxDiscrete value to be used by the TDigest object */ - def maxDiscrete(maxDiscreteP: Int) = this.copy(maxDiscreteV = maxDiscreteP) - - def deterministic: Boolean = false - - def inputSchema: StructType = StructType(StructField("tdigests", TDigestArrayUDT) :: Nil) - - def bufferSchema: StructType = StructType(StructField("tdigests", TDigestArrayUDT) :: Nil) - - def dataType: DataType = TDigestArrayUDT - - def initialize(buf: MutableAggregationBuffer): Unit = { - buf(0) = TDigestArraySQL(Array.empty[TDigest]) - } - - def update(buf: MutableAggregationBuffer, input: Row): Unit = this.merge(buf, input) - - def merge(buf1: MutableAggregationBuffer, buf2: Row): Unit = { - if (!buf2.isNullAt(0)) { - val tds2 = buf2.getAs[TDigestArraySQL](0).tdigests - if (!tds2.isEmpty) { - val tdt = buf1.getAs[TDigestArraySQL](0).tdigests - val tds1 = if (!tdt.isEmpty) tdt else { - Array.fill(tds2.length) { TDigest.empty(deltaV, maxDiscreteV) } - } - require(tds1.length == tds2.length) - for { j <- 0 until tds1.length } { tds1(j) ++= tds2(j) } - buf1(0) = TDigestArraySQL(tds1) - } - } - } - - def evaluate(buf: Row): Any = buf.getAs[TDigestArraySQL](0) -} - -object pythonBindings { - def tdigestIntUDAF(delta: Double, maxDiscrete: Int) = - TDigestUDAF[Int](delta, maxDiscrete) - - def tdigestLongUDAF(delta: Double, maxDiscrete: Int) = - TDigestUDAF[Long](delta, maxDiscrete) - - def tdigestFloatUDAF(delta: Double, maxDiscrete: Int) = - TDigestUDAF[Float](delta, maxDiscrete) - - def tdigestDoubleUDAF(delta: Double, maxDiscrete: Int) = - TDigestUDAF[Double](delta, maxDiscrete) - - def tdigestMLVecUDAF(delta: Double, maxDiscrete: Int) = - TDigestMLVecUDAF(delta, maxDiscrete) - - def tdigestMLLibVecUDAF(delta: Double, maxDiscrete: Int) = - TDigestMLLibVecUDAF(delta, maxDiscrete) - - def tdigestIntArrayUDAF(delta: Double, maxDiscrete: Int) = - TDigestArrayUDAF[Int](delta, maxDiscrete) - - def tdigestLongArrayUDAF(delta: Double, maxDiscrete: Int) = - TDigestArrayUDAF[Long](delta, maxDiscrete) - - def tdigestFloatArrayUDAF(delta: Double, maxDiscrete: Int) = - TDigestArrayUDAF[Float](delta, maxDiscrete) - - def tdigestDoubleArrayUDAF(delta: Double, maxDiscrete: Int) = - TDigestArrayUDAF[Double](delta, maxDiscrete) - - def tdigestReduceUDAF(delta: Double, maxDiscrete: Int) = - TDigestReduceUDAF(delta, maxDiscrete) - - def tdigestArrayReduceUDAF(delta: Double, maxDiscrete: Int) = - TDigestArrayReduceUDAF(delta, maxDiscrete) -} diff --git a/src/main/scala/org/isarnproject/sketches/udaf/package.scala b/src/main/scala/org/isarnproject/sketches/udaf/package.scala deleted file mode 100644 index 45c8228..0000000 --- a/src/main/scala/org/isarnproject/sketches/udaf/package.scala +++ /dev/null @@ -1,176 +0,0 @@ -/* -Copyright 2017 Erik Erlandson -Licensed 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.isarnproject.sketches - -import scala.language.implicitConversions - -import org.apache.spark.sql.types._ -import org.apache.spark.sql.catalyst.util._ - -import org.apache.spark.isarnproject.sketches.udt._ - -/** package-wide methods, implicits and definitions for sketching UDAFs */ -package object udaf { - /** - * Obtain a UDAF for sketching a single numeric Dataset column using a t-digest - * @tparam N The numeric type of the column; Double, Int, etc - * @return A UDAF that can be applied to a Dataset column - * @example - * {{{ - * import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ - * // create a UDAF for a t-digest, adding custom settings for delta and maxDiscrete - * val udafTD = tdigestUDAF[Double].delta(0.1).maxDiscrete(25) - * // apply the UDAF to get a t-digest for a data column - * val agg = data.agg(udafTD($"NumericColumn")) - * // extract the t-digest - * val td = agg.getAs[TDigestSQL](0).tdigest - * }}} - */ - def tdigestUDAF[N](implicit - num: Numeric[N], - dataType: TDigestUDAFDataType[N]) - = TDigestUDAF(TDigest.deltaDefault, 0) - - /** - * Obtain a UDAF for sketching a numeric array-data Dataset column, using a t-digest for - * each element. - * @tparam N The numeric type of the array-data column; Double, Int, etc - * @return A UDAF that can be applied to a Dataset array-data column - * @example - * {{{ - * import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ - * // create a UDAF for t-digest array, adding custom settings for delta and maxDiscrete - * val udafTD = tdigestArrayUDAF[Double].delta(0.1).maxDiscrete(25) - * // apply the UDAF to get an array of t-digests for each element in the array-data - * val agg = data.agg(udafTD($"NumericArrayColumn")) - * // extract the t-digest array - * val tdArray = agg.getAs[TDigestArraySQL](0).tdigests - * }}} - */ - def tdigestArrayUDAF[N](implicit - num: Numeric[N], - dataType: TDigestUDAFDataType[N]) - = TDigestArrayUDAF(TDigest.deltaDefault, 0) - - /** - * Obtain a UDAF for sketching an ML Vector Dataset column, using a t-digest for - * each element in the vector - * @return A UDAF that can be applied to a ML Vector column - * @example - * {{{ - * import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ - * // create a UDAF for t-digest array, adding custom settings for delta and maxDiscrete - * val udafTD = tdigestMLVecUDAF[Double].delta(0.1).maxDiscrete(25) - * // apply the UDAF to get an array of t-digests for each element in the array-data - * val agg = data.agg(udafTD($"MLVecColumn")) - * // extract the t-digest array - * val tdArray = agg.getAs[TDigestArraySQL](0).tdigests - * }}} - */ - def tdigestMLVecUDAF = TDigestMLVecUDAF(TDigest.deltaDefault, 0) - - /** - * Obtain a UDAF for sketching an MLLib Vector Dataset column, using a t-digest for - * each element in the vector - * @return A UDAF that can be applied to a MLLib Vector column - * @example - * {{{ - * import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ - * // create a UDAF for t-digest array, adding custom settings for delta and maxDiscrete - * val udafTD = tdigestMLLibVecUDAF[Double].delta(0.1).maxDiscrete(25) - * // apply the UDAF to get an array of t-digests for each element in the array-data - * val agg = data.agg(udafTD($"MLLibVecColumn")) - * // extract the t-digest array - * val tdArray = agg.getAs[TDigestArraySQL](0).tdigests - * }}} - */ - def tdigestMLLibVecUDAF = TDigestMLLibVecUDAF(TDigest.deltaDefault, 0) - - /** - * Obtain a UDAF for aggregating (reducing) a column (or grouping) of t-digests - * @return A UDAF that can be applied to a column or grouping of t-digests - * @example - * {{{ - * scala> import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ - * // column of t-digests (might also be result of aggregating from groupBy) - * scala> tds.show() - * +--------------------+ - * | tdigests| - * +--------------------+ - * |TDigestSQL(TDiges...| - * |TDigestSQL(TDiges...| - * |TDigestSQL(TDiges...| - * +--------------------+ - * - * // apply tdigestReduceUDAF to reduce the t-digests to a single combined t-digest - * scala> val td = tds.agg(tdigestReduceUDAF($"tdigests").alias("tdigest")) - * td: org.apache.spark.sql.DataFrame = [tdigest: tdigest] - * - * scala> td.show() - * +--------------------+ - * | tdigest| - * +--------------------+ - * |TDigestSQL(TDiges...| - * +--------------------+ - * }}} - */ - def tdigestReduceUDAF = TDigestReduceUDAF(TDigest.deltaDefault, 0) - - /** - * Obtain a UDAF for aggregating (reducing) a column of t-digest vectors - * @return A UDAF that can be applied to a column of t-digest vectors - * @example - * {{{ - * scala> import org.isarnproject.sketches.udaf._, org.apache.spark.isarnproject.sketches.udt._ - * // column of t-digest arrays (might also be result of aggregating from groupBy) - * scala> tds.show() - * +--------------------+ - * | tdarrays| - * +--------------------+ - * |TDigestArraySQL([...| - * |TDigestArraySQL([...| - * |TDigestArraySQL([...| - * +--------------------+ - * - * // apply tdigestArrayReduceUDAF to reduce the t-digest arrays to single array - * scala> val td = tds.agg(tdigestArrayReduceUDAF($"tdigests").alias("tdarray")) - * td: org.apache.spark.sql.DataFrame = [tdarray: tdigestarray] - * - * scala> td.show() - * +---------------------+ - * | tdarray| - * +---------------------+ - * | TDigestArraySQL([...| - * +---------------------+ - * }}} - */ - def tdigestArrayReduceUDAF = TDigestArrayReduceUDAF(TDigest.deltaDefault, 0) - - /** implicitly unpack a TDigestSQL to extract its TDigest payload */ - implicit def implicitTDigestSQLToTDigest(tdsql: TDigestSQL): TDigest = tdsql.tdigest - - /** implicitly unpack a TDigestArraySQL to extract its Array[TDigest] payload */ - implicit def implicitTDigestArraySQLToTDigestArray(tdasql: TDigestArraySQL): Array[TDigest] = - tdasql.tdigests - - /** For declaring implicit values that map numeric types to corresponding DataType values */ - case class TDigestUDAFDataType[N](tpe: DataType) - - implicit val tDigestUDAFDataTypeByte = TDigestUDAFDataType[Byte](ByteType) - implicit val tDigestUDAFDataTypeShort = TDigestUDAFDataType[Short](ShortType) - implicit val tDigestUDAFDataTypeInt = TDigestUDAFDataType[Int](IntegerType) - implicit val tDigestUDAFDataTypeLong = TDigestUDAFDataType[Long](LongType) - implicit val tDigestUDAFDataTypeFloat = TDigestUDAFDataType[Float](FloatType) - implicit val tDigestUDAFDataTypeDouble = TDigestUDAFDataType[Double](DoubleType) -} diff --git a/src/test/scala/org/isarnproject/pipelines/spark/fi.scala b/src/test/scala/org/isarnproject/pipelines/spark/fi.scala new file mode 100644 index 0000000..e423116 --- /dev/null +++ b/src/test/scala/org/isarnproject/pipelines/spark/fi.scala @@ -0,0 +1,61 @@ +/* +Copyright 2017-2020 Erik Erlandson +Licensed 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.isarnproject.pipelines.spark.fi + +import scala.util.Random._ + +import utest._ + +import org.isarnproject.testing.spark.SparkTestSuite + +import org.apache.spark.ml.regression.LinearRegression +import org.apache.spark.ml.linalg.Vectors + +object FeatureImportanceSuite extends SparkTestSuite { + + // set the seed before generating any data + setSeed(7337L * 3773L) + + val raw1 = Vector.fill(1000){ Vectors.dense(nextGaussian, nextGaussian, nextGaussian) } + .map{v => (5*v(0) + 2*v(2), v)} + val train1 = spark.createDataFrame(raw1).toDF("label", "features").cache() + + // Spark DataFrames and RDDs are lazy. + // Make sure data are actually created prior to testing, or ordering + // may change based on test ordering + val count1 = train1.count() + + val tests = Tests { + val lr = new LinearRegression() + .setMaxIter(10) + .setRegParam(0.3) + .setElasticNetParam(0.8) + val lrModel = lr.fit(train1) + + val fi = new TDigestFI().setCompression(0.3) + val fiMod = fi.fit(train1) + .setTargetModel(lrModel) + .setDeviationMeasure("rms-dev") + .setFeatureNames(Array.tabulate(3){j=>s"x$j"}) + + val imp = fiMod.transform(train1) + val impmap = imp.collect.map { r => + (r.getAs[String](0), r.getAs[Double](1)) } + .toMap + + approx(impmap("x0"), 6.65, 0.5) + approx(impmap("x1"), 0.00, 0.001) + approx(impmap("x2"), 2.50, 0.5) + } +} diff --git a/src/test/scala/org/isarnproject/sketches/spark/tdigest.scala b/src/test/scala/org/isarnproject/sketches/spark/tdigest.scala new file mode 100644 index 0000000..6c07c84 --- /dev/null +++ b/src/test/scala/org/isarnproject/sketches/spark/tdigest.scala @@ -0,0 +1,197 @@ +/* +Copyright 2017-2020 Erik Erlandson +Licensed 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.isarnproject.sketches.spark.tdigest + +import scala.util.Random._ + +import utest._ + +import org.isarnproject.testing.spark.SparkTestSuite +import org.apache.spark.sql.functions._ +import org.apache.spark.ml.linalg.{ Vectors => VectorsML } +import org.apache.spark.mllib.linalg.{ Vectors => VectorsMLLib } + +import org.isarnproject.sketches.java.TDigest + +object TDigestAggregationSuite extends SparkTestSuite { + + import CDFFunctions._ + + // set the seed before generating any data + setSeed(7337L * 3773L) + + // don't use lazy values because then data generation order may be undefined, + // due to test execution order + val data1 = spark.createDataFrame(Vector.fill(10001){(nextInt(10), nextGaussian)}) + .toDF("j","x") + .cache() + + val data2 = spark.createDataFrame(Vector.fill(10002){(nextInt(10), Vector.fill(3){nextGaussian})}) + .toDF("j", "x") + .cache() + + val data3 = spark.createDataFrame(Vector.fill(10003){(nextInt(10), VectorsML.dense(nextGaussian,nextGaussian,nextGaussian))}) + .toDF("j", "x") + .cache() + + val data4 = spark.createDataFrame(Vector.fill(10004){(nextInt(10), VectorsMLLib.dense(nextGaussian,nextGaussian,nextGaussian))}) + .toDF("j", "x") + .cache() + + // Spark DataFrames and RDDs are lazy. + // Make sure data are actually created prior to testing, or ordering + // may change based on test ordering + val count1 = data1.count() + val count2 = data2.count() + val count3 = data3.count() + val count4 = data4.count() + + // Kolmogorov-Smirnov D tolerance + val epsD = 0.02 + + val tests = Tests { + test("TDigestAggregator") { + assert(data1.rdd.partitions.size > 1) + val udf = TDigestAggregator.udf[Double](compression = 0.25, maxDiscrete = 25) + val agg = data1.agg(udf(col("j")), udf(col("x"))).first + val (tdj, tdx) = (agg.getAs[TDigest](0), agg.getAs[TDigest](1)) + approx(tdj.mass(), count1) + approx(tdx.mass(), count1) + assert(KSD(tdj, discreteUniformCDF(0, 9)) < epsD) + assert(KSD(tdx, gaussianCDF(0,1)) < epsD) + } + + test("TDigestArrayAggregator") { + assert(data2.rdd.partitions.size > 1) + val udfj = TDigestAggregator.udf[Int](maxDiscrete = 25) + val udfx = TDigestArrayAggregator.udf[Double](compression = 0.25) + val agg = data2.agg(udfj(col("j")), udfx(col("x"))).first + val (tdj, tdx) = (agg.getAs[TDigest](0), agg.getAs[Seq[TDigest]](1)) + approx(tdj.mass(), count2) + assert(KSD(tdj, discreteUniformCDF(0, 9)) < epsD) + for { td <- tdx } { + approx(td.mass(), count2) + assert(KSD(td, gaussianCDF(0, 1)) < epsD) + } + } + + test("TDigestMLVecAggregator") { + assert(data3.rdd.partitions.size > 1) + val udfj = TDigestAggregator.udf[Int](maxDiscrete = 25) + val udfx = TDigestMLVecAggregator.udf(compression = 0.25) + val agg = data3.agg(udfj(col("j")), udfx(col("x"))).first + val (tdj, tdx) = (agg.getAs[TDigest](0), agg.getAs[Seq[TDigest]](1)) + approx(tdj.mass(), count3) + assert(KSD(tdj, discreteUniformCDF(0, 9)) < epsD) + for { td <- tdx } { + approx(td.mass(), count3) + assert(KSD(td, gaussianCDF(0, 1)) < epsD) + } + } + + test("TDigestMLLibVecAggregator") { + assert(data4.rdd.partitions.size > 1) + val udfj = TDigestAggregator.udf[Int](maxDiscrete = 25) + val udfx = TDigestMLLibVecAggregator.udf(compression = 0.25) + val agg = data4.agg(udfj(col("j")), udfx(col("x"))).first + val (tdj, tdx) = (agg.getAs[TDigest](0), agg.getAs[Seq[TDigest]](1)) + approx(tdj.mass(), count4) + assert(KSD(tdj, discreteUniformCDF(0, 9)) < epsD) + for { td <- tdx } { + approx(td.mass(), count4) + assert(KSD(td, gaussianCDF(0, 1)) < epsD) + } + } + + test("TDigestReduceAggregator") { + assert(data1.rdd.partitions.size > 1) + val udf = TDigestAggregator.udf[Double](compression = 0.25) + val grp = data1.groupBy("j").agg(udf(col("x")).alias("td")) + assert(grp.count() == 10) + val udfred = TDigestReduceAggregator.udf(compression = 0.25) + val agg = grp.agg(udfred(col("td"))).first + val tdx = agg.getAs[TDigest](0) + approx(tdx.mass(), count1) + assert(KSD(tdx, gaussianCDF(0,1)) < epsD) + } + + test("TDigestArrayReduceAggregator") { + assert(data2.rdd.partitions.size > 1) + val udf = TDigestArrayAggregator.udf[Double](compression = 0.25) + val grp = data2.groupBy("j").agg(udf(col("x")).alias("td")) + assert(grp.count() == 10) + val udfred = TDigestArrayReduceAggregator.udf(compression = 0.25) + val agg = grp.agg(udfred(col("td"))).first + val tdx = agg.getAs[Seq[TDigest]](0) + for { td <- tdx } { + approx(td.mass(), count2) + assert(KSD(td, gaussianCDF(0, 1)) < epsD) + } + } + } +} + +object CDFFunctions { + type CDF = Double => Double + + // Kolmogorov Smirnov D-statistic + def KSD(td: TDigest, cdf: CDF, n: Int = 1000): Double = { + require(td.size() > 1) + require(n > 0) + val xmin = td.cdfInverse(0) + val xmax = td.cdfInverse(1) + val step = (xmax - xmin) / n.toDouble + val tdcdf = if (td.size() <= td.getMaxDiscrete()) td.cdfDiscrete(_) else td.cdf(_) + (BigDecimal(xmin) until xmax by step) + .iterator + .map(_.toDouble) + .map(x => math.abs(tdcdf(x) - cdf(x))) + .max + } + + def gaussianCDF(mean: Double = 0, stdv: Double = 1): CDF = { + require(stdv > 0.0) + val z = stdv * math.sqrt(2.0) + (x: Double) => (1.0 + erf((x - mean) / z)) / 2.0 + } + + def discreteUniformCDF(xmin: Int, xmax: Int): CDF = { + require(xmax > xmin) + require(xmin >= 0) + val z = (1 + xmax - xmin).toDouble + (x: Double) => { + if (x < xmin.toDouble) 0.0 else if (x >= xmax.toDouble) 1.0 else { + val xi = x.toInt + (1 + xi - xmin).toDouble / z + } + } + } + + // https://en.wikipedia.org/wiki/Error_function#Approximation_with_elementary_functions + def erf(x: Double): Double = { + // erf is an odd function + if (x < 0.0) -erf(-x) else { + val t = 1.0 / (1.0 + (0.47047 * x)) + var u = t + var s = 0.0 + s += 0.3480242 * u + u *= t + s -= 0.0958798 * u + u *= t + s += 0.7478556 * u + s *= math.exp(-(x * x)) + 1.0 - s + } + } +} diff --git a/src/test/scala/org/isarnproject/testing/spark.scala b/src/test/scala/org/isarnproject/testing/spark.scala new file mode 100644 index 0000000..f89f771 --- /dev/null +++ b/src/test/scala/org/isarnproject/testing/spark.scala @@ -0,0 +1,49 @@ +/* +Copyright 2017-2020 Erik Erlandson +Licensed 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.isarnproject.testing.spark + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SparkSession + +import utest._ + +abstract class SparkTestSuite extends TestSuite { + + private lazy val sparkConf: SparkConf = { + val conf = new SparkConf() + .setAppName("SparkTestSuite") + conf + } + + private lazy val sparkSession: SparkSession = { + val ss = SparkSession.builder + .master("local[2]") + .config(sparkConf) + .getOrCreate() + ss.sparkContext.setLogLevel("WARN") + ss + } + + def spark: SparkSession = sparkSession + + def approx(x: Double, t: Double, eps: Double = 1e-4): Unit = + assert(math.abs(x - t) < eps) + + override def utestAfterAll(): Unit = { + super.utestAfterAll() + // it is important that test suites are not run in parallel + // so that this doesn't kill a cluster while some other suite is using it + spark.stop() + } +}