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/07/26 17:00:12 UTC

[GitHub] [datasketches-java] davecromberge commented on a diff in pull request #407: Added SortedView to REQ, Renamed variables for clarity. JavaDoc updates.

davecromberge commented on code in PR #407:
URL: https://github.com/apache/datasketches-java/pull/407#discussion_r929940744


##########
README.md:
##########
@@ -143,11 +143,6 @@ In Eclipse, open the project *Properties / Java Build Path / Module Dependencies
     * Enter *Target module* = ALL-UNNAMED
     * Select button: *exports*
     * Hit *OK*
-* Select *Expose Package...*
-    * Enter *Package* = sun.nio.ch
-    * Enter *Target module* = ALL-UNNAMED
-    * Select button: *opens*
-    * Hit *OK*

Review Comment:
   I had deleted this from the README but we probably need to restore it - otherwise the user receives warnings when running tests through the IDE if this is not present.



##########
pom.xml:
##########
@@ -679,7 +682,6 @@ under the License.
                   --add-exports java.base/jdk.internal.misc=ALL-UNNAMED
                   --add-exports java.base/jdk.internal.ref=ALL-UNNAMED
                   --add-opens java.base/java.nio=ALL-UNNAMED
-                  --add-opens java.base/sun.nio.ch=ALL-UNNAMED

Review Comment:
   As above, this line should be restored.



