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 2020/07/08 00:31:53 UTC

[GitHub] [incubator-datasketches-java] leerho opened a new pull request #324: Refactor theta tuple

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


   This is a massive PR and will be difficult to review.  
   
   - I've had to do lots of shuffling of methods between classes to place them where they make the most sense. 
   - I was able to consolidate the compacting code into one new `CompactOperations` class that handles all the corner cases.  This cleaned up a lot of code that was duplicative. 
   - I've extended the set operations code in both Theta and Tuple to be more flexible and with similar API. 
       - All three operators (`Union, Intersection, AnotB`) can be executed using stateful or stateless operations.  
       - This makes them easier to remember and the fact that they behave the same in both Theta and Tuple should help users. 
       - I have not yet implemented these changes in the `ArrayOfDoubles` implementation.
   - Lots of documentation fixes where I came across them.  
   
   Go ahead and review this the best you can. Independent of the reviews I want to run some characterization tests to make sure I didn't screw something up.  It passes all the unit tests plus more I added, .... but you never know :)


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

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] [incubator-datasketches-java] leerho commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657888425


   The 3 bugs:
   - The EMPTY_SKETCH_MASK in EmptyCompactSketch was not masking out the lgNomLongs and lgArrLongs fields. This would cause it to reject an otherwise valid empty compact sketch.
   - The `loadCompactMemory()` as part of `CompactOperations` (this is new) was not initializing the `lgNomLongs` and `lgArrLongs` fields to zero. This contributed to the 1st bug, especially when using off-heap memory.
   - The new centralized `memoryToCompact()` as part of `CompactOperations` was not properly loading the flags field into the destination compact 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.

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455612832



##########
File path: src/main/java/org/apache/datasketches/theta/CompactOperations.java
##########
@@ -229,10 +233,20 @@ static final Memory loadCompactMemory(
     }
     final byte famID = (byte) Family.COMPACT.getID();
 
+    //Caution: The following loads directly into Memory without creating a heap byte[] first,
+    // which would act as a pre-clearing, initialization mechanism. So it is important to make sure
+    // that all fields are initialized, even those that are not used by the CompactSketch.
+    // Otherwise, uninitialized fields could be filled with off-heap garbage, which could cause
+    // other problems downstream if those fields are not filtered out first.
+    // As written below, all fields are initialized avoiding an extra copy.
+
+    //The first 8 bytes (pre0)
     insertPreLongs(dstMem, preLongs); //RF not used = 0
     insertSerVer(dstMem, SER_VER);
     insertFamilyID(dstMem, famID);
-    //ignore lgNomLongs, lgArrLongs bytes for compact sketches
+    //The following initializes the lgNomLongs and lgArrLongs to 0.
+    //They are not used in CompactSketches.
+    dstMem.putShort(LG_NOM_LONGS_BYTE, (short)0);

Review comment:
       I missed that on first inspection - thanks for explaining.  In the context of a performance critical piece of code, having this low level implementation 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.

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455390315



##########
File path: src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
##########
@@ -333,6 +333,55 @@ public void checkHeapCompactSketchCompact() {
     assertEquals(csk.getCurrentPreambleLongs(), 2);
   }
 
+  @Test
+  public void checkDirectCompactSketchCompact() {
+    WritableMemory wmem1, wmem2;
+    CompactSketch csk1, csk2;
+    int bytes;
+    int lgK = 6;
+
+    //empty
+    UpdateSketch sk = Sketches.updateSketchBuilder().setLogNominalEntries(lgK).build();
+    bytes = sk.getCompactBytes();
+    wmem1 = WritableMemory.allocate(bytes);
+    wmem2 = WritableMemory.allocate(bytes);
+    csk1 = sk.compact(true, wmem1); //place into memory
+    assertTrue(csk1 instanceof DirectCompactSketch);
+    csk2 = csk1.compact(true, wmem2);
+    assertTrue(csk2 instanceof DirectCompactSketch);
+
+    //single
+    sk.update(1);
+    bytes = sk.getCompactBytes();
+    wmem1 = WritableMemory.allocate(bytes);
+    wmem2 = WritableMemory.allocate(bytes);
+    csk1 = sk.compact(true, wmem1); //place into memory
+    assertTrue(csk1 instanceof DirectCompactSketch);
+    csk2 = csk1.compact(true, wmem2);
+    assertTrue(csk2 instanceof DirectCompactSketch);
+
+    //exact
+    sk.update(2);
+    bytes = sk.getCompactBytes();
+    wmem1 = WritableMemory.allocate(bytes);
+    wmem2 = WritableMemory.allocate(bytes);
+    csk1 = sk.compact(true, wmem1); //place into memory
+    assertTrue(csk1 instanceof DirectCompactSketch);
+    csk2 = csk1.compact(true, wmem2);
+    assertTrue(csk2 instanceof DirectCompactSketch);

Review comment:
       What is the significance of the additional value, when compared to a sketch containing a single value?  Considering that lgK = 6, it is clearly still in exact mode, is the intention here to see that the additional value serialises properly into memory?




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-660511373


   This pull request **fixes 17 alerts** when merging dfd3f086c3cde640046f2d27d647c481a2bf45b1 into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-e76a89a6e528f228af3cf19d63d5caa2a5ae0e48)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452148144



##########
File path: src/main/java/org/apache/datasketches/BoundsOnRatiosInSampledSets.java
##########
@@ -82,6 +82,28 @@ public static double getEstimateOfBoverA(final long a, final long b) {
     return (double) b / a;
   }
 
+  /**
+   * Return the estimate of A. See class javadoc.
+   * @param a See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.
+   * @return the approximate lower bound
+   */
+  public static double getEstimateOfA(final long a, final double f) {
+    checkInputs(a, 1, f);
+    return a / f;
+  }
+
+  /**
+   * Return the estimate of B. See class javadoc.
+   * @param b See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.

Review comment:
       Should the size documentation here refer instead to `b`?

##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -24,18 +24,33 @@
 
 /**
  * The API for the set difference operation <i>A and not B</i> operations.
- * This is a stateless operation. However, to make the API
- * more consistent with the other set operations the intended use is:
+ * This class include both stateful and stateless operations.

Review comment:
       There is a small typo here, `include` should be `includes`.

##########
File path: src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
##########
@@ -0,0 +1,495 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EMPTY;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_HEAP;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_MEMORY_UNORDERED;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EXACT;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.NULL;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+import java.util.Random;
+
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@SuppressWarnings({"javadoc","deprecation"})
+public class SetOpsCornerCasesTest {
+
+  /*******************************************/
+  Random rand = new Random(9001); //deterministic
+
+  @Test
+  public void checkSetOpsRandom() {
+    int hiA = 0, loB = 0, hiB = 0;
+    for (int i = 0; i < 1000; i++) {
+      hiA = rand.nextInt(128);      //skA fed values between 0 and 127
+      loB = rand.nextInt(64);
+      hiB = loB + rand.nextInt(64); //skB fed up to 63 values starting at loB
+      compareSetOpsRandom(64, 0, hiA, loB, hiB);
+    }
+  }
+
+  private static void compareSetOpsRandom(int k, int loA, int hiA, int loB, int hiB) {
+    UpdateSketch tskA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+    UpdateSketch tskB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+
+    for (int i = loA; i < hiA; i++) { tskA.update(i); }
+    for (int i = loB; i < hiB; i++) { tskB.update(i); }
+
+    CompactSketch rcskStdU = doStdUnion(tskA, tskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tskA, tskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tskA, tskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tskA, tskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tskA, tskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tskA, tskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tskA, tskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tskA, tskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tskA, tskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  /*******************************************/
+
+  @Test
+  //Check all corner cases against standard Union, Intersection, and AnotB.
+  //The unordered case is not tested
+  public void compareCornerCases() {
+    int k = 64;
+    for (State stateA : State.values()) {
+      for (State stateB : State.values()) {
+        if ((stateA == EST_MEMORY_UNORDERED) || (stateB == EST_MEMORY_UNORDERED)) { continue; }
+        cornerCaseChecks(stateA, stateB, k);
+        //cornerCaseChecksMemory(stateA, stateB, k);

Review comment:
       Did you mean to exclude these the memory corner case checks?

##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -47,55 +62,125 @@ public Family getFamily() {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is a stateful input operation. This method sets the given Sketch as the first
+   * argument <i>A</i> of a stateful <i>AnotB</i> operation. This overwrites the internal state of
+   * this AnotB operator with the contents of the given sketch. This sets the stage for multiple
+   * stateful subsequent {@link #notB(Sketch)} operations. The ultimate result is obtained using
+   * the {@link #getResult(boolean)} or {@link #getResult(boolean, WritableMemory, boolean)}.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * @param skA The incoming sketch for the first argument, <i>A</i>.
    */
-  public abstract CompactSketch getResult();
+  public abstract void setA(Sketch skA);
+
+  /**
+   * Performs a stateful <i>AND NOT</i> operation with the existing internal state of this AnotB
+   * operator. Use {@link #getResult(boolean)} or {@link #getResult(boolean, WritableMemory, boolean)}
+   * to obtain the result.
+   *
+   * <p>An input argument of null or empty is ignored.</p>
+   *
+   * @param skB The incoming sketch for the second (or following) argument <i>B</i>.
+   */
+  public abstract void notB(Sketch skB);
+
+  /**
+   * Gets the result of this operation as an ordered CompactSketch on the Java heap.
+   * @param reset If true, clears this operator to the empty state after result is returned.
+   * @return the result of this operation as a CompactSketch.
+   */
+  public abstract CompactSketch getResult(boolean reset);
 
   /**
-   * Gets the result of this set operation as a CompactSketch of the chosen form
+   * Gets the result of this stateful set operation as a CompactSketch of the chosen form. The

Review comment:
       It's a little unclear what the chosen form refers to - does that refer to the chosen ordering?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -121,6 +121,11 @@
     }
   }
 
+  @Override
+  public CompactSketch<S> compact() {
+    return this;
+  }

Review comment:
       I notice that the Theta CompactSketch exposes a method to query the size of the compacted bytes.  It would be useful to have the same method on a Tuple CompactSketch, if possible.
   It's often necessary for internal monitoring and querying to record the sizes being stored and retrieved.

##########
File path: src/test/java/org/apache/datasketches/theta/HeapAlphaSketchTest.java
##########
@@ -291,9 +290,9 @@ public void checkAlphaToCompactForms() {
     assertEquals(comp4.getUpperBound(2), comp1ub);
     assertEquals(comp4.isEmpty(), false);
     assertEquals(comp4.isEstimationMode(), estimating);
-    assertEquals(comp1bytes, comp4.getCurrentBytes(true)); //flag is not relevant
+    assertEquals(comp1bytes, comp4.getCompactBytes()); //flag is not relevant

Review comment:
       The previous three comments relating to the irrelevance of the flag can now also be removed.

##########
File path: src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
##########
@@ -82,15 +83,15 @@
     }
   }
 
-  IntersectionImplR(final short seedHash) {
-    seedHash_ = seedHash;
-    mem_ = null;
-    lgArrLongs_ = 0;
-    curCount_ = -1;
-    thetaLong_ = Long.MAX_VALUE;
-    empty_ = false;
-    hashTable_ = null;
-  }
+  //  IntersectionImplR(final short seedHash) {
+  //    seedHash_ = seedHash;
+  //    mem_ = null;
+  //    lgArrLongs_ = 0;
+  //    curCount_ = -1;
+  //    thetaLong_ = Long.MAX_VALUE;
+  //    empty_ = false;
+  //    hashTable_ = null;
+  //  }

Review comment:
       Did you mean to remove this constructor?

##########
File path: src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
##########
@@ -171,7 +171,7 @@ public void checkAnotBEarlyStop() {
       CompactSketch csk2 = usk2.compact(true, null);
 
       Sketch rsk = PairwiseSetOperations.aNotB(csk1, csk2);
-      double result1 = rsk.getEstimate();
+      double result1 = rsk.getEstimate(); //null ptr

Review comment:
       What is the significance a possible null pointer here?

##########
File path: src/test/java/org/apache/datasketches/theta/SketchTest.java
##########
@@ -65,50 +65,48 @@ public void checkGetCurrentBytes() {
     int lowQSPreLongs = Family.QUICKSELECT.getMinPreLongs();
     int lowCompPreLongs = Family.COMPACT.getMinPreLongs();
     UpdateSketch sketch = UpdateSketch.builder().setNominalEntries(k).build(); // QS Sketch
-    assertEquals(sketch.getCurrentPreambleLongs(false), lowQSPreLongs);
-    assertEquals(sketch.getCurrentPreambleLongs(true), 1); //compact form
-    assertEquals(sketch.getCurrentDataLongs(false), k*2);
-    assertEquals(sketch.getCurrentDataLongs(true), 0); //compact form
-    assertEquals(sketch.getCurrentBytes(false), (k*2*8) + (lowQSPreLongs << 3));
-    assertEquals(sketch.getCurrentBytes(true), lowCompPreLongs << 3);
+    assertEquals(sketch.getCurrentPreambleLongs(), lowQSPreLongs);
+    assertEquals(sketch.getCompactPreambleLongs(), 1); //compact form
+    assertEquals(sketch.getCurrentDataLongs(), k*2);
+    assertEquals(sketch.getCurrentBytes(), (k*2*8) + (lowQSPreLongs << 3));
+    assertEquals(sketch.getCompactBytes(), lowCompPreLongs << 3);

Review comment:
       Removing the boolean flags from the interface in favour of explicit method names is a nice improvement.

##########
File path: src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
##########
@@ -0,0 +1,495 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EMPTY;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_HEAP;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_MEMORY_UNORDERED;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EXACT;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.NULL;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+import java.util.Random;
+
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@SuppressWarnings({"javadoc","deprecation"})
+public class SetOpsCornerCasesTest {
+
+  /*******************************************/
+  Random rand = new Random(9001); //deterministic
+
+  @Test
+  public void checkSetOpsRandom() {
+    int hiA = 0, loB = 0, hiB = 0;
+    for (int i = 0; i < 1000; i++) {
+      hiA = rand.nextInt(128);      //skA fed values between 0 and 127
+      loB = rand.nextInt(64);
+      hiB = loB + rand.nextInt(64); //skB fed up to 63 values starting at loB
+      compareSetOpsRandom(64, 0, hiA, loB, hiB);
+    }
+  }
+
+  private static void compareSetOpsRandom(int k, int loA, int hiA, int loB, int hiB) {
+    UpdateSketch tskA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+    UpdateSketch tskB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+
+    for (int i = loA; i < hiA; i++) { tskA.update(i); }
+    for (int i = loB; i < hiB; i++) { tskB.update(i); }
+
+    CompactSketch rcskStdU = doStdUnion(tskA, tskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tskA, tskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tskA, tskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tskA, tskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tskA, tskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tskA, tskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tskA, tskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tskA, tskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tskA, tskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  /*******************************************/
+
+  @Test
+  //Check all corner cases against standard Union, Intersection, and AnotB.
+  //The unordered case is not tested
+  public void compareCornerCases() {
+    int k = 64;
+    for (State stateA : State.values()) {
+      for (State stateB : State.values()) {
+        if ((stateA == EST_MEMORY_UNORDERED) || (stateB == EST_MEMORY_UNORDERED)) { continue; }
+        cornerCaseChecks(stateA, stateB, k);
+        //cornerCaseChecksMemory(stateA, stateB, k);
+      }
+    }
+  }
+
+  @Test
+  public void checkExactNullSpecificCase() {
+    cornerCaseChecksMemory(State.EXACT, State.NULL, 64);
+  }
+
+  private static void cornerCaseChecksMemory(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    WritableMemory wmem = WritableMemory.allocate(SetOperation.getMaxUnionBytes(k));
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU); //heap, heap
+
+    rcskStdU = doStdUnion(tcskA, tcskB, k, wmem);
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, wmem);
+    checkCornerCase(rcskStdPairU, rcskStdU); //direct, direct
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxIntersectionBytes(k));
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI); //empty, empty
+
+    rcskStdI = doStdIntersection(tcskA, tcskB, wmem);
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdPairI, rcskStdI); //empty, empty //direct, direct???
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxAnotBResultBytes(k));
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB); //heap, heap
+
+    rcskStdAnotB = doStdAnotB(tcskA, tcskB, wmem);
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB); //direct, heap
+  }
+
+  private static void cornerCaseChecks(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  private static CompactSketch doStdUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    union.update(tskA);
+    union.update(tskB);
+    return union.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    return union.union(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    inter.update(tskA);
+    inter.update(tskB);
+    return inter.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    return inter.intersect(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    return anotb.aNotB(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdStatefulAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    anotb.setA(tskA);
+    anotb.notB(tskB);
+    anotb.getResult(false);
+    return anotb.getResult(true, wmem, true);
+  }
+
+  private static CompactSketch doPwUnion(Sketch tskA, Sketch tskB, int k) {
+    CompactSketch tcskA, tcskB;
+    if (tskA == null) { tcskA = null; }
+    else { tcskA = (tskA instanceof CompactSketch) ? (CompactSketch) tskA : tskA.compact(); }
+    if (tskB == null) { tcskB = null; }
+    else { tcskB = (tskB instanceof CompactSketch) ? (CompactSketch) tskB : tskB.compact(); }
+    return PairwiseSetOperations.union(tcskA, tcskB, k);
+  }
+
+  private static CompactSketch doPwIntersection(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.intersect(tskA, tskB);
+  }
+
+  private static CompactSketch doPwAnotB(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.aNotB(tskA, tskB);
+  }
+
+
+  private static void checkCornerCase(Sketch rskA, Sketch rskB) {
+    double estA = rskA.getEstimate();
+    double estB = rskB.getEstimate();
+    boolean emptyA = rskA.isEmpty();
+    boolean emptyB = rskB.isEmpty();
+    long thetaLongA = rskA.getThetaLong();
+    long thetaLongB = rskB.getThetaLong();
+    int countA = rskA.getRetainedEntries(true);
+    int countB = rskB.getRetainedEntries(true);
+    Assert.assertEquals(estB, estA, 0.0);
+    Assert.assertEquals(emptyB, emptyA);
+    Assert.assertEquals(thetaLongB, thetaLongA);
+    Assert.assertEquals(countB, countA);
+    Assert.assertEquals(rskA.getClass().getSimpleName(), rskB.getClass().getSimpleName());
+  }
+
+  /*******************************************/
+
+  @Test
+  public void checkUnionNotOrdered() {
+    int k = 64;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap = generate(EST_HEAP, k);
+    CompactSketch skHeapUO = generate(EST_MEMORY_UNORDERED, k);
+
+    PairwiseSetOperations.union(skNull, skHeapUO, k);
+    PairwiseSetOperations.union(skEmpty, skHeapUO, k);
+    PairwiseSetOperations.union(skHeapUO, skNull, k);
+    PairwiseSetOperations.union(skHeapUO, skEmpty, k);
+    PairwiseSetOperations.union(skHeapUO, skHeap, k);
+    PairwiseSetOperations.union(skHeap, skHeapUO, k);
+  }
+
+  @Test
+  public void checkSeedHash() {
+    int k = 64;
+    UpdateSketch tmp1 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp1.update(1);
+    tmp1.update(3);
+    CompactSketch skSmallSeed2A = tmp1.compact(true, null);
+
+    UpdateSketch tmp2 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp2.update(1);
+    tmp2.update(2);
+    CompactSketch skSmallSeed2B = tmp2.compact(true, null);
+
+    CompactSketch skExact = generate(EXACT, k);
+    CompactSketch skHeap = generate(EST_HEAP, 2 * k);
+    //Intersect
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //A NOT B
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //Union
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+  }
+
+  @Test
+  public void checkPwUnionReduceToK() {
+    int k = 16;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap1 = generate(EST_HEAP, k);
+    CompactSketch skHeap2 = generate(EST_HEAP, k);
+    CompactSketch csk;
+    csk = PairwiseSetOperations.union(skNull, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skEmpty, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skNull, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skEmpty, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skHeap2, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+  }
+
+
+
+
+  @Test
+  public void printlnTest() {
+    println("PRINTING: "+this.getClass().getName());
+  }
+
+  /**
+   * @param s value to print
+   */
+  static void println(String s) {
+    //System.out.println(s); //disable here
+  }
+
+  @Test
+  public void checkGenerator() {
+    int k = 16;
+    CompactSketch csk;
+
+    csk = generate(State.NULL, 0);
+    assertNull(csk);
+
+    csk = generate(State.EMPTY, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.SINGLE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 1);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EXACT, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), k);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_HEAP, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THLT1_CNT0_FALSE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THEQ1_CNT0_TRUE, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, false);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_MEMORY_UNORDERED, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), true);
+    assertEquals(csk.isOrdered(), false);
+  }
+
+  enum State {NULL, EMPTY, SINGLE, EXACT, EST_HEAP, THLT1_CNT0_FALSE, THEQ1_CNT0_TRUE, EST_MEMORY_UNORDERED}
+
+  private static CompactSketch generate(State state, int k) {
+    UpdateSketch sk = null;
+    CompactSketch csk = null;
+
+    switch(state) {
+      case NULL : {
+        //already null
+        break;
+      }
+      case EMPTY : { //results in EmptyCompactSketch
+        csk = Sketches.updateSketchBuilder().setNominalEntries(k).build().compact(true, null);
+        break;
+      }
+      case SINGLE : { //results in SingleItemSketches most of the time
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        sk.update(1);
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EXACT : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < k; i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EST_HEAP : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case THLT1_CNT0_FALSE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        sk.update(7); //above theta
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, F}
+        break;
+      }
+      case THEQ1_CNT0_TRUE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, T}
+        break;
+      }
+      case EST_MEMORY_UNORDERED : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        int bytes = Sketch.getMaxCompactSketchBytes(sk.getRetainedEntries(true));
+        byte[] byteArr = new byte[bytes];
+        WritableMemory mem = WritableMemory.wrap(byteArr);
+        csk = sk.compact(false, mem);
+        break;
+      }
+    }
+    return csk;
+  }

