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/08 17:17:05 UTC

[GitHub] [datasketches-java] leerho opened a new pull request, #390: Direct kll double

leerho opened a new pull request, #390:
URL: https://github.com/apache/datasketches-java/pull/390

   This PR is pretty big, but it contains a massive rewrite of the KLL sketches and supporting classes to reduce duplicate code, where possible, and to enable the Direct mode of operation, leveraging critical processes for use by both heap and off-heap KLL sketches as well as _double_ and _float_ primitives.  This PR contains:
   
   - KllFloatsSketch
   - KllDoublesSketch
   - KllDirectFloatsSketch
   - KllDirectDoublesSketch
   
   There is a lot of duplication across the test classes, but that is unavoidable for now.  I have added some documentation for all the sketches in the package-info.java class, which will show up in the javadocs.  Updating the website is still to be done.


-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r843235201


##########
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 has been removed.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849878728


##########
src/test/java/org/apache/datasketches/kll/KllDoublesSketchTest.java:
##########
@@ -326,16 +327,26 @@ public void serializeDeserializeOneItem() {
     sketch1.update(1);
     final byte[] bytes = sketch1.toByteArray();
     final KllDoublesSketch sketch2 = KllDoublesSketch.heapify(Memory.wrap(bytes));
-    assertEquals(bytes.length, sketch1.getSerializedSizeBytes());
+    assertEquals(bytes.length, sketch1.getCurrentCompactSerializedSizeBytes());
     assertFalse(sketch2.isEmpty());
     assertEquals(sketch2.getNumRetained(), 1);
     assertEquals(sketch2.getN(), 1);
     assertEquals(sketch2.getNormalizedRankError(false), sketch1.getNormalizedRankError(false));
     assertFalse(Double.isNaN(sketch2.getMinValue()));
     assertFalse(Double.isNaN(sketch2.getMaxValue()));
-    assertEquals(sketch2.getSerializedSizeBytes(), 8 + Double.BYTES);
+    assertEquals(sketch2.getCurrentCompactSerializedSizeBytes(), 8 + Double.BYTES);
   }
 
+  //@Test //not implemented from C++ yet
+  //public void deserializeOneItemV1() throws Exception {
+  //  final byte[] bytes = getResourceBytes("kll_sketch_float_one_item_v1.sk");
+  //  final KllFloatsSketch sketch = KllFloatsSketch.heapify(Memory.wrap(bytes));
+  //  assertFalse(sketch.isEmpty());
+  //  assertFalse(sketch.isEstimationMode());
+  //  assertEquals(sketch.getN(), 1);
+  //  assertEquals(sketch.getNumRetained(), 1);
+  //}

Review Comment:
   No, but there was such a case for the prior floats sketch



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849877729


##########
src/main/java/org/apache/datasketches/kll/KllHelper.java:
##########
@@ -19,32 +19,189 @@
 
 package org.apache.datasketches.kll;
 
+import static java.lang.Math.pow;
 import static org.apache.datasketches.Util.floorPowerOf2;
+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.KllSketch.CDF_COEF;
+import static org.apache.datasketches.kll.KllSketch.CDF_EXP;
+import static org.apache.datasketches.kll.KllSketch.PMF_COEF;
+import static org.apache.datasketches.kll.KllSketch.PMF_EXP;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
 
-class KllHelper {
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.kll.KllSketch.SketchType;
+
+/**
+ * This class provides some useful sketch analysis tools that are used internally and also can be used by
+ * interested users to understand the internal structure of the sketch as well as the growth properties of the
+ * sketch given a stream length.
+ *
+ * @author lrhodes
+ *
+ */
+public class KllHelper {
+
+  public static class GrowthStats {
+    SketchType sketchType;
+    int k;
+    int m;
+    long givenN;
+    long maxN;
+    int numLevels;
+    int maxItems;
+    int compactBytes;
+    int updatableBytes;
+  }
+
+  public static class LevelStats {
+    long n;
+    int numLevels;
+    int items;
+
+    LevelStats(final long n, final int numLevels, final int items) {
+      this.n = n;
+      this.numLevels = numLevels;
+      this.items = items;
+    }
+  }
 
   /**
-   * Copy the old array into a new larger array.
-   * The extra space is at the top.
-   * @param oldArr the given old array with data
-   * @param newLen the new length larger than the oldArr.length.
-   * @return the new array
+   * This is the exact powers of 3 from 3^0 to 3^30 where the exponent is the index
    */
-  static int[] growIntArray(final int[] oldArr, final int newLen) {
-    final int oldLen = oldArr.length;
-    assert newLen > oldLen;
-    final int[] newArr = new int[newLen];
-    System.arraycopy(oldArr, 0, newArr, 0, oldLen);
-    return newArr;
+  private static final long[] powersOfThree =
+      new long[] {1, 3, 9, 27, 81, 243, 729, 2187, 6561, 19683, 59049, 177147, 531441,
+  1594323, 4782969, 14348907, 43046721, 129140163, 387420489, 1162261467,
+  3486784401L, 10460353203L, 31381059609L, 94143178827L, 282429536481L,
+  847288609443L, 2541865828329L, 7625597484987L, 22876792454961L, 68630377364883L,
+  205891132094649L};
+
+  /**
+   * Given k, m, and numLevels, this computes and optionally prints the structure of the sketch when the given
+   * number of levels are completely filled.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param numLevels the given number of levels of the sketch
+   * @param printSketchStructure if true will print the details of the sketch structure at the given numLevels.
+   * @return LevelStats with the final summary of the sketch's cumulative N,
+   * and cumulative items at the given numLevels.
+   */
+  public static LevelStats getFinalSketchStatsAtNumLevels(
+      final int k,
+      final int m,
+      final int numLevels,
+      final boolean printSketchStructure) {
+    int cumItems = 0;
+    long cumN = 0;
+    if (printSketchStructure) {
+      println("SKETCH STRUCTURE:");
+      println("Given K        : " + k);
+      println("Given M        : " + m);
+      println("Given NumLevels: " + numLevels);
+      printf("%6s %8s %12s %18s %18s\n", "Level", "Items", "CumItems", "N at Level", "CumN");
+    }
+    for (int level = 0; level < numLevels; level++) {
+      final LevelStats lvlStats = getLevelCapacityItems(k, m, numLevels, level);
+      cumItems += lvlStats.items;
+      cumN += lvlStats.n;
+      if (printSketchStructure) {
+        printf("%6d %,8d %,12d %,18d %,18d\n", level, lvlStats.items, cumItems, lvlStats.n, cumN);
+      }
+    }
+    return new LevelStats(cumN, numLevels, cumItems);
   }
 
   /**
-   * Returns the upper bound of the number of levels based on <i>n</i>.
-   * @param n the length of the stream
-   * @return floor( log_2(n) )
+   * Given k, m, n, and the sketch type, this computes (and optionally prints) the growth scheme for a sketch as it
+   * grows large enough to accommodate a stream length of n items.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param n the desired stream length
+   * @param sketchType the given sketch type (DOUBLES_SKETCH or FLOATS_SKETCH)
+   * @param printGrowthScheme if true the entire growth scheme of the sketch will be printed.
+   * @return GrowthStats with the final values of the growth scheme
    */
-  static int ubOnNumLevels(final long n) {
-    return 1 + Long.numberOfTrailingZeros(floorPowerOf2(n));
+  public static GrowthStats getGrowthSchemeForGivenN(
+      final int k,
+      final int m,
+      final long n,
+      final SketchType sketchType,
+      final boolean printGrowthScheme) {
+    int numLevels = 0;
+    LevelStats lvlStats;
+    final GrowthStats gStats = new GrowthStats();
+    gStats.k = k;
+    gStats.m = m;
+    gStats.givenN = n;
+    gStats.sketchType = sketchType;
+    if (printGrowthScheme) {
+      println("GROWTH SCHEME:");
+      println("Given SketchType: " + sketchType.toString());
+      println("Given K         : " + k);
+      println("Given M         : " + m);
+      println("Given N         : " + n);
+      printf("%10s %10s %20s %13s %15s\n", "NumLevels", "MaxItems", "MaxN", "CompactBytes", "UpdatableBytes");
+    }

Review Comment:
   Maybe, this has been rewritten, so I'll take a look.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849876885


##########
src/main/java/org/apache/datasketches/kll/package-info.java:
##########
@@ -147,6 +148,25 @@
  * <li>Then <i>v<sub>lo</sub> &le; v &le; v<sub>hi</sub></i>, with 99% confidence.</li>
  * </ul>
  *
+ * <p>The current implementations of the KLL sketch in the DataSketches Java library component include:</p>
+ *
+ * <ul>
+ * <li><b>KllFloatsSketch</b>: This operates on the Java heap and uses the java <i>float</i> primitive for the
+ * smallest possible size. It can be serialized to a compact, immutable form or to an updatable form suitable for
+ * use by the Kll Direct sketches.</li>
+ * <li><b>KllDoublesSketch</b>: This operates on the Java heap and uses the java <i>double</i> primitive for a much
+ * larger range of numeric values, and is larger as a result. It can be serialized to a compact, immutable form or
+ * to an updatable form suitable for use by the Kll Direct sketches.</li>
+ * <li><b>KllDirectFloatsSketch</b>: This is intended to operate off-heap and performs all of its operations in one
+ * contiguous chunk of memory. It uses the java <i>float</i> primitive for the smallest possible size off-heap.</li>
+ * <li><b>KllDirectDoublesSketch</b>:   This is intended to operate off-heap and performs all of its operations in one
+ * contiguous chunk of memory. It uses the java <i>double</i> primitive for a much larger range of numeric values,
+ * and is larger as a result.</li>
+ * </ul>
+ *
+ * <p>Please visit our website: <a href="https://datasketches.apache.org">DataSketches Home Page</a> for more
+ * information.</p>

Review Comment:
   If you remove the size penalty by going from floats to doubles, and compare "apples to apples", the KllDoublesSketch is considerably smaller than the classic DoublesSketch.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r843233613


##########
src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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 org.apache.datasketches.Util.zeroPad;
+
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+//@formatter:off
+
+/**
+ * This class defines the serialized data structure and provides access methods for the key fields.
+ *
+ * <p>The intent of the design of this class was to isolate the detailed knowledge of the bit and
+ * byte layout of the serialized form of the sketches derived from the base sketch classes into one place.
+ * This allows the possibility of the introduction of different serialization
+ * schemes with minimal impact on the rest of the library.</p>
+ *
+ * <p>
+ * LAYOUT: The low significance bytes of this <i>long</i> based data structure are on the right.
+ * The multi-byte primitives are stored in native byte order.
+ * The single byte fields are treated as unsigned.</p>
+ *
+ * <p>An empty sketch requires only 8 bytes, which is only preamble.
+ * A serialized, non-empty KllDoublesSketch requires at least 16 bytes of preamble.
+ * A serialized, non-empty KllFloatsSketch requires at least 12 bytes of preamble.</p>
+ *
+ * <pre>{@code
+ * Serialized float sketch layout, more than one item:
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||---------------------------------N_LONG---------------------------------------|
+ *      ||         |       |        |   20   |   19   |    18   |   17   |      16      |
+ *  2   ||<-------Levels Arr Start----------]| unused |NumLevels|------Min K------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-------Min/Max Arr Start---------]|[<----------Levels Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-----Float Items Arr Start-------]|[<---------Min/Max Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||         |       |        |        |[<-------Float Items Arr End--------------|
+ *
+ * Serialized float sketch layout, Empty (8 bytes) and Single Item (12 bytes):
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||                                   |-------------Single Item------------------|
+ *
+ *
+ *
+ * Serialized double sketch layout, more than one item:
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||---------------------------------N_LONG---------------------------------------|
+ *      ||   23    |   22  |   21   |   20   |   19   |    18   |   17   |      16      |
+ *  2   ||<-------Levels Arr Start----------]| unused |NumLevels|------Min K------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-------Min/Max Arr Start---------]|[<----------Levels Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<----Double Items Arr Start-------]|[<---------Min/Max Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||         |       |        |        |[<------Double Items Arr End--------------|
+ *
+ * Serialized double sketch layout, Empty (8 bytes) and Single Item (16 bytes):
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||                                                               |      8       |
+ *  1   ||------------------------------Single Item-------------------------------------|
+ *
+ * The structure of the data block depends on Layout:
+ *
+ *   For FLOAT_SINGLE_COMPACT or DOUBLE_SINGLE_COMPACT:
+ *     The single data item is at offset DATA_START_ADR_SINGLE_ITEM = 8
+ *
+ *   For FLOAT_FULL_COMPACT:
+ *     The int[] levels array starts at offset DATA_START_ADR_FLOAT = 20 with a length of numLevels integers;
+ *     Followed by Float Min_Value, then Float Max_Value
+ *     Followed by an array of Floats of length retainedItems()
+ *
+ *   For DOUBLE_FULL_COMPACT
+ *     The int[] levels array starts at offset DATA_START_ADR_DOUBLE = 20 with a length of numLevels integers;
+ *     Followed by Double Min_Value, then Double Max_Value
+ *     Followed by an array of Doubles of length retainedItems()
+ *
+ *   For FLOAT_UPDATABLE
+ *     The int[] levels array starts at offset DATA_START_ADR_FLOAT = 20 with a length of (numLevels + 1) integers;
+ *     Followed by Float Min_Value, then Float Max_Value
+ *     Followed by an array of Floats of length KllHelper.computeTotalItemCapacity(...).
+ *
+ *   For DOUBLE_UPDATABLE
+ *     The int[] levels array starts at offset DATA_START_ADR_DOUBLE = 20 with a length of (numLevels + 1) integers;
+ *     Followed by Double Min_Value, then Double Max_Value
+ *     Followed by an array of Doubles of length KllHelper.computeTotalItemCapacity(...).
+ *
+ * }</pre>
+ *
+ *  @author Lee Rhodes
+ */
+final class KllPreambleUtil {
+
+  private KllPreambleUtil() {}
+
+  static final String LS = System.getProperty("line.separator");
+
+  /**
+   * The default value of K
+   */
+  public static final int DEFAULT_K = 200;
+  public static final int DEFAULT_M = 8;
+  public static final int MAX_K = (1 << 16) - 1; // serialized as an unsigned short
+  public static final int MAX_M = 8;
+  public static final int MIN_M = 2;
+
+  // Preamble byte addresses
+  static final int PREAMBLE_INTS_BYTE_ADR     = 0;
+  static final int SER_VER_BYTE_ADR           = 1;
+  static final int FAMILY_BYTE_ADR            = 2;
+  static final int FLAGS_BYTE_ADR             = 3;
+  static final int K_SHORT_ADR                = 4;  // to 5
+  static final int M_BYTE_ADR                 = 6;
+  //                                            7 is reserved for future use
+  // SINGLE ITEM ONLY
+  static final int DATA_START_ADR_SINGLE_ITEM = 8;
+
+  // MULTI-ITEM
+  static final int N_LONG_ADR                 = 8;  // to 15
+  static final int MIN_K_SHORT_ADR         = 16; // to 17
+  static final int NUM_LEVELS_BYTE_ADR        = 18;
+
+  // FLOAT SKETCH                               19 is reserved for future use in float sketch
+  static final int DATA_START_ADR_FLOAT       = 20; // float sketch, not single item
+
+  // DOUBLE SKETCH                              19 to 23 is reserved for future use in double sketch
+  static final int DATA_START_ADR_DOUBLE      = 20; // double sketch, not single item
+
+  // Other static values
+  static final byte SERIAL_VERSION_EMPTY_FULL = 1; // Empty or full preamble, NOT single item format
+  static final byte SERIAL_VERSION_SINGLE     = 2; // only single-item format
+  static final byte SERIAL_VERSION_UPDATABLE  = 3; //
+  static final int PREAMBLE_INTS_EMPTY_SINGLE = 2; // for empty or single item
+  static final int PREAMBLE_INTS_FLOAT        = 5; // not empty nor single item, full preamble float
+  static final int PREAMBLE_INTS_DOUBLE       = 5; // not empty nor single item, full preamble double
+
+  // Flag bit masks
+  static final int EMPTY_BIT_MASK             = 1;
+  static final int LEVEL_ZERO_SORTED_BIT_MASK = 2;
+  static final int SINGLE_ITEM_BIT_MASK       = 4;
+  static final int DOUBLES_SKETCH_BIT_MASK    = 8;
+  static final int UPDATABLE_BIT_MASK         = 16;
+
+  enum Layout {
+    FLOAT_FULL_COMPACT,       FLOAT_EMPTY_COMPACT,      FLOAT_SINGLE_COMPACT,
+    DOUBLE_FULL_COMPACT,      DOUBLE_EMPTY_COMPACT,     DOUBLE_SINGLE_COMPACT,
+    FLOAT_UPDATABLE,  DOUBLE_UPDATABLE }
+
+  /**
+   * Returns a human readable string summary of the internal state of the given byte array.
+   * Used primarily in testing.
+   *
+   * @param byteArr the given byte array.
+   * @return the summary string.
+   */
+  static String toString(final byte[] byteArr) {
+    final Memory mem = Memory.wrap(byteArr);
+    return toString(mem);
+  }
+
+  /**
+   * Returns a human readable string summary of the internal state of the given Memory.
+   * Used primarily in testing.
+   *
+   * @param mem the given Memory
+   * @return the summary string.
+   */
+  static String toString(final Memory mem) {
+    return memoryToString(mem);
+  }
+
+  static String memoryToString(final Memory mem) {
+    final KllMemoryValidate memChk = new KllMemoryValidate(mem);
+    final int flags = memChk.flags & 0XFF;
+    final String flagsStr = (flags) + ", 0x" + (Integer.toHexString(flags)) + ", "
+        + zeroPad(Integer.toBinaryString(flags), 8);
+    final int preInts = memChk.preInts;
+    final StringBuilder sb = new StringBuilder();
+    sb.append(Util.LS).append("### KLL SKETCH MEMORY SUMMARY:").append(LS);
+    sb.append("Byte   0   : Preamble Ints      : ").append(preInts).append(LS);
+    sb.append("Byte   1   : SerVer             : ").append(memChk.serVer).append(LS);
+    sb.append("Byte   2   : FamilyID           : ").append(memChk.familyID).append(LS);
+    sb.append("             FamilyName         : ").append(memChk.famName).append(LS);
+    sb.append("Byte   3   : Flags Field        : ").append(flagsStr).append(LS);
+    sb.append("         Bit Flag Name").append(LS);
+    sb.append("           0 EMPTY COMPACT      : ").append(memChk.empty).append(LS);
+    sb.append("           1 LEVEL_ZERO_SORTED  : ").append(memChk.level0Sorted).append(LS);
+    sb.append("           2 SINGLE_ITEM COMPACT: ").append(memChk.singleItem).append(LS);
+    sb.append("           3 DOUBLES_SKETCH     : ").append(memChk.doublesSketch).append(LS);
+    sb.append("           4 UPDATABLE          : ").append(memChk.updatable).append(LS);
+    sb.append("Bytes  4-5 : K                  : ").append(memChk.k).append(LS);
+    sb.append("Byte   6   : Min Level Cap, M   : ").append(memChk.m).append(LS);
+    sb.append("Byte   7   : (Reserved)         : ").append(LS);
+
+    switch (memChk.layout) {
+      case DOUBLE_FULL_COMPACT:
+      case FLOAT_FULL_COMPACT:
+      case FLOAT_UPDATABLE:
+      case DOUBLE_UPDATABLE:
+      {
+        sb.append("Bytes  8-15: N                  : ").append(memChk.n).append(LS);
+        sb.append("Bytes 16-17: DyMinK             : ").append(memChk.dyMinK).append(LS);
+        sb.append("Byte  18   : NumLevels          : ").append(memChk.numLevels).append(LS);
+        break;
+      }
+      case FLOAT_EMPTY_COMPACT:
+      case FLOAT_SINGLE_COMPACT:
+      case DOUBLE_EMPTY_COMPACT:
+      case DOUBLE_SINGLE_COMPACT:
+      {
+        sb.append("Assumed    : N                  : ").append(memChk.n).append(LS);
+        sb.append("Assumed    : DyMinK             : ").append(memChk.dyMinK).append(LS);
+        sb.append("Assumed    : NumLevels          : ").append(memChk.numLevels).append(LS);
+        break;
+      }
+      default: break; //can never happen
+    }
+    sb.append("PreambleBytes                   : ").append(preInts * 4).append(LS);
+    sb.append("Sketch Bytes                    : ").append(memChk.sketchBytes).append(LS);
+    sb.append("Memory Capacity Bytes           : ").append(mem.getCapacity()).append(LS);
+    sb.append("### END KLL Sketch Memory Summary").append(LS);
+    return sb.toString();
+  }
+
+  static int extractPreInts(final Memory mem) {
+    return mem.getByte(PREAMBLE_INTS_BYTE_ADR) & 0XFF;
+  }
+
+  static int extractSerVer(final Memory mem) {
+    return mem.getByte(SER_VER_BYTE_ADR) & 0XFF;
+  }
+
+  static int extractFamilyID(final Memory mem) {
+    return mem.getByte(FAMILY_BYTE_ADR) & 0XFF;
+  }
+
+  static int extractFlags(final Memory mem) {
+    return mem.getByte(FLAGS_BYTE_ADR) & 0XFF;
+  }
+
+  static boolean extractEmptyFlag(final Memory mem) {
+    return (extractFlags(mem) & EMPTY_BIT_MASK) != 0;
+  }
+
+  static boolean extractLevelZeroSortedFlag(final Memory mem) {
+    return (extractFlags(mem) & LEVEL_ZERO_SORTED_BIT_MASK) != 0;
+  }
+
+  static boolean extractSingleItemFlag(final Memory mem) {
+    return (extractFlags(mem) & SINGLE_ITEM_BIT_MASK) != 0;
+  }
+
+  static boolean extractDoubleSketchFlag(final Memory mem) {
+    return (extractFlags(mem) & DOUBLES_SKETCH_BIT_MASK) != 0;
+  }
+
+  static boolean extractUpdatableFlag(final Memory mem) {
+    return (extractFlags(mem) & UPDATABLE_BIT_MASK) != 0;
+  }
+
+  static int extractK(final Memory mem) {
+    return mem.getShort(K_SHORT_ADR) & 0XFFFF;
+  }
+
+  static int extractM(final Memory mem) {
+    return mem.getByte(M_BYTE_ADR) & 0XFF;
+  }
+
+  static long extractN(final Memory mem) {
+    return mem.getLong(N_LONG_ADR);
+  }
+
+  static int extractMinK(final Memory mem) {
+    return mem.getShort(MIN_K_SHORT_ADR) & 0XFFFF;
+  }
+
+  static int extractNumLevels(final Memory mem) {
+    return mem.getByte(NUM_LEVELS_BYTE_ADR) & 0XFF;
+  }
+
+  static void insertPreInts(final WritableMemory wmem, final int value) {
+    wmem.putByte(PREAMBLE_INTS_BYTE_ADR, (byte) value);
+  }
+
+  static void insertSerVer(final WritableMemory wmem, final int value) {
+    wmem.putByte(SER_VER_BYTE_ADR, (byte) value);
+  }
+
+  static void insertFamilyID(final WritableMemory wmem, final int value) {
+    wmem.putByte(FAMILY_BYTE_ADR, (byte) value);
+  }
+
+  static void insertFlags(final WritableMemory wmem, final int value) {
+    wmem.putByte(FLAGS_BYTE_ADR, (byte) value);
+  }
+
+  static void insertEmptyFlag(final WritableMemory wmem,  final boolean empty) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, empty ? flags | EMPTY_BIT_MASK : flags & ~EMPTY_BIT_MASK);
+  }
+
+  static void insertLevelZeroSortedFlag(final WritableMemory wmem,  final boolean levelZeroSorted) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, levelZeroSorted ? flags | LEVEL_ZERO_SORTED_BIT_MASK : flags & ~LEVEL_ZERO_SORTED_BIT_MASK);
+  }
+
+  static void insertSingleItemFlag(final WritableMemory wmem,  final boolean singleItem) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, singleItem ? flags | SINGLE_ITEM_BIT_MASK : flags & ~SINGLE_ITEM_BIT_MASK);
+  }
+
+  static void insertDoubleSketchFlag(final WritableMemory wmem,  final boolean doubleSketch) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, doubleSketch ? flags | DOUBLES_SKETCH_BIT_MASK : flags & ~DOUBLES_SKETCH_BIT_MASK);
+  }
+
+  static void insertUpdatableFlag(final WritableMemory wmem,  final boolean updatable) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, updatable ? flags | UPDATABLE_BIT_MASK : flags & ~UPDATABLE_BIT_MASK);
+  }
+
+  static void insertK(final WritableMemory wmem, final int value) {
+    wmem.putShort(K_SHORT_ADR, (short) value);
+  }
+
+  static void insertM(final WritableMemory wmem, final int value) {
+    wmem.putByte(M_BYTE_ADR, (byte) value);
+  }
+
+  static void insertN(final WritableMemory wmem, final long value) {
+    wmem.putLong(N_LONG_ADR, value);
+  }
+
+  static void insertMinK(final WritableMemory wmem, final int value) {
+    wmem.putShort(MIN_K_SHORT_ADR, (short) value);
+  }
+
+  static void insertNumLevels(final WritableMemory wmem, final int value) {
+    wmem.putByte(NUM_LEVELS_BYTE_ADR, (byte) value);
+  }
+

