You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by GitBox <gi...@apache.org> on 2022/04/01 16:33:25 UTC

[GitHub] [datasketches-java] will-lauer commented on a change in pull request #390: Direct kll double

will-lauer commented on a change in pull request #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r840687634



##########
File path: src/main/java/org/apache/datasketches/kll/KllSketch.java
##########
@@ -0,0 +1,1676 @@
+/*
+ * 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.kll;
+
+import static java.lang.Math.abs;
+import static java.lang.Math.ceil;
+import static java.lang.Math.exp;
+import static java.lang.Math.log;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.Math.round;
+import static org.apache.datasketches.Util.isOdd;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_SINGLE_ITEM;
+import static org.apache.datasketches.kll.KllPreambleUtil.N_LONG_ADR;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_EMPTY_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_EMPTY_FULL;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertDoubleSketchFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertEmptyFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertM;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertPreInts;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSerVer;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSingleItemFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertUpdatableFlag;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
+import static org.apache.datasketches.kll.KllSketch.SketchType.FLOATS_SKETCH;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/*
+ * Sampled stream data (floats or doubles) is stored as an array or as part of a Memory object.
+ * This array is partitioned into sections called levels and the indices into the array of items
+ * are tracked by a small integer array called levels or levels array.
+ * The data for level i lies in positions levelsArray[i] through levelsArray[i + 1] - 1 inclusive.
+ * Hence, the levelsArray must contain (numLevels + 1) indices.
+ * The valid portion of items array is completely packed and sorted, except for level 0,
+ * which is filled from the top down. Any items below the index levelsArray[0] is garbage and will be
+ * overwritten by subsequent updates.
+ *
+ * Invariants:
+ * 1) After a compaction, or an update, or a merge, all levels are sorted except for level zero.
+ * 2) After a compaction, (sum of capacities) - (sum of items) >= 1,
+ *  so there is room for least 1 more item in level zero.
+ * 3) There are no gaps except at the bottom, so if levels_[0] = 0,
+ *  the sketch is exactly filled to capacity and must be compacted or the itemsArray and levelsArray
+ *  must be expanded to include more levels.
+ * 4) Sum of weights of all retained items == N.
+ * 5) Current total item capacity = itemsArray.length = levelsArray[numLevels].
+ */
+
+/**
+ * This class is the root of the KLL sketch class hierarchy. It includes the public API that is independent
+ * of either sketch type (float or double) and independent of whether the sketch is targeted for use on the
+ * heap or Direct (off-heap.
+ *
+ * <p>Please refer to the documentation in the package-info:<br>
+ * {@link org.apache.datasketches.kll}</p>
+ *
+ * @author Lee Rhodes, Kevin Lang
+ */
+public abstract class KllSketch {
+  static final double EPS_DELTA_THRESHOLD = 1E-6;
+  static final double MIN_EPS = 4.7634E-5;
+  static final double PMF_COEF = 2.446;
+  static final double PMF_EXP = 0.9433;
+  static final double CDF_COEF = 2.296;
+  static final double CDF_EXP = 0.9723;
+  static final Random random = new Random();
+  static final boolean compatible = true; //rank 0.0 and 1.0. compatible with classic Quantiles Sketch
+  SketchType sketchType;
+  WritableMemory wmem;
+  MemoryRequestServer memReqSvr;
+  boolean direct;
+
+  /**
+   * The default value of K
+   */
+  public static final int DEFAULT_K = 200;
+
+  /**
+   * The default value of M
+   */
+  static final int DEFAULT_M = 8;
+
+  /**
+   * The maximum value of K
+   */
+  public static final int MAX_K = (1 << 16) - 1; // serialized as an unsigned short
+
+  /**
+   * The maximum value of M
+   */
+  static final int MAX_M = 8;
+
+  /**
+   * The minimum value of M
+   */
+  static final int MIN_M = 2;
+
+  /**
+   *
+   * @param sketchType either DOUBLE_SKETCH or FLOAT_SKETCH
+   * @param wmem  the current WritableMemory or null
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   */
+  KllSketch(final SketchType sketchType, final WritableMemory wmem, final MemoryRequestServer memReqSvr) {
+   this.sketchType = sketchType;
+   this.wmem = wmem;
+   if (wmem != null) {
+     this.direct = true;
+     this.memReqSvr = memReqSvr;
+   } else {
+     this.direct = false;
+     this.memReqSvr = null;
+   }
+  }
+
+public enum SketchType { FLOATS_SKETCH, DOUBLES_SKETCH }
+
+  //Static methods
+
+  /**
+   * Gets the approximate value of <em>k</em> to use given epsilon, the normalized rank error.
+   * @param epsilon the normalized rank error between zero and one.
+   * @param pmf if true, this function returns the value of <em>k</em> assuming the input epsilon
+   * is the desired "double-sided" epsilon for the getPMF() function. Otherwise, this function
+   * returns the value of <em>k</em> assuming the input epsilon is the desired "single-sided"
+   * epsilon for all the other queries.
+   *
+   * <p>Please refer to the documentation in the package-info:<br>
+   * {@link org.apache.datasketches.kll}</p>
+   * @return the value of <i>k</i> given a value of epsilon.
+   */
+  // constants were derived as the best fit to 99 percentile empirically measured max error in
+  // thousands of trials
+  public static int getKFromEpsilon(final double epsilon, final boolean pmf) {
+    //Ensure that eps is >= than the lowest possible eps given MAX_K and pmf=false.
+    final double eps = max(epsilon, MIN_EPS);
+    final double kdbl = pmf
+        ? exp(log(PMF_COEF / eps) / PMF_EXP)
+        : exp(log(CDF_COEF / eps) / CDF_EXP);
+    final double krnd = round(kdbl);
+    final double del = abs(krnd - kdbl);
+    final int k = (int) (del < EPS_DELTA_THRESHOLD ? krnd : ceil(kdbl));
+    return max(KllSketch.MIN_M, min(KllSketch.MAX_K, k));
+  }
+
+  /**
+   * Returns upper bound on the compact serialized size of a FloatsSketch given a parameter
+   * <em>k</em> and stream length. This method can be used if allocation of storage
+   * is necessary beforehand.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @return upper bound on the compact serialized size
+   * @deprecated use {@link #getMaxSerializedSizeBytes(int, long, SketchType, boolean)} instead.
+   */
+  @Deprecated
+  public static int getMaxSerializedSizeBytes(final int k, final long n) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, FLOATS_SKETCH, false);
+    return gStats.compactBytes;
+  }
+
+  /**
+   * Returns upper bound on the serialized size of a KllSketch given the following parameters.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @param sketchType either DOUBLES_SKETCH or FLOATS_SKETCH
+   * @param updatable true if updatable form, otherwise the standard compact form.
+   * @return upper bound on the serialized size of a KllSketch.
+   */
+  public static int getMaxSerializedSizeBytes(final int k, final long n,
+      final SketchType sketchType, final boolean updatable) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, sketchType, false);
+    return updatable ? gStats.updatableBytes : gStats.compactBytes;
+  }
+
+  /**
+   * Gets the normalized rank error given k and pmf.
+   * Static method version of the <i>getNormalizedRankError(boolean)</i>.
+   * @param k the configuration parameter
+   * @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, the normalized rank error for the getPMF() function.
+   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
+   */
+  public static double getNormalizedRankError(final int k, final boolean pmf) {
+    return KllHelper.getNormalizedRankError(k, pmf);
+  }
+
+  /**
+   * Returns the current number of bytes this Sketch would require if serialized.
+   * @return the number of bytes this sketch would require if serialized.
+   */
+  public int getSerializedSizeBytes() {
+    return (direct)
+        ? getCurrentUpdatableSerializedSizeBytes()
+        : getCurrentCompactSerializedSizeBytes();
+  }
+
+  static int getSerializedSizeBytes(final int numLevels, final int numItems,
+      final SketchType sketchType, final boolean updatable) {
+    int levelsBytes = 0;
+    if (!updatable) {
+      if (numItems == 0) { return N_LONG_ADR; }
+      if (numItems == 1) {
+        return DATA_START_ADR_SINGLE_ITEM + (sketchType == DOUBLES_SKETCH ? Double.BYTES : Float.BYTES);
+      }
+      levelsBytes = numLevels * Integer.BYTES;
+    } else {
+      levelsBytes = (numLevels + 1) * Integer.BYTES;
+    }
+    if (sketchType == DOUBLES_SKETCH) {
+      return DATA_START_ADR_DOUBLE + levelsBytes + (numItems + 2) * Double.BYTES; //+2 is for min & max
+    } else {
+      return DATA_START_ADR_FLOAT + levelsBytes + (numItems + 2) * Float.BYTES;
+    }
+  }
+
+  final static boolean isCompatible() {

Review comment:
       This method is unnecessary and probably should be removed. Its final and static, returning a constant. There is no way override it and it isn't necessary for implementing an interface. Any use of this method could be reduced to just `true`, simplifying the code.

##########
File path: src/main/java/org/apache/datasketches/kll/KllSketch.java
##########
@@ -0,0 +1,1676 @@
+/*
+ * 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.kll;
+
+import static java.lang.Math.abs;
+import static java.lang.Math.ceil;
+import static java.lang.Math.exp;
+import static java.lang.Math.log;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.Math.round;
+import static org.apache.datasketches.Util.isOdd;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_SINGLE_ITEM;
+import static org.apache.datasketches.kll.KllPreambleUtil.N_LONG_ADR;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_EMPTY_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_EMPTY_FULL;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertDoubleSketchFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertEmptyFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertM;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertPreInts;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSerVer;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSingleItemFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertUpdatableFlag;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
+import static org.apache.datasketches.kll.KllSketch.SketchType.FLOATS_SKETCH;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/*
+ * Sampled stream data (floats or doubles) is stored as an array or as part of a Memory object.
+ * This array is partitioned into sections called levels and the indices into the array of items
+ * are tracked by a small integer array called levels or levels array.
+ * The data for level i lies in positions levelsArray[i] through levelsArray[i + 1] - 1 inclusive.
+ * Hence, the levelsArray must contain (numLevels + 1) indices.
+ * The valid portion of items array is completely packed and sorted, except for level 0,
+ * which is filled from the top down. Any items below the index levelsArray[0] is garbage and will be
+ * overwritten by subsequent updates.
+ *
+ * Invariants:
+ * 1) After a compaction, or an update, or a merge, all levels are sorted except for level zero.
+ * 2) After a compaction, (sum of capacities) - (sum of items) >= 1,
+ *  so there is room for least 1 more item in level zero.
+ * 3) There are no gaps except at the bottom, so if levels_[0] = 0,
+ *  the sketch is exactly filled to capacity and must be compacted or the itemsArray and levelsArray
+ *  must be expanded to include more levels.
+ * 4) Sum of weights of all retained items == N.
+ * 5) Current total item capacity = itemsArray.length = levelsArray[numLevels].
+ */
+
+/**
+ * This class is the root of the KLL sketch class hierarchy. It includes the public API that is independent
+ * of either sketch type (float or double) and independent of whether the sketch is targeted for use on the
+ * heap or Direct (off-heap.
+ *
+ * <p>Please refer to the documentation in the package-info:<br>
+ * {@link org.apache.datasketches.kll}</p>
+ *
+ * @author Lee Rhodes, Kevin Lang
+ */
+public abstract class KllSketch {
+  static final double EPS_DELTA_THRESHOLD = 1E-6;
+  static final double MIN_EPS = 4.7634E-5;
+  static final double PMF_COEF = 2.446;
+  static final double PMF_EXP = 0.9433;
+  static final double CDF_COEF = 2.296;
+  static final double CDF_EXP = 0.9723;
+  static final Random random = new Random();
+  static final boolean compatible = true; //rank 0.0 and 1.0. compatible with classic Quantiles Sketch
+  SketchType sketchType;
+  WritableMemory wmem;
+  MemoryRequestServer memReqSvr;
+  boolean direct;
+
+  /**
+   * The default value of K
+   */
+  public static final int DEFAULT_K = 200;
+
+  /**
+   * The default value of M
+   */
+  static final int DEFAULT_M = 8;
+
+  /**
+   * The maximum value of K
+   */
+  public static final int MAX_K = (1 << 16) - 1; // serialized as an unsigned short
+
+  /**
+   * The maximum value of M
+   */
+  static final int MAX_M = 8;
+
+  /**
+   * The minimum value of M
+   */
+  static final int MIN_M = 2;
+
+  /**
+   *
+   * @param sketchType either DOUBLE_SKETCH or FLOAT_SKETCH
+   * @param wmem  the current WritableMemory or null
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   */
+  KllSketch(final SketchType sketchType, final WritableMemory wmem, final MemoryRequestServer memReqSvr) {
+   this.sketchType = sketchType;
+   this.wmem = wmem;
+   if (wmem != null) {
+     this.direct = true;
+     this.memReqSvr = memReqSvr;
+   } else {
+     this.direct = false;
+     this.memReqSvr = null;
+   }
+  }
+
+public enum SketchType { FLOATS_SKETCH, DOUBLES_SKETCH }
+
+  //Static methods
+
+  /**
+   * Gets the approximate value of <em>k</em> to use given epsilon, the normalized rank error.
+   * @param epsilon the normalized rank error between zero and one.
+   * @param pmf if true, this function returns the value of <em>k</em> assuming the input epsilon
+   * is the desired "double-sided" epsilon for the getPMF() function. Otherwise, this function
+   * returns the value of <em>k</em> assuming the input epsilon is the desired "single-sided"
+   * epsilon for all the other queries.
+   *
+   * <p>Please refer to the documentation in the package-info:<br>
+   * {@link org.apache.datasketches.kll}</p>
+   * @return the value of <i>k</i> given a value of epsilon.
+   */
+  // constants were derived as the best fit to 99 percentile empirically measured max error in
+  // thousands of trials
+  public static int getKFromEpsilon(final double epsilon, final boolean pmf) {
+    //Ensure that eps is >= than the lowest possible eps given MAX_K and pmf=false.
+    final double eps = max(epsilon, MIN_EPS);
+    final double kdbl = pmf
+        ? exp(log(PMF_COEF / eps) / PMF_EXP)
+        : exp(log(CDF_COEF / eps) / CDF_EXP);
+    final double krnd = round(kdbl);
+    final double del = abs(krnd - kdbl);
+    final int k = (int) (del < EPS_DELTA_THRESHOLD ? krnd : ceil(kdbl));
+    return max(KllSketch.MIN_M, min(KllSketch.MAX_K, k));
+  }
+
+  /**
+   * Returns upper bound on the compact serialized size of a FloatsSketch given a parameter
+   * <em>k</em> and stream length. This method can be used if allocation of storage
+   * is necessary beforehand.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @return upper bound on the compact serialized size
+   * @deprecated use {@link #getMaxSerializedSizeBytes(int, long, SketchType, boolean)} instead.
+   */
+  @Deprecated
+  public static int getMaxSerializedSizeBytes(final int k, final long n) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, FLOATS_SKETCH, false);
+    return gStats.compactBytes;
+  }
+
+  /**
+   * Returns upper bound on the serialized size of a KllSketch given the following parameters.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @param sketchType either DOUBLES_SKETCH or FLOATS_SKETCH
+   * @param updatable true if updatable form, otherwise the standard compact form.
+   * @return upper bound on the serialized size of a KllSketch.
+   */
+  public static int getMaxSerializedSizeBytes(final int k, final long n,
+      final SketchType sketchType, final boolean updatable) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, sketchType, false);
+    return updatable ? gStats.updatableBytes : gStats.compactBytes;
+  }
+
+  /**
+   * Gets the normalized rank error given k and pmf.
+   * Static method version of the <i>getNormalizedRankError(boolean)</i>.
+   * @param k the configuration parameter
+   * @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, the normalized rank error for the getPMF() function.
+   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
+   */
+  public static double getNormalizedRankError(final int k, final boolean pmf) {
+    return KllHelper.getNormalizedRankError(k, pmf);
+  }
+
+  /**
+   * Returns the current number of bytes this Sketch would require if serialized.
+   * @return the number of bytes this sketch would require if serialized.
+   */
+  public int getSerializedSizeBytes() {
+    return (direct)
+        ? getCurrentUpdatableSerializedSizeBytes()
+        : getCurrentCompactSerializedSizeBytes();
+  }
+
+  static int getSerializedSizeBytes(final int numLevels, final int numItems,
+      final SketchType sketchType, final boolean updatable) {
+    int levelsBytes = 0;
+    if (!updatable) {
+      if (numItems == 0) { return N_LONG_ADR; }
+      if (numItems == 1) {
+        return DATA_START_ADR_SINGLE_ITEM + (sketchType == DOUBLES_SKETCH ? Double.BYTES : Float.BYTES);
+      }
+      levelsBytes = numLevels * Integer.BYTES;
+    } else {
+      levelsBytes = (numLevels + 1) * Integer.BYTES;
+    }
+    if (sketchType == DOUBLES_SKETCH) {
+      return DATA_START_ADR_DOUBLE + levelsBytes + (numItems + 2) * Double.BYTES; //+2 is for min & max
+    } else {
+      return DATA_START_ADR_FLOAT + levelsBytes + (numItems + 2) * Float.BYTES;
+    }
+  }
+
+  final static boolean isCompatible() {
+    return compatible;
+  }
+
+  enum Error { TGT_IS_IMMUTABLE, SRC_IS_NOT_DIRECT, SRC_IS_NOT_DOUBLE,
+   SRC_IS_NOT_FLOAT, SRC_CANNOT_BE_DIRECT, MUST_NOT_CALL }
+
+  final static void kllSketchThrow(final Error errType) {
+    String msg = "";
+    switch (errType) {
+      case TGT_IS_IMMUTABLE: msg = "Given sketch Memory is immutable, cannot write."; break;
+      case SRC_IS_NOT_DIRECT: msg = "Given sketch must be of type Direct."; break;
+      case SRC_IS_NOT_DOUBLE: msg = "Given sketch must be of type Double."; break;
+      case SRC_IS_NOT_FLOAT: msg = "Given sketch must be of type Float."; break;
+      case SRC_CANNOT_BE_DIRECT: msg = "Given sketch must not be of type Direct."; break;
+      case MUST_NOT_CALL: msg = "This is an artifact of inheritance and should never be called."; break;
+      default: msg = "Unknown error."; break;
+    }
+    throw new SketchesArgumentException(msg);
+  }
+
+  //Public Non-static methods
+
+  /**
+   * Returns the current compact number of bytes this sketch would require to store.
+   * @return the current compact number of bytes this sketch would require to store.
+   */
+  public final int getCurrentCompactSerializedSizeBytes() {
+    return KllSketch.getSerializedSizeBytes(getNumLevels(), getNumRetained(), sketchType, false);
+  }
+
+  /**
+   * Returns the current updatable number of bytes this sketch would require to store.
+   * @return the current updatable number of bytes this sketch would require to store.
+   */
+  public final int getCurrentUpdatableSerializedSizeBytes() {
+    final int itemCap = KllHelper.computeTotalItemCapacity(getK(), getM(), getNumLevels());
+    return KllSketch.getSerializedSizeBytes(getNumLevels(), itemCap, sketchType, true);
+  }
+
+  /**
+   * Returns the user configured parameter k
+   * @return the user configured parameter k
+   */
+  public abstract int getK();
+
+  /**
+   * Returns the configured parameter m

Review comment:
       What is `M`? This would be a good place to expand the documentation.

##########
File path: src/main/java/org/apache/datasketches/kll/KllSketch.java
##########
@@ -0,0 +1,1676 @@
+/*
+ * 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.kll;
+
+import static java.lang.Math.abs;
+import static java.lang.Math.ceil;
+import static java.lang.Math.exp;
+import static java.lang.Math.log;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.Math.round;
+import static org.apache.datasketches.Util.isOdd;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_SINGLE_ITEM;
+import static org.apache.datasketches.kll.KllPreambleUtil.N_LONG_ADR;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_EMPTY_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_EMPTY_FULL;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertDoubleSketchFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertEmptyFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertM;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertPreInts;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSerVer;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSingleItemFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertUpdatableFlag;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
+import static org.apache.datasketches.kll.KllSketch.SketchType.FLOATS_SKETCH;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/*
+ * Sampled stream data (floats or doubles) is stored as an array or as part of a Memory object.
+ * This array is partitioned into sections called levels and the indices into the array of items
+ * are tracked by a small integer array called levels or levels array.
+ * The data for level i lies in positions levelsArray[i] through levelsArray[i + 1] - 1 inclusive.
+ * Hence, the levelsArray must contain (numLevels + 1) indices.
+ * The valid portion of items array is completely packed and sorted, except for level 0,
+ * which is filled from the top down. Any items below the index levelsArray[0] is garbage and will be
+ * overwritten by subsequent updates.
+ *
+ * Invariants:
+ * 1) After a compaction, or an update, or a merge, all levels are sorted except for level zero.
+ * 2) After a compaction, (sum of capacities) - (sum of items) >= 1,
+ *  so there is room for least 1 more item in level zero.
+ * 3) There are no gaps except at the bottom, so if levels_[0] = 0,
+ *  the sketch is exactly filled to capacity and must be compacted or the itemsArray and levelsArray
+ *  must be expanded to include more levels.
+ * 4) Sum of weights of all retained items == N.
+ * 5) Current total item capacity = itemsArray.length = levelsArray[numLevels].
+ */
+
+/**
+ * This class is the root of the KLL sketch class hierarchy. It includes the public API that is independent
+ * of either sketch type (float or double) and independent of whether the sketch is targeted for use on the
+ * heap or Direct (off-heap.
+ *
+ * <p>Please refer to the documentation in the package-info:<br>
+ * {@link org.apache.datasketches.kll}</p>
+ *
+ * @author Lee Rhodes, Kevin Lang
+ */
+public abstract class KllSketch {
+  static final double EPS_DELTA_THRESHOLD = 1E-6;
+  static final double MIN_EPS = 4.7634E-5;
+  static final double PMF_COEF = 2.446;
+  static final double PMF_EXP = 0.9433;
+  static final double CDF_COEF = 2.296;
+  static final double CDF_EXP = 0.9723;
+  static final Random random = new Random();
+  static final boolean compatible = true; //rank 0.0 and 1.0. compatible with classic Quantiles Sketch
+  SketchType sketchType;
+  WritableMemory wmem;
+  MemoryRequestServer memReqSvr;
+  boolean direct;
+
+  /**
+   * The default value of K
+   */
+  public static final int DEFAULT_K = 200;
+
+  /**
+   * The default value of M
+   */
+  static final int DEFAULT_M = 8;
+
+  /**
+   * The maximum value of K
+   */
+  public static final int MAX_K = (1 << 16) - 1; // serialized as an unsigned short
+
+  /**
+   * The maximum value of M
+   */
+  static final int MAX_M = 8;
+
+  /**
+   * The minimum value of M
+   */
+  static final int MIN_M = 2;
+
+  /**
+   *
+   * @param sketchType either DOUBLE_SKETCH or FLOAT_SKETCH
+   * @param wmem  the current WritableMemory or null
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   */
+  KllSketch(final SketchType sketchType, final WritableMemory wmem, final MemoryRequestServer memReqSvr) {
+   this.sketchType = sketchType;
+   this.wmem = wmem;
+   if (wmem != null) {
+     this.direct = true;
+     this.memReqSvr = memReqSvr;
+   } else {
+     this.direct = false;
+     this.memReqSvr = null;
+   }
+  }
+
+public enum SketchType { FLOATS_SKETCH, DOUBLES_SKETCH }
+
+  //Static methods
+
+  /**
+   * Gets the approximate value of <em>k</em> to use given epsilon, the normalized rank error.
+   * @param epsilon the normalized rank error between zero and one.
+   * @param pmf if true, this function returns the value of <em>k</em> assuming the input epsilon
+   * is the desired "double-sided" epsilon for the getPMF() function. Otherwise, this function
+   * returns the value of <em>k</em> assuming the input epsilon is the desired "single-sided"
+   * epsilon for all the other queries.
+   *
+   * <p>Please refer to the documentation in the package-info:<br>
+   * {@link org.apache.datasketches.kll}</p>
+   * @return the value of <i>k</i> given a value of epsilon.
+   */
+  // constants were derived as the best fit to 99 percentile empirically measured max error in
+  // thousands of trials
+  public static int getKFromEpsilon(final double epsilon, final boolean pmf) {
+    //Ensure that eps is >= than the lowest possible eps given MAX_K and pmf=false.
+    final double eps = max(epsilon, MIN_EPS);
+    final double kdbl = pmf
+        ? exp(log(PMF_COEF / eps) / PMF_EXP)
+        : exp(log(CDF_COEF / eps) / CDF_EXP);
+    final double krnd = round(kdbl);
+    final double del = abs(krnd - kdbl);
+    final int k = (int) (del < EPS_DELTA_THRESHOLD ? krnd : ceil(kdbl));
+    return max(KllSketch.MIN_M, min(KllSketch.MAX_K, k));
+  }
+
+  /**
+   * Returns upper bound on the compact serialized size of a FloatsSketch given a parameter
+   * <em>k</em> and stream length. This method can be used if allocation of storage
+   * is necessary beforehand.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @return upper bound on the compact serialized size
+   * @deprecated use {@link #getMaxSerializedSizeBytes(int, long, SketchType, boolean)} instead.
+   */
+  @Deprecated
+  public static int getMaxSerializedSizeBytes(final int k, final long n) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, FLOATS_SKETCH, false);
+    return gStats.compactBytes;
+  }
+
+  /**
+   * Returns upper bound on the serialized size of a KllSketch given the following parameters.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @param sketchType either DOUBLES_SKETCH or FLOATS_SKETCH
+   * @param updatable true if updatable form, otherwise the standard compact form.
+   * @return upper bound on the serialized size of a KllSketch.
+   */
+  public static int getMaxSerializedSizeBytes(final int k, final long n,
+      final SketchType sketchType, final boolean updatable) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, sketchType, false);
+    return updatable ? gStats.updatableBytes : gStats.compactBytes;
+  }
+
+  /**
+   * Gets the normalized rank error given k and pmf.
+   * Static method version of the <i>getNormalizedRankError(boolean)</i>.
+   * @param k the configuration parameter
+   * @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, the normalized rank error for the getPMF() function.
+   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
+   */
+  public static double getNormalizedRankError(final int k, final boolean pmf) {
+    return KllHelper.getNormalizedRankError(k, pmf);
+  }
+
+  /**
+   * Returns the current number of bytes this Sketch would require if serialized.
+   * @return the number of bytes this sketch would require if serialized.
+   */
+  public int getSerializedSizeBytes() {
+    return (direct)
+        ? getCurrentUpdatableSerializedSizeBytes()
+        : getCurrentCompactSerializedSizeBytes();
+  }
+
+  static int getSerializedSizeBytes(final int numLevels, final int numItems,

Review comment:
       Is numItems the stream length here, or the number of retained items. Its unclear without javadoc for this method.

##########
File path: src/main/java/org/apache/datasketches/kll/KllSketch.java
##########
@@ -0,0 +1,1676 @@
+/*
+ * 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.kll;
+
+import static java.lang.Math.abs;
+import static java.lang.Math.ceil;
+import static java.lang.Math.exp;
+import static java.lang.Math.log;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static java.lang.Math.round;
+import static org.apache.datasketches.Util.isOdd;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_SINGLE_ITEM;
+import static org.apache.datasketches.kll.KllPreambleUtil.N_LONG_ADR;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_EMPTY_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_EMPTY_FULL;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_SINGLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertDoubleSketchFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertEmptyFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertM;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertPreInts;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSerVer;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertSingleItemFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertUpdatableFlag;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
+import static org.apache.datasketches.kll.KllSketch.SketchType.FLOATS_SKETCH;
+
+import java.util.Arrays;
+import java.util.Random;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/*
+ * Sampled stream data (floats or doubles) is stored as an array or as part of a Memory object.
+ * This array is partitioned into sections called levels and the indices into the array of items
+ * are tracked by a small integer array called levels or levels array.
+ * The data for level i lies in positions levelsArray[i] through levelsArray[i + 1] - 1 inclusive.
+ * Hence, the levelsArray must contain (numLevels + 1) indices.
+ * The valid portion of items array is completely packed and sorted, except for level 0,
+ * which is filled from the top down. Any items below the index levelsArray[0] is garbage and will be
+ * overwritten by subsequent updates.
+ *
+ * Invariants:
+ * 1) After a compaction, or an update, or a merge, all levels are sorted except for level zero.
+ * 2) After a compaction, (sum of capacities) - (sum of items) >= 1,
+ *  so there is room for least 1 more item in level zero.
+ * 3) There are no gaps except at the bottom, so if levels_[0] = 0,
+ *  the sketch is exactly filled to capacity and must be compacted or the itemsArray and levelsArray
+ *  must be expanded to include more levels.
+ * 4) Sum of weights of all retained items == N.
+ * 5) Current total item capacity = itemsArray.length = levelsArray[numLevels].
+ */
+
+/**
+ * This class is the root of the KLL sketch class hierarchy. It includes the public API that is independent
+ * of either sketch type (float or double) and independent of whether the sketch is targeted for use on the
+ * heap or Direct (off-heap.
+ *
+ * <p>Please refer to the documentation in the package-info:<br>
+ * {@link org.apache.datasketches.kll}</p>
+ *
+ * @author Lee Rhodes, Kevin Lang
+ */
+public abstract class KllSketch {
+  static final double EPS_DELTA_THRESHOLD = 1E-6;
+  static final double MIN_EPS = 4.7634E-5;
+  static final double PMF_COEF = 2.446;
+  static final double PMF_EXP = 0.9433;
+  static final double CDF_COEF = 2.296;
+  static final double CDF_EXP = 0.9723;
+  static final Random random = new Random();
+  static final boolean compatible = true; //rank 0.0 and 1.0. compatible with classic Quantiles Sketch
+  SketchType sketchType;
+  WritableMemory wmem;
+  MemoryRequestServer memReqSvr;
+  boolean direct;
+
+  /**
+   * The default value of K
+   */
+  public static final int DEFAULT_K = 200;
+
+  /**
+   * The default value of M
+   */
+  static final int DEFAULT_M = 8;
+
+  /**
+   * The maximum value of K
+   */
+  public static final int MAX_K = (1 << 16) - 1; // serialized as an unsigned short
+
+  /**
+   * The maximum value of M
+   */
+  static final int MAX_M = 8;
+
+  /**
+   * The minimum value of M
+   */
+  static final int MIN_M = 2;
+
+  /**
+   *
+   * @param sketchType either DOUBLE_SKETCH or FLOAT_SKETCH
+   * @param wmem  the current WritableMemory or null
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   */
+  KllSketch(final SketchType sketchType, final WritableMemory wmem, final MemoryRequestServer memReqSvr) {
+   this.sketchType = sketchType;
+   this.wmem = wmem;
+   if (wmem != null) {
+     this.direct = true;
+     this.memReqSvr = memReqSvr;
+   } else {
+     this.direct = false;
+     this.memReqSvr = null;
+   }
+  }
+
+public enum SketchType { FLOATS_SKETCH, DOUBLES_SKETCH }
+
+  //Static methods
+
+  /**
+   * Gets the approximate value of <em>k</em> to use given epsilon, the normalized rank error.
+   * @param epsilon the normalized rank error between zero and one.
+   * @param pmf if true, this function returns the value of <em>k</em> assuming the input epsilon
+   * is the desired "double-sided" epsilon for the getPMF() function. Otherwise, this function
+   * returns the value of <em>k</em> assuming the input epsilon is the desired "single-sided"
+   * epsilon for all the other queries.
+   *
+   * <p>Please refer to the documentation in the package-info:<br>
+   * {@link org.apache.datasketches.kll}</p>
+   * @return the value of <i>k</i> given a value of epsilon.
+   */
+  // constants were derived as the best fit to 99 percentile empirically measured max error in
+  // thousands of trials
+  public static int getKFromEpsilon(final double epsilon, final boolean pmf) {
+    //Ensure that eps is >= than the lowest possible eps given MAX_K and pmf=false.
+    final double eps = max(epsilon, MIN_EPS);
+    final double kdbl = pmf
+        ? exp(log(PMF_COEF / eps) / PMF_EXP)
+        : exp(log(CDF_COEF / eps) / CDF_EXP);
+    final double krnd = round(kdbl);
+    final double del = abs(krnd - kdbl);
+    final int k = (int) (del < EPS_DELTA_THRESHOLD ? krnd : ceil(kdbl));
+    return max(KllSketch.MIN_M, min(KllSketch.MAX_K, k));
+  }
+
+  /**
+   * Returns upper bound on the compact serialized size of a FloatsSketch given a parameter
+   * <em>k</em> and stream length. This method can be used if allocation of storage
+   * is necessary beforehand.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @return upper bound on the compact serialized size
+   * @deprecated use {@link #getMaxSerializedSizeBytes(int, long, SketchType, boolean)} instead.
+   */
+  @Deprecated
+  public static int getMaxSerializedSizeBytes(final int k, final long n) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, FLOATS_SKETCH, false);
+    return gStats.compactBytes;
+  }
+
+  /**
+   * Returns upper bound on the serialized size of a KllSketch given the following parameters.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param n stream length
+   * @param sketchType either DOUBLES_SKETCH or FLOATS_SKETCH
+   * @param updatable true if updatable form, otherwise the standard compact form.
+   * @return upper bound on the serialized size of a KllSketch.
+   */
+  public static int getMaxSerializedSizeBytes(final int k, final long n,
+      final SketchType sketchType, final boolean updatable) {
+    final KllHelper.GrowthStats gStats =
+        KllHelper.getGrowthSchemeForGivenN(k, KllSketch.DEFAULT_M, n, sketchType, false);
+    return updatable ? gStats.updatableBytes : gStats.compactBytes;
+  }
+
+  /**
+   * Gets the normalized rank error given k and pmf.
+   * Static method version of the <i>getNormalizedRankError(boolean)</i>.
+   * @param k the configuration parameter
+   * @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, the normalized rank error for the getPMF() function.
+   * Otherwise, it is the "single-sided" normalized rank error for all the other queries.
+   */
+  public static double getNormalizedRankError(final int k, final boolean pmf) {
+    return KllHelper.getNormalizedRankError(k, pmf);
+  }
+
+  /**
+   * Returns the current number of bytes this Sketch would require if serialized.
+   * @return the number of bytes this sketch would require if serialized.
+   */
+  public int getSerializedSizeBytes() {
+    return (direct)
+        ? getCurrentUpdatableSerializedSizeBytes()
+        : getCurrentCompactSerializedSizeBytes();
+  }
+
+  static int getSerializedSizeBytes(final int numLevels, final int numItems,
+      final SketchType sketchType, final boolean updatable) {
+    int levelsBytes = 0;
+    if (!updatable) {
+      if (numItems == 0) { return N_LONG_ADR; }
+      if (numItems == 1) {
+        return DATA_START_ADR_SINGLE_ITEM + (sketchType == DOUBLES_SKETCH ? Double.BYTES : Float.BYTES);
+      }
+      levelsBytes = numLevels * Integer.BYTES;
+    } else {
+      levelsBytes = (numLevels + 1) * Integer.BYTES;
+    }
+    if (sketchType == DOUBLES_SKETCH) {
+      return DATA_START_ADR_DOUBLE + levelsBytes + (numItems + 2) * Double.BYTES; //+2 is for min & max
+    } else {
+      return DATA_START_ADR_FLOAT + levelsBytes + (numItems + 2) * Float.BYTES;
+    }
+  }
+
+  final static boolean isCompatible() {
+    return compatible;
+  }
+
+  enum Error { TGT_IS_IMMUTABLE, SRC_IS_NOT_DIRECT, SRC_IS_NOT_DOUBLE,
+   SRC_IS_NOT_FLOAT, SRC_CANNOT_BE_DIRECT, MUST_NOT_CALL }
+
+  final static void kllSketchThrow(final Error errType) {
+    String msg = "";
+    switch (errType) {
+      case TGT_IS_IMMUTABLE: msg = "Given sketch Memory is immutable, cannot write."; break;
+      case SRC_IS_NOT_DIRECT: msg = "Given sketch must be of type Direct."; break;
+      case SRC_IS_NOT_DOUBLE: msg = "Given sketch must be of type Double."; break;
+      case SRC_IS_NOT_FLOAT: msg = "Given sketch must be of type Float."; break;
+      case SRC_CANNOT_BE_DIRECT: msg = "Given sketch must not be of type Direct."; break;
+      case MUST_NOT_CALL: msg = "This is an artifact of inheritance and should never be called."; break;

Review comment:
       Design-wise, it probably makes more sense to merge this case statement with the enum above, since they should always be kept in sync with each other. Then this method would simply become
   
       final static void kllSketchThrow(final Error errType) {
           throw new SketchesArgumentException(errType.getMessage());
       }




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@datasketches.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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