Review comment:
       Are similar corner case tests necessary for the tuple set operations as well?  I am curious about how they work, and the mechanics of the test seems to compare two sketches for similarity.  What is the significance of comparing pairwise operations to the results of the builder?  Is the former a brute force approach?

##########
File path: src/test/java/org/apache/datasketches/theta/UnionImplTest.java
##########
@@ -54,14 +54,14 @@ public void checkUpdateWithSketch() {
     assertEquals(union.getResult().getEstimate(), k, 0.0);
   }
 
-  @Test(expectedExceptions = SketchesArgumentException.class)
-  public void checkCorruptedCompactFlag() {
+  @Test
+  public void checkUnorderedCompactFlag() {
     int k = 16;
     WritableMemory mem = WritableMemory.wrap(new byte[(k*8) + 24]);
     UpdateSketch sketch = Sketches.updateSketchBuilder().setNominalEntries(k).build();
     for (int i=0; i<k; i++) { sketch.update(i); }
     CompactSketch sketchInDirectOrd = sketch.compact(true, mem);
-    sketch.compact(false, mem); //corrupt memory
+    sketch.compact(false, mem); //change the order bit

Review comment:
       The prior version expected an exception to be thrown when the order bit was changed, is this still the case?

##########
File path: src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
##########
@@ -222,22 +216,80 @@ public void checkMemTooSmallOrdered() {
   @Test
   public void checkCompactCachePart() {
     //phony values except for curCount = 0.
-    long[] result = CompactSketch.compactCachePart(null, 4, 0, 0L, false);
+    long[] result = IntersectionImplR.compactCachePart(null, 4, 0, 0L, false);
     assertEquals(result.length, 0);
   }
 
   @Test
   public void checkDirectCompactSingleItemSketch() {
+    State state;
     UpdateSketch sk = Sketches.updateSketchBuilder().build();
-    CompactSketch csk = sk.compact(true, WritableMemory.allocate(16));
-    int bytes = csk.getCurrentBytes(true);
-    assertEquals(bytes, 8);
+
+    CompactSketch csko; //ordered
+    CompactSketch csku; //unordered
+
+    WritableMemory wmem = WritableMemory.allocate(16);
+    csko = sk.compact(true, wmem); //empty, direct, ordered
+    //ClassType, Count, Bytes, Compact, Empty, Direct, Memory, Ordered, Estimation
+    state = new State("DirectCompactSketch", 0, 8, true, true, false, true, true, false);
+    state.check(csko);
+
+    wmem = WritableMemory.allocate(16);
+    csku = sk.compact(false, wmem); //empty, direct, unordered
+    state = new State("DirectCompactSketch", 0, 8, true, true, false, true, true, false);
+    state.check(csku);
+
     sk.update(1);
-    csk = sk.compact(true, WritableMemory.allocate(16));
-    bytes = csk.getCurrentBytes(true);
-    assertEquals(bytes, 16);
-    assertTrue(csk == csk.compact());
-    assertTrue(csk == csk.compact(true, null));
+    wmem = WritableMemory.allocate(16);
+    csko = sk.compact(true, wmem); //Single, direct, ordered
+    state = new State("DirectCompactSketch", 1, 16, true, false, false, true, true, false);
+    state.check(csko);
+
+    wmem = WritableMemory.allocate(16);
+    csku = sk.compact(false, wmem); //Single, direct, unordered
+    state = new State("DirectCompactSketch", 1, 16, true, false, false, true, true, false);
+    state.check(csku);
+
+    CompactSketch csk2o; //ordered
+    CompactSketch csk2u; //unordered
+
+    csk2o = csku.compact(); //single, heap, ordered
+    state = new State("SingleItemSketch", 1, 16, true, false, false, false, true, false);
+    state.check(csk2o);
+
+    csk2o = csku.compact(true, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csku.compact(false, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csko.compact(true, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csko.compact(false, null); //single, heap, ordered

Review comment:
       Lines 263 and 26 should read `unordered` due to the first argument = false.




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657111212


   This pull request **fixes 17 alerts** when merging 9ee4403d7c6c26de3b69ad7ab5ee93058be19c23 into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-ad99315655c6699fd4d86382006c4eba4a93fcc1)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455403972



##########
File path: src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
##########
@@ -128,20 +128,19 @@ static HeapAlphaSketch heapifyInstance(final Memory srcMem, final long seed) {
     checkMemIntegrity(srcMem, seed, preambleLongs, lgNomLongs, lgArrLongs);
 
     final float p = extractP(srcMem);                             //bytes 12-15
-    final int lgRF = extractLgResizeFactor(srcMem);               //byte 0
-    final ResizeFactor myRF = ResizeFactor.getRF(lgRF);
+    final int memlgRF = extractLgResizeFactor(srcMem);            //byte 0
+    final ResizeFactor memRF = ResizeFactor.getRF(memlgRF);
 
     final double nomLongs = (1L << lgNomLongs);
     final double alpha = nomLongs / (nomLongs + 1.0);
     final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * LONG_MAX_VALUE_AS_DOUBLE);
 
-    if ((myRF == ResizeFactor.X1)
-            && (lgArrLongs != startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
-      throw new SketchesArgumentException("Possible corruption: ResizeFactor X1, but provided "
-              + "array too small for sketch size");
+    if (isResizeFactorIncorrect(srcMem, lgNomLongs, lgArrLongs)) {
+      throw new SketchesArgumentException("Possible corruption: ResizeFactor  "
+          + "inconsistent with lgNomLongs and lgArrLongs.");

Review comment:
       Yes.  Look more closely at the object being modified in the case where X2 is set.  It is a WritableMemory.  In the case where  I have to throw, it is just Memory, which is immutable.  Big difference.




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-659078494


   This pull request **fixes 17 alerts** when merging 46451a20d5b8e745657deb4310b7173718277a6f into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-1b191141324c5dee51e8a38151e064523e8a2209)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452902752



##########
File path: src/main/java/org/apache/datasketches/BoundsOnRatiosInSampledSets.java
##########
@@ -82,6 +82,28 @@ public static double getEstimateOfBoverA(final long a, final long b) {
     return (double) b / a;
   }
 
+  /**
+   * Return the estimate of A. See class javadoc.
+   * @param a See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.
+   * @return the approximate lower bound
+   */
+  public static double getEstimateOfA(final long a, final double f) {
+    checkInputs(a, 1, f);
+    return a / f;
+  }
+
+  /**
+   * Return the estimate of B. See class javadoc.
+   * @param b See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.

Review comment:
       Thank you for addressing my concerns about serialisation - your answer also cleared my misconceptions about the library version being tied to the serialisation.




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452628457



##########
File path: src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
##########
@@ -82,15 +83,15 @@
     }
   }
 
-  IntersectionImplR(final short seedHash) {
-    seedHash_ = seedHash;
-    mem_ = null;
-    lgArrLongs_ = 0;
-    curCount_ = -1;
-    thetaLong_ = Long.MAX_VALUE;
-    empty_ = false;
-    hashTable_ = null;
-  }
+  //  IntersectionImplR(final short seedHash) {
+  //    seedHash_ = seedHash;
+  //    mem_ = null;
+  //    lgArrLongs_ = 0;
+  //    curCount_ = -1;
+  //    thetaLong_ = Long.MAX_VALUE;
+  //    empty_ = false;
+  //    hashTable_ = null;
+  //  }

Review comment:
       Yes.  It was placed there at one time when we thought we needed it.  But nothing calls it, it is dead code.  I left it there in case I thought of another reason to put it back.




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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452913393



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -121,6 +121,11 @@
     }
   }
 
+  @Override
+  public CompactSketch<S> compact() {
+    return this;
+  }

Review comment:
       Great - just to add some detail - it is possible now to compute this by serialising the sketch and measuring the resulting byte size.  However, this is obviously wasteful and my question above is only predicated on whether the internal bytes are already available.




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455403972



##########
File path: src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
##########
@@ -128,20 +128,19 @@ static HeapAlphaSketch heapifyInstance(final Memory srcMem, final long seed) {
     checkMemIntegrity(srcMem, seed, preambleLongs, lgNomLongs, lgArrLongs);
 
     final float p = extractP(srcMem);                             //bytes 12-15
-    final int lgRF = extractLgResizeFactor(srcMem);               //byte 0
-    final ResizeFactor myRF = ResizeFactor.getRF(lgRF);
+    final int memlgRF = extractLgResizeFactor(srcMem);            //byte 0
+    final ResizeFactor memRF = ResizeFactor.getRF(memlgRF);
 
     final double nomLongs = (1L << lgNomLongs);
     final double alpha = nomLongs / (nomLongs + 1.0);
     final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * LONG_MAX_VALUE_AS_DOUBLE);
 
-    if ((myRF == ResizeFactor.X1)
-            && (lgArrLongs != startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
-      throw new SketchesArgumentException("Possible corruption: ResizeFactor X1, but provided "
-              + "array too small for sketch size");
+    if (isResizeFactorIncorrect(srcMem, lgNomLongs, lgArrLongs)) {
+      throw new SketchesArgumentException("Possible corruption: ResizeFactor  "
+          + "inconsistent with lgNomLongs and lgArrLongs.");

Review comment:
       The DirectQuickSelectSketch input is a WritableMemory srcMem, and the operation is a WritableWrap, which allows the sketch to be **operated on** while in off-heap memory.  Here I clearly want to make sure the ResizeFactor is correct in case I need to continue to grow the sketch off-heap.
   
   In the HeapQuickSelectSketch the operation is heapifyInstance which means I am putting a sketch that was off-heap onto the Java heap, thus discarding the srcMem object.  Here I only need to make sure I have a working value of ResizeFactor in case I need to continue growing the sketch on-heap.
   
   In the HeapAlphaSketch I could have used the same strategy as the HeapQuickSelectSketch, instead of throwing.
   
   This is a very observant catch!  Thank you!




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452628909



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -121,6 +121,11 @@
     }
   }
 
+  @Override
+  public CompactSketch<S> compact() {
+    return this;
+  }

Review comment:
       In order to do that we would have to add a method getSerializationBytes() or equivalent to the Summary Interface.  This is not a bad idea. Let me think about it.  




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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455396253



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -121,6 +121,11 @@
     }
   }
 
+  @Override
+  public CompactSketch<S> compact() {
+    return this;
+  }

Review comment:
       I have looked through the additional changes, and yes, all the rest of the points raised were resolved.  Thank you for addressing them.  I have also read through the code related to the bug fixes mentioned and the additional commentary on clearing memory is valuable.
   
   Concerning discussion thread here, I am interested in how the summary may be restricted.  Were you considering adding an additional method to the summary type to provide the size for the tuple?  
   If it is more involved, I am happy to pick this up on [DISCUSS] or not let this block the PR.




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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455378653



##########
File path: src/main/java/org/apache/datasketches/theta/CompactOperations.java
##########
@@ -229,10 +233,20 @@ static final Memory loadCompactMemory(
     }
     final byte famID = (byte) Family.COMPACT.getID();
 
+    //Caution: The following loads directly into Memory without creating a heap byte[] first,
+    // which would act as a pre-clearing, initialization mechanism. So it is important to make sure
+    // that all fields are initialized, even those that are not used by the CompactSketch.
+    // Otherwise, uninitialized fields could be filled with off-heap garbage, which could cause
+    // other problems downstream if those fields are not filtered out first.
+    // As written below, all fields are initialized avoiding an extra copy.
+
+    //The first 8 bytes (pre0)
     insertPreLongs(dstMem, preLongs); //RF not used = 0
     insertSerVer(dstMem, SER_VER);
     insertFamilyID(dstMem, famID);
-    //ignore lgNomLongs, lgArrLongs bytes for compact sketches
+    //The following initializes the lgNomLongs and lgArrLongs to 0.
+    //They are not used in CompactSketches.
+    dstMem.putShort(LG_NOM_LONGS_BYTE, (short)0);

Review comment:
       Does clearing LG_NOM_LONGS_BYTE to zero have the effect of clearing both lgNomLongs and lgArrLongs?  From the preamble output, it appeared that they were two separate fields, and I didn't see an explicit byte cleared for lgArrLongs.




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

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] [incubator-datasketches-java] davecromberge commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657119735


   Lee, I've just done a fresh checkout of this branch on my machine, and run this command in my terminal:
   
   `mvn clean compile test -Dgpg.skip=true`
   
   I get the same failure as the build server.  I am using Adopt OpenJDK version 8.0.252.j9-adpt.  I'll try and open in my IDE and see if I can also reproduce it there.


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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455383769



##########
File path: src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
##########
@@ -128,20 +128,19 @@ static HeapAlphaSketch heapifyInstance(final Memory srcMem, final long seed) {
     checkMemIntegrity(srcMem, seed, preambleLongs, lgNomLongs, lgArrLongs);
 
     final float p = extractP(srcMem);                             //bytes 12-15
-    final int lgRF = extractLgResizeFactor(srcMem);               //byte 0
-    final ResizeFactor myRF = ResizeFactor.getRF(lgRF);
+    final int memlgRF = extractLgResizeFactor(srcMem);            //byte 0
+    final ResizeFactor memRF = ResizeFactor.getRF(memlgRF);
 
     final double nomLongs = (1L << lgNomLongs);
     final double alpha = nomLongs / (nomLongs + 1.0);
     final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * LONG_MAX_VALUE_AS_DOUBLE);
 
-    if ((myRF == ResizeFactor.X1)
-            && (lgArrLongs != startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
-      throw new SketchesArgumentException("Possible corruption: ResizeFactor X1, but provided "
-              + "array too small for sketch size");
+    if (isResizeFactorIncorrect(srcMem, lgNomLongs, lgArrLongs)) {
+      throw new SketchesArgumentException("Possible corruption: ResizeFactor  "
+          + "inconsistent with lgNomLongs and lgArrLongs.");

Review comment:
       This is a different approach taken than that of the DirectQuickSelect sketch, where X2 is selected as a resize factor when the arguments are incorrect, as per the comment:
   ```//If incorrect it sets it to X2 which always works.```
   Is this by design?
   




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657155515


   This pull request **fixes 17 alerts** when merging 6b6448201515c2e3909e59248f37fd0ed866d672 into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-d9375134ca75394e0e5ebfa41bc9821bd62a7e9d)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452904819



##########
File path: src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
##########
@@ -0,0 +1,495 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EMPTY;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_HEAP;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_MEMORY_UNORDERED;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EXACT;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.NULL;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+import java.util.Random;
+
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@SuppressWarnings({"javadoc","deprecation"})
+public class SetOpsCornerCasesTest {
+
+  /*******************************************/
+  Random rand = new Random(9001); //deterministic
+
+  @Test
+  public void checkSetOpsRandom() {
+    int hiA = 0, loB = 0, hiB = 0;
+    for (int i = 0; i < 1000; i++) {
+      hiA = rand.nextInt(128);      //skA fed values between 0 and 127
+      loB = rand.nextInt(64);
+      hiB = loB + rand.nextInt(64); //skB fed up to 63 values starting at loB
+      compareSetOpsRandom(64, 0, hiA, loB, hiB);
+    }
+  }
+
+  private static void compareSetOpsRandom(int k, int loA, int hiA, int loB, int hiB) {
+    UpdateSketch tskA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+    UpdateSketch tskB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+
+    for (int i = loA; i < hiA; i++) { tskA.update(i); }
+    for (int i = loB; i < hiB; i++) { tskB.update(i); }
+
+    CompactSketch rcskStdU = doStdUnion(tskA, tskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tskA, tskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tskA, tskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tskA, tskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tskA, tskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tskA, tskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tskA, tskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tskA, tskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tskA, tskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  /*******************************************/
+
+  @Test
+  //Check all corner cases against standard Union, Intersection, and AnotB.
+  //The unordered case is not tested
+  public void compareCornerCases() {
+    int k = 64;
+    for (State stateA : State.values()) {
+      for (State stateB : State.values()) {
+        if ((stateA == EST_MEMORY_UNORDERED) || (stateB == EST_MEMORY_UNORDERED)) { continue; }
+        cornerCaseChecks(stateA, stateB, k);
+        //cornerCaseChecksMemory(stateA, stateB, k);

Review comment:
       No I did not.  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.

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r453223984



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -121,6 +121,11 @@
     }
   }
 
+  @Override
+  public CompactSketch<S> compact() {
+    return this;
+  }

Review comment:
       This is the only item that is not resolved yet.  




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452908794



##########
File path: src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
##########
@@ -0,0 +1,495 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EMPTY;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_HEAP;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_MEMORY_UNORDERED;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EXACT;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.NULL;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+import java.util.Random;
+
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@SuppressWarnings({"javadoc","deprecation"})
+public class SetOpsCornerCasesTest {
+
+  /*******************************************/
+  Random rand = new Random(9001); //deterministic
+
+  @Test
+  public void checkSetOpsRandom() {
+    int hiA = 0, loB = 0, hiB = 0;
+    for (int i = 0; i < 1000; i++) {
+      hiA = rand.nextInt(128);      //skA fed values between 0 and 127
+      loB = rand.nextInt(64);
+      hiB = loB + rand.nextInt(64); //skB fed up to 63 values starting at loB
+      compareSetOpsRandom(64, 0, hiA, loB, hiB);
+    }
+  }
+
+  private static void compareSetOpsRandom(int k, int loA, int hiA, int loB, int hiB) {
+    UpdateSketch tskA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+    UpdateSketch tskB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+
+    for (int i = loA; i < hiA; i++) { tskA.update(i); }
+    for (int i = loB; i < hiB; i++) { tskB.update(i); }
+
+    CompactSketch rcskStdU = doStdUnion(tskA, tskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tskA, tskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tskA, tskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tskA, tskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tskA, tskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tskA, tskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tskA, tskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tskA, tskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tskA, tskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  /*******************************************/
+
+  @Test
+  //Check all corner cases against standard Union, Intersection, and AnotB.
+  //The unordered case is not tested
+  public void compareCornerCases() {
+    int k = 64;
+    for (State stateA : State.values()) {
+      for (State stateB : State.values()) {
+        if ((stateA == EST_MEMORY_UNORDERED) || (stateB == EST_MEMORY_UNORDERED)) { continue; }
+        cornerCaseChecks(stateA, stateB, k);
+        //cornerCaseChecksMemory(stateA, stateB, k);
+      }
+    }
+  }
+
+  @Test
+  public void checkExactNullSpecificCase() {
+    cornerCaseChecksMemory(State.EXACT, State.NULL, 64);
+  }
+
+  private static void cornerCaseChecksMemory(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    WritableMemory wmem = WritableMemory.allocate(SetOperation.getMaxUnionBytes(k));
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU); //heap, heap
+
+    rcskStdU = doStdUnion(tcskA, tcskB, k, wmem);
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, wmem);
+    checkCornerCase(rcskStdPairU, rcskStdU); //direct, direct
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxIntersectionBytes(k));
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI); //empty, empty
+
+    rcskStdI = doStdIntersection(tcskA, tcskB, wmem);
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdPairI, rcskStdI); //empty, empty //direct, direct???
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxAnotBResultBytes(k));
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB); //heap, heap
+
+    rcskStdAnotB = doStdAnotB(tcskA, tcskB, wmem);
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB); //direct, heap
+  }
+
+  private static void cornerCaseChecks(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  private static CompactSketch doStdUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    union.update(tskA);
+    union.update(tskB);
+    return union.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    return union.union(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    inter.update(tskA);
+    inter.update(tskB);
+    return inter.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    return inter.intersect(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    return anotb.aNotB(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdStatefulAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    anotb.setA(tskA);
+    anotb.notB(tskB);
+    anotb.getResult(false);
+    return anotb.getResult(true, wmem, true);
+  }
+
+  private static CompactSketch doPwUnion(Sketch tskA, Sketch tskB, int k) {
+    CompactSketch tcskA, tcskB;
+    if (tskA == null) { tcskA = null; }
+    else { tcskA = (tskA instanceof CompactSketch) ? (CompactSketch) tskA : tskA.compact(); }
+    if (tskB == null) { tcskB = null; }
+    else { tcskB = (tskB instanceof CompactSketch) ? (CompactSketch) tskB : tskB.compact(); }
+    return PairwiseSetOperations.union(tcskA, tcskB, k);
+  }
+
+  private static CompactSketch doPwIntersection(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.intersect(tskA, tskB);
+  }
+
+  private static CompactSketch doPwAnotB(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.aNotB(tskA, tskB);
+  }
+
+
+  private static void checkCornerCase(Sketch rskA, Sketch rskB) {
+    double estA = rskA.getEstimate();
+    double estB = rskB.getEstimate();
+    boolean emptyA = rskA.isEmpty();
+    boolean emptyB = rskB.isEmpty();
+    long thetaLongA = rskA.getThetaLong();
+    long thetaLongB = rskB.getThetaLong();
+    int countA = rskA.getRetainedEntries(true);
+    int countB = rskB.getRetainedEntries(true);
+    Assert.assertEquals(estB, estA, 0.0);
+    Assert.assertEquals(emptyB, emptyA);
+    Assert.assertEquals(thetaLongB, thetaLongA);
+    Assert.assertEquals(countB, countA);
+    Assert.assertEquals(rskA.getClass().getSimpleName(), rskB.getClass().getSimpleName());
+  }
+
+  /*******************************************/
+
+  @Test
+  public void checkUnionNotOrdered() {
+    int k = 64;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap = generate(EST_HEAP, k);
+    CompactSketch skHeapUO = generate(EST_MEMORY_UNORDERED, k);
+
+    PairwiseSetOperations.union(skNull, skHeapUO, k);
+    PairwiseSetOperations.union(skEmpty, skHeapUO, k);
+    PairwiseSetOperations.union(skHeapUO, skNull, k);
+    PairwiseSetOperations.union(skHeapUO, skEmpty, k);
+    PairwiseSetOperations.union(skHeapUO, skHeap, k);
+    PairwiseSetOperations.union(skHeap, skHeapUO, k);
+  }
+
+  @Test
+  public void checkSeedHash() {
+    int k = 64;
+    UpdateSketch tmp1 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp1.update(1);
+    tmp1.update(3);
+    CompactSketch skSmallSeed2A = tmp1.compact(true, null);
+
+    UpdateSketch tmp2 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp2.update(1);
+    tmp2.update(2);
+    CompactSketch skSmallSeed2B = tmp2.compact(true, null);
+
+    CompactSketch skExact = generate(EXACT, k);
+    CompactSketch skHeap = generate(EST_HEAP, 2 * k);
+    //Intersect
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //A NOT B
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //Union
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+  }
+
+  @Test
+  public void checkPwUnionReduceToK() {
+    int k = 16;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap1 = generate(EST_HEAP, k);
+    CompactSketch skHeap2 = generate(EST_HEAP, k);
+    CompactSketch csk;
+    csk = PairwiseSetOperations.union(skNull, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skEmpty, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skNull, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skEmpty, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skHeap2, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+  }
+
+
+
+
+  @Test
+  public void printlnTest() {
+    println("PRINTING: "+this.getClass().getName());
+  }
+
+  /**
+   * @param s value to print
+   */
+  static void println(String s) {
+    //System.out.println(s); //disable here
+  }
+
+  @Test
+  public void checkGenerator() {
+    int k = 16;
+    CompactSketch csk;
+
+    csk = generate(State.NULL, 0);
+    assertNull(csk);
+
+    csk = generate(State.EMPTY, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.SINGLE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 1);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EXACT, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), k);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_HEAP, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THLT1_CNT0_FALSE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THEQ1_CNT0_TRUE, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, false);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_MEMORY_UNORDERED, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), true);
+    assertEquals(csk.isOrdered(), false);
+  }
+
+  enum State {NULL, EMPTY, SINGLE, EXACT, EST_HEAP, THLT1_CNT0_FALSE, THEQ1_CNT0_TRUE, EST_MEMORY_UNORDERED}
+
+  private static CompactSketch generate(State state, int k) {
+    UpdateSketch sk = null;
+    CompactSketch csk = null;
+
+    switch(state) {
+      case NULL : {
+        //already null
+        break;
+      }
+      case EMPTY : { //results in EmptyCompactSketch
+        csk = Sketches.updateSketchBuilder().setNominalEntries(k).build().compact(true, null);
+        break;
+      }
+      case SINGLE : { //results in SingleItemSketches most of the time
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        sk.update(1);
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EXACT : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < k; i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EST_HEAP : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case THLT1_CNT0_FALSE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        sk.update(7); //above theta
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, F}
+        break;
+      }
+      case THEQ1_CNT0_TRUE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, T}
+        break;
+      }
+      case EST_MEMORY_UNORDERED : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        int bytes = Sketch.getMaxCompactSketchBytes(sk.getRetainedEntries(true));
+        byte[] byteArr = new byte[bytes];
+        WritableMemory mem = WritableMemory.wrap(byteArr);
+        csk = sk.compact(false, mem);
+        break;
+      }
+    }
+    return csk;
+  }

Review comment:
       Actually we do.  But the corner-case tests are executed via one of the concrete implementations and in this case in the `../tuple/adouble` package.  And because of the way generics work, we only need to exercise these tests from one of the concrete implementations.  The way I wrote them in the `adouble`package is probably a better way to do it, and I haven't rewritten the tests in theta yet to take advantage of 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.

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455369268



##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -47,55 +62,174 @@ public Family getFamily() {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is part of a multistep, stateful AnotB operation and sets the given Tuple sketch as the
+   * first argument <i>A</i> of <i>A-AND-NOT-B</i>. This overwrites the internal state of this
+   * AnotB operator with the contents of the given sketch.
+   * This sets the stage for multiple following <i>notB</i> steps.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * <p>Rationale: In mathematics a "null set" is a set with no members, which we call an empty set.
+   * That is distinctly different from the java <i>null</i>, which represents a nonexistant object.
+   * In most cases it is a programming error due to some object that was not properly initialized.
+   * With a null as the first argument, we cannot know what the user's intent is.
+   * Since it is very likely that a <i>null</i> is a programming error, we throw a an exception.</p>
+   *
+   * <p>An enpty input argument will set the internal state to empty.</p>
+   *
+   * <p>Rationale: An empty set is a mathematically legal concept. Although it makes any subsequent,
+   * valid argument for B irrelvant, we must allow this and assume the user knows what they are
+   * doing.</p>
+   *
+   * <p>Performing {@link #getResult(boolean)} just after this step will return a compact form of
+   * the given argument.</p>
+   *
+   * @param skA The incoming sketch for the first argument, <i>A</i>.
    */
-  public abstract CompactSketch getResult();
+  public abstract void setA(Sketch skA);
 
   /**
-   * Gets the result of this set operation as a CompactSketch of the chosen form
-   * @param dstOrdered
-   * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>.
+   * This is part of a multistep, stateful AnotB operation and sets the given Tuple sketch as the
+   * second (or <i>n+1</i>th) argument <i>B</i> of <i>A-AND-NOT-B</i>.
+   * Performs an <i>AND NOT</i> operation with the existing internal state of this AnotB operator.
    *
-   * @param dstMem
-   * <a href="{@docRoot}/resources/dictionary.html#dstMem">See Destination Memory</a>.
+   * <p>An input argument of null or empty is ignored.</p>
+   *
+   * <p>Rationale: A <i>null</i> for the second or following arguments is more tollerable because
+   * <i>A NOT null</i> is still <i>A</i> even if we don't know exactly what the null represents. It
+   * clearly does not have any content that overlaps with <i>A</i>. Also, because this can be part of
+   * a multistep operation with multiple <i>notB</i> steps. Other following steps can still produce
+   * a valid result.</p>
+   *
+   * <p>Use {@link #getResult(boolean)} to obtain the result.</p>
    *
-   * @return the result of this set operation as a CompactSketch of the chosen form
+   * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
-  public abstract CompactSketch getResult(boolean dstOrdered, WritableMemory dstMem);
+  public abstract void notB(Sketch skB);

Review comment:
       This additional documentation is great, especially the clarity provided for exception cases vs legal empty sets.  There are two references to a Tuple sketch, perhaps you mean `Theta` instead?




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

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] [incubator-datasketches-java] AlexanderSaydakov commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455458636



##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -47,55 +62,174 @@ public Family getFamily() {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is part of a multistep, stateful AnotB operation and sets the given Theta sketch as the
+   * first argument <i>A</i> of <i>A-AND-NOT-B</i>. This overwrites the internal state of this
+   * AnotB operator with the contents of the given sketch.
+   * This sets the stage for multiple following <i>notB</i> steps.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * <p>Rationale: In mathematics a "null set" is a set with no members, which we call an empty set.
+   * That is distinctly different from the java <i>null</i>, which represents a nonexistant object.
+   * In most cases it is a programming error due to some object that was not properly initialized.
+   * With a null as the first argument, we cannot know what the user's intent is.
+   * Since it is very likely that a <i>null</i> is a programming error, we throw a an exception.</p>
+   *
+   * <p>An enpty input argument will set the internal state to empty.</p>
+   *
+   * <p>Rationale: An empty set is a mathematically legal concept. Although it makes any subsequent,
+   * valid argument for B irrelvant, we must allow this and assume the user knows what they are
+   * doing.</p>
+   *
+   * <p>Performing {@link #getResult(boolean)} just after this step will return a compact form of
+   * the given argument.</p>
+   *
+   * @param skA The incoming sketch for the first argument, <i>A</i>.
    */
-  public abstract CompactSketch getResult();
+  public abstract void setA(Sketch skA);
 
   /**
-   * Gets the result of this set operation as a CompactSketch of the chosen form
-   * @param dstOrdered
-   * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>.
+   * This is part of a multistep, stateful AnotB operation and sets the given Theta sketch as the
+   * second (or <i>n+1</i>th) argument <i>B</i> of <i>A-AND-NOT-B</i>.
+   * Performs an <i>AND NOT</i> operation with the existing internal state of this AnotB operator.
    *
-   * @param dstMem
-   * <a href="{@docRoot}/resources/dictionary.html#dstMem">See Destination Memory</a>.
+   * <p>An input argument of null or empty is ignored.</p>
+   *
+   * <p>Rationale: A <i>null</i> for the second or following arguments is more tollerable because
+   * <i>A NOT null</i> is still <i>A</i> even if we don't know exactly what the null represents. It
+   * clearly does not have any content that overlaps with <i>A</i>. Also, because this can be part of
+   * a multistep operation with multiple <i>notB</i> steps. Other following steps can still produce
+   * a valid result.</p>
+   *
+   * <p>Use {@link #getResult(boolean)} to obtain the result.</p>
    *
-   * @return the result of this set operation as a CompactSketch of the chosen form
+   * @param skB The incoming Theta sketch for the second (or following) argument <i>B</i>.
    */
-  public abstract CompactSketch getResult(boolean dstOrdered, WritableMemory dstMem);
+  public abstract void notB(Sketch skB);
 
   /**
-   * Perform A-and-not-B set operation on the two given sketches.
-   * A null sketch is interpreted as an empty sketch.
+   * Gets the result of the mutistep, stateful operation AnotB that have been executed with calls
+   * to {@link #setA(Sketch)} and ({@link #notB(Sketch)} or
+   * {@link #notB(org.apache.datasketches.theta.Sketch)}).
+   *
+   * @param reset If <i>true</i>, clears this operator to the empty state after this result is
+   * returned. Set this to <i>false</i> if you wish to obtain an intermediate result.
+   * @return the result of this operation as an ordered, on-heap {@link CompactSketch}.
+   */
+  public abstract CompactSketch getResult(boolean reset);
+
+  /**
+   * Gets the result of this stateful set operation as a CompactSketch of the form based on
+   * the input arguments.
+   * The stateful input operations are {@link #setA(Sketch)} and {@link #notB(Sketch)}.
+   *
+   * @param dstOrdered If <i>true</i>, the result will be an ordered {@link CompactSketch}.
+   * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>.
+   *
+   * @param dstMem if not <i>null</i> the given Memory will be the target location of the result.
+   * <a href="{@docRoot}/resources/dictionary.html#dstMem">See Destination Memory</a>.
    *
-   * @param a The incoming sketch for the first argument
-   * @param b The incoming sketch for the second argument
+   * @param reset If <i>true</i>, clears this operator to the empty state after this result is
+   * returned. Set this to <i>false</i> if you wish to obtain an intermediate result.
+   *
+   * @return the result of this operation as a {@link CompactSketch} of the chosen form.
    */
-  public abstract void update(Sketch a, Sketch b);
+  public abstract CompactSketch getResult(boolean dstOrdered, WritableMemory dstMem, boolean reset);
 
   /**
    * Perform A-and-not-B set operation on the two given sketches and return the result as an
    * ordered CompactSketch on the heap.
-   * @param a The incoming sketch for the first argument
-   * @param b The incoming sketch for the second argument
+   *
+   * <p>This a stateless operation and has no impact on the internal state of this operator.
+   * Thus, this is not an accumulating update and does not interact with the {@link #setA(Sketch)},
+   * {@link #notB(Sketch)}, {@link #getResult(boolean)}, or
+   * {@link #getResult(boolean, WritableMemory, boolean)} methods.</p>
+   *
+   * <p>If either argument is null an exception is thrown.</p>
+   *
+   * <p>Rationale: In mathematics a "null set" is a set with no members, which we call an empty set.
+   * That is distinctly different from the java <i>null</i>, which represents a nonexistant object.
+   * In most cases it is a programming error due to some object that was not properly initialized.
+   * With a null as the first argument, we cannot know what the user's intent is.
+   * With a null as the second argument, we can't ignore it as we must return a result and there is
+   * no following possible viable arguments for the second argument.
+   * Since it is very likely that a <i>null</i> is a programming error, we throw a an exception.</p>
+   *
+   * @param skA The incoming sketch for the first argument.
+   * @param skB The incoming sketch for the second argument.
    * @return an ordered CompactSketch on the heap
    */
-  public CompactSketch aNotB(final Sketch a, final Sketch b) {
-    return aNotB(a, b, true, null);
+  public CompactSketch aNotB(final Sketch skA, final Sketch skB) {
+    return aNotB(skA, skB, true, null);
   }
 
   /**
    * Perform A-and-not-B set operation on the two given sketches and return the result as a
    * CompactSketch.
-   * @param a The incoming sketch for the first argument
-   * @param b The incoming sketch for the second argument
+   *
+   * <p>This a stateless operation and has no impact on the internal state of this operator.
+   * Thus, this is not an accumulating update and does not interact with the {@link #setA(Sketch)},
+   * {@link #notB(Sketch)}, {@link #getResult(boolean)}, or
+   * {@link #getResult(boolean, WritableMemory, boolean)} methods.</p>
+   *
+   * <p>If either argument is null an exception is thrown.</p>
+   *
+   * <p>Rationale: In mathematics a "null set" is a set with no members, which we call an empty set.
+   * That is distinctly different from the java <i>null</i>, which represents a nonexistant object.

Review comment:
       nonexistent




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455402662



##########
File path: src/main/java/org/apache/datasketches/theta/CompactOperations.java
##########
@@ -229,10 +233,20 @@ static final Memory loadCompactMemory(
     }
     final byte famID = (byte) Family.COMPACT.getID();
 
+    //Caution: The following loads directly into Memory without creating a heap byte[] first,
+    // which would act as a pre-clearing, initialization mechanism. So it is important to make sure
+    // that all fields are initialized, even those that are not used by the CompactSketch.
+    // Otherwise, uninitialized fields could be filled with off-heap garbage, which could cause
+    // other problems downstream if those fields are not filtered out first.
+    // As written below, all fields are initialized avoiding an extra copy.
+
+    //The first 8 bytes (pre0)
     insertPreLongs(dstMem, preLongs); //RF not used = 0
     insertSerVer(dstMem, SER_VER);
     insertFamilyID(dstMem, famID);
-    //ignore lgNomLongs, lgArrLongs bytes for compact sketches
+    //The following initializes the lgNomLongs and lgArrLongs to 0.
+    //They are not used in CompactSketches.
+    dstMem.putShort(LG_NOM_LONGS_BYTE, (short)0);

Review comment:
       lgNomLongs is a byte followed by lgArrLongs, which is also a byte.  2 bytes = 1 short.  so clearing the address of lgNomLongs with a short clears both.  The power of Memory (and its danger, if you don't know what you are doing) is that it acts like a combination of C struct and C union so that you can write 8 bytes as a byte[] and then read it back as a long or a double. 
   




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657149160


   This pull request **fixes 17 alerts** when merging 3df81f65ac001df2afc178c488eea06010c581a1 into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-e31623022681d34e840e1297864ee6baea56adbf)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r456807013



##########
File path: src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
##########
@@ -207,11 +207,11 @@ public SetOperation build(final Family family, final WritableMemory dstMem) {
       }
       case A_NOT_B: {
         if (dstMem == null) {
-          setOp = new HeapAnotB(bSeed);
+          setOp = new AnotBimpl(bSeed);
         }
         else {
-          throw new SketchesArgumentException(
-            "AnotB is a stateless operation and cannot be persisted.");
+          throw new SketchesArgumentException( //TODO we should be able to do this now.

Review comment:
       Removed the TODO, but still not allowing AnotB to be persisted.  




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

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] [incubator-datasketches-java] leerho merged pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho merged pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324


   


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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455402662



##########
File path: src/main/java/org/apache/datasketches/theta/CompactOperations.java
##########
@@ -229,10 +233,20 @@ static final Memory loadCompactMemory(
     }
     final byte famID = (byte) Family.COMPACT.getID();
 
+    //Caution: The following loads directly into Memory without creating a heap byte[] first,
+    // which would act as a pre-clearing, initialization mechanism. So it is important to make sure
+    // that all fields are initialized, even those that are not used by the CompactSketch.
+    // Otherwise, uninitialized fields could be filled with off-heap garbage, which could cause
+    // other problems downstream if those fields are not filtered out first.
+    // As written below, all fields are initialized avoiding an extra copy.
+
+    //The first 8 bytes (pre0)
     insertPreLongs(dstMem, preLongs); //RF not used = 0
     insertSerVer(dstMem, SER_VER);
     insertFamilyID(dstMem, famID);
-    //ignore lgNomLongs, lgArrLongs bytes for compact sketches
+    //The following initializes the lgNomLongs and lgArrLongs to 0.
+    //They are not used in CompactSketches.
+    dstMem.putShort(LG_NOM_LONGS_BYTE, (short)0);

Review comment:
       lgNomLongs is a byte followed by lgArrLongs, which is also a byte.  2 bytes = 1 short.  so clearing the address of lgNomLongs with a short clears both.  The power of Memory (and its danger, if you don't know what you are doing) is that it acts like a combination of C struct and C union so that you can write 8 bytes as a byte[] and then read it back as a long or a double.  And visa versa!
   




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455407107



##########
File path: src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
##########
@@ -333,6 +333,55 @@ public void checkHeapCompactSketchCompact() {
     assertEquals(csk.getCurrentPreambleLongs(), 2);
   }
 
+  @Test
+  public void checkDirectCompactSketchCompact() {
+    WritableMemory wmem1, wmem2;
+    CompactSketch csk1, csk2;
+    int bytes;
+    int lgK = 6;
+
+    //empty
+    UpdateSketch sk = Sketches.updateSketchBuilder().setLogNominalEntries(lgK).build();
+    bytes = sk.getCompactBytes();
+    wmem1 = WritableMemory.allocate(bytes);
+    wmem2 = WritableMemory.allocate(bytes);
+    csk1 = sk.compact(true, wmem1); //place into memory
+    assertTrue(csk1 instanceof DirectCompactSketch);
+    csk2 = csk1.compact(true, wmem2);
+    assertTrue(csk2 instanceof DirectCompactSketch);
+
+    //single
+    sk.update(1);
+    bytes = sk.getCompactBytes();
+    wmem1 = WritableMemory.allocate(bytes);
+    wmem2 = WritableMemory.allocate(bytes);
+    csk1 = sk.compact(true, wmem1); //place into memory
+    assertTrue(csk1 instanceof DirectCompactSketch);
+    csk2 = csk1.compact(true, wmem2);
+    assertTrue(csk2 instanceof DirectCompactSketch);
+
+    //exact
+    sk.update(2);
+    bytes = sk.getCompactBytes();
+    wmem1 = WritableMemory.allocate(bytes);
+    wmem2 = WritableMemory.allocate(bytes);
+    csk1 = sk.compact(true, wmem1); //place into memory
+    assertTrue(csk1 instanceof DirectCompactSketch);
+    csk2 = csk1.compact(true, wmem2);
+    assertTrue(csk2 instanceof DirectCompactSketch);

Review comment:
       Yes. But I need to add more asserts to make the difference between these cases more clear.




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

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] [incubator-datasketches-java] leerho commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-655869324


   I have completed the following c14n tests:
   - Accuracy
   - Update Speed
   - Serialization Speed
   - Deserialization Speed
   - Estimation Speed
   
   They all look really good.


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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452623175



##########
File path: src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
##########
@@ -171,7 +171,7 @@ public void checkAnotBEarlyStop() {
       CompactSketch csk2 = usk2.compact(true, null);
 
       Sketch rsk = PairwiseSetOperations.aNotB(csk1, csk2);
-      double result1 = rsk.getEstimate();
+      double result1 = rsk.getEstimate(); //null ptr

Review comment:
       These were comments I was making in the code as I was troubleshooting and forgot to go back and clean up.  The entire PairwiseSetOperations class is now deprecated since the main set operation classes now cover these pair operations as part of their API.   Eventually, when we move to version 3.0, we can delete the deprecated classes and the tests associated with them.




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452625920



##########
File path: src/test/java/org/apache/datasketches/theta/HeapAlphaSketchTest.java
##########
@@ -291,9 +290,9 @@ public void checkAlphaToCompactForms() {
     assertEquals(comp4.getUpperBound(2), comp1ub);
     assertEquals(comp4.isEmpty(), false);
     assertEquals(comp4.isEstimationMode(), estimating);
-    assertEquals(comp1bytes, comp4.getCurrentBytes(true)); //flag is not relevant
+    assertEquals(comp1bytes, comp4.getCompactBytes()); //flag is not relevant

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.

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] [incubator-datasketches-java] leerho commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657117651


   This failure is really odd.  I cannot reproduce it locally either in Eclipse or from Maven.


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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657134659


   This pull request **fixes 17 alerts** when merging 88c23df0bbfcb105072c052395ca1e2052e77577 into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-901e94aaa2dde01bb4bd94fdcb2fa94c343e26bc)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452532816



##########
File path: src/main/java/org/apache/datasketches/BoundsOnRatiosInSampledSets.java
##########
@@ -82,6 +82,28 @@ public static double getEstimateOfBoverA(final long a, final long b) {
     return (double) b / a;
   }
 
+  /**
+   * Return the estimate of A. See class javadoc.
+   * @param a See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.
+   * @return the approximate lower bound
+   */
+  public static double getEstimateOfA(final long a, final double f) {
+    checkInputs(a, 1, f);
+    return a / f;
+  }
+
+  /**
+   * Return the estimate of B. See class javadoc.
+   * @param b See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.

Review comment:
       You are correct! Excellent catch!
   
   WRT Question above: The Serialization Version only relates to the storage layout and independent of the code release version.  And, yes, it our intention to retain "binary compatibility" across languages and backward compatibility across time.  We are already binary compatible with sketch binaries produced 8 years ago.   This means we can read the old sketch binaries, but when serializing, we serialize to the latest SerVer=3.




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455409724



##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -47,55 +62,174 @@ public Family getFamily() {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is part of a multistep, stateful AnotB operation and sets the given Tuple sketch as the
+   * first argument <i>A</i> of <i>A-AND-NOT-B</i>. This overwrites the internal state of this
+   * AnotB operator with the contents of the given sketch.
+   * This sets the stage for multiple following <i>notB</i> steps.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * <p>Rationale: In mathematics a "null set" is a set with no members, which we call an empty set.
+   * That is distinctly different from the java <i>null</i>, which represents a nonexistant object.
+   * In most cases it is a programming error due to some object that was not properly initialized.
+   * With a null as the first argument, we cannot know what the user's intent is.
+   * Since it is very likely that a <i>null</i> is a programming error, we throw a an exception.</p>
+   *
+   * <p>An enpty input argument will set the internal state to empty.</p>
+   *
+   * <p>Rationale: An empty set is a mathematically legal concept. Although it makes any subsequent,
+   * valid argument for B irrelvant, we must allow this and assume the user knows what they are
+   * doing.</p>
+   *
+   * <p>Performing {@link #getResult(boolean)} just after this step will return a compact form of
+   * the given argument.</p>
+   *
+   * @param skA The incoming sketch for the first argument, <i>A</i>.
    */
-  public abstract CompactSketch getResult();
+  public abstract void setA(Sketch skA);
 
   /**
-   * Gets the result of this set operation as a CompactSketch of the chosen form
-   * @param dstOrdered
-   * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>.
+   * This is part of a multistep, stateful AnotB operation and sets the given Tuple sketch as the
+   * second (or <i>n+1</i>th) argument <i>B</i> of <i>A-AND-NOT-B</i>.
+   * Performs an <i>AND NOT</i> operation with the existing internal state of this AnotB operator.
    *
-   * @param dstMem
-   * <a href="{@docRoot}/resources/dictionary.html#dstMem">See Destination Memory</a>.
+   * <p>An input argument of null or empty is ignored.</p>
+   *
+   * <p>Rationale: A <i>null</i> for the second or following arguments is more tollerable because
+   * <i>A NOT null</i> is still <i>A</i> even if we don't know exactly what the null represents. It
+   * clearly does not have any content that overlaps with <i>A</i>. Also, because this can be part of
+   * a multistep operation with multiple <i>notB</i> steps. Other following steps can still produce
+   * a valid result.</p>
+   *
+   * <p>Use {@link #getResult(boolean)} to obtain the result.</p>
    *
-   * @return the result of this set operation as a CompactSketch of the chosen form
+   * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
-  public abstract CompactSketch getResult(boolean dstOrdered, WritableMemory dstMem);
+  public abstract void notB(Sketch skB);

Review comment:
       Good catch.  This is a copy/paste error. Thank you!




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-657888373


   This pull request **fixes 17 alerts** when merging d87b9e18f8a3c3ae8b2b66a35f13a65dc081a16a into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-b17c7fbb4c725002aa9674c6e24f39ccded698ce)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] AlexanderSaydakov commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455465567



##########
File path: src/main/java/org/apache/datasketches/theta/AnotBimpl.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.HashOperations.convertToHashTable;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
+import static org.apache.datasketches.Util.checkSeedHashes;
+import static org.apache.datasketches.Util.simpleIntLog2;
+
+import java.util.Arrays;
+
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * Implements the A-and-not-B operations.
+ * @author Lee Rhodes
+ * @author Kevin Lang
+ */
+final class AnotBimpl extends AnotB {
+  private final short seedHash_;
+  private boolean empty_;
+  private long thetaLong_;
+  private long[] hashArr_ = new long[0]; //compact array w curCount_ entries
+  private int curCount_;
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by SetOperation.Builder.
+   *
+   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
+   */
+  AnotBimpl(final long seed) {
+    this(computeSeedHash(seed));
+  }
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by PairwiseSetOperation.
+   *
+   * @param seedHash 16 bit hash of the chosen update seed.
+   */
+  AnotBimpl(final short seedHash) {
+    seedHash_ = seedHash;
+    reset();
+  }
+
+  @Override
+  public void setA(final Sketch skA) {
+    if (skA == null) {
+      reset();
+      throw new SketchesArgumentException("The input argument must not be null");
+    }
+    if (skA.isEmpty()) {
+      reset();
+      return;
+    }
+    //skA is not empty
+    checkSeedHashes(seedHash_, skA.getSeedHash());
+    empty_ = false;
+    thetaLong_ = skA.getThetaLong();
+    final CompactSketch cskA = (skA instanceof CompactSketch)
+        ? (CompactSketch) skA
+        : ((UpdateSketch) skA).compact();
+    hashArr_ = skA.isDirect() ? cskA.getCache() : cskA.getCache().clone();
+    curCount_ = cskA.getRetainedEntries(true);
+  }
+
+  @Override
+  public void notB(final Sketch skB) {
+    if (empty_ || (skB == null) || skB.isEmpty()) { return; }
+    //skB is not empty
+    checkSeedHashes(seedHash_, skB.getSeedHash());
+    final long thetaLongB = skB.getThetaLong();
+    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+
+    //Build hashtable and removes hashes of skB >= theta
+    final int countB = skB.getRetainedEntries(true);
+    CompactSketch cskB = null;
+    UpdateSketch uskB = null;
+    final long[] hashTableB;
+    if (skB instanceof CompactSketch) {
+      cskB = (CompactSketch) skB;
+      hashTableB = convertToHashTable(cskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD);
+    } else {
+      uskB = (UpdateSketch) skB;
+      hashTableB = (thetaLong_ < thetaLongB)
+          ? convertToHashTable(uskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD)

Review comment:
       why convert hash table to another hash table? theta check can be done before looking up in existing one




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

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] [incubator-datasketches-java] AlexanderSaydakov commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455474372



##########
File path: src/main/java/org/apache/datasketches/theta/AnotBimpl.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.HashOperations.convertToHashTable;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
+import static org.apache.datasketches.Util.checkSeedHashes;
+import static org.apache.datasketches.Util.simpleIntLog2;
+
+import java.util.Arrays;
+
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * Implements the A-and-not-B operations.
+ * @author Lee Rhodes
+ * @author Kevin Lang
+ */
+final class AnotBimpl extends AnotB {
+  private final short seedHash_;
+  private boolean empty_;
+  private long thetaLong_;
+  private long[] hashArr_ = new long[0]; //compact array w curCount_ entries
+  private int curCount_;
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by SetOperation.Builder.
+   *
+   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
+   */
+  AnotBimpl(final long seed) {
+    this(computeSeedHash(seed));
+  }
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by PairwiseSetOperation.
+   *
+   * @param seedHash 16 bit hash of the chosen update seed.
+   */
+  AnotBimpl(final short seedHash) {
+    seedHash_ = seedHash;
+    reset();
+  }
+
+  @Override
+  public void setA(final Sketch skA) {
+    if (skA == null) {
+      reset();
+      throw new SketchesArgumentException("The input argument must not be null");
+    }
+    if (skA.isEmpty()) {
+      reset();
+      return;
+    }
+    //skA is not empty
+    checkSeedHashes(seedHash_, skA.getSeedHash());
+    empty_ = false;
+    thetaLong_ = skA.getThetaLong();
+    final CompactSketch cskA = (skA instanceof CompactSketch)
+        ? (CompactSketch) skA
+        : ((UpdateSketch) skA).compact();
+    hashArr_ = skA.isDirect() ? cskA.getCache() : cskA.getCache().clone();
+    curCount_ = cskA.getRetainedEntries(true);
+  }
+
+  @Override
+  public void notB(final Sketch skB) {
+    if (empty_ || (skB == null) || skB.isEmpty()) { return; }
+    //skB is not empty
+    checkSeedHashes(seedHash_, skB.getSeedHash());
+    final long thetaLongB = skB.getThetaLong();
+    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+
+    //Build hashtable and removes hashes of skB >= theta
+    final int countB = skB.getRetainedEntries(true);
+    CompactSketch cskB = null;
+    UpdateSketch uskB = null;
+    final long[] hashTableB;
+    if (skB instanceof CompactSketch) {
+      cskB = (CompactSketch) skB;
+      hashTableB = convertToHashTable(cskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD);
+    } else {
+      uskB = (UpdateSketch) skB;
+      hashTableB = (thetaLong_ < thetaLongB)
+          ? convertToHashTable(uskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD)
+          : uskB.getCache();
+      cskB = uskB.compact();
+    }
+
+    //build temporary arrays of skA
+    final long[] tmpHashArrA = new long[curCount_];
+
+    //search for non matches and build temp arrays
+    final int lgHTBLen = simpleIntLog2(hashTableB.length);
+    int nonMatches = 0;
+    for (int i = 0; i < curCount_; i++) {
+      final long hash = hashArr_[i];
+      if ((hash != 0) && (hash < thetaLong_)) { //skips hashes of A >= theta
+        final int index = hashSearch(hashTableB, lgHTBLen, hash);
+        if (index == -1) {
+          tmpHashArrA[nonMatches] = hash;
+          nonMatches++;
+        }
+      }
+    }
+    hashArr_ = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
+    curCount_ = nonMatches;
+    empty_ = (nonMatches == 0) && (thetaLong_ == Long.MAX_VALUE);
+  }
+
+  @Override
+  public CompactSketch getResult(final boolean reset) {
+    return getResult(true, null, reset);
+  }
+
+  @Override
+  public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem,
+      final boolean reset) {
+    final CompactSketch result =
+        CompactOperations.componentsToCompact(
+            thetaLong_, curCount_, seedHash_, empty_, true, false, dstOrdered, dstMem, hashArr_);

Review comment:
       it seems that hashArr_ is not a copy, which it should be unless reset is requested




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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452909171



##########
File path: src/test/java/org/apache/datasketches/theta/UnionImplTest.java
##########
@@ -54,14 +54,14 @@ public void checkUpdateWithSketch() {
     assertEquals(union.getResult().getEstimate(), k, 0.0);
   }
 
-  @Test(expectedExceptions = SketchesArgumentException.class)
-  public void checkCorruptedCompactFlag() {
+  @Test
+  public void checkUnorderedCompactFlag() {
     int k = 16;
     WritableMemory mem = WritableMemory.wrap(new byte[(k*8) + 24]);
     UpdateSketch sketch = Sketches.updateSketchBuilder().setNominalEntries(k).build();
     for (int i=0; i<k; i++) { sketch.update(i); }
     CompactSketch sketchInDirectOrd = sketch.compact(true, mem);
-    sketch.compact(false, mem); //corrupt memory
+    sketch.compact(false, mem); //change the order bit

Review comment:
       It makes sense to remove the various implementations of the different variants.  When performing set operations between sets with different order bits, are the unordered sets sorted first internally?  I ask this because searching the internal hash keys should still work in theory, if the keys are unsorted.




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-655213144


   This pull request **fixes 17 alerts** when merging 644ab859645efce4fb14665533117edbed17bf16 into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-8405e6cd099498fc626da0ae530272e1d2b68c0f)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r453006832



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -121,6 +121,11 @@
     }
   }
 
+  @Override
+  public CompactSketch<S> compact() {
+    return this;
+  }

Review comment:
       Clearly doing toByteArray() would be very wasteful. The only knowledge we have of the Summary is the two current methods: we can copy it and serialize it to bytes.  When we are serializing the sketch we have to call toByteArray() anyway, so we grab its size at that point.  
   
   But adding this method would give us that visibility and I think it could be useful.  However, we don't want to make the requirements on the Summary too restrictive as it may limit what the users want to do with it.  
   
   Perhaps the best way to resolve this would be to put a [DISCUSS] issue on this topic on our dev@ list and see if anyone responds pro or con.
   
   




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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452911163



##########
File path: src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
##########
@@ -0,0 +1,495 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EMPTY;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_HEAP;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_MEMORY_UNORDERED;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EXACT;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.NULL;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+import java.util.Random;
+
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@SuppressWarnings({"javadoc","deprecation"})
+public class SetOpsCornerCasesTest {
+
+  /*******************************************/
+  Random rand = new Random(9001); //deterministic
+
+  @Test
+  public void checkSetOpsRandom() {
+    int hiA = 0, loB = 0, hiB = 0;
+    for (int i = 0; i < 1000; i++) {
+      hiA = rand.nextInt(128);      //skA fed values between 0 and 127
+      loB = rand.nextInt(64);
+      hiB = loB + rand.nextInt(64); //skB fed up to 63 values starting at loB
+      compareSetOpsRandom(64, 0, hiA, loB, hiB);
+    }
+  }
+
+  private static void compareSetOpsRandom(int k, int loA, int hiA, int loB, int hiB) {
+    UpdateSketch tskA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+    UpdateSketch tskB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+
+    for (int i = loA; i < hiA; i++) { tskA.update(i); }
+    for (int i = loB; i < hiB; i++) { tskB.update(i); }
+
+    CompactSketch rcskStdU = doStdUnion(tskA, tskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tskA, tskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tskA, tskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tskA, tskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tskA, tskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tskA, tskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tskA, tskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tskA, tskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tskA, tskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  /*******************************************/
+
+  @Test
+  //Check all corner cases against standard Union, Intersection, and AnotB.
+  //The unordered case is not tested
+  public void compareCornerCases() {
+    int k = 64;
+    for (State stateA : State.values()) {
+      for (State stateB : State.values()) {
+        if ((stateA == EST_MEMORY_UNORDERED) || (stateB == EST_MEMORY_UNORDERED)) { continue; }
+        cornerCaseChecks(stateA, stateB, k);
+        //cornerCaseChecksMemory(stateA, stateB, k);
+      }
+    }
+  }
+
+  @Test
+  public void checkExactNullSpecificCase() {
+    cornerCaseChecksMemory(State.EXACT, State.NULL, 64);
+  }
+
+  private static void cornerCaseChecksMemory(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    WritableMemory wmem = WritableMemory.allocate(SetOperation.getMaxUnionBytes(k));
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU); //heap, heap
+
+    rcskStdU = doStdUnion(tcskA, tcskB, k, wmem);
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, wmem);
+    checkCornerCase(rcskStdPairU, rcskStdU); //direct, direct
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxIntersectionBytes(k));
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI); //empty, empty
+
+    rcskStdI = doStdIntersection(tcskA, tcskB, wmem);
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdPairI, rcskStdI); //empty, empty //direct, direct???
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxAnotBResultBytes(k));
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB); //heap, heap
+
+    rcskStdAnotB = doStdAnotB(tcskA, tcskB, wmem);
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB); //direct, heap
+  }
+
+  private static void cornerCaseChecks(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  private static CompactSketch doStdUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    union.update(tskA);
+    union.update(tskB);
+    return union.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    return union.union(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    inter.update(tskA);
+    inter.update(tskB);
+    return inter.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    return inter.intersect(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    return anotb.aNotB(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdStatefulAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    anotb.setA(tskA);
+    anotb.notB(tskB);
+    anotb.getResult(false);
+    return anotb.getResult(true, wmem, true);
+  }
+
+  private static CompactSketch doPwUnion(Sketch tskA, Sketch tskB, int k) {
+    CompactSketch tcskA, tcskB;
+    if (tskA == null) { tcskA = null; }
+    else { tcskA = (tskA instanceof CompactSketch) ? (CompactSketch) tskA : tskA.compact(); }
+    if (tskB == null) { tcskB = null; }
+    else { tcskB = (tskB instanceof CompactSketch) ? (CompactSketch) tskB : tskB.compact(); }
+    return PairwiseSetOperations.union(tcskA, tcskB, k);
+  }
+
+  private static CompactSketch doPwIntersection(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.intersect(tskA, tskB);
+  }
+
+  private static CompactSketch doPwAnotB(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.aNotB(tskA, tskB);
+  }
+
+
+  private static void checkCornerCase(Sketch rskA, Sketch rskB) {
+    double estA = rskA.getEstimate();
+    double estB = rskB.getEstimate();
+    boolean emptyA = rskA.isEmpty();
+    boolean emptyB = rskB.isEmpty();
+    long thetaLongA = rskA.getThetaLong();
+    long thetaLongB = rskB.getThetaLong();
+    int countA = rskA.getRetainedEntries(true);
+    int countB = rskB.getRetainedEntries(true);
+    Assert.assertEquals(estB, estA, 0.0);
+    Assert.assertEquals(emptyB, emptyA);
+    Assert.assertEquals(thetaLongB, thetaLongA);
+    Assert.assertEquals(countB, countA);
+    Assert.assertEquals(rskA.getClass().getSimpleName(), rskB.getClass().getSimpleName());
+  }
+
+  /*******************************************/
+
+  @Test
+  public void checkUnionNotOrdered() {
+    int k = 64;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap = generate(EST_HEAP, k);
+    CompactSketch skHeapUO = generate(EST_MEMORY_UNORDERED, k);
+
+    PairwiseSetOperations.union(skNull, skHeapUO, k);
+    PairwiseSetOperations.union(skEmpty, skHeapUO, k);
+    PairwiseSetOperations.union(skHeapUO, skNull, k);
+    PairwiseSetOperations.union(skHeapUO, skEmpty, k);
+    PairwiseSetOperations.union(skHeapUO, skHeap, k);
+    PairwiseSetOperations.union(skHeap, skHeapUO, k);
+  }
+
+  @Test
+  public void checkSeedHash() {
+    int k = 64;
+    UpdateSketch tmp1 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp1.update(1);
+    tmp1.update(3);
+    CompactSketch skSmallSeed2A = tmp1.compact(true, null);
+
+    UpdateSketch tmp2 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp2.update(1);
+    tmp2.update(2);
+    CompactSketch skSmallSeed2B = tmp2.compact(true, null);
+
+    CompactSketch skExact = generate(EXACT, k);
+    CompactSketch skHeap = generate(EST_HEAP, 2 * k);
+    //Intersect
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //A NOT B
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //Union
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+  }
+
+  @Test
+  public void checkPwUnionReduceToK() {
+    int k = 16;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap1 = generate(EST_HEAP, k);
+    CompactSketch skHeap2 = generate(EST_HEAP, k);
+    CompactSketch csk;
+    csk = PairwiseSetOperations.union(skNull, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skEmpty, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skNull, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skEmpty, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skHeap2, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+  }
+
+
+
+
+  @Test
+  public void printlnTest() {
+    println("PRINTING: "+this.getClass().getName());
+  }
+
+  /**
+   * @param s value to print
+   */
+  static void println(String s) {
+    //System.out.println(s); //disable here
+  }
+
+  @Test
+  public void checkGenerator() {
+    int k = 16;
+    CompactSketch csk;
+
+    csk = generate(State.NULL, 0);
+    assertNull(csk);
+
+    csk = generate(State.EMPTY, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.SINGLE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 1);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EXACT, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), k);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_HEAP, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THLT1_CNT0_FALSE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THEQ1_CNT0_TRUE, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, false);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_MEMORY_UNORDERED, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), true);
+    assertEquals(csk.isOrdered(), false);
+  }
+
+  enum State {NULL, EMPTY, SINGLE, EXACT, EST_HEAP, THLT1_CNT0_FALSE, THEQ1_CNT0_TRUE, EST_MEMORY_UNORDERED}
+
+  private static CompactSketch generate(State state, int k) {
+    UpdateSketch sk = null;
+    CompactSketch csk = null;
+
+    switch(state) {
+      case NULL : {
+        //already null
+        break;
+      }
+      case EMPTY : { //results in EmptyCompactSketch
+        csk = Sketches.updateSketchBuilder().setNominalEntries(k).build().compact(true, null);
+        break;
+      }
+      case SINGLE : { //results in SingleItemSketches most of the time
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        sk.update(1);
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EXACT : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < k; i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EST_HEAP : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case THLT1_CNT0_FALSE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        sk.update(7); //above theta
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, F}
+        break;
+      }
+      case THEQ1_CNT0_TRUE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, T}
+        break;
+      }
+      case EST_MEMORY_UNORDERED : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        int bytes = Sketch.getMaxCompactSketchBytes(sk.getRetainedEntries(true));
+        byte[] byteArr = new byte[bytes];
+        WritableMemory mem = WritableMemory.wrap(byteArr);
+        csk = sk.compact(false, mem);
+        break;
+      }
+    }
+    return csk;
+  }

