You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by le...@apache.org on 2024/03/26 21:57:46 UTC

(datasketches-java) branch extend_KS_to_kll created (now ce189132)

This is an automated email from the ASF dual-hosted git repository.

leerho pushed a change to branch extend_KS_to_kll
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git


      at ce189132 This extends the usability of the Kolmogorov-Smirnov test

This branch includes the following new commits:

     new ce189132 This extends the usability of the Kolmogorov-Smirnov test

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org


(datasketches-java) 01/01: This extends the usability of the Kolmogorov-Smirnov test

Posted by le...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

leerho pushed a commit to branch extend_KS_to_kll
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git

commit ce189132f794c88898d322443ce2f4d7d704531f
Author: Lee Rhodes <le...@users.noreply.github.com>
AuthorDate: Tue Mar 26 14:57:32 2024 -0700

    This extends the usability of the Kolmogorov-Smirnov test
    
    from the Classic DoublesSketch to KllDoublesSketch and the
    KllFloatsSketch.
    
    In order to do this I needed to add "getNormalizedRankError(boolean) to
    the root interface QuantilesAPI. All of the sketches (except REQ) had
    this anyway. In REQ I just throw an unsupported operation exception,
    because it makes no sense for REQ.
    
    This also means the KolmogorovSmirnov class and its test move to the
    "quantilescommon" package.
---
 .../org/apache/datasketches/kll/KllSketch.java     |  14 +-
 .../datasketches/quantiles/DoublesSketch.java      |  10 +-
 .../apache/datasketches/quantiles/ItemsSketch.java |   8 +-
 .../datasketches/quantiles/KolmogorovSmirnov.java  | 114 -------
 .../quantilescommon/KolmogorovSmirnov.java         | 167 ++++++++++
 .../datasketches/quantilescommon/QuantilesAPI.java |  11 +
 .../org/apache/datasketches/req/ReqSketch.java     |   9 +
 .../quantiles/KolmogorovSmirnovTest.java           | 148 ---------
 .../quantilescommon/KolmogorovSmirnovTest.java     | 355 +++++++++++++++++++++
 9 files changed, 545 insertions(+), 291 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/kll/KllSketch.java b/src/main/java/org/apache/datasketches/kll/KllSketch.java
index bbf269cb..2d7385f6 100644
--- a/src/main/java/org/apache/datasketches/kll/KllSketch.java
+++ b/src/main/java/org/apache/datasketches/kll/KllSketch.java
@@ -193,15 +193,7 @@ public abstract class KllSketch implements QuantilesAPI {
     return KllHelper.getNormalizedRankError(k, pmf);
   }
 
-  /**
-   * Gets the approximate rank error of this sketch normalized as a fraction between zero and one.
-   * The epsilon returned is a best fit to 99 percent confidence empirically measured max error
-   * in thousands of trials.
-   * @param pmf if true, returns the "double-sided" normalized rank error for the getPMF() function.
-   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
-   * @return if pmf is true, returns the "double-sided" normalized rank error for the getPMF() function.
-   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
-   */
+  @Override
   public final double getNormalizedRankError(final boolean pmf) {
     return getNormalizedRankError(getMinK(), pmf);
   }