Review Comment:
   This has been fixed.
   



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842168694


##########
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;
+  }

Review Comment:
   Fixed.
   



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842193618


##########
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:
   Added more complete comments in a number of places.



-- 
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


[GitHub] [datasketches-java] leerho closed pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho closed pull request #390: Direct kll double
URL: https://github.com/apache/datasketches-java/pull/390


-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849871859


##########
src/main/java/org/apache/datasketches/kll/KllMemoryValidate.java:
##########
@@ -0,0 +1,349 @@
+/*
+ * 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 org.apache.datasketches.Family.idToFamily;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.SRC_NOT_KLL;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.EMPTYBIT_AND_PREINTS;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.UPDATABLEBIT_AND_SER_VER;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.EMPTYBIT_AND_SER_VER;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.EMPTYBIT_AND_SINGLEBIT;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.SINGLEBIT_AND_SER_VER;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.DOUBLEBIT_AND_PREINTS;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.FLOATBIT_AND_PREINTS;
+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.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.extractDoubleSketchFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractEmptyFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractFlags;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractM;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractN;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractPreInts;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractSerVer;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractSingleItemFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractUpdatableFlag;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.kll.KllPreambleUtil.Layout;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class performs all the error checking of an incoming Memory object and extracts the key fields in the process.
+ * This is used by all sketches that read or import Memory objects.
+ *
+ * @author lrhodes
+ *
+ */
+final class KllMemoryValidate {
+  // first 8 bytes
+  final int preInts; // = extractPreInts(srcMem);
+  final int serVer;
+  final int familyID;
+  final String famName;
+  final int flags;
+  boolean empty;
+  boolean singleItem;
+  final boolean level0Sorted;
+  final boolean doublesSketch;
+  final boolean updatable;
+  final int k;
+  final int m;
+  final int memCapacity;
+
+  Layout layout;
+  // depending on the layout, the next 8-16 bytes of the preamble, may be filled with assumed values.
+  // For example, if the layout is compact & empty, n = 0, if compact and single, n = 1, etc.
+  long n;
+  // next 4 bytes
+  int dyMinK;
+  int numLevels;
+  // derived
+  int capacityItems; //capacity of Items array for exporting and for Updatable form
+  int itemsRetained; //actual items retained in Compact form
+  int itemsArrStart;
+  int sketchBytes;
+  Memory levelsArrCompact; //if sk = empty or single, this is derived
+  Memory minMaxArrCompact; //if sk = empty or single, this is derived
+  Memory itemsArrCompact;  //if sk = empty or single, this is derived
+  WritableMemory levelsArrUpdatable;
+  WritableMemory minMaxArrUpdatable;
+  WritableMemory itemsArrUpdatable;
+
+  KllMemoryValidate(final Memory srcMem) {
+    memCapacity = (int) srcMem.getCapacity();
+    preInts = extractPreInts(srcMem);
+    serVer = extractSerVer(srcMem);
+
+    familyID = extractFamilyID(srcMem);
+    if (familyID != Family.KLL.getID()) { memoryValidateThrow(SRC_NOT_KLL, familyID); }
+    famName = idToFamily(familyID).toString();
+    flags = extractFlags(srcMem);
+    empty = extractEmptyFlag(srcMem);
+    level0Sorted  = extractLevelZeroSortedFlag(srcMem);
+    singleItem    = extractSingleItemFlag(srcMem);
+    doublesSketch = extractDoubleSketchFlag(srcMem);
+    updatable    = extractUpdatableFlag(srcMem);
+    k = extractK(srcMem);
+    m = extractM(srcMem);
+    KllHelper.checkM(m);
+    KllHelper.checkK(k, m);
+    if ((serVer == SERIAL_VERSION_UPDATABLE) ^ updatable) { memoryValidateThrow(UPDATABLEBIT_AND_SER_VER, 0); }
+
+    if (updatable) { updatableMemoryValidate((WritableMemory) srcMem); }
+    else { compactMemoryValidate(srcMem); }
+  }
+
+  void compactMemoryValidate(final Memory srcMem) {
+    if (empty && singleItem) { memoryValidateThrow(EMPTYBIT_AND_SINGLEBIT, 0); }
+    final int sw = (empty ? 1 : 0) | (singleItem ? 4 : 0) | (doublesSketch ? 8 : 0);
+    switch (sw) {
+      case 0: { //FLOAT_FULL_COMPACT
+        if (preInts != PREAMBLE_INTS_FLOAT) { memoryValidateThrow(FLOATBIT_AND_PREINTS, preInts); }
+        if (serVer != SERIAL_VERSION_EMPTY_FULL) { memoryValidateThrow(EMPTYBIT_AND_SER_VER, serVer); }
+        layout = Layout.FLOAT_FULL_COMPACT;
+        n = extractN(srcMem);
+        dyMinK = extractMinK(srcMem);
+        numLevels = extractNumLevels(srcMem);
+        int offset = DATA_START_ADR_FLOAT;
+        // LEVELS MEM
+        final int[] myLevelsArr = new int[numLevels + 1];
+        srcMem.getIntArray(offset, myLevelsArr, 0, numLevels); //copies all except the last one
+        myLevelsArr[numLevels] = KllHelper.computeTotalItemCapacity(k, m, numLevels); //load the last one
+        levelsArrCompact = Memory.wrap(myLevelsArr); //separate from srcMem,
+        offset += (int)levelsArrCompact.getCapacity() - Integer.BYTES; // but one larger than srcMem
+        // MIN/MAX MEM
+        minMaxArrCompact = srcMem.region(offset, 2L * Float.BYTES);
+        offset += (int)minMaxArrCompact.getCapacity();
+        // ITEMS MEM
+        itemsArrStart = offset;
+        capacityItems = myLevelsArr[numLevels];
+        itemsRetained = capacityItems - myLevelsArr[0];
+        final float[] myItemsArr = new float[capacityItems];
+        srcMem.getFloatArray(offset, myItemsArr, myLevelsArr[0], itemsRetained);
+        itemsArrCompact = Memory.wrap(myItemsArr);
+        sketchBytes = offset + itemsRetained * Float.BYTES;
+        break;
+      }
+      case 1: { //FLOAT_EMPTY_COMPACT
+        if (preInts != PREAMBLE_INTS_EMPTY_SINGLE) { memoryValidateThrow(EMPTYBIT_AND_PREINTS, preInts); }
+        if (serVer != SERIAL_VERSION_EMPTY_FULL) { memoryValidateThrow(EMPTYBIT_AND_SER_VER, serVer); }
+        layout = Layout.FLOAT_EMPTY_COMPACT;
+        n = 0;           //assumed
+        dyMinK = k;      //assumed
+        numLevels = 1;   //assumed
+        // LEVELS MEM
+        levelsArrCompact = Memory.wrap(new int[] {k, k});
+        // MIN/MAX MEM
+        minMaxArrCompact = Memory.wrap(new float[] {Float.NaN, Float.NaN});
+        // ITEMS MEM
+        capacityItems = k;
+        itemsRetained = 0;
+        itemsArrCompact = Memory.wrap(new float[k]);
+        sketchBytes = DATA_START_ADR_SINGLE_ITEM; //also used for empty
+        itemsArrStart = DATA_START_ADR_SINGLE_ITEM;
+        break;
+      }
+      case 4: { //FLOAT_SINGLE_COMPACT
+        if (preInts != PREAMBLE_INTS_EMPTY_SINGLE) { memoryValidateThrow(EMPTYBIT_AND_PREINTS, preInts); }
+        if (serVer != SERIAL_VERSION_SINGLE) { memoryValidateThrow(SINGLEBIT_AND_SER_VER, serVer); }
+        layout = Layout.FLOAT_SINGLE_COMPACT;
+        n = 1;
+        dyMinK = k;
+        numLevels = 1;
+        // LEVELS MEM
+        levelsArrCompact = Memory.wrap(new int[] {k - 1, k});
+        final float minMax = srcMem.getFloat(DATA_START_ADR_SINGLE_ITEM);
+        // MIN/MAX MEM
+        minMaxArrCompact = Memory.wrap(new float[] {minMax, minMax});
+        // ITEMS MEM
+        capacityItems = k;
+        itemsRetained = 1;
+        final float[] myFloatItems = new float[k];
+        myFloatItems[k - 1] = minMax;
+        itemsArrCompact = Memory.wrap(myFloatItems);
+        sketchBytes = DATA_START_ADR_SINGLE_ITEM + Float.BYTES;
+        itemsArrStart = DATA_START_ADR_SINGLE_ITEM;
+        break;
+      }
+      case 8: { //DOUBLE_FULL_COMPACT
+        if (preInts != PREAMBLE_INTS_DOUBLE) { memoryValidateThrow(DOUBLEBIT_AND_PREINTS, preInts); }
+        if (serVer != SERIAL_VERSION_EMPTY_FULL) { memoryValidateThrow(EMPTYBIT_AND_SER_VER, serVer); }
+        layout = Layout.DOUBLE_FULL_COMPACT;
+        n = extractN(srcMem);
+        dyMinK = extractMinK(srcMem);
+        numLevels = extractNumLevels(srcMem);
+        int offset = DATA_START_ADR_DOUBLE;
+        // LEVELS MEM
+        final int[] myLevelsArr = new int[numLevels + 1];
+        srcMem.getIntArray(offset, myLevelsArr, 0, numLevels); //all except the last one
+        myLevelsArr[numLevels] = KllHelper.computeTotalItemCapacity(k, m, numLevels); //load the last one
+        levelsArrCompact = Memory.wrap(myLevelsArr); //separate from srcMem
+        offset += (int)levelsArrCompact.getCapacity() - Integer.BYTES;
+        // MIN/MAX MEM
+        minMaxArrCompact = srcMem.region(offset, 2L * Double.BYTES);
+        offset += (int)minMaxArrCompact.getCapacity();
+        // ITEMS MEM
+        itemsArrStart = offset;
+        capacityItems = myLevelsArr[numLevels];
+        itemsRetained = capacityItems - myLevelsArr[0];
+        final double[] myItemsArr = new double[capacityItems];
+        srcMem.getDoubleArray(offset, myItemsArr, myLevelsArr[0], itemsRetained);
+        itemsArrCompact = Memory.wrap(myItemsArr);
+        sketchBytes = offset + itemsRetained * Double.BYTES;
+        break;
+      }
+      case 9: { //DOUBLE_EMPTY_COMPACT
+        if (preInts != PREAMBLE_INTS_EMPTY_SINGLE) { memoryValidateThrow(EMPTYBIT_AND_PREINTS, preInts); }
+        if (serVer != SERIAL_VERSION_EMPTY_FULL) { memoryValidateThrow(EMPTYBIT_AND_SER_VER, serVer); }
+        layout = Layout.DOUBLE_EMPTY_COMPACT;
+        n = 0;
+        dyMinK = k;
+        numLevels = 1;
+
+        // LEVELS MEM
+        levelsArrCompact = Memory.wrap(new int[] {k, k});
+        // MIN/MAX MEM
+        minMaxArrCompact = Memory.wrap(new double[] {Double.NaN, Double.NaN});
+        // ITEMS MEM
+        capacityItems = k;
+        itemsRetained = 0;
+        itemsArrCompact = Memory.wrap(new double[k]);
+        sketchBytes = DATA_START_ADR_SINGLE_ITEM; //also used for empty
+        itemsArrStart = DATA_START_ADR_SINGLE_ITEM;
+        break;
+      }
+      case 12: { //DOUBLE_SINGLE_COMPACT
+        if (preInts != PREAMBLE_INTS_EMPTY_SINGLE) { memoryValidateThrow(EMPTYBIT_AND_PREINTS, preInts); }
+        if (serVer != SERIAL_VERSION_SINGLE) { memoryValidateThrow(SINGLEBIT_AND_SER_VER, serVer); }
+        layout = Layout.DOUBLE_SINGLE_COMPACT;
+        n = 1;
+        dyMinK = k;
+        numLevels = 1;
+
+        // LEVELS MEM
+        levelsArrCompact = Memory.wrap(new int[] {k - 1, k});
+        final double minMax = srcMem.getDouble(DATA_START_ADR_SINGLE_ITEM);
+        // MIN/MAX MEM
+        minMaxArrCompact = Memory.wrap(new double[] {minMax, minMax});
+        // ITEMS MEM
+        capacityItems = k;
+        itemsRetained = 1;
+        final double[] myDoubleItems = new double[k];
+        myDoubleItems[k - 1] = minMax;
+        itemsArrCompact = Memory.wrap(myDoubleItems);
+        sketchBytes = DATA_START_ADR_SINGLE_ITEM + Double.BYTES;
+        itemsArrStart = DATA_START_ADR_SINGLE_ITEM;
+        break;
+      }
+      default: break; //can't happen
+    }
+  }
+
+  void updatableMemoryValidate(final WritableMemory wSrcMem) {
+    if (doublesSketch) { //DOUBLE_UPDATABLE
+      if (preInts != PREAMBLE_INTS_DOUBLE) { memoryValidateThrow(DOUBLEBIT_AND_PREINTS, preInts); }
+      layout = Layout.DOUBLE_UPDATABLE;
+      n = extractN(wSrcMem);
+      empty = n == 0;       //empty & singleItem are set for convenience
+      singleItem = n == 1;  // there is no error checking on these bits
+      dyMinK = extractMinK(wSrcMem);
+      numLevels = extractNumLevels(wSrcMem);
+
+      int offset = DATA_START_ADR_DOUBLE;
+      //LEVELS
+      levelsArrUpdatable = wSrcMem.writableRegion(offset, (numLevels + 1L) * Integer.BYTES);
+      offset += (int)levelsArrUpdatable.getCapacity();
+      //MIN/MAX
+      minMaxArrUpdatable = wSrcMem.writableRegion(offset, 2L * Double.BYTES);
+      offset += (int)minMaxArrUpdatable.getCapacity();
+      //ITEMS
+      capacityItems = levelsArrUpdatable.getInt((long)numLevels * Integer.BYTES);
+      final int itemsArrBytes = capacityItems * Double.BYTES;
+      itemsArrStart = offset;
+      itemsArrUpdatable = wSrcMem.writableRegion(offset, itemsArrBytes);
+      sketchBytes = offset + itemsArrBytes;
+    }
+    else { //FLOAT_UPDATABLE
+      if (preInts != PREAMBLE_INTS_FLOAT) { memoryValidateThrow(FLOATBIT_AND_PREINTS, preInts); }
+      layout = Layout.FLOAT_UPDATABLE;
+      n = extractN(wSrcMem);
+      empty = n == 0;       //empty & singleItem are set for convenience
+      singleItem = n == 1;  // there is no error checking on these bits
+      dyMinK = extractMinK(wSrcMem);
+      numLevels = extractNumLevels(wSrcMem);
+      int offset = DATA_START_ADR_FLOAT;
+      //LEVELS
+      levelsArrUpdatable = wSrcMem.writableRegion(offset, (numLevels + 1L) * Integer.BYTES);
+      offset += (int)levelsArrUpdatable.getCapacity();
+      //MIN/MAX
+      minMaxArrUpdatable = wSrcMem.writableRegion(offset, 2L * Float.BYTES);
+      offset += (int)minMaxArrUpdatable.getCapacity();
+      //ITEMS
+      capacityItems = levelsArrUpdatable.getInt((long)numLevels * Integer.BYTES);
+      final int itemsArrBytes = capacityItems * Float.BYTES;
+      itemsArrStart = offset;
+      itemsArrUpdatable = wSrcMem.writableRegion(offset, itemsArrBytes);
+      sketchBytes = itemsArrStart + itemsArrBytes;
+    }
+  }
+
+  enum MemoryInputError { SRC_NOT_KLL, EMPTYBIT_AND_PREINTS, EMPTYBIT_AND_SER_VER,
+    SINGLEBIT_AND_SER_VER, DOUBLEBIT_AND_PREINTS, FLOATBIT_AND_PREINTS, UPDATABLEBIT_AND_SER_VER,
+    EMPTYBIT_AND_SINGLEBIT }
+
+  private static void memoryValidateThrow(final MemoryInputError errType, final int value) {
+    String msg = "";
+    switch (errType) {
+      case SRC_NOT_KLL: msg = "FamilyID Field must be: " + Family.KLL.getID() + ", NOT: " + value; break;
+      case EMPTYBIT_AND_PREINTS: msg =
+          "Empty Bit: 1 -> PreInts: " + PREAMBLE_INTS_EMPTY_SINGLE + ", NOT: " + value; break;
+      case EMPTYBIT_AND_SER_VER: msg =
+          "Empty Bit: 1 -> SerVer: " + SERIAL_VERSION_EMPTY_FULL + ", NOT: " + value; break;
+      case SINGLEBIT_AND_SER_VER: msg =
+          "Single Item Bit: 1 -> SerVer: " + SERIAL_VERSION_SINGLE + ", NOT: " + value; break;
+      case DOUBLEBIT_AND_PREINTS: msg =
+          "Double Sketch Bit: 1 -> PreInts: " + PREAMBLE_INTS_DOUBLE + ", NOT: " + value; break;
+      case FLOATBIT_AND_PREINTS: msg =
+          "Double Sketch Bit: 0 -> PreInts: " + PREAMBLE_INTS_FLOAT + ", NOT: " + value; break;
+      case UPDATABLEBIT_AND_SER_VER: msg =
+          "((SerVer == 3) ^ (Updatable Bit)) must = 0."; break;
+      case EMPTYBIT_AND_SINGLEBIT: msg =
+          "Empty flag bit and SingleItem flag bit cannot both be set. Flags: " + value; break;
+      default: msg = "Unknown error"; break;
+    }
+    throw new SketchesArgumentException(msg);
+  }
+
+}