##########
src/main/java/org/apache/datasketches/req/ReqSketchSortedViewIterator.java:
##########
@@ -0,0 +1,110 @@
+/*
+
+ * 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.req;
+
+/**
+ * Iterator over KllDoublesSketchSortedView.
+ *
+ * <p>The recommended iteration loop:</p>
+ * <pre>{@code
+ *   ReqSketchSortedViewIterator itr = sketch.getSortedView().iterator();
+ *   while (itr.next()) {
+ *     float v = itr.getValue();
+ *     ...
+ *   }
+ * }</pre>
+ */
+public class ReqSketchSortedViewIterator {
+
+  private final float[] values;
+  private final long[] cumWeights;
+  private int index;
+
+  ReqSketchSortedViewIterator(final float[] values, final long[] cumWeights) {
+    this.values = values;
+    this.cumWeights = cumWeights;
+    index = -1;
+  }

Review Comment:
   Does it make sense to include the following constructor invariant:
   ```
   assert(values.size == cumWeights.size)  // values and cumulative weights have equal number of elements
   ```



##########
src/main/java/org/apache/datasketches/req/ReqSketchSortedView.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.req;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.datasketches.InequalitySearch;
+
+/**
+ * Supports searches for quantiles, Ranks, Iterator and Sorted View
+ * @author Lee Rhodes
+ */

Review Comment:
   Would it be possible to add more documentation as to the purpose of this view - is it internal or would users use it for debugging?



##########
src/main/java/org/apache/datasketches/req/BaseReqSketch.java:
##########
@@ -266,14 +274,15 @@ abstract class BaseReqSketch {
 
   /**
    * Sets the chosen criterion for value comparison
-   * @deprecated
-   *
    * @param ltEq (Less-than-or Equals) If true, the sketch will use the &le; criterion for comparing
    * values.  Otherwise, the criterion is strictly &lt;, the default.
    * This can be set anytime prior to a <i>getRank(float)</i> or <i>getQuantile(double)</i> or
    * equivalent query.
    * @return this
+   * @deprecated in the future the ltEq comparison parameter will not be saved at the class level in preference to
+   * the comparison parameter being specified for each API call. This method will be removed.
    */
+  @Deprecated

Review Comment:
   A version should be added to the deprecation notice e.g. since v3.0.X



##########
src/main/java/org/apache/datasketches/InequalitySearch.java:
##########
@@ -85,12 +85,11 @@ int getIndex(final long[] arr, final int a, final int b, final long v) {
 
     @Override
     int resolve(final int lo, final int hi, final int low, final int high) {
-      if (lo >= high) { return high; }
-      return -1;
+      return (lo >= high) ? high : -1;

Review Comment:
   Would this method also require public visibility?



##########
src/main/java/org/apache/datasketches/QuantilesHelper.java:
##########
@@ -104,7 +104,7 @@ private static int searchForChunkContainingPos(final long[] arr, final long pos,
     // the following three asserts can probably go away eventually, since it is fairly clear
     // that if these invariants hold at the beginning of the search, they will be maintained
     assert l < r;
-    assert arr[l] <= pos;
+    //assert arr[l] <= pos; //TODO inclusive fails with Kevins code

Review Comment:
   It's not quite clear what is to be done here - is the intention to restore the assertion and make it pass or does the assertion need to be updated to LE and not LEQ?



##########
src/main/java/org/apache/datasketches/req/ReqSketchSortedViewIterator.java:
##########
@@ -0,0 +1,110 @@
+/*
+
+ * 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.req;
+
+/**
+ * Iterator over KllDoublesSketchSortedView.
+ *
+ * <p>The recommended iteration loop:</p>
+ * <pre>{@code
+ *   ReqSketchSortedViewIterator itr = sketch.getSortedView().iterator();
+ *   while (itr.next()) {
+ *     float v = itr.getValue();
+ *     ...
+ *   }
+ * }</pre>
+ */

Review Comment:
   Minor doc typo, should read `Iterator over ReqSketchSortedView`.



##########
src/test/java/org/apache/datasketches/req/ReqSketchTest.java:
##########
@@ -303,7 +302,7 @@ private static void checkSerDeImpl(final int k, final boolean hra, final int cou
     assertEquals(sk2.getHighRankAccuracy(),sk1.getHighRankAccuracy());
     assertEquals(sk2.getK(), sk1.getK());
     assertEquals(sk2.getMaxNomSize(), sk1.getMaxNomSize());
-    assertEquals(sk2.getLtEq(), sk1.getLtEq());
+    //assertEquals(sk2.getLtEq(), sk1.getLtEq());

Review Comment:
   Is this assertion tied to the earlier TODO wrt Kevin's code?



##########
src/test/java/org/apache/datasketches/CrossCheckQuantilesTest.java:
##########
@@ -0,0 +1,356 @@
+/*
+ * 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;
+
+import static org.apache.datasketches.CrossCheckQuantilesTest.PrimType.DOUBLE;
+import static org.apache.datasketches.CrossCheckQuantilesTest.PrimType.FLOAT;
+import static org.apache.datasketches.CrossCheckQuantilesTest.SkType.CLASSIC;
+import static org.apache.datasketches.CrossCheckQuantilesTest.SkType.KLL;
+import static org.apache.datasketches.CrossCheckQuantilesTest.SkType.REQ;
+import static org.apache.datasketches.CrossCheckQuantilesTest.SkType.REQ_NO_DEDUP;
+import static org.apache.datasketches.CrossCheckQuantilesTest.SkType.REQ_SV;
+import static org.testng.Assert.assertEquals;
+
+import org.apache.datasketches.kll.KllFloatsSketch;
+import org.apache.datasketches.quantiles.DoublesSketch;
+import org.apache.datasketches.quantiles.UpdateDoublesSketch;
+import org.apache.datasketches.req.ReqSketch;
+import org.apache.datasketches.req.ReqSketchBuilder;
+import org.apache.datasketches.req.ReqSketchSortedView;
+import org.testng.annotations.Test;
+
+public class CrossCheckQuantilesTest {
+
+  enum SkType { REQ, REQ_SV, REQ_NO_DEDUP, KLL, CLASSIC }
+
+  enum PrimType { DOUBLE, FLOAT }
+
+  final int k = 32; //all sketches are in exact mode
+  final boolean hra = false; //for the REQ sketch
+  final boolean INCLUSIVE = true;
+  final boolean NON_INCLUSIVE = false;
+
+  @Test
+  public void checkQuantileSketches() {
+    checkQAndR(REQ, FLOAT, NON_INCLUSIVE); //must do REQ first to compute expected results!
+    checkQAndR(REQ, FLOAT, INCLUSIVE);
+
+    checkQAndR(REQ_SV, FLOAT, NON_INCLUSIVE);
+    checkQAndR(REQ_SV, FLOAT, INCLUSIVE);
+
+    checkQAndR(REQ_NO_DEDUP, FLOAT, NON_INCLUSIVE);
+    checkQAndR(REQ_NO_DEDUP, FLOAT, INCLUSIVE);
+
+    checkQAndR(KLL, FLOAT, NON_INCLUSIVE);
+    checkQAndR(KLL, FLOAT, INCLUSIVE);
+
+    checkQAndR(CLASSIC, DOUBLE, NON_INCLUSIVE);
+    checkQAndR(CLASSIC, DOUBLE, INCLUSIVE);
+    println("");
+  }
+
+  double[] testRankResults_NI = null;
+  double[] testRankResults_I = null;
+  float[] testQuantileFResults_NI = null;
+  float[] testQuantileFResults_I = null;
+  double[] testQuantileDResults_NI = null;
+  double[] testQuantileDResults_I = null;
+
+  private void checkQAndR(final SkType skType, final PrimType type, final boolean inclusive) {
+    String head = "CHECK " + skType.toString();
+    if (inclusive) { println("\n---------- " + head + " INCLUSIVE ----------\n"); }
+    else { println("\n########## " + head + " NON-INCLUSIVE ##########\n"); }
+
+    //CREATE EMPTY SKETCHES
+    ReqSketchBuilder reqBldr = ReqSketch.builder();
+    reqBldr.setK(4).setHighRankAccuracy(hra).setLessThanOrEqual(inclusive);
+    ReqSketch reqSk = reqBldr.build();
+
+    KllFloatsSketch kllSk = KllFloatsSketch.newHeapInstance(k);
+
+    UpdateDoublesSketch udSk = DoublesSketch.builder().setK(k).build();

Review Comment:
   Why is k set to 4 for the REQ sketch, but the constant k=32 used for the classic and KLL sketches? Is the former interpreted as lgK?



##########
src/main/java/org/apache/datasketches/req/ReqSketchSortedViewIterator.java:
##########
@@ -0,0 +1,110 @@
+/*
+
+ * 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.req;
+
+/**
+ * Iterator over KllDoublesSketchSortedView.
+ *
+ * <p>The recommended iteration loop:</p>
+ * <pre>{@code
+ *   ReqSketchSortedViewIterator itr = sketch.getSortedView().iterator();
+ *   while (itr.next()) {
+ *     float v = itr.getValue();
+ *     ...
+ *   }
+ * }</pre>
+ */
+public class ReqSketchSortedViewIterator {
+
+  private final float[] values;
+  private final long[] cumWeights;
+  private int index;
+
+  ReqSketchSortedViewIterator(final float[] values, final long[] cumWeights) {
+    this.values = values;
+    this.cumWeights = cumWeights;
+    index = -1;
+  }
+
+  /**
+   * Gets the current value.
+   *
+   * <p>Don't call this before calling next() for the first time
+   * or after getting false from next(). </p>
+   * @return the current value
+   */
+  public float getValue() {
+    return values[index];
+  }
+
+  /**
+   * Gets the cumulative weight for the current value.
+   *
+   * <p>Don't call this before calling next() for the first time
+   * or after getting false from next().</p>
+   * @param inclusive If true, includes the weight of the current value.
+   * Otherwise, returns the cumulative weightof the previous value.
+   * @return cumulative weight for the current value.
+   */
+  public long getCumulativeWeight(final boolean inclusive) {
+    return inclusive ? cumWeights[index]
+        : (index == 0) ? 0 : cumWeights[index - 1];
+  }
+
+  /**
+   * Gets the normalized rank for the current value or previous value.
+   *
+   * <p>Don't call this before calling next() for the first time
+   * or after getting false from next().</p>
+   * @param inclusive if true, returns the normalized rank of the current value.
+   * Otherwise, returns the normalized rank of the previous value.
+   * @return normalized rank for the current value or previous value.
+   */
+  public double getNormalizedRank(final boolean inclusive) {
+    final double N = cumWeights[ cumWeights.length - 1];
+    return getCumulativeWeight(inclusive) / N;
+  }

Review Comment:
   If this is a performance sensitive operation initialising N in the constructor might be more performant.



##########
src/main/java/org/apache/datasketches/req/ReqSketchSortedView.java:
##########
@@ -0,0 +1,232 @@
+/*
+ * 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.req;
+
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.datasketches.InequalitySearch;
+
+/**
+ * Supports searches for quantiles, Ranks, Iterator and Sorted View
+ * @author Lee Rhodes
+ */
+public class ReqSketchSortedView {
+  private static final String LS = System.getProperty("line.separator");
+  private float[] values;
+  private long[] cumWeights;
+  private final boolean hra; //used in merge
+  private final long N;
+  private final boolean dedup;
+
+  /**
+   * Construct this sorted view with the given sketch.
+   * The number of values in this sorted view will be the same as the number of retained values in the sketch.
+   * @param sk the given sketch
+   */
+  public ReqSketchSortedView(final ReqSketch sk) {
+    this(sk, false);
+  }
+
+  /**
+   * Construct this sorted view with the given sketch and option to deduplicate the values.
+   * The weights will be combined for the duplicate values.
+   * The getQuantile() and getRank() methods will work properly.
+   * @param sk the given sketch
+   * @param dedup if true, duplicate values will be combined into a single value with the combined weights.
+   */

Review Comment:
   What does it mean for the `getQuantile` and `getRank` methods to work properly?  Also, is it clear that combining weights is through addition and not some other aggregate operation such as max?



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