@@ -279,10 +271,6 @@ public abstract class KllSketch implements QuantilesAPI {
    */
   public abstract void merge(KllSketch other);
 
-  /**
-   * Returns human readable summary information about this sketch.
-   * Used for debugging.
-   */
   @Override
   public final String toString() {
     return toString(false, false);
diff --git a/src/main/java/org/apache/datasketches/quantiles/DoublesSketch.java b/src/main/java/org/apache/datasketches/quantiles/DoublesSketch.java
index fd0698b8..1d148268 100644
--- a/src/main/java/org/apache/datasketches/quantiles/DoublesSketch.java
+++ b/src/main/java/org/apache/datasketches/quantiles/DoublesSketch.java
@@ -266,15 +266,7 @@ public abstract class DoublesSketch implements QuantilesDoublesAPI {
   @Override
   public abstract long getN();
 
-  /**
-   * Gets the approximate rank error of this sketch normalized as a fraction between zero and one.
-   * The epsilon returned is a best fit to 99 percent confidence empirically measured max error
-   * in thousands of trials.
-   * @param pmf if true, returns the "double-sided" normalized rank error for the getPMF() function.
-   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
-   * @return if pmf is true, returns the normalized rank error for the getPMF() function.
-   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
-   */
+  @Override
   public double getNormalizedRankError(final boolean pmf) {
     return getNormalizedRankError(k_, pmf);
   }
diff --git a/src/main/java/org/apache/datasketches/quantiles/ItemsSketch.java b/src/main/java/org/apache/datasketches/quantiles/ItemsSketch.java
index 13aee81e..9361c6cd 100644
--- a/src/main/java/org/apache/datasketches/quantiles/ItemsSketch.java
+++ b/src/main/java/org/apache/datasketches/quantiles/ItemsSketch.java
@@ -361,13 +361,7 @@ public final class ItemsSketch<T> implements QuantilesGenericAPI<T> {
     return n_;
   }
 
-  /**
-   * Gets the approximate rank error of this sketch normalized as a fraction between zero and one.
-   * @param pmf if true, returns the "double-sided" normalized rank error for the getPMF() function.
-   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
-   * @return if pmf is true, returns the normalized rank error for the getPMF() function.
-   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
-   */
+  @Override
   public double getNormalizedRankError(final boolean pmf) {
     return getNormalizedRankError(k_, pmf);
   }
diff --git a/src/main/java/org/apache/datasketches/quantiles/KolmogorovSmirnov.java b/src/main/java/org/apache/datasketches/quantiles/KolmogorovSmirnov.java
deleted file mode 100644
index 6b8079ea..00000000
--- a/src/main/java/org/apache/datasketches/quantiles/KolmogorovSmirnov.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.datasketches.quantiles;
-
-import org.apache.datasketches.quantilescommon.DoublesSketchSortedView;
-
-/**
- * Kolmogorov-Smirnov Test
- * See <a href="https://en.wikipedia.org/wiki/Kolmogorov-Smirnov_test">Kolmogorov–Smirnov Test</a>
- */
-final class KolmogorovSmirnov {
-
-  //TODO This KS test will have to be redesigned to accommodate REQ sketches.
-
-  /**
-   * Computes the raw delta area between two quantile sketches for the
-   * <i>kolmogorovSmirnovTest(DoublesSketch, DoublesSketch, double)</i>
-   * method.
-   * @param sketch1 Input DoubleSketch 1
-   * @param sketch2 Input DoubleSketch 2
-   * @return the raw delta area between two quantile sketches
-   */
-  public static double computeKSDelta(final DoublesSketch sketch1, final DoublesSketch sketch2) {
-    final DoublesSketchSortedView p = sketch1.getSortedView();
-    final DoublesSketchSortedView q = sketch2.getSortedView();
-
-    final double[] pSamplesArr = p.getQuantiles();
-    final double[] qSamplesArr = q.getQuantiles();
-    final long[] pCumWtsArr = p.getCumulativeWeights();
-    final long[] qCumWtsArr = q.getCumulativeWeights();
-    final int pSamplesArrLen = pSamplesArr.length;
-    final int qSamplesArrLen = qSamplesArr.length;
-
-    final double n1 = sketch1.getN();
-    final double n2 = sketch2.getN();
-
-    double deltaHeight = 0;
-    int i = 0;
-    int j = 0;
-
-    while ((i < pSamplesArrLen - 1) && (j < qSamplesArrLen - 1)) {
-      deltaHeight = Math.max(deltaHeight, Math.abs(pCumWtsArr[i] / n1 - qCumWtsArr[j] / n2));
-      if (pSamplesArr[i] < qSamplesArr[j]) {
-        i++;
-      } else if (qSamplesArr[j] < pSamplesArr[i]) {
-        j++;
-      } else {
-        i++;
-        j++;
-      }
-    }
-
-    deltaHeight = Math.max(deltaHeight, Math.abs(pCumWtsArr[i] / n1 - qCumWtsArr[j] / n2));
-    return deltaHeight;
-  }
-
-  /**
-   * Computes the adjusted delta height threshold for the
-   * <i>kolmogorovSmirnovTest(DoublesSketch, DoublesSketch, double)</i>
-   * method.
-   * This adjusts the computed threshold by the error epsilons of the two given sketches.
-   * @param sketch1 Input DoubleSketch 1
-   * @param sketch2 Input DoubleSketch 2
-   * @param tgtPvalue Target p-value. Typically .001 to .1, e.g., .05.
-   * @return the adjusted threshold to be compared with the raw delta area.
-   */
-  public static double computeKSThreshold(final DoublesSketch sketch1,
-                                          final DoublesSketch sketch2,
-                                          final double tgtPvalue) {
-    final double r1 = sketch1.getNumRetained();
-    final double r2 = sketch2.getNumRetained();
-    final double alpha = tgtPvalue;
-    final double alphaFactor = Math.sqrt(-0.5 * Math.log(0.5 * alpha));
-    final double deltaAreaThreshold = alphaFactor * Math.sqrt((r1 + r2) / (r1 * r2));
-    final double eps1 = sketch1.getNormalizedRankError(false);
-    final double eps2 = sketch2.getNormalizedRankError(false);
-    return deltaAreaThreshold + eps1 + eps2;
-  }
-
-  /**
-   * Performs the Kolmogorov-Smirnov Test between two quantiles sketches.
-   * Note: if the given sketches have insufficient data or if the sketch sizes are too small,
-   * this will return false.
-   * @param sketch1 Input DoubleSketch 1
-   * @param sketch2 Input DoubleSketch 2
-   * @param tgtPvalue Target p-value. Typically .001 to .1, e.g., .05.
-   * @return Boolean indicating whether we can reject the null hypothesis (that the sketches
-   * reflect the same underlying distribution) using the provided tgtPValue.
-   */
-  public static boolean kolmogorovSmirnovTest(final DoublesSketch sketch1,
-      final DoublesSketch sketch2, final double tgtPvalue) {
-    final double delta = computeKSDelta(sketch1, sketch2);
-    final double thresh = computeKSThreshold(sketch1, sketch2, tgtPvalue);
-    return delta > thresh;
-  }
-
-}
diff --git a/src/main/java/org/apache/datasketches/quantilescommon/KolmogorovSmirnov.java b/src/main/java/org/apache/datasketches/quantilescommon/KolmogorovSmirnov.java
new file mode 100644
index 00000000..2856cff0
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/quantilescommon/KolmogorovSmirnov.java
@@ -0,0 +1,167 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches.quantilescommon;
+
+import static org.apache.datasketches.quantilescommon.QuantilesAPI.UNSUPPORTED_MSG;
+
+import org.apache.datasketches.req.ReqSketch;
+
+/**
+ * Kolmogorov-Smirnov Test
+ * See <a href="https://en.wikipedia.org/wiki/Kolmogorov-Smirnov_test">Kolmogorov–Smirnov Test</a>
+ */
+public final class KolmogorovSmirnov {
+
+  /**
+   * Computes the raw delta between two QuantilesDoublesAPI sketches for the <i>kolmogorovSmirnovTest(...)</i> method.
+   * @param sketch1 first Input QuantilesDoublesAPI
+   * @param sketch2 second Input QuantilesDoublesAPI
+   * @return the raw delta area between two QuantilesDoublesAPI sketches
+   */
+  public static double computeKSDelta(final QuantilesDoublesAPI sketch1, final QuantilesDoublesAPI sketch2) {
+    final DoublesSortedView p = sketch1.getSortedView();
+    final DoublesSortedView q = sketch2.getSortedView();
+
+    final double[] pSamplesArr = p.getQuantiles();
+    final double[] qSamplesArr = q.getQuantiles();
+    final long[] pCumWtsArr = p.getCumulativeWeights();
+    final long[] qCumWtsArr = q.getCumulativeWeights();
+    final int pSamplesArrLen = pSamplesArr.length;
+    final int qSamplesArrLen = qSamplesArr.length;
+
+    final double n1 = sketch1.getN();
+    final double n2 = sketch2.getN();
+
+    double deltaHeight = 0;
+    int i = 0;
+    int j = 0;
+
+    while ((i < pSamplesArrLen - 1) && (j < qSamplesArrLen - 1)) {
+      deltaHeight = Math.max(deltaHeight, Math.abs(pCumWtsArr[i] / n1 - qCumWtsArr[j] / n2));
+      if (pSamplesArr[i] < qSamplesArr[j]) {
+        i++;
+      } else if (qSamplesArr[j] < pSamplesArr[i]) {
+        j++;
+      } else {
+        i++;
+        j++;
+      }
+    }
+
+    deltaHeight = Math.max(deltaHeight, Math.abs(pCumWtsArr[i] / n1 - qCumWtsArr[j] / n2));
+    return deltaHeight;
+  }
+
+  /**
+   * Computes the raw delta between two QuantilesFloatsAPI sketches for the <i>kolmogorovSmirnovTest(...)</i> method.
+   * method.
+   * @param sketch1 first Input QuantilesFloatsAPI sketch
+   * @param sketch2 second Input QuantilesFloatsAPI sketch
+   * @return the raw delta area between two QuantilesFloatsAPI sketches
+   */
+  public static double computeKSDelta(final QuantilesFloatsAPI sketch1, final QuantilesFloatsAPI sketch2) {
+    final FloatsSortedView p = sketch1.getSortedView();
+    final FloatsSortedView q = sketch2.getSortedView();
+
+    final float[] pSamplesArr = p.getQuantiles();
+    final float[] qSamplesArr = q.getQuantiles();
+    final long[] pCumWtsArr = p.getCumulativeWeights();
+    final long[] qCumWtsArr = q.getCumulativeWeights();
+    final int pSamplesArrLen = pSamplesArr.length;
+    final int qSamplesArrLen = qSamplesArr.length;
+
+    final double n1 = sketch1.getN();
+    final double n2 = sketch2.getN();
+
+    double deltaHeight = 0;
+    int i = 0;
+    int j = 0;
+
+    while ((i < pSamplesArrLen - 1) && (j < qSamplesArrLen - 1)) {
+      deltaHeight = Math.max(deltaHeight, Math.abs(pCumWtsArr[i] / n1 - qCumWtsArr[j] / n2));
+      if (pSamplesArr[i] < qSamplesArr[j]) {
+        i++;
+      } else if (qSamplesArr[j] < pSamplesArr[i]) {
+        j++;
+      } else {
+        i++;
+        j++;
+      }
+    }
+
+    deltaHeight = Math.max(deltaHeight, Math.abs(pCumWtsArr[i] / n1 - qCumWtsArr[j] / n2));
+    return deltaHeight;
+  }
+
+  /**
+   * Computes the adjusted delta height threshold for the <i>kolmogorovSmirnovTest(...)</i> method.
+   * This adjusts the computed threshold by the error epsilons of the two given sketches.
+   * The two sketches must be of the same primitive type, double or float.
+   * This will not work with the REQ sketch.
+   * @param sketch1 first Input QuantilesAPI sketch
+   * @param sketch2 second Input QuantilesAPI sketch
+   * @param tgtPvalue Target p-value. Typically .001 to .1, e.g., .05.
+   * @return the adjusted threshold to be compared with the raw delta area.
+   */
+  public static double computeKSThreshold(final QuantilesAPI sketch1,
+                                          final QuantilesAPI sketch2,
+                                          final double tgtPvalue) {
+    final double r1 = sketch1.getNumRetained();
+    final double r2 = sketch2.getNumRetained();
+    final double alpha = tgtPvalue;
+    final double alphaFactor = Math.sqrt(-0.5 * Math.log(0.5 * alpha));
+    final double deltaAreaThreshold = alphaFactor * Math.sqrt((r1 + r2) / (r1 * r2));
+    final double eps1 = sketch1.getNormalizedRankError(false);
+    final double eps2 = sketch2.getNormalizedRankError(false);
+    return deltaAreaThreshold + eps1 + eps2;
+  }
+
+  /**
+   * Performs the Kolmogorov-Smirnov Test between two QuantilesAPI sketches.
+   * Note: if the given sketches have insufficient data or if the sketch sizes are too small,
+   * this will return false. The two sketches must be of the same primitive type, double or float.
+   * This will not work with the REQ sketch.
+   * @param sketch1 first Input QuantilesAPI
+   * @param sketch2 second Input QuantilesAPI
+   * @param tgtPvalue Target p-value. Typically .001 to .1, e.g., .05.
+   * @return Boolean indicating whether we can reject the null hypothesis (that the sketches
+   * reflect the same underlying distribution) using the provided tgtPValue.
+   */
+  public static boolean kolmogorovSmirnovTest(final QuantilesAPI sketch1,
+      final QuantilesAPI sketch2, final double tgtPvalue) {
+
+    final double delta = isDoubleType(sketch1, sketch2)
+        ? computeKSDelta((QuantilesDoublesAPI)sketch1, (QuantilesDoublesAPI)sketch2)
+        : computeKSDelta((QuantilesFloatsAPI)sketch1, (QuantilesFloatsAPI)sketch2);
+    final double thresh = computeKSThreshold(sketch1, sketch2, tgtPvalue);
+    return delta > thresh;
+  }
+
+  private static boolean isDoubleType(final Object sk1, final Object sk2) {
+    if (sk1 instanceof ReqSketch || sk2 instanceof ReqSketch) {
+      throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+    }
+    final boolean isDbl = (sk1 instanceof QuantilesDoublesAPI && sk2 instanceof QuantilesDoublesAPI);
+    final boolean isFlt = (sk1 instanceof QuantilesFloatsAPI && sk2 instanceof QuantilesFloatsAPI);
+    if (isDbl ^ isFlt) { return isDbl; }
+    else { throw new UnsupportedOperationException(UNSUPPORTED_MSG); }
+  }
+
+}
diff --git a/src/main/java/org/apache/datasketches/quantilescommon/QuantilesAPI.java b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesAPI.java
index f433519e..0d02d2ec 100644
--- a/src/main/java/org/apache/datasketches/quantilescommon/QuantilesAPI.java
+++ b/src/main/java/org/apache/datasketches/quantilescommon/QuantilesAPI.java
@@ -225,6 +225,17 @@ public interface QuantilesAPI {
    */
   long getN();
 
+  /**
+   * Gets the approximate rank error of this sketch normalized as a fraction between zero and one.
+   * The epsilon returned is a best fit to 99 percent confidence empirically measured max error
+   * in thousands of trials.
+   * @param pmf if true, returns the "double-sided" normalized rank error for the getPMF() function.
+   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
+   * @return if pmf is true, returns the "double-sided" normalized rank error for the getPMF() function.
+   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
+   */
+  double getNormalizedRankError(boolean pmf);
+
   /**
    * Gets the number of quantiles retained by the sketch.
    * @return the number of quantiles retained by the sketch
diff --git a/src/main/java/org/apache/datasketches/req/ReqSketch.java b/src/main/java/org/apache/datasketches/req/ReqSketch.java
index 347b3aaf..7e19be1d 100644
--- a/src/main/java/org/apache/datasketches/req/ReqSketch.java
+++ b/src/main/java/org/apache/datasketches/req/ReqSketch.java
@@ -235,6 +235,15 @@ public final class ReqSketch extends BaseReqSketch {
     return totalN;
   }
 
+  @Override
+  /**
+   * This is an unsupported operation for this sketch
+   */
+  public double getNormalizedRankError(final boolean pmf) {
+    throw new UnsupportedOperationException(UNSUPPORTED_MSG);
+
+  }
+
   @Override
   public double[] getPMF(final float[] splitPoints, final QuantileSearchCriteria searchCrit) {
     if (isEmpty()) { throw new IllegalArgumentException(QuantilesAPI.EMPTY_MSG); }
diff --git a/src/test/java/org/apache/datasketches/quantiles/KolmogorovSmirnovTest.java b/src/test/java/org/apache/datasketches/quantiles/KolmogorovSmirnovTest.java
deleted file mode 100644
index fd8e4f71..00000000
--- a/src/test/java/org/apache/datasketches/quantiles/KolmogorovSmirnovTest.java
+++ /dev/null
@@ -1,148 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.datasketches.quantiles;
-
-import static org.testng.Assert.assertEquals;
-import static org.testng.Assert.assertFalse;
-import static org.testng.Assert.assertTrue;
-
-import java.util.Random;
-
-import org.testng.annotations.Test;
-
-public class KolmogorovSmirnovTest {
-
- @Test
- public void checkDisjointDistribution() {
-   final int k = 256;
-   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
-   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
-
-   final Random rand = new Random(1);
-
-   final int n =  (3 * k) - 1;
-   for (int i = 0; i < n; ++i) {
-     final double x = rand.nextGaussian();
-     s1.update(x + 500);
-     s2.update(x);
-   }
-   //assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 1.0, 1E-6);
-   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
- }
-
- @Test
- public void checkIdenticalDistribution() {
-   final int k = 256;
-   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
-
-   final Random rand = new Random(1);
-
-   final int n =  (3 * k) - 1;
-   for (int i = 0; i < n; ++i) {
-     final double x = rand.nextGaussian();
-     s1.update(x);
-   }
-
-   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s1), 0.0, 0.0);
-   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s1));
- }
-
-
- @Test
- public void checkSameDistributionDifferentSketches() {
-   final int k = 256;
-   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
-   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
-
-   final Random rand = new Random(1);
-
-   final int n =  (3 * k) - 1;
-   for (int i = 0; i < n; ++i) {
-     final double x = rand.nextGaussian();
-     s1.update(x);
-     s2.update(x);
-   }
-
-   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 0, .01);
-   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
- }
-
- @Test
- public void mediumResolution() {
-   final int k = 2048;
-   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
-   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
-   final double tgtPvalue = .05;
-
-   final Random rand = new Random(1);
-
-   final int n =  (3 * k) - 1;
-   for (int i = 0; i < n; ++i) {
-     final double x = rand.nextGaussian();
-     s1.update(x + .05);
-     s2.update(x);
-   }
-
-   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
-   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
-   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
-   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
-       + "\nNull Hypoth Rejected = " + reject);
-   assertFalse(reject);
- }
-
- @Test
- public void highResolution() {
-   final int k = 8192;
-   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
-   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
-   final double tgtPvalue = .05;
-
-   final Random rand = new Random(1);
-
-   final int n =  (3 * k) - 1;
-   for (int i = 0; i < n; ++i) {
-     final double x = rand.nextGaussian();
-     s1.update(x + .05);
-     s2.update(x);
-   }
-
-   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
-   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
-   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
-   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
-       + "\nNull Hypoth Rejected = " + reject);
-   assertTrue(reject);
- }
-
-
-  @Test
-  public void printlnTest() {
-    println("PRINTING: "+this.getClass().getName());
-  }
-
-  /**
-   * @param s value to print
-   */
-  static void println(String s) {
-    //System.out.println(s); //disable here
-  }
-
-}
diff --git a/src/test/java/org/apache/datasketches/quantilescommon/KolmogorovSmirnovTest.java b/src/test/java/org/apache/datasketches/quantilescommon/KolmogorovSmirnovTest.java
new file mode 100644
index 00000000..eeb005cc
--- /dev/null
+++ b/src/test/java/org/apache/datasketches/quantilescommon/KolmogorovSmirnovTest.java
@@ -0,0 +1,355 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches.quantilescommon;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertTrue;
+
+import java.util.Random;
+
+import org.apache.datasketches.kll.KllDoublesSketch;
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.datasketches.kll.KllSketch;
+import org.apache.datasketches.quantiles.DoublesSketch;
+import org.apache.datasketches.quantiles.UpdateDoublesSketch;
+import org.testng.annotations.Test;
+
+public class KolmogorovSmirnovTest {
+
+ @Test
+ public void checkDisjointDistributionClassicDoubles() {
+   final int k = 256;
+   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
+   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x + 500);
+     s2.update(x);
+   }
+   final double delta = DoublesSketch.getNormalizedRankError(k, false);
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 1.0, delta);
+ }
+
+ @Test
+ public void checkDisjointDistributionKllDoubles() {
+   final int k = 256;
+   final KllDoublesSketch s1 = KllDoublesSketch.newHeapInstance(k);
+   final KllDoublesSketch s2 = KllDoublesSketch.newHeapInstance(k);
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x + 500);
+     s2.update(x);
+   }
+   final double delta = KllSketch.getNormalizedRankError(k, false);
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 1.0, delta);
+ }
+
+ @Test
+ public void checkDisjointDistributionKllFloats() {
+   final int k = 256;
+   final KllFloatsSketch s1 = KllFloatsSketch.newHeapInstance(k);
+   final KllFloatsSketch s2 = KllFloatsSketch.newHeapInstance(k);
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final float x = (float)rand.nextGaussian();
+     s1.update(x + 500);
+     s2.update(x);
+   }
+   final double delta = KllSketch.getNormalizedRankError(k, false);
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 1.0, delta);
+ }
+
+ @Test
+ public void checkIdenticalDistributionClassicDoubles() {
+   final int k = 256;
+   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x);
+   }
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s1));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s1), 0.0, 0.0);
+ }
+
+ @Test
+ public void checkIdenticalDistributionKllDoubles() {
+   final int k = 256;
+   final KllDoublesSketch s1 = KllDoublesSketch.newHeapInstance(k);
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x);
+   }
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s1));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s1), 0.0, 0.0);
+ }
+
+ @Test
+ public void checkIdenticalDistributionKllFloats() {
+   final int k = 256;
+   final KllFloatsSketch s1 = KllFloatsSketch.newHeapInstance(k);
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final float x = (float)rand.nextGaussian();
+     s1.update(x);
+   }
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s1));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s1), 0.0, 0.0);
+ }
+
+ @Test
+ public void checkSameDistributionDifferentClassicDoublesSketches() {
+   final int k = 256;
+   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
+   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x);
+     s2.update(x);
+   }
+   final double delta = DoublesSketch.getNormalizedRankError(k, false);
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 0, delta);
+ }
+
+ @Test
+ public void checkSameDistributionDifferentKllDoublesSketches() {
+   final int k = 256;
+   final KllDoublesSketch s1 = KllDoublesSketch.newHeapInstance(k);
+   final KllDoublesSketch s2 = KllDoublesSketch.newHeapInstance(k);
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x);
+     s2.update(x);
+   }
+   final double delta = KllSketch.getNormalizedRankError(k, false);
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 0, delta);
+ }
+
+ @Test
+ public void checkSameDistributionDifferentKllFloatsSketches() {
+   final int k = 256;
+   final KllFloatsSketch s1 = KllFloatsSketch.newHeapInstance(k);
+   final KllFloatsSketch s2 = KllFloatsSketch.newHeapInstance(k);
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final float x = (float)rand.nextGaussian();
+     s1.update(x);
+     s2.update(x);
+   }
+   final double delta = KllSketch.getNormalizedRankError(k, false);
+   println("D = " + KolmogorovSmirnov.computeKSDelta(s1, s2));
+   assertEquals(KolmogorovSmirnov.computeKSDelta(s1, s2), 0, delta);
+ }
+
+ @Test
+ public void mediumResolutionClassicDoubles() {
+   final int k = 2048;
+   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
+   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
+   final double tgtPvalue = .05;
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x + .05);
+     s2.update(x);
+   }
+
+   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
+   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
+   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
+   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
+       + "\nNull Hypoth Rejected = " + reject);
+   assertFalse(reject);
+ }
+
+ @Test
+ public void mediumResolutionKllDoubles() {
+   final int k = 2048;
+   final KllDoublesSketch s1 = KllDoublesSketch.newHeapInstance(k);
+   final KllDoublesSketch s2 = KllDoublesSketch.newHeapInstance(k);
+   final double tgtPvalue = .05;
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x + .05);
+     s2.update(x);
+   }
+
+   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
+   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
+   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
+   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
+       + "\nNull Hypoth Rejected = " + reject);
+   assertFalse(reject);
+ }
+
+ @Test
+ public void mediumResolutionKllFloats() {
+   final int k = 2048;
+   final KllFloatsSketch s1 = KllFloatsSketch.newHeapInstance(k);
+   final KllFloatsSketch s2 = KllFloatsSketch.newHeapInstance(k);
+   final double tgtPvalue = .05;
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final float x = (float)rand.nextGaussian();
+     s1.update(x + .05F);
+     s2.update(x);
+   }
+
+   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
+   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
+   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
+   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
+       + "\nNull Hypoth Rejected = " + reject);
+   assertFalse(reject);
+ }
+
+ @Test
+ public void highResolutionClassicDoubles() {
+   final int k = 8192;
+   final UpdateDoublesSketch s1 = DoublesSketch.builder().setK(k).build();
+   final UpdateDoublesSketch s2 = DoublesSketch.builder().setK(k).build();
+   final double tgtPvalue = .05;
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x + .05);
+     s2.update(x);
+   }
+
+   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
+   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
+   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
+   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
+       + "\nNull Hypoth Rejected = " + reject);
+   assertTrue(reject);
+ }
+
+ @Test
+ public void highResolutionKllDoubles() {
+   final int k = 8192;
+   final KllDoublesSketch s1 = KllDoublesSketch.newHeapInstance(k);
+   final KllDoublesSketch s2 = KllDoublesSketch.newHeapInstance(k);
+   final double tgtPvalue = .05;
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final double x = rand.nextGaussian();
+     s1.update(x + .05);
+     s2.update(x);
+   }
+
+   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
+   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
+   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
+   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
+       + "\nNull Hypoth Rejected = " + reject);
+   assertTrue(reject);
+ }
+
+ @Test
+ public void highResolutionKllFloats() {
+   final int k = 8192;
+   final KllFloatsSketch s1 = KllFloatsSketch.newHeapInstance(k);
+   final KllFloatsSketch s2 = KllFloatsSketch.newHeapInstance(k);
+   final double tgtPvalue = .05;
+
+   final Random rand = new Random(1);
+
+   final int n =  (3 * k) - 1;
+   for (int i = 0; i < n; ++i) {
+     final float x = (float)rand.nextGaussian();
+     s1.update(x + .05F);
+     s2.update(x);
+   }
+
+   double D = KolmogorovSmirnov.computeKSDelta(s1, s2);
+   double thresh = KolmogorovSmirnov.computeKSThreshold(s1, s2, tgtPvalue);
+   final boolean reject = KolmogorovSmirnov.kolmogorovSmirnovTest(s1, s2, tgtPvalue);
+   println("pVal = " + tgtPvalue + "\nK = " + k + "\nD = " + D + "\nTh = " + thresh
+       + "\nNull Hypoth Rejected = " + reject);
+   assertTrue(reject);
+ }
+
+  @Test
+  public void printlnTest() {
+    println("PRINTING: "+this.getClass().getName());
+  }
+
+  /**
+   * @param s value to print
+   */
+  static void println(String s) {
+    //System.out.println(s); //disable here
+  }
+
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org
For additional commands, e-mail: commits-help@datasketches.apache.org