Review Comment:
   KllMemoryValidate is only used to validate incoming Memory objects to protect against possibly corrupted binary objects or accidental use of other sketch Memory objects being submitted to this sketch.  Once the sketch has been initialized, it is not every used again.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842170964


##########
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.

Review Comment:
   I'm going to change the wording to " After a compaction, or an update, or a merge, every level is sorted except for level zero."
   
   



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842183155


##########
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:
   If you do a quick references check you will find this is used in only two places:
   - getCurrentCompactSerializedSizeBytes()
   - getCurrentUpdatableSerializedSizeBytes()
   In the former case, numItems refers to retained items, in the latter case it refers to the total capacity (items) of the current sketch configured with K and at the given number of levels.
   
   I don't normally write Javadocs on internal methods unless they are quite complex or subtle in operation. 
   But for you :) , I changed the name of the method to be a bit more clear and added a code comment on numItems.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849869310


##########
src/main/java/org/apache/datasketches/kll/KllMemoryValidate.java:
##########
@@ -0,0 +1,349 @@
+/*
+ * 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 org.apache.datasketches.Family.idToFamily;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.SRC_NOT_KLL;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.EMPTYBIT_AND_PREINTS;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.UPDATABLEBIT_AND_SER_VER;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.EMPTYBIT_AND_SER_VER;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.EMPTYBIT_AND_SINGLEBIT;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.SINGLEBIT_AND_SER_VER;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.DOUBLEBIT_AND_PREINTS;
+import static org.apache.datasketches.kll.KllMemoryValidate.MemoryInputError.FLOATBIT_AND_PREINTS;
+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.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.extractDoubleSketchFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractEmptyFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractFlags;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractM;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractN;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractPreInts;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractSerVer;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractSingleItemFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractUpdatableFlag;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.kll.KllPreambleUtil.Layout;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class performs all the error checking of an incoming Memory object and extracts the key fields in the process.
+ * This is used by all sketches that read or import Memory objects.
+ *
+ * @author lrhodes
+ *
+ */
+final class KllMemoryValidate {
+  // first 8 bytes
+  final int preInts; // = extractPreInts(srcMem);
+  final int serVer;
+  final int familyID;
+  final String famName;
+  final int flags;
+  boolean empty;
+  boolean singleItem;
+  final boolean level0Sorted;
+  final boolean doublesSketch;
+  final boolean updatable;
+  final int k;
+  final int m;
+  final int memCapacity;
+
+  Layout layout;
+  // depending on the layout, the next 8-16 bytes of the preamble, may be filled with assumed values.
+  // For example, if the layout is compact & empty, n = 0, if compact and single, n = 1, etc.
+  long n;
+  // next 4 bytes
+  int dyMinK;
+  int numLevels;
+  // derived
+  int capacityItems; //capacity of Items array for exporting and for Updatable form
+  int itemsRetained; //actual items retained in Compact form
+  int itemsArrStart;
+  int sketchBytes;
+  Memory levelsArrCompact; //if sk = empty or single, this is derived
+  Memory minMaxArrCompact; //if sk = empty or single, this is derived
+  Memory itemsArrCompact;  //if sk = empty or single, this is derived
+  WritableMemory levelsArrUpdatable;
+  WritableMemory minMaxArrUpdatable;
+  WritableMemory itemsArrUpdatable;
+
+  KllMemoryValidate(final Memory srcMem) {
+    memCapacity = (int) srcMem.getCapacity();
+    preInts = extractPreInts(srcMem);
+    serVer = extractSerVer(srcMem);
+
+    familyID = extractFamilyID(srcMem);
+    if (familyID != Family.KLL.getID()) { memoryValidateThrow(SRC_NOT_KLL, familyID); }
+    famName = idToFamily(familyID).toString();
+    flags = extractFlags(srcMem);
+    empty = extractEmptyFlag(srcMem);
+    level0Sorted  = extractLevelZeroSortedFlag(srcMem);
+    singleItem    = extractSingleItemFlag(srcMem);
+    doublesSketch = extractDoubleSketchFlag(srcMem);
+    updatable    = extractUpdatableFlag(srcMem);
+    k = extractK(srcMem);
+    m = extractM(srcMem);
+    KllHelper.checkM(m);
+    KllHelper.checkK(k, m);
+    if ((serVer == SERIAL_VERSION_UPDATABLE) ^ updatable) { memoryValidateThrow(UPDATABLEBIT_AND_SER_VER, 0); }
+
+    if (updatable) { updatableMemoryValidate((WritableMemory) srcMem); }
+    else { compactMemoryValidate(srcMem); }
+  }
+
+  void compactMemoryValidate(final Memory srcMem) {
+    if (empty && singleItem) { memoryValidateThrow(EMPTYBIT_AND_SINGLEBIT, 0); }
+    final int sw = (empty ? 1 : 0) | (singleItem ? 4 : 0) | (doublesSketch ? 8 : 0);
+    switch (sw) {
+      case 0: { //FLOAT_FULL_COMPACT
+        if (preInts != PREAMBLE_INTS_FLOAT) { memoryValidateThrow(FLOATBIT_AND_PREINTS, preInts); }
+        if (serVer != SERIAL_VERSION_EMPTY_FULL) { memoryValidateThrow(EMPTYBIT_AND_SER_VER, serVer); }
+        layout = Layout.FLOAT_FULL_COMPACT;
+        n = extractN(srcMem);
+        dyMinK = extractMinK(srcMem);
+        numLevels = extractNumLevels(srcMem);
+        int offset = DATA_START_ADR_FLOAT;
+        // LEVELS MEM
+        final int[] myLevelsArr = new int[numLevels + 1];
+        srcMem.getIntArray(offset, myLevelsArr, 0, numLevels); //copies all except the last one
+        myLevelsArr[numLevels] = KllHelper.computeTotalItemCapacity(k, m, numLevels); //load the last one
+        levelsArrCompact = Memory.wrap(myLevelsArr); //separate from srcMem,
+        offset += (int)levelsArrCompact.getCapacity() - Integer.BYTES; // but one larger than srcMem
+        // MIN/MAX MEM
+        minMaxArrCompact = srcMem.region(offset, 2L * Float.BYTES);
+        offset += (int)minMaxArrCompact.getCapacity();
+        // ITEMS MEM
+        itemsArrStart = offset;
+        capacityItems = myLevelsArr[numLevels];
+        itemsRetained = capacityItems - myLevelsArr[0];
+        final float[] myItemsArr = new float[capacityItems];
+        srcMem.getFloatArray(offset, myItemsArr, myLevelsArr[0], itemsRetained);
+        itemsArrCompact = Memory.wrap(myItemsArr);
+        sketchBytes = offset + itemsRetained * Float.BYTES;
+        break;
+      }

Review Comment:
   This has been rewritten.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842029958


##########
src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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 org.apache.datasketches.Util.zeroPad;
+
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+//@formatter:off
+
+/**
+ * This class defines the serialized data structure and provides access methods for the key fields.
+ *
+ * <p>The intent of the design of this class was to isolate the detailed knowledge of the bit and
+ * byte layout of the serialized form of the sketches derived from the base sketch classes into one place.
+ * This allows the possibility of the introduction of different serialization
+ * schemes with minimal impact on the rest of the library.</p>
+ *
+ * <p>
+ * LAYOUT: The low significance bytes of this <i>long</i> based data structure are on the right.
+ * The multi-byte primitives are stored in native byte order.
+ * The single byte fields are treated as unsigned.</p>
+ *
+ * <p>An empty sketch requires only 8 bytes, which is only preamble.
+ * A serialized, non-empty KllDoublesSketch requires at least 16 bytes of preamble.
+ * A serialized, non-empty KllFloatsSketch requires at least 12 bytes of preamble.</p>
+ *
+ * <pre>{@code
+ * Serialized float sketch layout, more than one item:
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||---------------------------------N_LONG---------------------------------------|
+ *      ||         |       |        |   20   |   19   |    18   |   17   |      16      |
+ *  2   ||<-------Levels Arr Start----------]| unused |NumLevels|--Dynamic-Min K--------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-------Min/Max Arr Start---------]|[<----------Levels Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-----Float Items Arr Start-------]|[<---------Min/Max Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||         |       |        |        |[<-------Float Items Arr End--------------|
+ *
+ * Serialized float sketch layout, Empty (8 bytes) and Single Item (12 bytes):
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||                                   |-------------Single Item------------------|
+ *
+ *
+ *
+ * Serialized double sketch layout, more than one item:
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||---------------------------------N_LONG---------------------------------------|
+ *      ||   23    |   22  |   21   |   20   |   19   |    18   |   17   |      16      |
+ *  2   ||<-------Levels Arr Start----------]| unused |NumLevels|--Dynamic-Min K--------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-------Min/Max Arr Start---------]|[<----------Levels Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<----Double Items Arr Start-------]|[<---------Min/Max Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||         |       |        |        |[<------Double Items Arr End--------------|
+ *
+ * Serialized double sketch layout, Empty (8 bytes) and Single Item (16 bytes):
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||                                                               |      8       |
+ *  1   ||------------------------------Single Item-------------------------------------|
+ *
+ * The structure of the data block depends on Layout:
+ *
+ *   For FLOAT_SINGLE_COMPACT or DOUBLE_SINGLE_COMPACT:
+ *     The single data item is at offset DATA_START_ADR_SINGLE_ITEM = 8
+ *
+ *   For FLOAT_FULL_COMPACT:
+ *     The int[] levels array starts at offset DATA_START_ADR_FLOAT = 20 with a length of numLevels integers;
+ *     Followed by Float Min_Value, then Float Max_Value
+ *     Followed by an array of Floats of length retainedItems()
+ *
+ *   For DOUBLE_FULL_COMPACT
+ *     The int[] levels array starts at offset DATA_START_ADR_DOUBLE = 20 with a length of numLevels integers;
+ *     Followed by Double Min_Value, then Double Max_Value
+ *     Followed by an array of Doubles of length retainedItems()
+ *
+ *   For FLOAT_UPDATABLE
+ *     The int[] levels array starts at offset DATA_START_ADR_FLOAT = 20 with a length of (numLevels + 1) integers;
+ *     Followed by Float Min_Value, then Float Max_Value
+ *     Followed by an array of Floats of length KllHelper.computeTotalItemCapacity(...).
+ *
+ *   For DOUBLE_UPDATABLE
+ *     The int[] levels array starts at offset DATA_START_ADR_DOUBLE = 20 with a length of (numLevels + 1) integers;
+ *     Followed by Double Min_Value, then Double Max_Value
+ *     Followed by an array of Doubles of length KllHelper.computeTotalItemCapacity(...).
+ *
+ * }</pre>
+ *
+ *  @author Lee Rhodes
+ */
+final class KllPreambleUtil {
+
+  private KllPreambleUtil() {}
+
+  static final String LS = System.getProperty("line.separator");
+
+  /**
+   * The default value of K
+   */
+  public static final int DEFAULT_K = 200;
+  public static final int DEFAULT_M = 8;
+  static final int MAX_K = (1 << 16) - 1; // serialized as an unsigned short
+
+  // Preamble byte addresses
+  static final int PREAMBLE_INTS_BYTE_ADR     = 0;
+  static final int SER_VER_BYTE_ADR           = 1;
+  static final int FAMILY_BYTE_ADR            = 2;
+  static final int FLAGS_BYTE_ADR             = 3;
+  static final int K_SHORT_ADR                = 4;  // to 5
+  static final int M_BYTE_ADR                 = 6;
+  //                                            7 is reserved for future use
+  // SINGLE ITEM ONLY
+  static final int DATA_START_ADR_SINGLE_ITEM = 8;
+
+  // MULTI-ITEM
+  static final int N_LONG_ADR                 = 8;  // to 15
+  static final int DY_MIN_K_SHORT_ADR         = 16; // to 17
+  static final int NUM_LEVELS_BYTE_ADR        = 18;
+
+  // FLOAT SKETCH                               19 is reserved for future use in float sketch
+  static final int DATA_START_ADR_FLOAT       = 20; // float sketch, not single item
+
+  // DOUBLE SKETCH                              19 to 23 is reserved for future use in double sketch
+  static final int DATA_START_ADR_DOUBLE      = 20; // double sketch, not single item //TODO??

Review Comment:
   The comment in line 155 was obsolete.  I will fix.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r843232798


##########
src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java:
##########
@@ -0,0 +1,372 @@
+/*
+ * 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 org.apache.datasketches.Util.zeroPad;
+
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+//@formatter:off
+
+/**
+ * This class defines the serialized data structure and provides access methods for the key fields.
+ *
+ * <p>The intent of the design of this class was to isolate the detailed knowledge of the bit and
+ * byte layout of the serialized form of the sketches derived from the base sketch classes into one place.
+ * This allows the possibility of the introduction of different serialization
+ * schemes with minimal impact on the rest of the library.</p>
+ *

Review Comment:
   Panama or possibly others.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842171630


##########
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].
+ */

Review Comment:
   Yes, and to compute quantiles as well. 



-- 
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


[GitHub] [datasketches-java] davecromberge commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
davecromberge commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r850207627


##########
src/main/java/org/apache/datasketches/kll/KllDirectSketch.java:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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 org.apache.datasketches.kll.KllPreambleUtil.extractMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractM;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractN;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class implements all the methods for the Direct (off-heap) sketches that are independent
+ * of the sketch type (float or double).
+ */
+abstract class KllDirectSketch extends KllSketch {
+  final boolean updatable = true;
+  WritableMemory levelsArrUpdatable;
+  WritableMemory minMaxArrUpdatable;
+  WritableMemory itemsArrUpdatable;

Review Comment:
   This makes sense now, thanks.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849879363


##########
src/main/java/org/apache/datasketches/kll/KllDirectFloatsSketch.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.UPDATABLE_BIT_MASK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFlags;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+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.KllSketch.Error.MUST_NOT_CALL;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_DIRECT;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_FLOAT;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+//Intentional extra blank line so the code lines up with KllDirectDoublesSketch

Review Comment:
   Pretty much, yes.
   



-- 
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


[GitHub] [datasketches-java] leerho closed pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho closed pull request #390: Direct kll double
URL: https://github.com/apache/datasketches-java/pull/390


-- 
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


[GitHub] [datasketches-java] leerho commented on pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#issuecomment-1088166351

   @AlexanderSaydakov thanks!  Over the weekend it occured to me that the structure wasn't quite right and also, the current API doesn't provide for heapify of an updatable Memory format.  
   
   I will fix.


-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849863486


##########
src/main/java/org/apache/datasketches/kll/KllHeapSketch.java:
##########
@@ -0,0 +1,153 @@
+/*
+ * 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 org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class implements all the methods for the heap sketches that are independent
+ * of the sketch type (float or double).
+ *
+ * @author lrhodes
+ */
+abstract class KllHeapSketch extends KllSketch {
+  private final int k;    // configured value of K.
+  private final int m;    // configured value of M.
+  private long n_;        // number of items input into this sketch.
+  private int minK_;    // dynamic minK for error estimation after merging with different k.
+  private int numLevels_; // one-based number of current levels.
+  private int[] levels_;  // array of index offsets into the items[]. Size = numLevels + 1.
+  private boolean isLevelZeroSorted_;
+
+  /**
+   * Heap constructor.
+   * @param k user configured size of sketch. Range [m, 2^16]
+   * @param m user configured minimum level width
+   * @param sketchType either DOUBLE_SKETCH or FLOAT_SKETCH
+   */
+  KllHeapSketch(final int k, final int m, final SketchType sketchType) {
+    super(sketchType, null, null);
+    KllHelper.checkM(m);
+    KllHelper.checkK(k, m);
+    this.k = k;
+    this.m = m;
+    n_ = 0;
+    minK_ = k;
+    numLevels_ = 1;
+    levels_ = new int[] {k, k};
+    isLevelZeroSorted_ = false;
+  }
+
+  @Override
+  public int getK() {
+    return k;
+  }
+
+  @Override
+  int getM() {
+    return m;
+  }
+
+  @Override
+  public long getN() {
+    return n_;
+  }
+
+  @Override
+  int getMinK() {
+    return minK_;
+  }
+
+  @Override
+  int[] getLevelsArray() {
+    return levels_;
+  }
+
+  @Override
+  int getLevelsArrayAt(final int index) { return levels_[index]; }
+
+  @Override
+  int getNumLevels() {
+    return numLevels_;
+  }
+
+  @Override
+  void incN() {
+    n_++;
+  }
+
+  @Override
+  void incNumLevels() {
+    numLevels_++;
+  }
+
+  @Override
+  boolean isLevelZeroSorted() {
+    return isLevelZeroSorted_;
+  }
+
+  @Override
+  void setMinK(final int minK) {
+    minK_ = minK;
+  }
+
+  @Override
+  void setItemsArrayUpdatable(final WritableMemory itemsMem) { } //dummy
+
+  @Override
+  void setLevelsArray(final int[] levelsArr) {
+    levels_ = levelsArr;
+  }
+
+  @Override
+  void setLevelsArrayAt(final int index, final int value) { levels_[index] = value; }
+
+  @Override
+  void setLevelsArrayAtMinusEq(final int index, final int minusEq) {
+    levels_[index] -= minusEq;
+  }
+
+  @Override
+  void setLevelsArrayAtPlusEq(final int index, final int plusEq) {
+    levels_[index] += plusEq;
+  }
+
+  @Override
+  void setLevelsArrayUpdatable(final WritableMemory levelsMem) { } //dummy
+
+  @Override
+  void setLevelZeroSorted(final boolean sorted) {
+    this.isLevelZeroSorted_ = sorted;
+  }
+
+  @Override
+  void setMinMaxArrayUpdatable(final WritableMemory minMaxMem) { } //dummy
+
+  @Override
+  void setN(final long n) {
+    n_ = n;
+  }
+
+  @Override
+  void setNumLevels(final int numLevels) {
+    numLevels_ = numLevels;
+  }
+
+}

Review Comment:
   There is a noticeable performance cost to doing all update operations in a Memory object as opposed to the heap. 
   Also, writing the code to accomplish Direct, off-heap (within a Memory object) updating is much more difficult. As a result only a few of our sketches are capable of this Direct, off-heap updating. 



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849879201


##########
src/main/java/org/apache/datasketches/kll/KllDirectDoublesSketch.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.DOUBLES_SKETCH_BIT_MASK;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.UPDATABLE_BIT_MASK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFlags;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+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.KllSketch.Error.MUST_NOT_CALL;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_DIRECT;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_DOUBLE;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class implements an off-heap doubles KllSketch via a WritableMemory instance of the sketch.
+ *
+ * <p>Please refer to the documentation in the package-info:<br>
+ * {@link org.apache.datasketches.kll}</p>
+ *
+ * @author Lee Rhodes, Kevin Lang
+ */
+public final class KllDirectDoublesSketch extends KllDirectSketch {
+
+  /**
+   * The actual constructor.
+   * @param wmem the current WritableMemory
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @param memVal the MemoryValadate object
+   */

Review Comment:
   Thanks,



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849878947


##########
src/main/java/org/apache/datasketches/QuantilesHelper.java:
##########
@@ -43,15 +43,28 @@ public static long convertToPrecedingCummulative(final long[] array) {
   /**
    * Returns the linear zero-based index (position) of a value in the hypothetical sorted stream of
    * values of size n.
-   * @param phi the fractional position where: 0 &le; &#966; &le; 1.0.
+   * @param rank the fractional position where: 0 &le; &#966; &le; 1.0.
    * @param n the size of the stream
    * @return the index, a value between 0 and n-1.
-   */ //also used by KLL
-  public static long posOfPhi(final double phi, final long n) {
-    final long pos = (long) Math.floor(phi * n);
+   */ //used by classic Quantiles and KLL
+  public static long posOfRank(final double rank, final long n) {
+    final long pos = (long) Math.floor(rank * n);
     return pos == n ? n - 1 : pos; //avoids ArrayIndexOutOfBoundException
   }
 
+  /**
+   * Returns the linear zero-based index (position) of a value in the hypothetical sorted stream of
+   * values of size n.
+   * @param rank the fractional position where: 0 &le; &#966; &le; 1.0.
+   * @param n the size of the stream
+   * @return the index, a value between 0 and n-1.
+   * @deprecated use {@link #posOfRank(double, long)} instead. Version 3.2.0.
+   */ //used by classic Quantiles and KLL
+  @Deprecated
+  public static long posOfPhi(final double rank, final long n) {
+    return posOfRank(rank, n);
+  }

Review Comment:
   Yes, this, I think has been corrected.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842169359


##########
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.

Review Comment:
   Only within the scope of a particular level. 



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842166599


##########
src/test/java/org/apache/datasketches/kll/KllDirectDoublesSketchTest.java:
##########
@@ -0,0 +1,607 @@
+/*
+ * 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 org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertNotNull;
+import static org.testng.Assert.assertNull;
+import static org.testng.Assert.assertTrue;
+
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.memory.DefaultMemoryRequestServer;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
+
+@SuppressWarnings("javadoc")
+public class KllDirectDoublesSketchTest {
+
+  private static final double PMF_EPS_FOR_K_8 = 0.35; // PMF rank error (epsilon) for k=8
+  private static final double PMF_EPS_FOR_K_128 = 0.025; // PMF rank error (epsilon) for k=128
+  private static final double PMF_EPS_FOR_K_256 = 0.013; // PMF rank error (epsilon) for k=256
+  private static final double NUMERIC_NOISE_TOLERANCE = 1E-6;
+  private static final DefaultMemoryRequestServer memReqSvr = new DefaultMemoryRequestServer();
+
+  @Test
+  public void empty() {
+    final KllDirectDoublesSketch sketch = getDDSketch(200, 0);
+    sketch.update(Double.NaN); // this must not change anything
+    assertTrue(sketch.isEmpty());
+    assertEquals(sketch.getN(), 0);
+    assertEquals(sketch.getNumRetained(), 0);
+    assertTrue(Double.isNaN(sketch.getRank(0)));
+    assertTrue(Double.isNaN(sketch.getMinValue()));
+    assertTrue(Double.isNaN(sketch.getMaxValue()));
+    assertTrue(Double.isNaN(sketch.getQuantile(0.5)));
+    assertNull(sketch.getQuantiles(new double[] {0}));
+    assertNull(sketch.getPMF(new double[] {0}));
+    assertNotNull(sketch.toString(true, true));
+    assertNotNull(sketch.toString());
+  }
+
+  @Test(expectedExceptions = SketchesArgumentException.class)
+  public void getQuantileInvalidArg() {
+    final KllDirectDoublesSketch sketch = getDDSketch(200, 0);
+    sketch.update(1);
+    sketch.getQuantile(-1.0);
+  }
+
+  @Test(expectedExceptions = SketchesArgumentException.class)
+  public void getQuantilesInvalidArg() {
+    final KllDirectDoublesSketch sketch = getDDSketch(200, 0);
+    sketch.update(1);
+    sketch.getQuantiles(new double[] {2.0});
+  }
+
+  @Test
+  public void oneItem() {
+    final KllDirectDoublesSketch sketch = getDDSketch(200, 0);
+    sketch.update(1);
+    assertFalse(sketch.isEmpty());
+    assertEquals(sketch.getN(), 1);
+    assertEquals(sketch.getNumRetained(), 1);
+    assertEquals(sketch.getRank(1), 0.0);
+    assertEquals(sketch.getRank(2), 1.0);
+    assertEquals(sketch.getMinValue(), 1.0);
+    assertEquals(sketch.getMaxValue(), 1.0);
+    assertEquals(sketch.getQuantile(0.5), 1.0);
+  }
+
+  @Test
+  public void manyItemsEstimationMode() {
+    final KllDirectDoublesSketch sketch = getDDSketch(200, 0);
+    final int n = 1_000_000;
+
+    for (int i = 0; i < n; i++) {
+      sketch.update(i);
+    }
+    assertEquals(sketch.getN(), n);
+
+    // test getRank
+    for (int i = 0; i < n; i++) {
+      final double trueRank = (double) i / n;
+      assertEquals(sketch.getRank(i), trueRank, PMF_EPS_FOR_K_256, "for value " + i);
+    }
+
+    // test getPMF
+    final double[] pmf = sketch.getPMF(new double[] {n / 2}); // split at median

Review Comment:
   fixed this and for KllDirectFloatsSketch too.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842140481


##########
src/main/java/org/apache/datasketches/kll/KllDirectSketch.java:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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 org.apache.datasketches.kll.KllPreambleUtil.extractMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractM;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractN;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class implements all the methods for the Direct (off-heap) sketches that are independent
+ * of the sketch type (float or double).
+ */
+abstract class KllDirectSketch extends KllSketch {
+  final boolean updatable = true;
+  WritableMemory levelsArrUpdatable;
+  WritableMemory minMaxArrUpdatable;
+  WritableMemory itemsArrUpdatable;
+
+  /**
+   * For the direct sketches it is important that the methods implemented here are designed to work dynamically
+   * as the sketch grows off-heap.
+   * @param sketchType either DOUBLE_SKETCH or FLOAT_SKETCH
+   * @param wmem the current WritableMemory
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   */

Review Comment:
   Yes, exactly.  There is no local storage.  So all the preamble fields and the data arrays must be always reference  the off-heap Memory segment. As the sketch grows the off-heap offsets of the base of the various arrays also change, thus dynamic.  
   
   In the heap environment, for example, I can have a separate array of ints for the levels Array and a separate array for the float items data, which makes referencing easy.  
   
   But in the off-heap environment, I have one contiguous array of bytes where the float items sit right on top of the integer levels array.  Thus if the levels array grows by 4 bytes (when adding a level), the start of the float items array is pushed up by 4 bytes.  So all the offsets into the items array must be recomputed.  
   
   To simplify this I use multiple Memory Regions, which are just views of the actual off-heap segment.  When the sketch grows, I just update the base address of the regions. When I write to an offset in a region, it correctly updates the bytes in the remote off-heap segment as if it were a local float array.  
   



-- 
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


[GitHub] [datasketches-java] davecromberge commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
davecromberge commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r844890777


##########
src/main/java/org/apache/datasketches/kll/KllDirectSketch.java:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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 org.apache.datasketches.kll.KllPreambleUtil.extractMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractM;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractN;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class implements all the methods for the Direct (off-heap) sketches that are independent
+ * of the sketch type (float or double).
+ */
+abstract class KllDirectSketch extends KllSketch {
+  final boolean updatable = true;
+  WritableMemory levelsArrUpdatable;
+  WritableMemory minMaxArrUpdatable;
+  WritableMemory itemsArrUpdatable;
+
+  /**
+   * For the direct sketches it is important that the methods implemented here are designed to work dynamically
+   * as the sketch grows off-heap.
+   * @param sketchType either DOUBLE_SKETCH or FLOAT_SKETCH
+   * @param wmem the current WritableMemory
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   */

Review Comment:
   Thanks for clarifying, makes sense.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849879637


##########
src/main/java/org/apache/datasketches/kll/KllDirectDoublesSketch.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.DOUBLES_SKETCH_BIT_MASK;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_DOUBLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.UPDATABLE_BIT_MASK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFlags;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+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.KllSketch.Error.MUST_NOT_CALL;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_DIRECT;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_DOUBLE;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class implements an off-heap doubles KllSketch via a WritableMemory instance of the sketch.
+ *
+ * <p>Please refer to the documentation in the package-info:<br>
+ * {@link org.apache.datasketches.kll}</p>
+ *
+ * @author Lee Rhodes, Kevin Lang
+ */
+public final class KllDirectDoublesSketch extends KllDirectSketch {
+
+  /**
+   * The actual constructor.
+   * @param wmem the current WritableMemory
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @param memVal the MemoryValadate object
+   */
+  private KllDirectDoublesSketch(final WritableMemory wmem, final MemoryRequestServer memReqSvr,
+      final KllMemoryValidate memVal) {
+    super(SketchType.DOUBLES_SKETCH, wmem, memReqSvr, memVal);
+  }
+
+  /**
+   * Wrap a sketch around the given source Memory containing sketch data that originated from
+   * this sketch.
+   * @param srcMem a WritableMemory that contains data.
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @return instance of this sketch
+   */
+  public static KllDirectDoublesSketch writableWrap(final WritableMemory srcMem, final MemoryRequestServer memReqSvr) {
+    final KllMemoryValidate memVal = new KllMemoryValidate(srcMem);
+    return new KllDirectDoublesSketch(srcMem, memReqSvr, memVal);
+  }
+
+  /**
+   * Create a new instance of this sketch using the default <i>m</i> of 8.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param dstMem the given destination WritableMemory object for use by the sketch
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @return a new instance of this sketch
+   */
+  public static KllDirectDoublesSketch newInstance(final int k, final WritableMemory dstMem,
+      final MemoryRequestServer memReqSvr) {
+    return newInstance(k, KllSketch.DEFAULT_M, dstMem, memReqSvr);
+  }
+
+  /**
+   * Create a new instance of this sketch.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param m parameter that controls the minimum level width.
+   * @param dstMem the given destination WritableMemory object for use by the sketch
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @return a new instance of this sketch
+   */
+  static KllDirectDoublesSketch newInstance(final int k, final int m, final WritableMemory dstMem,
+      final MemoryRequestServer memReqSvr) {
+    insertPreInts(dstMem, PREAMBLE_INTS_DOUBLE);
+    insertSerVer(dstMem, SERIAL_VERSION_UPDATABLE);
+    insertFamilyID(dstMem, Family.KLL.getID());
+    insertFlags(dstMem, DOUBLES_SKETCH_BIT_MASK | UPDATABLE_BIT_MASK);
+    insertK(dstMem, k);
+    insertM(dstMem, m);
+    insertN(dstMem, 0);
+    insertMinK(dstMem, k);
+    insertNumLevels(dstMem, 1);
+    int offset = DATA_START_ADR_DOUBLE;
+    dstMem.putIntArray(offset, new int[] {k, k}, 0, 2);
+    offset += 2 * Integer.BYTES;
+    dstMem.putDoubleArray(offset, new double[] {Double.NaN, Double.NaN}, 0, 2);
+    offset += 2 * Double.BYTES;
+    dstMem.putDoubleArray(offset, new double[k], 0, k);
+    final KllMemoryValidate memVal = new KllMemoryValidate(dstMem);
+    return new KllDirectDoublesSketch(dstMem, memReqSvr, memVal);
+  }
+
+  /**
+   * Returns an approximation to the Cumulative Distribution Function (CDF), which is the
+   * cumulative analog of the PMF, of the input stream given a set of splitPoint (values).
+   *
+   * <p>The resulting approximations have a probabilistic guarantee that can be obtained from the
+   * getNormalizedRankError(false) function.
+   *
+   * <p>If the sketch is empty this returns null.</p>
+   *
+   * @param splitPoints an array of <i>m</i> unique, monotonically increasing double values
+   * that divide the real number line into <i>m+1</i> consecutive disjoint intervals.
+   * The definition of an "interval" is inclusive of the left splitPoint (or minimum value) and
+   * exclusive of the right splitPoint, with the exception that the last interval will include
+   * the maximum value.
+   * It is not necessary to include either the min or max values in these split points.
+   *
+   * @return an array of m+1 double values on the interval [0.0, 1.0),
+   * which are a consecutive approximation to the CDF of the input stream given the splitPoints.
+   * The value at array position j of the returned CDF array is the sum of the returned values
+   * in positions 0 through j of the returned PMF array.
+   */
+  public double[] getCDF(final double[] splitPoints) {
+    return getDoublesPmfOrCdf(splitPoints, true);
+  }
+
+  /**
+   * Returns the max value of the stream.
+   * If the sketch is empty this returns NaN.
+   *
+   * @return the max value of the stream
+   */
+  public double getMaxValue() {
+    return getMaxDoubleValue();
+  }
+
+  /**
+   * Returns the min value of the stream.
+   * If the sketch is empty this returns NaN.
+   *
+   * @return the min value of the stream
+   */
+  public double getMinValue() {
+    return getMinDoubleValue();
+  }
+
+  /**
+   * Returns an approximation to the Probability Mass Function (PMF) of the input stream
+   * given a set of splitPoints (values).
+   *
+   * <p>The resulting approximations have a probabilistic guarantee that can be obtained from the
+   * getNormalizedRankError(true) function.
+   *
+   * <p>If the sketch is empty this returns null.</p>
+   *
+   * @param splitPoints an array of <i>m</i> unique, monotonically increasing double values
+   * that divide the real number line into <i>m+1</i> consecutive disjoint intervals.
+   * The definition of an "interval" is inclusive of the left splitPoint (or minimum value) and
+   * exclusive of the right splitPoint, with the exception that the last interval will include
+   * the maximum value.
+   * It is not necessary to include either the min or max values in these split points.
+   *
+   * @return an array of m+1 doubles on the interval [0.0, 1.0),
+   * each of which is an approximation to the fraction of the total input stream values
+   * (the mass) that fall into one of those intervals.
+   * The definition of an "interval" is inclusive of the left splitPoint and exclusive of the right
+   * splitPoint, with the exception that the last interval will include maximum value.
+   */
+  public double[] getPMF(final double[] splitPoints) {
+    return getDoublesPmfOrCdf(splitPoints, false);
+  }
+
+  /**
+   * Returns an approximation to the value of the data item
+   * that would be preceded by the given fraction of a hypothetical sorted
+   * version of the input stream so far.
+   *
+   * <p>We note that this method has a fairly large overhead (microseconds instead of nanoseconds)
+   * so it should not be called multiple times to get different quantiles from the same
+   * sketch. Instead use getQuantiles(), which pays the overhead only once.
+   *
+   * <p>If the sketch is empty this returns NaN.
+   *
+   * @param fraction the specified fractional position in the hypothetical sorted stream.
+   * These are also called normalized ranks or fractional ranks.
+   * If fraction = 0.0, the true minimum value of the stream is returned.
+   * If fraction = 1.0, the true maximum value of the stream is returned.
+   *
+   * @return the approximation to the value at the given fraction
+   */
+  public double getQuantile(final double fraction) {
+    return getDoublesQuantile(fraction);
+  }
+
+  /**
+   * Gets the lower bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%.
+   * @param fraction the given normalized rank as a fraction
+   * @return the lower bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%. Returns NaN if the sketch is empty.
+   */
+  public double getQuantileLowerBound(final double fraction) {
+    return getQuantile(max(0, fraction - KllHelper.getNormalizedRankError(getMinK(), false)));
+  }
+
+  /**
+   * This is a more efficient multiple-query version of getQuantile().
+   *
+   * <p>This returns an array that could have been generated by using getQuantile() with many
+   * different fractional ranks, but would be very inefficient.
+   * This method incurs the internal set-up overhead once and obtains multiple quantile values in
+   * a single query. It is strongly recommend that this method be used instead of multiple calls
+   * to getQuantile().
+   *
+   * <p>If the sketch is empty this returns null.
+   *
+   * @param fractions given array of fractional positions in the hypothetical sorted stream.
+   * These are also called normalized ranks or fractional ranks.
+   * These fractions must be in the interval [0.0, 1.0], inclusive.
+   *
+   * @return array of approximations to the given fractions in the same order as given fractions
+   * array.
+   */
+  public double[] getQuantiles(final double[] fractions) {
+    return getDoublesQuantiles(fractions);
+  }
+
+  /**
+   * This is also a more efficient multiple-query version of getQuantile() and allows the caller to
+   * specify the number of evenly spaced fractional ranks.
+   *
+   * <p>If the sketch is empty this returns null.
+   *
+   * @param numEvenlySpaced an integer that specifies the number of evenly spaced fractional ranks.
+   * This must be a positive integer greater than 0. A value of 1 will return the min value.
+   * A value of 2 will return the min and the max value. A value of 3 will return the min,
+   * the median and the max value, etc.
+   *
+   * @return array of approximations to the given fractions in the same order as given fractions
+   * array.
+   */
+  public double[] getQuantiles(final int numEvenlySpaced) {
+    if (isEmpty()) { return null; }
+    return getQuantiles(org.apache.datasketches.Util.evenlySpaced(0.0, 1.0, numEvenlySpaced));
+  }
+
+  /**
+   * Gets the upper bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%.
+   * @param fraction the given normalized rank as a fraction
+   * @return the upper bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%. Returns NaN if the sketch is empty.
+   */
+  public double getQuantileUpperBound(final double fraction) {
+    return getQuantile(min(1.0, fraction + KllHelper.getNormalizedRankError(getMinK(), false)));
+  }
+
+  /**
+   * Returns an approximation to the normalized (fractional) rank of the given value from 0 to 1,
+   * inclusive.
+   *
+   * <p>The resulting approximation has a probabilistic guarantee that can be obtained from the
+   * getNormalizedRankError(false) function.
+   *
+   * <p>If the sketch is empty this returns NaN.</p>
+   *
+   * @param value to be ranked
+   * @return an approximate rank of the given value
+   */
+  public double getRank(final double value) {
+    return getDoubleRank(value);
+  }
+
+  /**
+   * @return the iterator for this class
+   */
+  public KllDoublesSketchIterator iterator() {
+    return new KllDoublesSketchIterator(getDoubleItemsArray(), getLevelsArray(), getNumLevels());
+  }
+
+  /**
+   * Merges another sketch into this one.
+   * @param other sketch to merge into this one
+   */
+  public void merge(final KllSketch other) {
+    if (!other.isDirect()) { kllSketchThrow(SRC_IS_NOT_DIRECT); }
+    if (!other.isDoublesSketch()) { kllSketchThrow(SRC_IS_NOT_DOUBLE); }
+    mergeDoubleImpl(other);
+  }
+
+  /**
+   * Updates this sketch with the given data item.
+   *
+   * @param value an item from a stream of items. NaNs are ignored.
+   */
+  public void update(final double value) {
+    updateDouble(value);
+  }
+
+  @Override
+  double[] getDoubleItemsArray() {
+    final int items = getItemsArrLengthItems();
+    final double[] itemsArr = new double[items];
+    itemsArrUpdatable.getDoubleArray(0, itemsArr, 0, items);
+    return itemsArr;
+  }
+
+  @Override
+  double getDoubleItemsArrayAt(final int index) {
+    return itemsArrUpdatable.getDouble((long)index * Double.BYTES);
+  }
+
+  @Override
+  float[] getFloatItemsArray() { kllSketchThrow(MUST_NOT_CALL); return null; }
+
+  @Override
+  float getFloatItemsArrayAt(final int index) { kllSketchThrow(MUST_NOT_CALL); return Float.NaN; }
+
+  @Override
+  double getMaxDoubleValue() {
+    return minMaxArrUpdatable.getDouble(Double.BYTES);
+  }
+
+  @Override
+  float getMaxFloatValue() { kllSketchThrow(MUST_NOT_CALL); return Float.NaN; }
+
+  @Override
+  double getMinDoubleValue() {
+    return minMaxArrUpdatable.getDouble(0);
+  }
+
+  @Override
+  float getMinFloatValue() { kllSketchThrow(MUST_NOT_CALL); return Float.NaN; }
+
+  @Override
+  void setDoubleItemsArray(final double[] doubleItems) {
+    if (!updatable) { kllSketchThrow(TGT_IS_IMMUTABLE); }
+    itemsArrUpdatable.putDoubleArray(0, doubleItems, 0, doubleItems.length);
+  }
+
+  @Override
+  void setDoubleItemsArrayAt(final int index, final double value) {
+    itemsArrUpdatable.putDouble((long)index * Double.BYTES, value);
+  }

Review Comment:
   this has been rewritten.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842248605


##########
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:
   Fixed this one and a similar one in KllMemoryValidate.  Thanks, I shoulda thought of that :)



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842162311


##########
src/main/java/org/apache/datasketches/kll/KllDirectFloatsSketch.java:
##########
@@ -0,0 +1,389 @@
+/*
+ * 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.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.kll.KllPreambleUtil.DATA_START_ADR_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.PREAMBLE_INTS_FLOAT;
+import static org.apache.datasketches.kll.KllPreambleUtil.SERIAL_VERSION_UPDATABLE;
+import static org.apache.datasketches.kll.KllPreambleUtil.UPDATABLE_BIT_MASK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFamilyID;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertFlags;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertK;
+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.KllSketch.Error.MUST_NOT_CALL;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_DIRECT;
+import static org.apache.datasketches.kll.KllSketch.Error.SRC_IS_NOT_FLOAT;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+//Intentional extra blank line so the code lines up with KllDirectDoublesSketch
+/**
+ * This class implements an off-heap floats KllSketch via a WritableMemory instance of the sketch.
+ *
+ * <p>Please refer to the documentation in the package-info:<br>
+ * {@link org.apache.datasketches.kll}</p>
+ *
+ * @author Lee Rhodes, Kevin Lang
+ */
+public final class KllDirectFloatsSketch extends KllDirectSketch {
+
+  /**
+   * The actual constructor
+   * @param wmem the current WritableMemory
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @param memVal the MemoryValadate object
+   */
+  private KllDirectFloatsSketch(final WritableMemory wmem, final MemoryRequestServer memReqSvr,
+      final KllMemoryValidate memVal) {
+   super(SketchType.FLOATS_SKETCH, wmem, memReqSvr, memVal);
+  }
+
+  /**
+   * Wrap a sketch around the given source Memory containing sketch data that originated from
+   * this sketch.
+   * @param srcMem a WritableMemory that contains data.
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @return instance of this sketch
+   */
+  public static KllDirectFloatsSketch writableWrap(final WritableMemory srcMem, final MemoryRequestServer memReqSvr) {
+    final KllMemoryValidate memVal = new KllMemoryValidate(srcMem);
+    return new KllDirectFloatsSketch(srcMem, memReqSvr, memVal);
+  }
+
+  /**
+   * Create a new instance of this sketch using the default <i>m</i> of 8.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param dstMem the given destination WritableMemory object for use by the sketch
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @return a new instance of this sketch
+   */
+  public static KllDirectFloatsSketch newInstance(final int k, final WritableMemory dstMem,
+      final MemoryRequestServer memReqSvr) {
+    return newInstance(k, KllSketch.DEFAULT_M, dstMem, memReqSvr);
+  }
+
+  /**
+   * Create a new instance of this sketch.
+   * @param k parameter that controls size of the sketch and accuracy of estimates
+   * @param m parameter that controls the minimum level width.
+   * @param dstMem the given destination WritableMemory object for use by the sketch
+   * @param memReqSvr the given MemoryRequestServer to request a larger WritableMemory
+   * @return a new instance of this sketch
+   */
+  static KllDirectFloatsSketch newInstance(final int k, final int m, final WritableMemory dstMem,
+      final MemoryRequestServer memReqSvr) {
+    insertPreInts(dstMem, PREAMBLE_INTS_FLOAT);
+    insertSerVer(dstMem, SERIAL_VERSION_UPDATABLE);
+    insertFamilyID(dstMem, Family.KLL.getID());
+    insertFlags(dstMem, UPDATABLE_BIT_MASK);
+    insertK(dstMem, k);
+    insertM(dstMem, m);
+    insertN(dstMem, 0);
+    insertMinK(dstMem, k);
+    insertNumLevels(dstMem, 1);
+    int offset = DATA_START_ADR_FLOAT;
+    dstMem.putIntArray(offset, new int[] {k, k}, 0, 2);
+    offset += 2 * Integer.BYTES;
+    dstMem.putFloatArray(offset, new float[] {Float.NaN, Float.NaN}, 0, 2);
+    offset += 2 * Float.BYTES;
+    dstMem.putFloatArray(offset, new float[k], 0, k);
+    final KllMemoryValidate memVal = new KllMemoryValidate(dstMem);
+    return new KllDirectFloatsSketch(dstMem, memReqSvr, memVal);
+  }
+
+  /**
+   * Returns an approximation to the Cumulative Distribution Function (CDF), which is the
+   * cumulative analog of the PMF, of the input stream given a set of splitPoint (values).
+   *
+   * <p>The resulting approximations have a probabilistic guarantee that can be obtained from the
+   * getNormalizedRankError(false) function.
+   *
+   * <p>If the sketch is empty this returns null.</p>
+   *
+   * @param splitPoints an array of <i>m</i> unique, monotonically increasing float values
+   * that divide the real number line into <i>m+1</i> consecutive disjoint intervals.
+   * The definition of an "interval" is inclusive of the left splitPoint (or minimum value) and
+   * exclusive of the right splitPoint, with the exception that the last interval will include
+   * the maximum value.
+   * It is not necessary to include either the min or max values in these split points.
+   *
+   * @return an array of m+1 double values on the interval [0.0, 1.0),
+   * which are a consecutive approximation to the CDF of the input stream given the splitPoints.
+   * The value at array position j of the returned CDF array is the sum of the returned values
+   * in positions 0 through j of the returned PMF array.
+   */
+  public double[] getCDF(final float[] splitPoints) {
+    return getFloatsPmfOrCdf(splitPoints, true);
+  }
+
+  /**
+   * Returns the max value of the stream.
+   * If the sketch is empty this returns NaN.
+   *
+   * @return the max value of the stream
+   */
+  public float getMaxValue() {
+    return getMaxFloatValue();
+  }
+
+  /**
+   * Returns the min value of the stream.
+   * If the sketch is empty this returns NaN.
+   *
+   * @return the min value of the stream
+   */
+  public float getMinValue() {
+    return getMinFloatValue();
+  }
+
+  /**
+   * Returns an approximation to the Probability Mass Function (PMF) of the input stream
+   * given a set of splitPoints (values).
+   *
+   * <p>The resulting approximations have a probabilistic guarantee that can be obtained from the
+   * getNormalizedRankError(true) function.
+   *
+   * <p>If the sketch is empty this returns null.</p>
+   *
+   * @param splitPoints an array of <i>m</i> unique, monotonically increasing float values
+   * that divide the real number line into <i>m+1</i> consecutive disjoint intervals.
+   * The definition of an "interval" is inclusive of the left splitPoint (or minimum value) and
+   * exclusive of the right splitPoint, with the exception that the last interval will include
+   * the maximum value.
+   * It is not necessary to include either the min or max values in these split points.
+   *
+   * @return an array of m+1 doubles on the interval [0.0, 1.0),
+   * each of which is an approximation to the fraction of the total input stream values
+   * (the mass) that fall into one of those intervals.
+   * The definition of an "interval" is inclusive of the left splitPoint and exclusive of the right
+   * splitPoint, with the exception that the last interval will include maximum value.
+   */
+  public double[] getPMF(final float[] splitPoints) {
+    return getFloatsPmfOrCdf(splitPoints, false);
+  }
+
+  /**
+   * Returns an approximation to the value of the data item
+   * that would be preceded by the given fraction of a hypothetical sorted
+   * version of the input stream so far.
+   *
+   * <p>We note that this method has a fairly large overhead (microseconds instead of nanoseconds)
+   * so it should not be called multiple times to get different quantiles from the same
+   * sketch. Instead use getQuantiles(), which pays the overhead only once.
+   *
+   * <p>If the sketch is empty this returns NaN.
+   *
+   * @param fraction the specified fractional position in the hypothetical sorted stream.
+   * These are also called normalized ranks or fractional ranks.
+   * If fraction = 0.0, the true minimum value of the stream is returned.
+   * If fraction = 1.0, the true maximum value of the stream is returned.
+   *
+   * @return the approximation to the value at the given fraction
+   */
+  public float getQuantile(final double fraction) {
+    return getFloatsQuantile(fraction);
+  }
+
+  /**
+   * Gets the lower bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%.
+   * @param fraction the given normalized rank as a fraction
+   * @return the lower bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%. Returns NaN if the sketch is empty.
+   */
+  public float getQuantileLowerBound(final double fraction) {
+    return getQuantile(max(0, fraction - KllHelper.getNormalizedRankError(getMinK(), false)));
+  }
+
+  /**
+   * This is a more efficient multiple-query version of getQuantile().
+   *
+   * <p>This returns an array that could have been generated by using getQuantile() with many
+   * different fractional ranks, but would be very inefficient.
+   * This method incurs the internal set-up overhead once and obtains multiple quantile values in
+   * a single query. It is strongly recommend that this method be used instead of multiple calls
+   * to getQuantile().
+   *
+   * <p>If the sketch is empty this returns null.
+   *
+   * @param fractions given array of fractional positions in the hypothetical sorted stream.
+   * These are also called normalized ranks or fractional ranks.
+   * These fractions must be in the interval [0.0, 1.0], inclusive.
+   *
+   * @return array of approximations to the given fractions in the same order as given fractions
+   * array.
+   */
+  public float[] getQuantiles(final double[] fractions) {
+    return getFloatsQuantiles(fractions);
+  }
+
+  /**
+   * This is also a more efficient multiple-query version of getQuantile() and allows the caller to
+   * specify the number of evenly spaced fractional ranks.
+   *
+   * <p>If the sketch is empty this returns null.
+   *
+   * @param numEvenlySpaced an integer that specifies the number of evenly spaced fractional ranks.
+   * This must be a positive integer greater than 0. A value of 1 will return the min value.
+   * A value of 2 will return the min and the max value. A value of 3 will return the min,
+   * the median and the max value, etc.
+   *
+   * @return array of approximations to the given fractions in the same order as given fractions
+   * array.
+   */
+  public float[] getQuantiles(final int numEvenlySpaced) {
+    if (isEmpty()) { return null; }
+    return getQuantiles(org.apache.datasketches.Util.evenlySpaced(0.0, 1.0, numEvenlySpaced));
+  }
+
+  /**
+   * Gets the upper bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%.
+   * @param fraction the given normalized rank as a fraction
+   * @return the upper bound of the value interval in which the true quantile of the given rank
+   * exists with a confidence of at least 99%. Returns NaN if the sketch is empty.
+   */
+  public float getQuantileUpperBound(final double fraction) {
+    return getQuantile(min(1.0, fraction + KllHelper.getNormalizedRankError(getMinK(), false)));
+  }
+
+  /**
+   * Returns an approximation to the normalized (fractional) rank of the given value from 0 to 1,
+   * inclusive.
+   *
+   * <p>The resulting approximation has a probabilistic guarantee that can be obtained from the
+   * getNormalizedRankError(false) function.
+   *
+   * <p>If the sketch is empty this returns NaN.</p>
+   *
+   * @param value to be ranked
+   * @return an approximate rank of the given value
+   */
+  public double getRank(final float value) {
+    return getFloatRank(value);
+  }
+
+  /**
+   * @return the iterator for this class
+   */
+  public KllFloatsSketchIterator iterator() {
+    return new KllFloatsSketchIterator(getFloatItemsArray(), getLevelsArray(), getNumLevels());
+  }
+
+  /**
+   * Merges another sketch into this one.
+   * @param other sketch to merge into this one
+   */
+  public void merge(final KllSketch other) {
+    if (!other.isDirect()) { kllSketchThrow(SRC_IS_NOT_DIRECT); }
+    if (!other.isFloatsSketch()) { kllSketchThrow(SRC_IS_NOT_FLOAT); }
+    mergeFloatImpl(other);
+  }
+
+  /**
+   * Updates this sketch with the given data item.
+   *
+   * @param value an item from a stream of items. NaNs are ignored.
+   */
+  public void update(final float value) {
+    updateFloat(value);
+  }
+
+  @Override
+  double[] getDoubleItemsArray() { kllSketchThrow(MUST_NOT_CALL); return null; }
+
+  @Override
+  double getDoubleItemsArrayAt(final int index) { kllSketchThrow(MUST_NOT_CALL); return Double.NaN; }
+
+  @Override
+  float[] getFloatItemsArray() {
+    final int items = getItemsArrLengthItems();
+    final float[] itemsArr = new float[items];
+    itemsArrUpdatable.getFloatArray(0, itemsArr, 0, items);
+    return itemsArr;
+  }
+
+  @Override
+  float getFloatItemsArrayAt(final int index) {
+    return itemsArrUpdatable.getFloat((long)index * Float.BYTES);
+  }
+
+  @Override
+  double getMaxDoubleValue() { kllSketchThrow(MUST_NOT_CALL); return Double.NaN; }
+
+  @Override
+  float getMaxFloatValue() {
+    return minMaxArrUpdatable.getFloat(Float.BYTES);
+  }
+
+  @Override
+  double getMinDoubleValue() { kllSketchThrow(MUST_NOT_CALL); return Double.NaN; }
+
+  @Override
+  float getMinFloatValue() {
+    return minMaxArrUpdatable.getFloat(0);
+  }
+
+  @Override
+  void setDoubleItemsArray(final double[] doubleItems) { kllSketchThrow(MUST_NOT_CALL); }
+
+  @Override
+  void setDoubleItemsArrayAt(final int index, final double value) { kllSketchThrow(MUST_NOT_CALL); }
+
+  @Override
+  void setFloatItemsArray(final float[] floatItems) {
+    if (!updatable) { kllSketchThrow(TGT_IS_IMMUTABLE); }
+    itemsArrUpdatable.putFloatArray(0, floatItems, 0, floatItems.length);
+  }
+
+  @Override
+  void setFloatItemsArrayAt(final int index, final float value) {
+    itemsArrUpdatable.putFloat((long)index * Float.BYTES, value);

Review Comment:
   Thanks! Fixed this and the one for KllDirectDoublesSketch too.



-- 
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


[GitHub] [datasketches-java] leerho commented on pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#issuecomment-1090593350

   I have added "isSameResource(Memory)" to this PR.  This will probably be the last push to this PR, which will not be merged.  All of the issues, except for Alex's issue of the structure and lacking polymorphic utility, have been resolved, I think.
   
   Nonetheless, addressing Alex's issue will involve massive restructuring so I will open a new DirectKllDoubleV2 PR based on this one, so that I can refer back to this one if necessary. 


-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849860491


##########
src/main/java/org/apache/datasketches/kll/KllDirectSketch.java:
##########
@@ -0,0 +1,195 @@
+/*
+ * 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 org.apache.datasketches.kll.KllPreambleUtil.extractMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractK;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractM;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractN;
+import static org.apache.datasketches.kll.KllPreambleUtil.extractNumLevels;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertMinK;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertLevelZeroSortedFlag;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertN;
+import static org.apache.datasketches.kll.KllPreambleUtil.insertNumLevels;
+import static org.apache.datasketches.kll.KllSketch.Error.TGT_IS_IMMUTABLE;
+
+import org.apache.datasketches.memory.MemoryRequestServer;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * This class implements all the methods for the Direct (off-heap) sketches that are independent
+ * of the sketch type (float or double).
+ */
+abstract class KllDirectSketch extends KllSketch {
+  final boolean updatable = true;
+  WritableMemory levelsArrUpdatable;
+  WritableMemory minMaxArrUpdatable;
+  WritableMemory itemsArrUpdatable;

Review Comment:
   The "updatable" is a modifier that refers to the Memory storage format, which is new with this release.  The previous Memory storage format, which I am now referring to as "compact" is not suitable for direct updates into the Memory object.  On-heap sketches are always updatable so the the modifier has no meaning.   



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849875413


##########
src/main/java/org/apache/datasketches/kll/package-info.java:
##########
@@ -35,16 +35,17 @@
  *
  * <p>The <i>normalized rank</i> (<i>rank</i>) of any specific value is defined as its
  * <i>absolute rank</i> divided by <i>N</i>.
- * Thus, the <i>normalized rank</i> is a value between zero and one.
+ * Thus, the <i>normalized rank</i> is a value in the interval [0.0, 1.0).
  * In the documentation and Javadocs for this sketch <i>absolute rank</i> is never used so any
  * reference to just <i>rank</i> should be interpreted to mean <i>normalized rank</i>.
  *
  * <p>This sketch is configured with a parameter <i>k</i>, which affects the size of the sketch
  * and its estimation error.
  *
- * <p>The estimation error is commonly called <i>epsilon</i> (or <i>eps</i>) and is a fraction
- * between zero and one. Larger values of <i>k</i> result in smaller values of epsilon.
- * Epsilon is always with respect to the rank and cannot be applied to the
+ * <p>In the research literature, the estimation error is commonly called <i>epsilon</i>
+ * (or <i>eps</i>) and is a fraction between zero and one.
+ * Larger values of <i>k</i> result in smaller values of epsilon.
+ * The epsilon error is always with respect to the rank and cannot be applied to the

Review Comment:
   This is a complex question and deals with fundamental issues of the KLL algorithm and our implementation.  In short, the highest level has the most impact on error, and less so with each lower level.  This is the reason that we implement a "lazy" compression scheme to minimize the need to create higher levels, and the reason that each of the levels can vary in size to also avoid having to create higher number of levels.  



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r849868777


##########
src/main/java/org/apache/datasketches/kll/KllHelper.java:
##########
@@ -19,32 +19,189 @@
 
 package org.apache.datasketches.kll;
 
+import static java.lang.Math.pow;
 import static org.apache.datasketches.Util.floorPowerOf2;
+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.KllSketch.CDF_COEF;
+import static org.apache.datasketches.kll.KllSketch.CDF_EXP;
+import static org.apache.datasketches.kll.KllSketch.PMF_COEF;
+import static org.apache.datasketches.kll.KllSketch.PMF_EXP;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
 
-class KllHelper {
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.kll.KllSketch.SketchType;
+
+/**
+ * This class provides some useful sketch analysis tools that are used internally and also can be used by
+ * interested users to understand the internal structure of the sketch as well as the growth properties of the
+ * sketch given a stream length.
+ *
+ * @author lrhodes
+ *
+ */
+public class KllHelper {
+
+  public static class GrowthStats {
+    SketchType sketchType;
+    int k;
+    int m;
+    long givenN;
+    long maxN;
+    int numLevels;
+    int maxItems;
+    int compactBytes;
+    int updatableBytes;
+  }
+
+  public static class LevelStats {
+    long n;
+    int numLevels;
+    int items;
+
+    LevelStats(final long n, final int numLevels, final int items) {
+      this.n = n;
+      this.numLevels = numLevels;
+      this.items = items;
+    }
+  }
 
   /**
-   * Copy the old array into a new larger array.
-   * The extra space is at the top.
-   * @param oldArr the given old array with data
-   * @param newLen the new length larger than the oldArr.length.
-   * @return the new array
+   * This is the exact powers of 3 from 3^0 to 3^30 where the exponent is the index
    */
-  static int[] growIntArray(final int[] oldArr, final int newLen) {
-    final int oldLen = oldArr.length;
-    assert newLen > oldLen;
-    final int[] newArr = new int[newLen];
-    System.arraycopy(oldArr, 0, newArr, 0, oldLen);
-    return newArr;
+  private static final long[] powersOfThree =
+      new long[] {1, 3, 9, 27, 81, 243, 729, 2187, 6561, 19683, 59049, 177147, 531441,
+  1594323, 4782969, 14348907, 43046721, 129140163, 387420489, 1162261467,
+  3486784401L, 10460353203L, 31381059609L, 94143178827L, 282429536481L,
+  847288609443L, 2541865828329L, 7625597484987L, 22876792454961L, 68630377364883L,
+  205891132094649L};
+
+  /**
+   * Given k, m, and numLevels, this computes and optionally prints the structure of the sketch when the given
+   * number of levels are completely filled.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param numLevels the given number of levels of the sketch
+   * @param printSketchStructure if true will print the details of the sketch structure at the given numLevels.
+   * @return LevelStats with the final summary of the sketch's cumulative N,
+   * and cumulative items at the given numLevels.
+   */
+  public static LevelStats getFinalSketchStatsAtNumLevels(
+      final int k,
+      final int m,
+      final int numLevels,
+      final boolean printSketchStructure) {
+    int cumItems = 0;
+    long cumN = 0;
+    if (printSketchStructure) {
+      println("SKETCH STRUCTURE:");
+      println("Given K        : " + k);
+      println("Given M        : " + m);
+      println("Given NumLevels: " + numLevels);
+      printf("%6s %8s %12s %18s %18s\n", "Level", "Items", "CumItems", "N at Level", "CumN");
+    }
+    for (int level = 0; level < numLevels; level++) {
+      final LevelStats lvlStats = getLevelCapacityItems(k, m, numLevels, level);
+      cumItems += lvlStats.items;
+      cumN += lvlStats.n;
+      if (printSketchStructure) {
+        printf("%6d %,8d %,12d %,18d %,18d\n", level, lvlStats.items, cumItems, lvlStats.n, cumN);
+      }
+    }
+    return new LevelStats(cumN, numLevels, cumItems);
   }
 
   /**
-   * Returns the upper bound of the number of levels based on <i>n</i>.
-   * @param n the length of the stream
-   * @return floor( log_2(n) )
+   * Given k, m, n, and the sketch type, this computes (and optionally prints) the growth scheme for a sketch as it
+   * grows large enough to accommodate a stream length of n items.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param n the desired stream length
+   * @param sketchType the given sketch type (DOUBLES_SKETCH or FLOATS_SKETCH)
+   * @param printGrowthScheme if true the entire growth scheme of the sketch will be printed.
+   * @return GrowthStats with the final values of the growth scheme
    */
-  static int ubOnNumLevels(final long n) {
-    return 1 + Long.numberOfTrailingZeros(floorPowerOf2(n));
+  public static GrowthStats getGrowthSchemeForGivenN(
+      final int k,
+      final int m,
+      final long n,
+      final SketchType sketchType,
+      final boolean printGrowthScheme) {
+    int numLevels = 0;
+    LevelStats lvlStats;
+    final GrowthStats gStats = new GrowthStats();
+    gStats.k = k;
+    gStats.m = m;
+    gStats.givenN = n;
+    gStats.sketchType = sketchType;
+    if (printGrowthScheme) {
+      println("GROWTH SCHEME:");
+      println("Given SketchType: " + sketchType.toString());
+      println("Given K         : " + k);
+      println("Given M         : " + m);
+      println("Given N         : " + n);
+      printf("%10s %10s %20s %13s %15s\n", "NumLevels", "MaxItems", "MaxN", "CompactBytes", "UpdatableBytes");
+    }
+    int compactBytes;
+    int updatableBytes;
+    do {
+      numLevels++;
+      lvlStats = getFinalSketchStatsAtNumLevels(k, m, numLevels, false);
+      final int maxItems = lvlStats.items;
+      final long maxN = lvlStats.n;
+      if (sketchType == DOUBLES_SKETCH) {
+        compactBytes = maxItems * Double.BYTES + numLevels * Integer.BYTES + 2 * Double.BYTES + DATA_START_ADR_DOUBLE;
+        updatableBytes = compactBytes + Integer.BYTES;
+      } else {
+        compactBytes = maxItems * Float.BYTES + numLevels * Integer.BYTES + 2 * Float.BYTES + DATA_START_ADR_FLOAT;
+        updatableBytes = compactBytes + Integer.BYTES;
+      }
+      if (printGrowthScheme) {
+        printf("%10d %,10d %,20d %,13d %,15d\n", numLevels, maxItems, maxN, compactBytes, updatableBytes);
+      }
+    } while (lvlStats.n < n);
+    gStats.maxN = lvlStats.n;
+    gStats.numLevels = lvlStats.numLevels;
+    gStats.maxItems = lvlStats.items;
+    gStats.compactBytes = compactBytes;
+    gStats.updatableBytes = updatableBytes;
+    return gStats;
+  }
+
+  /**
+   * Given k, m, numLevels, this computes the item capacity of a single level.
+   * @param k the given user sketch configuration parameter
+   * @param m the given user sketch configuration parameter
+   * @param numLevels the given number of levels of the sketch
+   * @param level the specific level to compute its item capacity
+   * @return LevelStats with the computed N and items for the given level.
+   */
+  public static LevelStats getLevelCapacityItems(
+      final int k,
+      final int m,
+      final int numLevels,
+      final int level) {
+    final int items = KllHelper.levelCapacity(k, numLevels, level, m);
+    final long n = (long)items << level;
+    return new LevelStats(n, numLevels, items);
+  }
+
+  /**
+   * Checks the validity of the given value k
+   * @param k must be greater than 7 and less than 65536.
+   */
+  static void checkK(final int k, final int m) {
+    if (k < m || k > KllSketch.MAX_K) {
+      throw new SketchesArgumentException(
+          "K must be >= " + m + " and <= " + KllSketch.MAX_K + ": " + k);
+    }
+  }

Review Comment:
   the parameter "m" is now hidden from the user.  



-- 
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


[GitHub] [datasketches-java] davecromberge commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
davecromberge commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r850210619


##########
src/main/java/org/apache/datasketches/kll/package-info.java:
##########
@@ -35,16 +35,17 @@
  *
  * <p>The <i>normalized rank</i> (<i>rank</i>) of any specific value is defined as its
  * <i>absolute rank</i> divided by <i>N</i>.
- * Thus, the <i>normalized rank</i> is a value between zero and one.
+ * Thus, the <i>normalized rank</i> is a value in the interval [0.0, 1.0).
  * In the documentation and Javadocs for this sketch <i>absolute rank</i> is never used so any
  * reference to just <i>rank</i> should be interpreted to mean <i>normalized rank</i>.
  *
  * <p>This sketch is configured with a parameter <i>k</i>, which affects the size of the sketch
  * and its estimation error.
  *
- * <p>The estimation error is commonly called <i>epsilon</i> (or <i>eps</i>) and is a fraction
- * between zero and one. Larger values of <i>k</i> result in smaller values of epsilon.
- * Epsilon is always with respect to the rank and cannot be applied to the
+ * <p>In the research literature, the estimation error is commonly called <i>epsilon</i>
+ * (or <i>eps</i>) and is a fraction between zero and one.
+ * Larger values of <i>k</i> result in smaller values of epsilon.
+ * The epsilon error is always with respect to the rank and cannot be applied to the

Review Comment:
   Thanks - the additional detail you provided about error and how it relates to levels is helpful.



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842148130


##########
src/main/java/org/apache/datasketches/kll/KllPreambleUtil.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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 org.apache.datasketches.Util.zeroPad;
+
+import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+//@formatter:off
+
+/**
+ * This class defines the serialized data structure and provides access methods for the key fields.
+ *
+ * <p>The intent of the design of this class was to isolate the detailed knowledge of the bit and
+ * byte layout of the serialized form of the sketches derived from the base sketch classes into one place.
+ * This allows the possibility of the introduction of different serialization
+ * schemes with minimal impact on the rest of the library.</p>
+ *
+ * <p>
+ * LAYOUT: The low significance bytes of this <i>long</i> based data structure are on the right.
+ * The multi-byte primitives are stored in native byte order.
+ * The single byte fields are treated as unsigned.</p>
+ *
+ * <p>An empty sketch requires only 8 bytes, which is only preamble.
+ * A serialized, non-empty KllDoublesSketch requires at least 16 bytes of preamble.
+ * A serialized, non-empty KllFloatsSketch requires at least 12 bytes of preamble.</p>
+ *
+ * <pre>{@code
+ * Serialized float sketch layout, more than one item:
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||---------------------------------N_LONG---------------------------------------|
+ *      ||         |       |        |   20   |   19   |    18   |   17   |      16      |
+ *  2   ||<-------Levels Arr Start----------]| unused |NumLevels|------Min K------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-------Min/Max Arr Start---------]|[<----------Levels Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-----Float Items Arr Start-------]|[<---------Min/Max Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||         |       |        |        |[<-------Float Items Arr End--------------|
+ *
+ * Serialized float sketch layout, Empty (8 bytes) and Single Item (12 bytes):
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||                                   |-------------Single Item------------------|
+ *
+ *
+ *
+ * Serialized double sketch layout, more than one item:
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||   15    |   14  |   13   |   12   |   11   |   10    |    9   |      8       |
+ *  1   ||---------------------------------N_LONG---------------------------------------|
+ *      ||   23    |   22  |   21   |   20   |   19   |    18   |   17   |      16      |
+ *  2   ||<-------Levels Arr Start----------]| unused |NumLevels|------Min K------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<-------Min/Max Arr Start---------]|[<----------Levels Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||<----Double Items Arr Start-------]|[<---------Min/Max Arr End----------------|
+ *      ||         |       |        |        |        |         |        |              |
+ *  ?   ||         |       |        |        |[<------Double Items Arr End--------------|
+ *
+ * Serialized double sketch layout, Empty (8 bytes) and Single Item (16 bytes):
+ *  Adr:
+ *      ||    7    |   6   |    5   |    4   |    3   |    2    |    1   |      0       |
+ *  0   || unused  |   M   |--------K--------|  Flags |  FamID  | SerVer | PreambleInts |
+ *      ||                                                               |      8       |
+ *  1   ||------------------------------Single Item-------------------------------------|
+ *
+ * The structure of the data block depends on Layout:
+ *
+ *   For FLOAT_SINGLE_COMPACT or DOUBLE_SINGLE_COMPACT:
+ *     The single data item is at offset DATA_START_ADR_SINGLE_ITEM = 8
+ *
+ *   For FLOAT_FULL_COMPACT:
+ *     The int[] levels array starts at offset DATA_START_ADR_FLOAT = 20 with a length of numLevels integers;
+ *     Followed by Float Min_Value, then Float Max_Value
+ *     Followed by an array of Floats of length retainedItems()
+ *
+ *   For DOUBLE_FULL_COMPACT
+ *     The int[] levels array starts at offset DATA_START_ADR_DOUBLE = 20 with a length of numLevels integers;
+ *     Followed by Double Min_Value, then Double Max_Value
+ *     Followed by an array of Doubles of length retainedItems()
+ *
+ *   For FLOAT_UPDATABLE
+ *     The int[] levels array starts at offset DATA_START_ADR_FLOAT = 20 with a length of (numLevels + 1) integers;
+ *     Followed by Float Min_Value, then Float Max_Value
+ *     Followed by an array of Floats of length KllHelper.computeTotalItemCapacity(...).
+ *
+ *   For DOUBLE_UPDATABLE
+ *     The int[] levels array starts at offset DATA_START_ADR_DOUBLE = 20 with a length of (numLevels + 1) integers;
+ *     Followed by Double Min_Value, then Double Max_Value
+ *     Followed by an array of Doubles of length KllHelper.computeTotalItemCapacity(...).
+ *
+ * }</pre>
+ *
+ *  @author Lee Rhodes
+ */
+final class KllPreambleUtil {
+
+  private KllPreambleUtil() {}
+
+  static final String LS = System.getProperty("line.separator");
+
+  /**
+   * The default value of K
+   */
+  public static final int DEFAULT_K = 200;
+  public static final int DEFAULT_M = 8;
+  public static final int MAX_K = (1 << 16) - 1; // serialized as an unsigned short
+  public static final int MAX_M = 8;
+  public static final int MIN_M = 2;
+
+  // Preamble byte addresses
+  static final int PREAMBLE_INTS_BYTE_ADR     = 0;
+  static final int SER_VER_BYTE_ADR           = 1;
+  static final int FAMILY_BYTE_ADR            = 2;
+  static final int FLAGS_BYTE_ADR             = 3;
+  static final int K_SHORT_ADR                = 4;  // to 5
+  static final int M_BYTE_ADR                 = 6;
+  //                                            7 is reserved for future use
+  // SINGLE ITEM ONLY
+  static final int DATA_START_ADR_SINGLE_ITEM = 8;
+
+  // MULTI-ITEM
+  static final int N_LONG_ADR                 = 8;  // to 15
+  static final int MIN_K_SHORT_ADR         = 16; // to 17
+  static final int NUM_LEVELS_BYTE_ADR        = 18;
+
+  // FLOAT SKETCH                               19 is reserved for future use in float sketch
+  static final int DATA_START_ADR_FLOAT       = 20; // float sketch, not single item
+
+  // DOUBLE SKETCH                              19 to 23 is reserved for future use in double sketch
+  static final int DATA_START_ADR_DOUBLE      = 20; // double sketch, not single item
+
+  // Other static values
+  static final byte SERIAL_VERSION_EMPTY_FULL = 1; // Empty or full preamble, NOT single item format
+  static final byte SERIAL_VERSION_SINGLE     = 2; // only single-item format
+  static final byte SERIAL_VERSION_UPDATABLE  = 3; //
+  static final int PREAMBLE_INTS_EMPTY_SINGLE = 2; // for empty or single item
+  static final int PREAMBLE_INTS_FLOAT        = 5; // not empty nor single item, full preamble float
+  static final int PREAMBLE_INTS_DOUBLE       = 5; // not empty nor single item, full preamble double
+
+  // Flag bit masks
+  static final int EMPTY_BIT_MASK             = 1;
+  static final int LEVEL_ZERO_SORTED_BIT_MASK = 2;
+  static final int SINGLE_ITEM_BIT_MASK       = 4;
+  static final int DOUBLES_SKETCH_BIT_MASK    = 8;
+  static final int UPDATABLE_BIT_MASK         = 16;
+
+  enum Layout {
+    FLOAT_FULL_COMPACT,       FLOAT_EMPTY_COMPACT,      FLOAT_SINGLE_COMPACT,
+    DOUBLE_FULL_COMPACT,      DOUBLE_EMPTY_COMPACT,     DOUBLE_SINGLE_COMPACT,
+    FLOAT_UPDATABLE,  DOUBLE_UPDATABLE }
+
+  /**
+   * Returns a human readable string summary of the internal state of the given byte array.
+   * Used primarily in testing.
+   *
+   * @param byteArr the given byte array.
+   * @return the summary string.
+   */
+  static String toString(final byte[] byteArr) {
+    final Memory mem = Memory.wrap(byteArr);
+    return toString(mem);
+  }
+
+  /**
+   * Returns a human readable string summary of the internal state of the given Memory.
+   * Used primarily in testing.
+   *
+   * @param mem the given Memory
+   * @return the summary string.
+   */
+  static String toString(final Memory mem) {
+    return memoryToString(mem);
+  }
+
+  static String memoryToString(final Memory mem) {
+    final KllMemoryValidate memChk = new KllMemoryValidate(mem);
+    final int flags = memChk.flags & 0XFF;
+    final String flagsStr = (flags) + ", 0x" + (Integer.toHexString(flags)) + ", "
+        + zeroPad(Integer.toBinaryString(flags), 8);
+    final int preInts = memChk.preInts;
+    final StringBuilder sb = new StringBuilder();
+    sb.append(Util.LS).append("### KLL SKETCH MEMORY SUMMARY:").append(LS);
+    sb.append("Byte   0   : Preamble Ints      : ").append(preInts).append(LS);
+    sb.append("Byte   1   : SerVer             : ").append(memChk.serVer).append(LS);
+    sb.append("Byte   2   : FamilyID           : ").append(memChk.familyID).append(LS);
+    sb.append("             FamilyName         : ").append(memChk.famName).append(LS);
+    sb.append("Byte   3   : Flags Field        : ").append(flagsStr).append(LS);
+    sb.append("         Bit Flag Name").append(LS);
+    sb.append("           0 EMPTY COMPACT      : ").append(memChk.empty).append(LS);
+    sb.append("           1 LEVEL_ZERO_SORTED  : ").append(memChk.level0Sorted).append(LS);
+    sb.append("           2 SINGLE_ITEM COMPACT: ").append(memChk.singleItem).append(LS);
+    sb.append("           3 DOUBLES_SKETCH     : ").append(memChk.doublesSketch).append(LS);
+    sb.append("           4 UPDATABLE          : ").append(memChk.updatable).append(LS);
+    sb.append("Bytes  4-5 : K                  : ").append(memChk.k).append(LS);
+    sb.append("Byte   6   : Min Level Cap, M   : ").append(memChk.m).append(LS);
+    sb.append("Byte   7   : (Reserved)         : ").append(LS);
+
+    switch (memChk.layout) {
+      case DOUBLE_FULL_COMPACT:
+      case FLOAT_FULL_COMPACT:
+      case FLOAT_UPDATABLE:
+      case DOUBLE_UPDATABLE:
+      {
+        sb.append("Bytes  8-15: N                  : ").append(memChk.n).append(LS);
+        sb.append("Bytes 16-17: DyMinK             : ").append(memChk.dyMinK).append(LS);
+        sb.append("Byte  18   : NumLevels          : ").append(memChk.numLevels).append(LS);
+        break;
+      }
+      case FLOAT_EMPTY_COMPACT:
+      case FLOAT_SINGLE_COMPACT:
+      case DOUBLE_EMPTY_COMPACT:
+      case DOUBLE_SINGLE_COMPACT:
+      {
+        sb.append("Assumed    : N                  : ").append(memChk.n).append(LS);
+        sb.append("Assumed    : DyMinK             : ").append(memChk.dyMinK).append(LS);
+        sb.append("Assumed    : NumLevels          : ").append(memChk.numLevels).append(LS);
+        break;
+      }
+      default: break; //can never happen
+    }
+    sb.append("PreambleBytes                   : ").append(preInts * 4).append(LS);
+    sb.append("Sketch Bytes                    : ").append(memChk.sketchBytes).append(LS);
+    sb.append("Memory Capacity Bytes           : ").append(mem.getCapacity()).append(LS);
+    sb.append("### END KLL Sketch Memory Summary").append(LS);
+    return sb.toString();
+  }
+
+  static int extractPreInts(final Memory mem) {
+    return mem.getByte(PREAMBLE_INTS_BYTE_ADR) & 0XFF;
+  }
+
+  static int extractSerVer(final Memory mem) {
+    return mem.getByte(SER_VER_BYTE_ADR) & 0XFF;
+  }
+
+  static int extractFamilyID(final Memory mem) {
+    return mem.getByte(FAMILY_BYTE_ADR) & 0XFF;
+  }
+
+  static int extractFlags(final Memory mem) {
+    return mem.getByte(FLAGS_BYTE_ADR) & 0XFF;
+  }
+
+  static boolean extractEmptyFlag(final Memory mem) {
+    return (extractFlags(mem) & EMPTY_BIT_MASK) != 0;
+  }
+
+  static boolean extractLevelZeroSortedFlag(final Memory mem) {
+    return (extractFlags(mem) & LEVEL_ZERO_SORTED_BIT_MASK) != 0;
+  }
+
+  static boolean extractSingleItemFlag(final Memory mem) {
+    return (extractFlags(mem) & SINGLE_ITEM_BIT_MASK) != 0;
+  }
+
+  static boolean extractDoubleSketchFlag(final Memory mem) {
+    return (extractFlags(mem) & DOUBLES_SKETCH_BIT_MASK) != 0;
+  }
+
+  static boolean extractUpdatableFlag(final Memory mem) {
+    return (extractFlags(mem) & UPDATABLE_BIT_MASK) != 0;
+  }
+
+  static int extractK(final Memory mem) {
+    return mem.getShort(K_SHORT_ADR) & 0XFFFF;
+  }
+
+  static int extractM(final Memory mem) {
+    return mem.getByte(M_BYTE_ADR) & 0XFF;
+  }
+
+  static long extractN(final Memory mem) {
+    return mem.getLong(N_LONG_ADR);
+  }
+
+  static int extractMinK(final Memory mem) {
+    return mem.getShort(MIN_K_SHORT_ADR) & 0XFFFF;
+  }
+
+  static int extractNumLevels(final Memory mem) {
+    return mem.getByte(NUM_LEVELS_BYTE_ADR) & 0XFF;
+  }
+
+  static void insertPreInts(final WritableMemory wmem, final int value) {
+    wmem.putByte(PREAMBLE_INTS_BYTE_ADR, (byte) value);
+  }
+
+  static void insertSerVer(final WritableMemory wmem, final int value) {
+    wmem.putByte(SER_VER_BYTE_ADR, (byte) value);
+  }
+
+  static void insertFamilyID(final WritableMemory wmem, final int value) {
+    wmem.putByte(FAMILY_BYTE_ADR, (byte) value);
+  }
+
+  static void insertFlags(final WritableMemory wmem, final int value) {
+    wmem.putByte(FLAGS_BYTE_ADR, (byte) value);
+  }
+
+  static void insertEmptyFlag(final WritableMemory wmem,  final boolean empty) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, empty ? flags | EMPTY_BIT_MASK : flags & ~EMPTY_BIT_MASK);
+  }
+
+  static void insertLevelZeroSortedFlag(final WritableMemory wmem,  final boolean levelZeroSorted) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, levelZeroSorted ? flags | LEVEL_ZERO_SORTED_BIT_MASK : flags & ~LEVEL_ZERO_SORTED_BIT_MASK);
+  }
+
+  static void insertSingleItemFlag(final WritableMemory wmem,  final boolean singleItem) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, singleItem ? flags | SINGLE_ITEM_BIT_MASK : flags & ~SINGLE_ITEM_BIT_MASK);
+  }
+
+  static void insertDoubleSketchFlag(final WritableMemory wmem,  final boolean doubleSketch) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, doubleSketch ? flags | DOUBLES_SKETCH_BIT_MASK : flags & ~DOUBLES_SKETCH_BIT_MASK);
+  }
+
+  static void insertUpdatableFlag(final WritableMemory wmem,  final boolean updatable) {
+    final int flags = extractFlags(wmem);
+    insertFlags(wmem, updatable ? flags | UPDATABLE_BIT_MASK : flags & ~UPDATABLE_BIT_MASK);
+  }
+
+  static void insertK(final WritableMemory wmem, final int value) {
+    wmem.putShort(K_SHORT_ADR, (short) value);
+  }
+
+  static void insertM(final WritableMemory wmem, final int value) {
+    wmem.putByte(M_BYTE_ADR, (byte) value);
+  }
+
+  static void insertN(final WritableMemory wmem, final long value) {
+    wmem.putLong(N_LONG_ADR, value);
+  }
+
+  static void insertMinK(final WritableMemory wmem, final int value) {
+    wmem.putShort(MIN_K_SHORT_ADR, (short) value);
+  }
+
+  static void insertNumLevels(final WritableMemory wmem, final int value) {
+    wmem.putByte(NUM_LEVELS_BYTE_ADR, (byte) value);
+  }
+

Review Comment:
   Hmm. I was using the same convention for naming these methods that was established some years ago, perhaps improperly at the time.  Also, I wanted to distinguish these method names, which are explicitly dedicated to updating the off-heap Memory object from the "setNumLevels(int)" method, which is a virtual method of the sketch itself. Having two methods with different purposes with the same name would be confusing. 
   
   Now, I could be more verbose and use "setMemoryNumLevels()", but I was trying not to be so verbose.
   
   But I think your comment about semantics is a good one.  After I have fixed all the other issues, I will come back and relook at this.
   



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842155585


##########
src/main/java/org/apache/datasketches/kll/KllHelper.java:
##########
@@ -19,32 +19,192 @@
 
 package org.apache.datasketches.kll;
 
+import static java.lang.Math.pow;
 import static org.apache.datasketches.Util.floorPowerOf2;
+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.MAX_K;
+import static org.apache.datasketches.kll.KllPreambleUtil.MAX_M;
+import static org.apache.datasketches.kll.KllPreambleUtil.MIN_M;
+import static org.apache.datasketches.kll.KllSketch.CDF_COEF;
+import static org.apache.datasketches.kll.KllSketch.CDF_EXP;
+import static org.apache.datasketches.kll.KllSketch.PMF_COEF;
+import static org.apache.datasketches.kll.KllSketch.PMF_EXP;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
 
-class KllHelper {
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.kll.KllSketch.SketchType;
+
+/**
+ * This class provides some useful sketch analysis tools that are used internally and also can be used by
+ * interested users to understand the internal structure of the sketch as well as the growth properties of the
+ * sketch given a stream length.
+ *
+ * @author lrhodes
+ *
+ */
+public class KllHelper {
+
+  public static class GrowthStats {
+    SketchType sketchType;
+    int k;
+    int m;
+    long givenN;
+    long maxN;
+    int numLevels;
+    int maxItems;
+    int compactBytes;
+    int updatableBytes;
+  }
+
+  public static class LevelStats {
+    long n;
+    int numLevels;
+    int items;
+
+    LevelStats(final long n, final int numLevels, final int items) {
+      this.n = n;
+      this.numLevels = numLevels;
+      this.items = items;
+    }
+  }
 
   /**
-   * Copy the old array into a new larger array.
-   * The extra space is at the top.
-   * @param oldArr the given old array with data
-   * @param newLen the new length larger than the oldArr.length.
-   * @return the new array
+   * This is the exact powers of 3 from 3^0 to 3^30 where the exponent is the index
    */
-  static int[] growIntArray(final int[] oldArr, final int newLen) {
-    final int oldLen = oldArr.length;
-    assert newLen > oldLen;
-    final int[] newArr = new int[newLen];
-    System.arraycopy(oldArr, 0, newArr, 0, oldLen);
-    return newArr;
+  private static final long[] powersOfThree =
+      new long[] {1, 3, 9, 27, 81, 243, 729, 2187, 6561, 19683, 59049, 177147, 531441,
+  1594323, 4782969, 14348907, 43046721, 129140163, 387420489, 1162261467,
+  3486784401L, 10460353203L, 31381059609L, 94143178827L, 282429536481L,
+  847288609443L, 2541865828329L, 7625597484987L, 22876792454961L, 68630377364883L,
+  205891132094649L};
+
+  /**
+   * Given k, m, and numLevels, this computes and optionally prints the structure of the sketch when the given
+   * number of levels are completely filled.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param numLevels the given number of levels of the sketch
+   * @param printSketchStructure if true will print the details of the sketch structure at the given numLevels.
+   * @return LevelStats with the final summary of the sketch's cumulative N,
+   * and cumulative items at the given numLevels.
+   */
+  public static LevelStats getFinalSketchStatsAtNumLevels(
+      final int k,
+      final int m,
+      final int numLevels,
+      final boolean printSketchStructure) {
+    int cumItems = 0;
+    long cumN = 0;
+    if (printSketchStructure) {
+      println("SKETCH STRUCTURE:");
+      println("Given K        : " + k);
+      println("Given M        : " + m);
+      println("Given NumLevels: " + numLevels);
+      printf("%6s %8s %12s %18s %18s\n", "Level", "Items", "CumItems", "N at Level", "CumN");
+    }
+    for (int level = 0; level < numLevels; level++) {
+      final LevelStats lvlStats = getLevelCapacityItems(k, m, numLevels, level);
+      cumItems += lvlStats.items;
+      cumN += lvlStats.n;
+      if (printSketchStructure) {
+        printf("%6d %,8d %,12d %,18d %,18d\n", level, lvlStats.items, cumItems, lvlStats.n, cumN);
+      }
+    }
+    return new LevelStats(cumN, numLevels, cumItems);
   }
 
   /**
-   * Returns the upper bound of the number of levels based on <i>n</i>.
-   * @param n the length of the stream
-   * @return floor( log_2(n) )
+   * Given k, m, n, and the sketch type, this computes (and optionally prints) the growth scheme for a sketch as it
+   * grows large enough to accommodate a stream length of n items.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param n the desired stream length
+   * @param sketchType the given sketch type (DOUBLES_SKETCH or FLOATS_SKETCH)
+   * @param printGrowthScheme if true the entire growth scheme of the sketch will be printed.
+   * @return GrowthStats with the final values of the growth scheme
    */
-  static int ubOnNumLevels(final long n) {
-    return 1 + Long.numberOfTrailingZeros(floorPowerOf2(n));
+  public static GrowthStats getGrowthSchemeForGivenN(
+      final int k,
+      final int m,
+      final long n,
+      final SketchType sketchType,
+      final boolean printGrowthScheme) {
+    int numLevels = 0;
+    LevelStats lvlStats;
+    final GrowthStats gStats = new GrowthStats();
+    gStats.k = k;
+    gStats.m = m;
+    gStats.givenN = n;
+    gStats.sketchType = sketchType;
+    if (printGrowthScheme) {
+      println("GROWTH SCHEME:");
+      println("Given SketchType: " + sketchType.toString());
+      println("Given K         : " + k);
+      println("Given M         : " + m);
+      println("Given N         : " + n);
+      printf("%10s %10s %20s %13s %15s\n", "NumLevels", "MaxItems", "MaxN", "CompactBytes", "UpdatableBytes");
+    }
+    int compactBytes;
+    int updatableBytes;
+    do {
+      numLevels++;
+      lvlStats = getFinalSketchStatsAtNumLevels(k, m, numLevels, false);
+      final int maxItems = lvlStats.items;
+      final long maxN = lvlStats.n;
+      if (sketchType == DOUBLES_SKETCH) {
+        compactBytes = maxItems * Double.BYTES + numLevels * Integer.BYTES + 2 * Double.BYTES + DATA_START_ADR_DOUBLE;
+        updatableBytes = compactBytes + Integer.BYTES;
+      } else {
+        compactBytes = maxItems * Float.BYTES + numLevels * Integer.BYTES + 2 * Float.BYTES + DATA_START_ADR_FLOAT;
+        updatableBytes = compactBytes + Integer.BYTES;
+      }
+      if (printGrowthScheme) {
+        printf("%10d %,10d %,20d %,13d %,15d\n", numLevels, maxItems, maxN, compactBytes, updatableBytes);
+      }
+    } while (lvlStats.n < n);
+    gStats.maxN = lvlStats.n;
+    gStats.numLevels = lvlStats.numLevels;
+    gStats.maxItems = lvlStats.items;
+    gStats.compactBytes = compactBytes;
+    gStats.updatableBytes = updatableBytes;
+    return gStats;
+  }
+
+  /**
+   * Given k, m, numLevels, this computes the item capacity of a single level.
+   * @param k the given user sketch configuration parameter
+   * @param m the given user sketch configuration parameter
+   * @param numLevels the given number of levels of the sketch
+   * @param level the specific level to compute its item capacity
+   * @return LevelStats with the computed N and items for the given level.
+   */
+  public static LevelStats getLevelCapacityItems(
+      final int k,
+      final int m,
+      final int numLevels,
+      final int level) {
+    final int items = KllHelper.levelCapacity(k, numLevels, level, m);
+    final long n = (long)items << level;
+    return new LevelStats(n, numLevels, items);
+  }

Review Comment:
   Yes. To get a good understanding of how this works I would suggest the following.
   - Start with a simple KllFloatsSketch(20) and fill it with 20 items. 
   - Then print sketch.toString(true, true);  It will print the  structure of the sketch.
   - Now add one more item and again print sketch.toString(true, true); 
   This will show you how the sketch structure has changed.  It had to add a new level.
   
   Keep doing this and you will see that each new level is about 2/3 the size of the previous level, 
   until it reaches _m_, which is the minimum size of a level. All new levels after that will be the same
   size _m_.  
   
   This will also show you when the sketch compresses the current levels. 
   



-- 
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


[GitHub] [datasketches-java] leerho commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
leerho commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r842194914


##########
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:
   Good idea.  Will fix. Thanks!



-- 
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


[GitHub] [datasketches-java] davecromberge commented on a diff in pull request #390: Direct kll double

Posted by GitBox <gi...@apache.org>.
davecromberge commented on code in PR #390:
URL: https://github.com/apache/datasketches-java/pull/390#discussion_r844892564


##########
src/main/java/org/apache/datasketches/kll/KllHelper.java:
##########
@@ -19,32 +19,192 @@
 
 package org.apache.datasketches.kll;
 
+import static java.lang.Math.pow;
 import static org.apache.datasketches.Util.floorPowerOf2;
+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.MAX_K;
+import static org.apache.datasketches.kll.KllPreambleUtil.MAX_M;
+import static org.apache.datasketches.kll.KllPreambleUtil.MIN_M;
+import static org.apache.datasketches.kll.KllSketch.CDF_COEF;
+import static org.apache.datasketches.kll.KllSketch.CDF_EXP;
+import static org.apache.datasketches.kll.KllSketch.PMF_COEF;
+import static org.apache.datasketches.kll.KllSketch.PMF_EXP;
+import static org.apache.datasketches.kll.KllSketch.SketchType.DOUBLES_SKETCH;
 
-class KllHelper {
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.kll.KllSketch.SketchType;
+
+/**
+ * This class provides some useful sketch analysis tools that are used internally and also can be used by
+ * interested users to understand the internal structure of the sketch as well as the growth properties of the
+ * sketch given a stream length.
+ *
+ * @author lrhodes
+ *
+ */
+public class KllHelper {
+
+  public static class GrowthStats {
+    SketchType sketchType;
+    int k;
+    int m;
+    long givenN;
+    long maxN;
+    int numLevels;
+    int maxItems;
+    int compactBytes;
+    int updatableBytes;
+  }
+
+  public static class LevelStats {
+    long n;
+    int numLevels;
+    int items;
+
+    LevelStats(final long n, final int numLevels, final int items) {
+      this.n = n;
+      this.numLevels = numLevels;
+      this.items = items;
+    }
+  }
 
   /**
-   * Copy the old array into a new larger array.
-   * The extra space is at the top.
-   * @param oldArr the given old array with data
-   * @param newLen the new length larger than the oldArr.length.
-   * @return the new array
+   * This is the exact powers of 3 from 3^0 to 3^30 where the exponent is the index
    */
-  static int[] growIntArray(final int[] oldArr, final int newLen) {
-    final int oldLen = oldArr.length;
-    assert newLen > oldLen;
-    final int[] newArr = new int[newLen];
-    System.arraycopy(oldArr, 0, newArr, 0, oldLen);
-    return newArr;
+  private static final long[] powersOfThree =
+      new long[] {1, 3, 9, 27, 81, 243, 729, 2187, 6561, 19683, 59049, 177147, 531441,
+  1594323, 4782969, 14348907, 43046721, 129140163, 387420489, 1162261467,
+  3486784401L, 10460353203L, 31381059609L, 94143178827L, 282429536481L,
+  847288609443L, 2541865828329L, 7625597484987L, 22876792454961L, 68630377364883L,
+  205891132094649L};
+
+  /**
+   * Given k, m, and numLevels, this computes and optionally prints the structure of the sketch when the given
+   * number of levels are completely filled.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param numLevels the given number of levels of the sketch
+   * @param printSketchStructure if true will print the details of the sketch structure at the given numLevels.
+   * @return LevelStats with the final summary of the sketch's cumulative N,
+   * and cumulative items at the given numLevels.
+   */
+  public static LevelStats getFinalSketchStatsAtNumLevels(
+      final int k,
+      final int m,
+      final int numLevels,
+      final boolean printSketchStructure) {
+    int cumItems = 0;
+    long cumN = 0;
+    if (printSketchStructure) {
+      println("SKETCH STRUCTURE:");
+      println("Given K        : " + k);
+      println("Given M        : " + m);
+      println("Given NumLevels: " + numLevels);
+      printf("%6s %8s %12s %18s %18s\n", "Level", "Items", "CumItems", "N at Level", "CumN");
+    }
+    for (int level = 0; level < numLevels; level++) {
+      final LevelStats lvlStats = getLevelCapacityItems(k, m, numLevels, level);
+      cumItems += lvlStats.items;
+      cumN += lvlStats.n;
+      if (printSketchStructure) {
+        printf("%6d %,8d %,12d %,18d %,18d\n", level, lvlStats.items, cumItems, lvlStats.n, cumN);
+      }
+    }
+    return new LevelStats(cumN, numLevels, cumItems);
   }
 
   /**
-   * Returns the upper bound of the number of levels based on <i>n</i>.
-   * @param n the length of the stream
-   * @return floor( log_2(n) )
+   * Given k, m, n, and the sketch type, this computes (and optionally prints) the growth scheme for a sketch as it
+   * grows large enough to accommodate a stream length of n items.
+   * @param k the given user configured sketch parameter
+   * @param m the given user configured sketch parameter
+   * @param n the desired stream length
+   * @param sketchType the given sketch type (DOUBLES_SKETCH or FLOATS_SKETCH)
+   * @param printGrowthScheme if true the entire growth scheme of the sketch will be printed.
+   * @return GrowthStats with the final values of the growth scheme
    */
-  static int ubOnNumLevels(final long n) {
-    return 1 + Long.numberOfTrailingZeros(floorPowerOf2(n));
+  public static GrowthStats getGrowthSchemeForGivenN(
+      final int k,
+      final int m,
+      final long n,
+      final SketchType sketchType,
+      final boolean printGrowthScheme) {
+    int numLevels = 0;
+    LevelStats lvlStats;
+    final GrowthStats gStats = new GrowthStats();
+    gStats.k = k;
+    gStats.m = m;
+    gStats.givenN = n;
+    gStats.sketchType = sketchType;
+    if (printGrowthScheme) {
+      println("GROWTH SCHEME:");
+      println("Given SketchType: " + sketchType.toString());
+      println("Given K         : " + k);
+      println("Given M         : " + m);
+      println("Given N         : " + n);
+      printf("%10s %10s %20s %13s %15s\n", "NumLevels", "MaxItems", "MaxN", "CompactBytes", "UpdatableBytes");
+    }
+    int compactBytes;
+    int updatableBytes;
+    do {
+      numLevels++;
+      lvlStats = getFinalSketchStatsAtNumLevels(k, m, numLevels, false);
+      final int maxItems = lvlStats.items;
+      final long maxN = lvlStats.n;
+      if (sketchType == DOUBLES_SKETCH) {
+        compactBytes = maxItems * Double.BYTES + numLevels * Integer.BYTES + 2 * Double.BYTES + DATA_START_ADR_DOUBLE;
+        updatableBytes = compactBytes + Integer.BYTES;
+      } else {
+        compactBytes = maxItems * Float.BYTES + numLevels * Integer.BYTES + 2 * Float.BYTES + DATA_START_ADR_FLOAT;
+        updatableBytes = compactBytes + Integer.BYTES;
+      }
+      if (printGrowthScheme) {
+        printf("%10d %,10d %,20d %,13d %,15d\n", numLevels, maxItems, maxN, compactBytes, updatableBytes);
+      }
+    } while (lvlStats.n < n);
+    gStats.maxN = lvlStats.n;
+    gStats.numLevels = lvlStats.numLevels;
+    gStats.maxItems = lvlStats.items;
+    gStats.compactBytes = compactBytes;
+    gStats.updatableBytes = updatableBytes;
+    return gStats;
+  }
+
+  /**
+   * Given k, m, numLevels, this computes the item capacity of a single level.
+   * @param k the given user sketch configuration parameter
+   * @param m the given user sketch configuration parameter
+   * @param numLevels the given number of levels of the sketch
+   * @param level the specific level to compute its item capacity
+   * @return LevelStats with the computed N and items for the given level.
+   */
+  public static LevelStats getLevelCapacityItems(
+      final int k,
+      final int m,
+      final int numLevels,
+      final int level) {
+    final int items = KllHelper.levelCapacity(k, numLevels, level, m);
+    final long n = (long)items << level;
+    return new LevelStats(n, numLevels, items);
+  }

Review Comment:
   Interesting - I plugged the level capacity code into a REPL and was playing around with the values but I'll do the same for your procedure above.



-- 
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