Review comment:
       Thanks for looking into this - I will take a look at the tests in the `adouble` package.  Having the same tests for Tuple is re-assuring, since I am using more and more Tuple set operations in my use cases.




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452626358



##########
File path: src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
##########
@@ -222,22 +216,80 @@ public void checkMemTooSmallOrdered() {
   @Test
   public void checkCompactCachePart() {
     //phony values except for curCount = 0.
-    long[] result = CompactSketch.compactCachePart(null, 4, 0, 0L, false);
+    long[] result = IntersectionImplR.compactCachePart(null, 4, 0, 0L, false);
     assertEquals(result.length, 0);
   }
 
   @Test
   public void checkDirectCompactSingleItemSketch() {
+    State state;
     UpdateSketch sk = Sketches.updateSketchBuilder().build();
-    CompactSketch csk = sk.compact(true, WritableMemory.allocate(16));
-    int bytes = csk.getCurrentBytes(true);
-    assertEquals(bytes, 8);
+
+    CompactSketch csko; //ordered
+    CompactSketch csku; //unordered
+
+    WritableMemory wmem = WritableMemory.allocate(16);
+    csko = sk.compact(true, wmem); //empty, direct, ordered
+    //ClassType, Count, Bytes, Compact, Empty, Direct, Memory, Ordered, Estimation
+    state = new State("DirectCompactSketch", 0, 8, true, true, false, true, true, false);
+    state.check(csko);
+
+    wmem = WritableMemory.allocate(16);
+    csku = sk.compact(false, wmem); //empty, direct, unordered
+    state = new State("DirectCompactSketch", 0, 8, true, true, false, true, true, false);
+    state.check(csku);
+
     sk.update(1);
-    csk = sk.compact(true, WritableMemory.allocate(16));
-    bytes = csk.getCurrentBytes(true);
-    assertEquals(bytes, 16);
-    assertTrue(csk == csk.compact());
-    assertTrue(csk == csk.compact(true, null));
+    wmem = WritableMemory.allocate(16);
+    csko = sk.compact(true, wmem); //Single, direct, ordered
+    state = new State("DirectCompactSketch", 1, 16, true, false, false, true, true, false);
+    state.check(csko);
+
+    wmem = WritableMemory.allocate(16);
+    csku = sk.compact(false, wmem); //Single, direct, unordered
+    state = new State("DirectCompactSketch", 1, 16, true, false, false, true, true, false);
+    state.check(csku);
+
+    CompactSketch csk2o; //ordered
+    CompactSketch csk2u; //unordered
+
+    csk2o = csku.compact(); //single, heap, ordered
+    state = new State("SingleItemSketch", 1, 16, true, false, false, false, true, false);
+    state.check(csk2o);
+
+    csk2o = csku.compact(true, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csku.compact(false, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csko.compact(true, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csko.compact(false, null); //single, heap, ordered

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.

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r453001280



##########
File path: src/test/java/org/apache/datasketches/theta/UnionImplTest.java
##########
@@ -54,14 +54,14 @@ public void checkUpdateWithSketch() {
     assertEquals(union.getResult().getEstimate(), k, 0.0);
   }
 
-  @Test(expectedExceptions = SketchesArgumentException.class)
-  public void checkCorruptedCompactFlag() {
+  @Test
+  public void checkUnorderedCompactFlag() {
     int k = 16;
     WritableMemory mem = WritableMemory.wrap(new byte[(k*8) + 24]);
     UpdateSketch sketch = Sketches.updateSketchBuilder().setNominalEntries(k).build();
     for (int i=0; i<k; i++) { sketch.update(i); }
     CompactSketch sketchInDirectOrd = sketch.compact(true, mem);
-    sketch.compact(false, mem); //corrupt memory
+    sketch.compact(false, mem); //change the order bit

Review comment:
       In an UpdateSketch the hashes are in a Knuth Open Address Double Hash (OADH) power-of-2 sized array; essentially a hash table, but a very efficient one.  The placement of hashes in a hash table is random, so the hashes are never in any order. The CompactSketch has all the same hashes, but all the empty slots of the hash table have been removed.  When creating the CS, the user has the option to have the hashes sorted or not. Sorting costs a little more time when creating the CS, but when merging CSs into a union, we can take advantage of "early-stop" which results in dramatic improvement in merge speed performance.  We never "search" the hashes in a CS. There is no need to do that.
   
   The CS is the simplest representation of a sketch.  It contains primarily the hashes and Theta, that's it!  It is immutable and has no concept of "K" or "Nominal Entries".  Yet it can be used as input of all set operations.




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455481082



##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -47,55 +62,174 @@ public Family getFamily() {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is part of a multistep, stateful AnotB operation and sets the given Theta sketch as the
+   * first argument <i>A</i> of <i>A-AND-NOT-B</i>. This overwrites the internal state of this
+   * AnotB operator with the contents of the given sketch.
+   * This sets the stage for multiple following <i>notB</i> steps.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * <p>Rationale: In mathematics a "null set" is a set with no members, which we call an empty set.
+   * That is distinctly different from the java <i>null</i>, which represents a nonexistant object.

Review comment:
       Thank you. Actually found this spelling error in 6 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.

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452534520



##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -24,18 +24,33 @@
 
 /**
  * The API for the set difference operation <i>A and not B</i> operations.
- * This is a stateless operation. However, to make the API
- * more consistent with the other set operations the intended use is:
+ * This class include both stateful and stateless operations.

Review comment:
       Thank you!




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

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] [incubator-datasketches-java] lgtm-com[bot] commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
lgtm-com[bot] commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-659083483


   This pull request **fixes 17 alerts** when merging 00fd703bc37492620a538527382cdb704adbcbbd into eead07c3fdb9b32ed785b5d73ae4a9739b3a0e42 - [view on LGTM.com](https://lgtm.com/projects/g/apache/incubator-datasketches-java/rev/pr-0d8bb3f34e8919067630a9da1f52c122ddb4e22a)
   
   **fixed alerts:**
   
   * 17 for Dereferenced variable may be null


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

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] [incubator-datasketches-java] AlexanderSaydakov commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455457363



##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -47,55 +62,174 @@ public Family getFamily() {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is part of a multistep, stateful AnotB operation and sets the given Theta sketch as the
+   * first argument <i>A</i> of <i>A-AND-NOT-B</i>. This overwrites the internal state of this
+   * AnotB operator with the contents of the given sketch.
+   * This sets the stage for multiple following <i>notB</i> steps.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * <p>Rationale: In mathematics a "null set" is a set with no members, which we call an empty set.
+   * That is distinctly different from the java <i>null</i>, which represents a nonexistant object.

Review comment:
       nonexistent




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455932261



##########
File path: src/main/java/org/apache/datasketches/theta/AnotBimpl.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.HashOperations.convertToHashTable;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
+import static org.apache.datasketches.Util.checkSeedHashes;
+import static org.apache.datasketches.Util.simpleIntLog2;
+
+import java.util.Arrays;
+
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * Implements the A-and-not-B operations.
+ * @author Lee Rhodes
+ * @author Kevin Lang
+ */
+final class AnotBimpl extends AnotB {
+  private final short seedHash_;
+  private boolean empty_;
+  private long thetaLong_;
+  private long[] hashArr_ = new long[0]; //compact array w curCount_ entries
+  private int curCount_;
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by SetOperation.Builder.
+   *
+   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
+   */
+  AnotBimpl(final long seed) {
+    this(computeSeedHash(seed));
+  }
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by PairwiseSetOperation.
+   *
+   * @param seedHash 16 bit hash of the chosen update seed.
+   */
+  AnotBimpl(final short seedHash) {
+    seedHash_ = seedHash;
+    reset();
+  }
+
+  @Override
+  public void setA(final Sketch skA) {
+    if (skA == null) {
+      reset();
+      throw new SketchesArgumentException("The input argument must not be null");
+    }
+    if (skA.isEmpty()) {
+      reset();
+      return;
+    }
+    //skA is not empty
+    checkSeedHashes(seedHash_, skA.getSeedHash());
+    empty_ = false;
+    thetaLong_ = skA.getThetaLong();
+    final CompactSketch cskA = (skA instanceof CompactSketch)
+        ? (CompactSketch) skA
+        : ((UpdateSketch) skA).compact();
+    hashArr_ = skA.isDirect() ? cskA.getCache() : cskA.getCache().clone();
+    curCount_ = cskA.getRetainedEntries(true);
+  }
+
+  @Override
+  public void notB(final Sketch skB) {
+    if (empty_ || (skB == null) || skB.isEmpty()) { return; }
+    //skB is not empty
+    checkSeedHashes(seedHash_, skB.getSeedHash());
+    final long thetaLongB = skB.getThetaLong();
+    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+
+    //Build hashtable and removes hashes of skB >= theta
+    final int countB = skB.getRetainedEntries(true);
+    CompactSketch cskB = null;
+    UpdateSketch uskB = null;
+    final long[] hashTableB;
+    if (skB instanceof CompactSketch) {
+      cskB = (CompactSketch) skB;
+      hashTableB = convertToHashTable(cskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD);
+    } else {
+      uskB = (UpdateSketch) skB;
+      hashTableB = (thetaLong_ < thetaLongB)
+          ? convertToHashTable(uskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD)

Review comment:
       Good catch. Thank you! 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.

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455402662



##########
File path: src/main/java/org/apache/datasketches/theta/CompactOperations.java
##########
@@ -229,10 +233,20 @@ static final Memory loadCompactMemory(
     }
     final byte famID = (byte) Family.COMPACT.getID();
 
+    //Caution: The following loads directly into Memory without creating a heap byte[] first,
+    // which would act as a pre-clearing, initialization mechanism. So it is important to make sure
+    // that all fields are initialized, even those that are not used by the CompactSketch.
+    // Otherwise, uninitialized fields could be filled with off-heap garbage, which could cause
+    // other problems downstream if those fields are not filtered out first.
+    // As written below, all fields are initialized avoiding an extra copy.
+
+    //The first 8 bytes (pre0)
     insertPreLongs(dstMem, preLongs); //RF not used = 0
     insertSerVer(dstMem, SER_VER);
     insertFamilyID(dstMem, famID);
-    //ignore lgNomLongs, lgArrLongs bytes for compact sketches
+    //The following initializes the lgNomLongs and lgArrLongs to 0.
+    //They are not used in CompactSketches.
+    dstMem.putShort(LG_NOM_LONGS_BYTE, (short)0);

Review comment:
       lgNomLongs is a byte followed by lgArrLongs, which is also a byte.  2 bytes = 1 short.  so clearing the address of lgNomLongs clears both.  The power of Memory (and its danger, if you don't know what you are doing) is that it acts like a combination of C struct and C union so that you can write 8 bytes as a byte[] and then read it back as a long or a double. 
   




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

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] [incubator-datasketches-java] leerho commented on pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#issuecomment-656822672


   An excellent review, thanks to David.  Clearly this PR is not done.  I will be submitting corrections to resolve these issues soon.


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

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] [incubator-datasketches-java] AlexanderSaydakov commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455480747



##########
File path: src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
##########
@@ -207,11 +207,11 @@ public SetOperation build(final Family family, final WritableMemory dstMem) {
       }
       case A_NOT_B: {
         if (dstMem == null) {
-          setOp = new HeapAnotB(bSeed);
+          setOp = new AnotBimpl(bSeed);
         }
         else {
-          throw new SketchesArgumentException(
-            "AnotB is a stateless operation and cannot be persisted.");
+          throw new SketchesArgumentException( //TODO we should be able to do this now.

Review comment:
       is this TODO comment still relevant?




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

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] [incubator-datasketches-java] davecromberge commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
davecromberge commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r455383769



##########
File path: src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
##########
@@ -128,20 +128,19 @@ static HeapAlphaSketch heapifyInstance(final Memory srcMem, final long seed) {
     checkMemIntegrity(srcMem, seed, preambleLongs, lgNomLongs, lgArrLongs);
 
     final float p = extractP(srcMem);                             //bytes 12-15
-    final int lgRF = extractLgResizeFactor(srcMem);               //byte 0
-    final ResizeFactor myRF = ResizeFactor.getRF(lgRF);
+    final int memlgRF = extractLgResizeFactor(srcMem);            //byte 0
+    final ResizeFactor memRF = ResizeFactor.getRF(memlgRF);
 
     final double nomLongs = (1L << lgNomLongs);
     final double alpha = nomLongs / (nomLongs + 1.0);
     final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * LONG_MAX_VALUE_AS_DOUBLE);
 
-    if ((myRF == ResizeFactor.X1)
-            && (lgArrLongs != startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
-      throw new SketchesArgumentException("Possible corruption: ResizeFactor X1, but provided "
-              + "array too small for sketch size");
+    if (isResizeFactorIncorrect(srcMem, lgNomLongs, lgArrLongs)) {
+      throw new SketchesArgumentException("Possible corruption: ResizeFactor  "
+          + "inconsistent with lgNomLongs and lgArrLongs.");

Review comment:
       This is a different approach taken than that of the DirectQuickSelect/HeapQuickSelect sketches, where X2 is selected as a resize factor when the arguments are incorrect, as per the comment:
   ```//If incorrect it sets it to X2 which always works.```
   Is this by design?
   




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r456806600



##########
File path: src/main/java/org/apache/datasketches/theta/AnotBimpl.java
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.HashOperations.convertToHashTable;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
+import static org.apache.datasketches.Util.checkSeedHashes;
+import static org.apache.datasketches.Util.simpleIntLog2;
+
+import java.util.Arrays;
+
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * Implements the A-and-not-B operations.
+ * @author Lee Rhodes
+ * @author Kevin Lang
+ */
+final class AnotBimpl extends AnotB {
+  private final short seedHash_;
+  private boolean empty_;
+  private long thetaLong_;
+  private long[] hashArr_ = new long[0]; //compact array w curCount_ entries
+  private int curCount_;
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by SetOperation.Builder.
+   *
+   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
+   */
+  AnotBimpl(final long seed) {
+    this(computeSeedHash(seed));
+  }
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by PairwiseSetOperation.
+   *
+   * @param seedHash 16 bit hash of the chosen update seed.
+   */
+  AnotBimpl(final short seedHash) {
+    seedHash_ = seedHash;
+    reset();
+  }
+
+  @Override
+  public void setA(final Sketch skA) {
+    if (skA == null) {
+      reset();
+      throw new SketchesArgumentException("The input argument must not be null");
+    }
+    if (skA.isEmpty()) {
+      reset();
+      return;
+    }
+    //skA is not empty
+    checkSeedHashes(seedHash_, skA.getSeedHash());
+    empty_ = false;
+    thetaLong_ = skA.getThetaLong();
+    final CompactSketch cskA = (skA instanceof CompactSketch)
+        ? (CompactSketch) skA
+        : ((UpdateSketch) skA).compact();
+    hashArr_ = skA.isDirect() ? cskA.getCache() : cskA.getCache().clone();
+    curCount_ = cskA.getRetainedEntries(true);
+  }
+
+  @Override
+  public void notB(final Sketch skB) {
+    if (empty_ || (skB == null) || skB.isEmpty()) { return; }
+    //skB is not empty
+    checkSeedHashes(seedHash_, skB.getSeedHash());
+    final long thetaLongB = skB.getThetaLong();
+    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+
+    //Build hashtable and removes hashes of skB >= theta
+    final int countB = skB.getRetainedEntries(true);
+    CompactSketch cskB = null;
+    UpdateSketch uskB = null;
+    final long[] hashTableB;
+    if (skB instanceof CompactSketch) {
+      cskB = (CompactSketch) skB;
+      hashTableB = convertToHashTable(cskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD);
+    } else {
+      uskB = (UpdateSketch) skB;
+      hashTableB = (thetaLong_ < thetaLongB)
+          ? convertToHashTable(uskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD)
+          : uskB.getCache();
+      cskB = uskB.compact();
+    }
+
+    //build temporary arrays of skA
+    final long[] tmpHashArrA = new long[curCount_];
+
+    //search for non matches and build temp arrays
+    final int lgHTBLen = simpleIntLog2(hashTableB.length);
+    int nonMatches = 0;
+    for (int i = 0; i < curCount_; i++) {
+      final long hash = hashArr_[i];
+      if ((hash != 0) && (hash < thetaLong_)) { //skips hashes of A >= theta
+        final int index = hashSearch(hashTableB, lgHTBLen, hash);
+        if (index == -1) {
+          tmpHashArrA[nonMatches] = hash;
+          nonMatches++;
+        }
+      }
+    }
+    hashArr_ = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
+    curCount_ = nonMatches;
+    empty_ = (nonMatches == 0) && (thetaLong_ == Long.MAX_VALUE);
+  }
+
+  @Override
+  public CompactSketch getResult(final boolean reset) {
+    return getResult(true, null, reset);
+  }
+
+  @Override
+  public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem,
+      final boolean reset) {
+    final CompactSketch result =
+        CompactOperations.componentsToCompact(
+            thetaLong_, curCount_, seedHash_, empty_, true, false, dstOrdered, dstMem, hashArr_);

Review comment:
       Good catch.  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.

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452536019



##########
File path: src/main/java/org/apache/datasketches/theta/AnotB.java
##########
@@ -47,55 +62,125 @@ public Family getFamily() {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is a stateful input operation. This method sets the given Sketch as the first
+   * argument <i>A</i> of a stateful <i>AnotB</i> operation. This overwrites the internal state of
+   * this AnotB operator with the contents of the given sketch. This sets the stage for multiple
+   * stateful subsequent {@link #notB(Sketch)} operations. The ultimate result is obtained using
+   * the {@link #getResult(boolean)} or {@link #getResult(boolean, WritableMemory, boolean)}.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * @param skA The incoming sketch for the first argument, <i>A</i>.
    */
-  public abstract CompactSketch getResult();
+  public abstract void setA(Sketch skA);
+
+  /**
+   * Performs a stateful <i>AND NOT</i> operation with the existing internal state of this AnotB
+   * operator. Use {@link #getResult(boolean)} or {@link #getResult(boolean, WritableMemory, boolean)}
+   * to obtain the result.
+   *
+   * <p>An input argument of null or empty is ignored.</p>
+   *
+   * @param skB The incoming sketch for the second (or following) argument <i>B</i>.
+   */
+  public abstract void notB(Sketch skB);
+
+  /**
+   * Gets the result of this operation as an ordered CompactSketch on the Java heap.
+   * @param reset If true, clears this operator to the empty state after result is returned.
+   * @return the result of this operation as a CompactSketch.
+   */
+  public abstract CompactSketch getResult(boolean reset);
 
   /**
-   * Gets the result of this set operation as a CompactSketch of the chosen form
+   * Gets the result of this stateful set operation as a CompactSketch of the chosen form. The

Review comment:
       Fixed, thank you!




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

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452624970



##########
File path: src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
##########
@@ -0,0 +1,495 @@
+/*
+ * 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.theta;
+
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EMPTY;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_HEAP;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_MEMORY_UNORDERED;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EXACT;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.NULL;
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertNull;
+
+import java.util.Random;
+
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
+
+@SuppressWarnings({"javadoc","deprecation"})
+public class SetOpsCornerCasesTest {
+
+  /*******************************************/
+  Random rand = new Random(9001); //deterministic
+
+  @Test
+  public void checkSetOpsRandom() {
+    int hiA = 0, loB = 0, hiB = 0;
+    for (int i = 0; i < 1000; i++) {
+      hiA = rand.nextInt(128);      //skA fed values between 0 and 127
+      loB = rand.nextInt(64);
+      hiB = loB + rand.nextInt(64); //skB fed up to 63 values starting at loB
+      compareSetOpsRandom(64, 0, hiA, loB, hiB);
+    }
+  }
+
+  private static void compareSetOpsRandom(int k, int loA, int hiA, int loB, int hiB) {
+    UpdateSketch tskA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+    UpdateSketch tskB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+
+    for (int i = loA; i < hiA; i++) { tskA.update(i); }
+    for (int i = loB; i < hiB; i++) { tskB.update(i); }
+
+    CompactSketch rcskStdU = doStdUnion(tskA, tskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tskA, tskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tskA, tskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tskA, tskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tskA, tskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tskA, tskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tskA, tskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tskA, tskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tskA, tskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  /*******************************************/
+
+  @Test
+  //Check all corner cases against standard Union, Intersection, and AnotB.
+  //The unordered case is not tested
+  public void compareCornerCases() {
+    int k = 64;
+    for (State stateA : State.values()) {
+      for (State stateB : State.values()) {
+        if ((stateA == EST_MEMORY_UNORDERED) || (stateB == EST_MEMORY_UNORDERED)) { continue; }
+        cornerCaseChecks(stateA, stateB, k);
+        //cornerCaseChecksMemory(stateA, stateB, k);
+      }
+    }
+  }
+
+  @Test
+  public void checkExactNullSpecificCase() {
+    cornerCaseChecksMemory(State.EXACT, State.NULL, 64);
+  }
+
+  private static void cornerCaseChecksMemory(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    WritableMemory wmem = WritableMemory.allocate(SetOperation.getMaxUnionBytes(k));
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU); //heap, heap
+
+    rcskStdU = doStdUnion(tcskA, tcskB, k, wmem);
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, wmem);
+    checkCornerCase(rcskStdPairU, rcskStdU); //direct, direct
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxIntersectionBytes(k));
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI); //empty, empty
+
+    rcskStdI = doStdIntersection(tcskA, tcskB, wmem);
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdPairI, rcskStdI); //empty, empty //direct, direct???
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxAnotBResultBytes(k));
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB); //heap, heap
+
+    rcskStdAnotB = doStdAnotB(tcskA, tcskB, wmem);
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB); //direct, heap
+  }
+
+  private static void cornerCaseChecks(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  private static CompactSketch doStdUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    union.update(tskA);
+    union.update(tskB);
+    return union.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    return union.union(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    inter.update(tskA);
+    inter.update(tskB);
+    return inter.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    return inter.intersect(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    return anotb.aNotB(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdStatefulAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    anotb.setA(tskA);
+    anotb.notB(tskB);
+    anotb.getResult(false);
+    return anotb.getResult(true, wmem, true);
+  }
+
+  private static CompactSketch doPwUnion(Sketch tskA, Sketch tskB, int k) {
+    CompactSketch tcskA, tcskB;
+    if (tskA == null) { tcskA = null; }
+    else { tcskA = (tskA instanceof CompactSketch) ? (CompactSketch) tskA : tskA.compact(); }
+    if (tskB == null) { tcskB = null; }
+    else { tcskB = (tskB instanceof CompactSketch) ? (CompactSketch) tskB : tskB.compact(); }
+    return PairwiseSetOperations.union(tcskA, tcskB, k);
+  }
+
+  private static CompactSketch doPwIntersection(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.intersect(tskA, tskB);
+  }
+
+  private static CompactSketch doPwAnotB(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.aNotB(tskA, tskB);
+  }
+
+
+  private static void checkCornerCase(Sketch rskA, Sketch rskB) {
+    double estA = rskA.getEstimate();
+    double estB = rskB.getEstimate();
+    boolean emptyA = rskA.isEmpty();
+    boolean emptyB = rskB.isEmpty();
+    long thetaLongA = rskA.getThetaLong();
+    long thetaLongB = rskB.getThetaLong();
+    int countA = rskA.getRetainedEntries(true);
+    int countB = rskB.getRetainedEntries(true);
+    Assert.assertEquals(estB, estA, 0.0);
+    Assert.assertEquals(emptyB, emptyA);
+    Assert.assertEquals(thetaLongB, thetaLongA);
+    Assert.assertEquals(countB, countA);
+    Assert.assertEquals(rskA.getClass().getSimpleName(), rskB.getClass().getSimpleName());
+  }
+
+  /*******************************************/
+
+  @Test
+  public void checkUnionNotOrdered() {
+    int k = 64;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap = generate(EST_HEAP, k);
+    CompactSketch skHeapUO = generate(EST_MEMORY_UNORDERED, k);
+
+    PairwiseSetOperations.union(skNull, skHeapUO, k);
+    PairwiseSetOperations.union(skEmpty, skHeapUO, k);
+    PairwiseSetOperations.union(skHeapUO, skNull, k);
+    PairwiseSetOperations.union(skHeapUO, skEmpty, k);
+    PairwiseSetOperations.union(skHeapUO, skHeap, k);
+    PairwiseSetOperations.union(skHeap, skHeapUO, k);
+  }
+
+  @Test
+  public void checkSeedHash() {
+    int k = 64;
+    UpdateSketch tmp1 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp1.update(1);
+    tmp1.update(3);
+    CompactSketch skSmallSeed2A = tmp1.compact(true, null);
+
+    UpdateSketch tmp2 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
+    tmp2.update(1);
+    tmp2.update(2);
+    CompactSketch skSmallSeed2B = tmp2.compact(true, null);
+
+    CompactSketch skExact = generate(EXACT, k);
+    CompactSketch skHeap = generate(EST_HEAP, 2 * k);
+    //Intersect
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.intersect(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //A NOT B
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.aNotB(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    //Union
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2A);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skExact, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skSmallSeed2B, skExact);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+    try {
+      PairwiseSetOperations.union(skHeap, skSmallSeed2B);
+      Assert.fail();
+    } catch (Exception e) { } //pass
+  }
+
+  @Test
+  public void checkPwUnionReduceToK() {
+    int k = 16;
+    CompactSketch skNull = generate(NULL, k);
+    CompactSketch skEmpty = generate(EMPTY, k);
+    CompactSketch skHeap1 = generate(EST_HEAP, k);
+    CompactSketch skHeap2 = generate(EST_HEAP, k);
+    CompactSketch csk;
+    csk = PairwiseSetOperations.union(skNull, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skEmpty, skHeap1, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skNull, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skEmpty, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+    csk = PairwiseSetOperations.union(skHeap1, skHeap2, k);
+    Assert.assertEquals(csk.getRetainedEntries(true), k);
+  }
+
+
+
+
+  @Test
+  public void printlnTest() {
+    println("PRINTING: "+this.getClass().getName());
+  }
+
+  /**
+   * @param s value to print
+   */
+  static void println(String s) {
+    //System.out.println(s); //disable here
+  }
+
+  @Test
+  public void checkGenerator() {
+    int k = 16;
+    CompactSketch csk;
+
+    csk = generate(State.NULL, 0);
+    assertNull(csk);
+
+    csk = generate(State.EMPTY, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.SINGLE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 1);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EXACT, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), k);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_HEAP, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THLT1_CNT0_FALSE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.THEQ1_CNT0_TRUE, k);
+    assertEquals(csk.isEmpty(), true);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(true), 0);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, false);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
+    csk = generate(State.EST_MEMORY_UNORDERED, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), true);
+    assertEquals(csk.getRetainedEntries(true) > k, true);
+    assertEquals(csk.getThetaLong() < Long.MAX_VALUE, true);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), true);
+    assertEquals(csk.isOrdered(), false);
+  }
+
+  enum State {NULL, EMPTY, SINGLE, EXACT, EST_HEAP, THLT1_CNT0_FALSE, THEQ1_CNT0_TRUE, EST_MEMORY_UNORDERED}
+
+  private static CompactSketch generate(State state, int k) {
+    UpdateSketch sk = null;
+    CompactSketch csk = null;
+
+    switch(state) {
+      case NULL : {
+        //already null
+        break;
+      }
+      case EMPTY : { //results in EmptyCompactSketch
+        csk = Sketches.updateSketchBuilder().setNominalEntries(k).build().compact(true, null);
+        break;
+      }
+      case SINGLE : { //results in SingleItemSketches most of the time
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        sk.update(1);
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EXACT : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < k; i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case EST_HEAP : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        csk = sk.compact(true, null);
+        break;
+      }
+      case THLT1_CNT0_FALSE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        sk.update(7); //above theta
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, F}
+        break;
+      }
+      case THEQ1_CNT0_TRUE : {
+        sk = Sketches.updateSketchBuilder().setP((float)0.5).setNominalEntries(k).build();
+        assert(sk.getRetainedEntries(true) == 0);
+        csk = sk.compact(true, null); //compact as {Th < 1.0, 0, T}
+        break;
+      }
+      case EST_MEMORY_UNORDERED : {
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        for (int i = 0; i < (4 * k); i++) {
+          sk.update(i);
+        }
+        int bytes = Sketch.getMaxCompactSketchBytes(sk.getRetainedEntries(true));
+        byte[] byteArr = new byte[bytes];
+        WritableMemory mem = WritableMemory.wrap(byteArr);
+        csk = sk.compact(false, mem);
+        break;
+      }
+    }
+    return csk;
+  }

Review comment:
       Real good question.  I will have to look into 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.

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] [incubator-datasketches-java] leerho commented on a change in pull request #324: Refactor theta tuple

Posted by GitBox <gi...@apache.org>.
leerho commented on a change in pull request #324:
URL: https://github.com/apache/incubator-datasketches-java/pull/324#discussion_r452619693



##########
File path: src/test/java/org/apache/datasketches/theta/UnionImplTest.java
##########
@@ -54,14 +54,14 @@ public void checkUpdateWithSketch() {
     assertEquals(union.getResult().getEstimate(), k, 0.0);
   }
 
-  @Test(expectedExceptions = SketchesArgumentException.class)
-  public void checkCorruptedCompactFlag() {
+  @Test
+  public void checkUnorderedCompactFlag() {
     int k = 16;
     WritableMemory mem = WritableMemory.wrap(new byte[(k*8) + 24]);
     UpdateSketch sketch = Sketches.updateSketchBuilder().setNominalEntries(k).build();
     for (int i=0; i<k; i++) { sketch.update(i); }
     CompactSketch sketchInDirectOrd = sketch.compact(true, mem);
-    sketch.compact(false, mem); //corrupt memory
+    sketch.compact(false, mem); //change the order bit

Review comment:
       My you have a very good eye!  At first I thought you found a bug, but what you found was an inert test, which I have now improved, thanks!  
   
   As of version 1.3.0, there was an additional layer of classes that were ordered and unordered subclasses of the HeapCompactSketch and the DirectCompactSketch. But this was really unnecessary. So I eliminated those 4 classes as they only differed by the ordered bit.  So before, there were two ways to determine whether the CompactSketch was ordered or not: the class type, and the ordered bit (memory) or boolean (on heap).  This test was designed to test the situation where the indicators were out-of-sync and make sure an error was thrown.  
   
   With the new version there is only one indicator of order, either a boolean on heap or a bit set in Memory. 
   As a result this test was inert.  So instead of removing it, I added additional checks to make sure that the union operation would successfully and correctly merge the same hashes whether the sketch order flag was set or not.  




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

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