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 2021/10/18 20:07:13 UTC

[GitHub] [datasketches-java] leerho opened a new pull request #369: Fix Mikhail's bug

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


   This fixes Mikhail's bug along with a number of other related corner-case issues in the 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.

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -177,23 +215,61 @@ public void notB(final Sketch<S> skB) {
    *
    * @param skB The incoming Theta sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final org.apache.datasketches.theta.Sketch skB) {
-    if (empty_ || skB == null || skB.isEmpty()) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+    if (skB == null) { return; } //ignore
 
-    //process B
-    final DataArrays<S> daB = getResultArraysTheta(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
-
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;
+      }
+      case SKA_TRIM: {
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        final DataArrays<S> da = trimDataArrays(hashArr_, summaryArr_,thetaLong_);
+        hashArr_ = da.hashArr;
+        curCount_ = (hashArr_ == null) ? 0 : hashArr_.length;
+        summaryArr_ = da.summaryArr;
+        break;
+      }
+      case SKETCH_A: {
+        break; //result is already in A
+      }
+      case FULL_ANOTB: { //both A and B should have valid entries.
+        thetaLong_ = min(thetaLong_, skB.getThetaLong());

Review comment:
       You are correct!




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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.tuple;
+
+import static java.lang.Math.ceil;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.HashOperations.hashInsertOnly;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.ceilingPowerOf2;
+
+import java.lang.reflect.Array;
+
+@SuppressWarnings("unchecked")
+class HashTables<S extends Summary> {
+  long[] hashTable_ = null;
+  S[] summaryTable_ = null;
+  int lgTableSize_ = 0;
+  int count_ = 0;
+
+  HashTables() { }
+
+  //must have valid entries
+  void fromSketch(final Sketch<S> sketch) {
+    count_ = sketch.getRetainedEntries();
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final SketchIterator<S> it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.getHash();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = (S)it.getSummary().copy();
+      if (summaryTable_ == null) {
+        summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
+      }
+      summaryTable_[index] = mySummary;
+    }
+  }
+
+  //must have valid entries
+  void fromSketch(final org.apache.datasketches.theta.Sketch sketch, final S summary) {
+    count_ = sketch.getRetainedEntries(true);
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final org.apache.datasketches.theta.HashIterator it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.get();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = summary;

Review comment:
       should this be a copy as in the method above?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -118,9 +120,9 @@
         }
       }
     } else { // current serial format
-      offset++; // unused
+      offset++;

Review comment:
       was the comment unnecessary? perhaps it should be a bit more verbose like "skip unused byte"

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {

Review comment:
       perhaps "IntersectResult" would be a bit more descriptive? It took me a while to understand

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private InterResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54
+
+    private static final Map<Integer, CornerCase> idToCornerCaseMap = new HashMap<>();
+    private int id;
+    private String inputStr;
+    private InterResult interResult;
+    private AnotbResult anotbResult;
+
+    static {
+      for (final CornerCase cc : values()) {
+        idToCornerCaseMap.put(cc.getId(), cc);
+      }
+    }
+
+    private CornerCase(final int id, final String desc,
+        final InterResult interResult, final AnotbResult anotbResult) {
+      this.id = id;
+      this.inputStr = desc;
+      this.interResult = interResult;
+      this.anotbResult = anotbResult;
+    }
+
+    public int getId() {
+      return id;
+    }
+
+    public String getDesc() {
+      return inputStr;
+    }
+
+    public InterResult getInterResult() {
+      return interResult;
+    }
+
+    public AnotbResult getAnotbResult() {
+      return anotbResult;
+    }
+
+    public static CornerCase idToCornerCase(final int id) {
+      final CornerCase cc = idToCornerCaseMap.get(id);
+      if (cc == null) {
+        throw new SketchesArgumentException("Possible Corruption: Illegal CornerCase ID: " + id);
+      }
+      return cc;
+    }
+  } //end of enum
+
+  public static int createCornerCaseId(
+      final long thetaLongA, final int countA, final boolean emptyA,
+      final long thetaLongB, final int countB, final boolean emptyB) {
+    return ((thetaLongA < MAX) ? 0 : 1 << 5)
+         | ((countA == 0)      ? 0 : 1 << 4)
+         | (!emptyA            ? 0 : 1 << 3)
+         | ((thetaLongB < MAX) ? 0 : 1 << 2)
+         | ((countB == 0)      ? 0 : 1 << 1)
+         | (!emptyB            ? 0 : 1);
+  }
+
+//  @Test

Review comment:
       forgot to remove?

##########
File path: src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
##########
@@ -51,7 +51,7 @@
   private int lgCurrentCapacity_;
   private final int lgResizeFactor_;
   private int count_;
-  private final SummaryFactory<S> summaryFactory_;
+  //private final SummaryFactory<S> summaryFactory_;

Review comment:
       not needed?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       here was another "unused" comment

##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -80,13 +78,12 @@ void fromArrays(final long[] hashArr, final S[] summaryArr, final int count) {
     count_ = count;
     lgTableSize_ = getLgTableSize(count);
 
-    S mySummary = null;
     summaryTable_ = null;
     hashTable_ = new long[1 << lgTableSize_];
     for (int i = 0; i < count; i++) {
       final long hash = hashArr[i];
       final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
-      mySummary = summaryArr[i];
+      final S mySummary = summaryArr[i];

Review comment:
       I suppose in this case it is fine not to make copies. The sketch can take ownership of the given objects, right?




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       If you look at line 53 (corrected) it reads:
   `EXACT,        //{ 1.0, >0, F} Bin: 110  Oct: 06, specify only value`
   
   And line 59:
   `//NOTE: 0 values in getSketch are not used.`
   Examine lines 512-515, p is not used.
   
   So for line 118:
   `UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);`
   
   I could have used any integer value for this test, I just happened to choose GT_MIDP_V (3).  The 0 is not used. 




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

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

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



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


[GitHub] [datasketches-java] leerho commented on pull request #369: Fix Mikhail's bug

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


   If there are no further issues, I will merge this PR as it more than satisfies the objective of fixing Mikhail's bug.  Nonetheless, there are some other checks that I would like to do, but those can be in separate PRs.  I'll leave this open for another 24 hours.
   


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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -177,23 +215,61 @@ public void notB(final Sketch<S> skB) {
    *
    * @param skB The incoming Theta sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final org.apache.datasketches.theta.Sketch skB) {
-    if (empty_ || skB == null || skB.isEmpty()) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+    if (skB == null) { return; } //ignore
 
-    //process B
-    final DataArrays<S> daB = getResultArraysTheta(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
-
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;
+      }
+      case SKA_TRIM: {
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        final DataArrays<S> da = trimDataArrays(hashArr_, summaryArr_,thetaLong_);
+        hashArr_ = da.hashArr;
+        curCount_ = (hashArr_ == null) ? 0 : hashArr_.length;
+        summaryArr_ = da.summaryArr;
+        break;
+      }
+      case SKETCH_A: {
+        break; //result is already in A
+      }
+      case FULL_ANOTB: { //both A and B should have valid entries.
+        thetaLong_ = min(thetaLong_, skB.getThetaLong());

Review comment:
       thetaLongB is already available in this scope and can be used instead of skB.getThetaLong.

##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTupleTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.tuple.aninteger;
+
+import org.apache.datasketches.tuple.AnotB;
+import org.apache.datasketches.tuple.CompactSketch;
+import org.apache.datasketches.tuple.Intersection;
+import org.testng.annotations.Test;
+
+/**
+ * Issue #368, from Mikhail Lavrinovich 12 OCT 2021
+ * The failure was AnotB(estimating {<1.0,1,F}, Intersect(estimating{<1.0,1,F}, newDegenerative{<1.0,0,T},
+ * Which should be equal to AnotB(estimating{<1.0,1,F}, new{1.0,0,T} = estimating{<1.0, 1, F}. The AnotB
+ * threw a null pointer exception because it was not properly handling sketches with zero entries.
+ */
+public class MikhailsBugTupleTest {
+
+  @Test
+  public void mikhailsBug() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    AnotB.aNotB(x, intersect); // NPE was here
+  }
+
+  //@Test
+  public void withTuple() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    println("Tuple x: Estimating {<1.0,1,F}");
+    println(x.toString());
+    println("Tuple y: NewDegenerative {<1.0,0,T}");
+    println(y.toString());
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    println("Tuple Intersect(Estimating, NewDegen) = new {1.0, 0, T}");
+    println(intersect.toString());
+    CompactSketch<IntegerSummary> csk = AnotB.aNotB(x, intersect);
+    println("Tuple AnotB(Estimating, New) = estimating {<1.0, 1, F}");
+    println(csk.toString());

Review comment:
       Would it be a good idea to use SetOperationCornerCases to verify the corner case matches - or is this unnecessary?

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       The website assigns 6 as octal id for the EXACT sketch type, which is incorrect - I have created a PR to update this on the website docs.

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Where sketch A is a NewDegen {<1.0,0,T}, why does the result not preserve the min theta rule?

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;

Review comment:
       It is fascinating that Theta sketch estimates are still calculated when the result contains no retained entries.  As you mention on the website, this is probably too complex a process to explain but for reference does this process occur in `BinomialBoundsN.java`?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -240,6 +242,34 @@ public int getCountLessThanThetaLong(final long thetaLong) {
     return bytes;
   }
 
+  @SuppressWarnings("unchecked")
+  CompactSketch<S> trimToTheta(final long thetaLong) {
+    final QuickSelectSketch<S> qsSk =
+        new QuickSelectSketch<>(this.getRetainedEntries(), ResizeFactor.X1.lg(), null);
+    int countOut = 0;
+    final SketchIterator<S> it = iterator();
+
+    while (it.next()) {
+      final long hash = it.getHash();
+      final S summary = it.getSummary();
+      if (hash < thetaLong) {
+        qsSk.insert(it.getHash(), (S)summary.copy());
+        countOut++;
+      }
+    }
+
+    qsSk.setThetaLong(thetaLong);
+    if (countOut == 0) {
+      if (thetaLong == Long.MAX_VALUE) {
+        return new CompactSketch<>(null, null, thetaLong, true);
+      } else {
+        return new CompactSketch<>(null, null, thetaLong, false);
+      }

Review comment:
       In the case where thetaLong is less than the max value, because it was constructed as a NewDegen {<1.0,0,T}, would using this as a check for emptiness be valid?

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       What effect does a sampling probability of zero have on whether a sketch stores a value in its buffer?  This question is in reference to line 118 where an exact sketch is created with a value greater than the MIDP_V.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       I incorrectly interpreted the bit for empty, thanks for clarifying.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       Makes sense, thanks.

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Thanks for the additional details, I understand the reason now.




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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on pull request #369: Fix Mikhail's bug

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on pull request #369:
URL: https://github.com/apache/datasketches-java/pull/369#issuecomment-959820525


   I expect to take a closer look at the logic while porting the tests (and, perhaps, adjusting the implementation) to C++


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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -118,9 +120,9 @@
         }
       }
     } else { // current serial format
-      offset++; // unused
+      offset++;

Review comment:
       will fix.  Thanks!

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {

Review comment:
       will fix, thanks!

##########
File path: src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
##########
@@ -51,7 +51,7 @@
   private int lgCurrentCapacity_;
   private final int lgResizeFactor_;
   private int count_;
-  private final SummaryFactory<S> summaryFactory_;
+  //private final SummaryFactory<S> summaryFactory_;

Review comment:
       It is not needed here as it is already in the parent "Sketch"

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private InterResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54
+
+    private static final Map<Integer, CornerCase> idToCornerCaseMap = new HashMap<>();
+    private int id;
+    private String inputStr;
+    private InterResult interResult;
+    private AnotbResult anotbResult;
+
+    static {
+      for (final CornerCase cc : values()) {
+        idToCornerCaseMap.put(cc.getId(), cc);
+      }
+    }
+
+    private CornerCase(final int id, final String desc,
+        final InterResult interResult, final AnotbResult anotbResult) {
+      this.id = id;
+      this.inputStr = desc;
+      this.interResult = interResult;
+      this.anotbResult = anotbResult;
+    }
+
+    public int getId() {
+      return id;
+    }
+
+    public String getDesc() {
+      return inputStr;
+    }
+
+    public InterResult getInterResult() {
+      return interResult;
+    }
+
+    public AnotbResult getAnotbResult() {
+      return anotbResult;
+    }
+
+    public static CornerCase idToCornerCase(final int id) {
+      final CornerCase cc = idToCornerCaseMap.get(id);
+      if (cc == null) {
+        throw new SketchesArgumentException("Possible Corruption: Illegal CornerCase ID: " + id);
+      }
+      return cc;
+    }
+  } //end of enum
+
+  public static int createCornerCaseId(
+      final long thetaLongA, final int countA, final boolean emptyA,
+      final long thetaLongB, final int countB, final boolean emptyB) {
+    return ((thetaLongA < MAX) ? 0 : 1 << 5)
+         | ((countA == 0)      ? 0 : 1 << 4)
+         | (!emptyA            ? 0 : 1 << 3)
+         | ((thetaLongB < MAX) ? 0 : 1 << 2)
+         | ((countB == 0)      ? 0 : 1 << 1)
+         | (!emptyB            ? 0 : 1);
+  }
+
+//  @Test

Review comment:
       No, actually, it is a very useful when debugging, but I agree that it is in the wrong place. So I moved it to /tuple/MiscTest and leave a short reference to it above the relevant method here.  Thanks!

##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.tuple;
+
+import static java.lang.Math.ceil;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.HashOperations.hashInsertOnly;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.ceilingPowerOf2;
+
+import java.lang.reflect.Array;
+
+@SuppressWarnings("unchecked")
+class HashTables<S extends Summary> {
+  long[] hashTable_ = null;
+  S[] summaryTable_ = null;
+  int lgTableSize_ = 0;
+  int count_ = 0;
+
+  HashTables() { }
+
+  //must have valid entries
+  void fromSketch(final Sketch<S> sketch) {
+    count_ = sketch.getRetainedEntries();
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final SketchIterator<S> it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.getHash();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = (S)it.getSummary().copy();
+      if (summaryTable_ == null) {
+        summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
+      }
+      summaryTable_[index] = mySummary;
+    }
+  }
+
+  //must have valid entries
+  void fromSketch(final org.apache.datasketches.theta.Sketch sketch, final S summary) {
+    count_ = sketch.getRetainedEntries(true);
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final org.apache.datasketches.theta.HashIterator it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.get();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = summary;

Review comment:
       You are correct.  Thanks!




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -118,9 +120,9 @@
         }
       }
     } else { // current serial format
-      offset++; // unused
+      offset++;

Review comment:
       will fix.  Thanks!




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/Intersection.java
##########
@@ -107,14 +107,19 @@ public void intersect(final Sketch<S> tupleSketch) {
     if (tupleSketch == null) { throw new SketchesArgumentException("Sketch must not be null"); }
     final boolean firstCall = firstCall_;
     firstCall_ = false;
+    final boolean emptyIn = tupleSketch.isEmpty();
+    if (empty_ || emptyIn) { //empty rule
+      //Because of the definition of null above and the Empty Rule (which is OR), empty_ must be true.
+      //Whatever the current internal state, we make our local empty.
+      resetToEmpty();
+      return;

Review comment:
       OK - I understand this to mean that in order to mark the current state empty different boolean logic is applied to the state of each operand.  




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTuple.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.tuple.aninteger;
+
+import org.apache.datasketches.tuple.AnotB;
+import org.apache.datasketches.tuple.CompactSketch;
+import org.apache.datasketches.tuple.Intersection;
+import org.testng.annotations.Test;
+
+/**
+ * Issue #368, from Mikhail Lavrinovich 12 OCT 2021
+ * The failure was AnotB(estimating {<1.0,1,F}, Intersect(estimating{<1.0,1,F}, newDegenerative{<1.0,0,T},
+ * Which should be equal to AnotB(estimating{<1.0,1,F}, new{1.0,0,T} = estimating{<1.0, 1, F}. The AnotB
+ * threw a null pointer exception because it was not properly handling sketches with zero entries.
+ */

Review comment:
       > degenerate (technical definition): lacking some property, order, or distinctness of structure previously or usually present.
   
   The use of the term in the datasketches-java dictionary apply to sketches that are in exact mode, because they are _not_ sketching. They are essentially a list of hashes.  The property that is missing is the sketching property of probabilistic estimation. 
   
   Here I'm using degenerate in a different sense.  Sketches where Theta < 1.0 and have zero entries.  The property that is missing are the entries! 




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/theta/AnotBimpl.java
##########
@@ -119,17 +121,22 @@ public CompactSketch aNotB(final Sketch skA, final Sketch skB, final boolean dst
     }
     //Both skA & skB are not null
 
+    final long minThetaLong = Math.min(skA.getThetaLong(), skB.getThetaLong());
+
     if (skA.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
+    //A is not Empty
     checkSeedHashes(skA.getSeedHash(), seedHash_);
 
-    if (skB.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
+    if (skB.isEmpty() && skB.getRetainedEntries() == 0) {
+      return skA.compact(dstOrdered, dstMem);
+   }

Review comment:
       This redundant check was in both Theta and Tuple.




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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTuple.java
##########
@@ -0,0 +1,74 @@
+/*

Review comment:
       I think this file should have a "Test" suffix to follow our naming convention.




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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.tuple;
+
+import static java.lang.Math.ceil;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.HashOperations.hashInsertOnly;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.ceilingPowerOf2;
+
+import java.lang.reflect.Array;
+
+@SuppressWarnings("unchecked")
+class HashTables<S extends Summary> {
+  long[] hashTable_ = null;
+  S[] summaryTable_ = null;
+  int lgTableSize_ = 0;
+  int count_ = 0;
+
+  HashTables() { }
+
+  //must have valid entries
+  void fromSketch(final Sketch<S> sketch) {
+    count_ = sketch.getRetainedEntries();
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final SketchIterator<S> it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.getHash();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = (S)it.getSummary().copy();
+      if (summaryTable_ == null) {
+        summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
+      }
+      summaryTable_[index] = mySummary;
+    }
+  }
+
+  //must have valid entries
+  void fromSketch(final org.apache.datasketches.theta.Sketch sketch, final S summary) {
+    count_ = sketch.getRetainedEntries(true);
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final org.apache.datasketches.theta.HashIterator it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.get();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = summary;

Review comment:
       should this be a copy as in the method above?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -118,9 +120,9 @@
         }
       }
     } else { // current serial format
-      offset++; // unused
+      offset++;

Review comment:
       was the comment unnecessary? perhaps it should be a bit more verbose like "skip unused byte"

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {

Review comment:
       perhaps "IntersectResult" would be a bit more descriptive? It took me a while to understand

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private InterResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54
+
+    private static final Map<Integer, CornerCase> idToCornerCaseMap = new HashMap<>();
+    private int id;
+    private String inputStr;
+    private InterResult interResult;
+    private AnotbResult anotbResult;
+
+    static {
+      for (final CornerCase cc : values()) {
+        idToCornerCaseMap.put(cc.getId(), cc);
+      }
+    }
+
+    private CornerCase(final int id, final String desc,
+        final InterResult interResult, final AnotbResult anotbResult) {
+      this.id = id;
+      this.inputStr = desc;
+      this.interResult = interResult;
+      this.anotbResult = anotbResult;
+    }
+
+    public int getId() {
+      return id;
+    }
+
+    public String getDesc() {
+      return inputStr;
+    }
+
+    public InterResult getInterResult() {
+      return interResult;
+    }
+
+    public AnotbResult getAnotbResult() {
+      return anotbResult;
+    }
+
+    public static CornerCase idToCornerCase(final int id) {
+      final CornerCase cc = idToCornerCaseMap.get(id);
+      if (cc == null) {
+        throw new SketchesArgumentException("Possible Corruption: Illegal CornerCase ID: " + id);
+      }
+      return cc;
+    }
+  } //end of enum
+
+  public static int createCornerCaseId(
+      final long thetaLongA, final int countA, final boolean emptyA,
+      final long thetaLongB, final int countB, final boolean emptyB) {
+    return ((thetaLongA < MAX) ? 0 : 1 << 5)
+         | ((countA == 0)      ? 0 : 1 << 4)
+         | (!emptyA            ? 0 : 1 << 3)
+         | ((thetaLongB < MAX) ? 0 : 1 << 2)
+         | ((countB == 0)      ? 0 : 1 << 1)
+         | (!emptyB            ? 0 : 1);
+  }
+
+//  @Test

Review comment:
       forgot to remove?

##########
File path: src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
##########
@@ -51,7 +51,7 @@
   private int lgCurrentCapacity_;
   private final int lgResizeFactor_;
   private int count_;
-  private final SummaryFactory<S> summaryFactory_;
+  //private final SummaryFactory<S> summaryFactory_;

Review comment:
       not needed?




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {

Review comment:
       will fix, thanks!




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -118,9 +120,9 @@
         }
       }
     } else { // current serial format
-      offset++; // unused
+      offset++;

Review comment:
       will fix.  Thanks!

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {

Review comment:
       will fix, thanks!

##########
File path: src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
##########
@@ -51,7 +51,7 @@
   private int lgCurrentCapacity_;
   private final int lgResizeFactor_;
   private int count_;
-  private final SummaryFactory<S> summaryFactory_;
+  //private final SummaryFactory<S> summaryFactory_;

Review comment:
       It is not needed here as it is already in the parent "Sketch"

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private InterResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54
+
+    private static final Map<Integer, CornerCase> idToCornerCaseMap = new HashMap<>();
+    private int id;
+    private String inputStr;
+    private InterResult interResult;
+    private AnotbResult anotbResult;
+
+    static {
+      for (final CornerCase cc : values()) {
+        idToCornerCaseMap.put(cc.getId(), cc);
+      }
+    }
+
+    private CornerCase(final int id, final String desc,
+        final InterResult interResult, final AnotbResult anotbResult) {
+      this.id = id;
+      this.inputStr = desc;
+      this.interResult = interResult;
+      this.anotbResult = anotbResult;
+    }
+
+    public int getId() {
+      return id;
+    }
+
+    public String getDesc() {
+      return inputStr;
+    }
+
+    public InterResult getInterResult() {
+      return interResult;
+    }
+
+    public AnotbResult getAnotbResult() {
+      return anotbResult;
+    }
+
+    public static CornerCase idToCornerCase(final int id) {
+      final CornerCase cc = idToCornerCaseMap.get(id);
+      if (cc == null) {
+        throw new SketchesArgumentException("Possible Corruption: Illegal CornerCase ID: " + id);
+      }
+      return cc;
+    }
+  } //end of enum
+
+  public static int createCornerCaseId(
+      final long thetaLongA, final int countA, final boolean emptyA,
+      final long thetaLongB, final int countB, final boolean emptyB) {
+    return ((thetaLongA < MAX) ? 0 : 1 << 5)
+         | ((countA == 0)      ? 0 : 1 << 4)
+         | (!emptyA            ? 0 : 1 << 3)
+         | ((thetaLongB < MAX) ? 0 : 1 << 2)
+         | ((countB == 0)      ? 0 : 1 << 1)
+         | (!emptyB            ? 0 : 1);
+  }
+
+//  @Test

Review comment:
       No, actually, it is a very useful when debugging, but I agree that it is in the wrong place. So I moved it to /tuple/MiscTest and leave a short reference to it above the relevant method here.  Thanks!

##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.tuple;
+
+import static java.lang.Math.ceil;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.HashOperations.hashInsertOnly;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.ceilingPowerOf2;
+
+import java.lang.reflect.Array;
+
+@SuppressWarnings("unchecked")
+class HashTables<S extends Summary> {
+  long[] hashTable_ = null;
+  S[] summaryTable_ = null;
+  int lgTableSize_ = 0;
+  int count_ = 0;
+
+  HashTables() { }
+
+  //must have valid entries
+  void fromSketch(final Sketch<S> sketch) {
+    count_ = sketch.getRetainedEntries();
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final SketchIterator<S> it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.getHash();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = (S)it.getSummary().copy();
+      if (summaryTable_ == null) {
+        summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
+      }
+      summaryTable_[index] = mySummary;
+    }
+  }
+
+  //must have valid entries
+  void fromSketch(final org.apache.datasketches.theta.Sketch sketch, final S summary) {
+    count_ = sketch.getRetainedEntries(true);
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final org.apache.datasketches.theta.HashIterator it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.get();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = summary;

Review comment:
       You are correct.  Thanks!




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTupleTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.tuple.aninteger;
+
+import org.apache.datasketches.tuple.AnotB;
+import org.apache.datasketches.tuple.CompactSketch;
+import org.apache.datasketches.tuple.Intersection;
+import org.testng.annotations.Test;
+
+/**
+ * Issue #368, from Mikhail Lavrinovich 12 OCT 2021
+ * The failure was AnotB(estimating {<1.0,1,F}, Intersect(estimating{<1.0,1,F}, newDegenerative{<1.0,0,T},
+ * Which should be equal to AnotB(estimating{<1.0,1,F}, new{1.0,0,T} = estimating{<1.0, 1, F}. The AnotB
+ * threw a null pointer exception because it was not properly handling sketches with zero entries.
+ */
+public class MikhailsBugTupleTest {
+
+  @Test
+  public void mikhailsBug() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    AnotB.aNotB(x, intersect); // NPE was here
+  }
+
+  //@Test
+  public void withTuple() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    println("Tuple x: Estimating {<1.0,1,F}");
+    println(x.toString());
+    println("Tuple y: NewDegenerative {<1.0,0,T}");
+    println(y.toString());
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    println("Tuple Intersect(Estimating, NewDegen) = new {1.0, 0, T}");
+    println(intersect.toString());
+    CompactSketch<IntegerSummary> csk = AnotB.aNotB(x, intersect);
+    println("Tuple AnotB(Estimating, New) = estimating {<1.0, 1, F}");
+    println(csk.toString());

Review comment:
       This code is a replica of Mikhail's code as he wrote it, plus some print statements.  I make it a practice to place test code of discovered bugs, among the tests as a way of making sure we covered the case he found.  Nonetheless, the tests that I constructed are far more comprehensive and covers his case and many others. 




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

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

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



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


[GitHub] [datasketches-java] leerho commented on pull request #369: Fix Mikhail's bug

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


   If there are no further issues, I will merge this PR as it more than satisfies the objective of fixing Mikhail's bug.  Nonetheless, there are some other checks that I would like to do, but those can be in separate PRs.  I'll leave this open for another 24 hours.
   


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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/theta/AnotBimpl.java
##########
@@ -119,17 +121,22 @@ public CompactSketch aNotB(final Sketch skA, final Sketch skB, final boolean dst
     }
     //Both skA & skB are not null
 
+    final long minThetaLong = Math.min(skA.getThetaLong(), skB.getThetaLong());
+
     if (skA.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
+    //A is not Empty
     checkSeedHashes(skA.getSeedHash(), seedHash_);
 
-    if (skB.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
+    if (skB.isEmpty() && skB.getRetainedEntries() == 0) {
+      return skA.compact(dstOrdered, dstMem);
+   }

Review comment:
       Good catch, and you are correct.  Thank you for finding this.  This is a redundant check that I inserted while debugging.  I will remove it.   I will answer your other questions with some more extensive documentation, still being developed. :)




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.tuple;
+
+import static java.lang.Math.ceil;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.HashOperations.hashInsertOnly;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.ceilingPowerOf2;
+
+import java.lang.reflect.Array;
+
+@SuppressWarnings("unchecked")
+class HashTables<S extends Summary> {
+  long[] hashTable_ = null;
+  S[] summaryTable_ = null;
+  int lgTableSize_ = 0;
+  int count_ = 0;
+
+  HashTables() { }
+
+  //must have valid entries
+  void fromSketch(final Sketch<S> sketch) {
+    count_ = sketch.getRetainedEntries();
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final SketchIterator<S> it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.getHash();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = (S)it.getSummary().copy();
+      if (summaryTable_ == null) {
+        summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
+      }
+      summaryTable_[index] = mySummary;
+    }
+  }
+
+  //must have valid entries
+  void fromSketch(final org.apache.datasketches.theta.Sketch sketch, final S summary) {
+    count_ = sketch.getRetainedEntries(true);
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final org.apache.datasketches.theta.HashIterator it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.get();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = summary;

Review comment:
       You are correct.  Thanks!




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on pull request #369: Fix Mikhail's bug

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


   The related documentation on the website is very instructive and helped with my previous confusion about the different cases that were being tested.   The use of enums to convey the corner case in question back to the AnotB set operation was also a nice approach. 


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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       Agreed.  Sorry I missed 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.

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       This change is what you suggested in your previous review.  This is not "another", it is the same 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.

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -177,23 +215,61 @@ public void notB(final Sketch<S> skB) {
    *
    * @param skB The incoming Theta sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final org.apache.datasketches.theta.Sketch skB) {
-    if (empty_ || skB == null || skB.isEmpty()) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+    if (skB == null) { return; } //ignore
 
-    //process B
-    final DataArrays<S> daB = getResultArraysTheta(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
-
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;
+      }
+      case SKA_TRIM: {
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        final DataArrays<S> da = trimDataArrays(hashArr_, summaryArr_,thetaLong_);
+        hashArr_ = da.hashArr;
+        curCount_ = (hashArr_ == null) ? 0 : hashArr_.length;
+        summaryArr_ = da.summaryArr;
+        break;
+      }
+      case SKETCH_A: {
+        break; //result is already in A
+      }
+      case FULL_ANOTB: { //both A and B should have valid entries.
+        thetaLong_ = min(thetaLong_, skB.getThetaLong());

Review comment:
       thetaLongB is already available in this scope and can be used instead of skB.getThetaLong.

##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTupleTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.tuple.aninteger;
+
+import org.apache.datasketches.tuple.AnotB;
+import org.apache.datasketches.tuple.CompactSketch;
+import org.apache.datasketches.tuple.Intersection;
+import org.testng.annotations.Test;
+
+/**
+ * Issue #368, from Mikhail Lavrinovich 12 OCT 2021
+ * The failure was AnotB(estimating {<1.0,1,F}, Intersect(estimating{<1.0,1,F}, newDegenerative{<1.0,0,T},
+ * Which should be equal to AnotB(estimating{<1.0,1,F}, new{1.0,0,T} = estimating{<1.0, 1, F}. The AnotB
+ * threw a null pointer exception because it was not properly handling sketches with zero entries.
+ */
+public class MikhailsBugTupleTest {
+
+  @Test
+  public void mikhailsBug() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    AnotB.aNotB(x, intersect); // NPE was here
+  }
+
+  //@Test
+  public void withTuple() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    println("Tuple x: Estimating {<1.0,1,F}");
+    println(x.toString());
+    println("Tuple y: NewDegenerative {<1.0,0,T}");
+    println(y.toString());
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    println("Tuple Intersect(Estimating, NewDegen) = new {1.0, 0, T}");
+    println(intersect.toString());
+    CompactSketch<IntegerSummary> csk = AnotB.aNotB(x, intersect);
+    println("Tuple AnotB(Estimating, New) = estimating {<1.0, 1, F}");
+    println(csk.toString());

Review comment:
       Would it be a good idea to use SetOperationCornerCases to verify the corner case matches - or is this unnecessary?

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       The website assigns 6 as octal id for the EXACT sketch type, which is incorrect - I have created a PR to update this on the website docs.

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Where sketch A is a NewDegen {<1.0,0,T}, why does the result not preserve the min theta rule?

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;

Review comment:
       It is fascinating that Theta sketch estimates are still calculated when the result contains no retained entries.  As you mention on the website, this is probably too complex a process to explain but for reference does this process occur in `BinomialBoundsN.java`?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -240,6 +242,34 @@ public int getCountLessThanThetaLong(final long thetaLong) {
     return bytes;
   }
 
+  @SuppressWarnings("unchecked")
+  CompactSketch<S> trimToTheta(final long thetaLong) {
+    final QuickSelectSketch<S> qsSk =
+        new QuickSelectSketch<>(this.getRetainedEntries(), ResizeFactor.X1.lg(), null);
+    int countOut = 0;
+    final SketchIterator<S> it = iterator();
+
+    while (it.next()) {
+      final long hash = it.getHash();
+      final S summary = it.getSummary();
+      if (hash < thetaLong) {
+        qsSk.insert(it.getHash(), (S)summary.copy());
+        countOut++;
+      }
+    }
+
+    qsSk.setThetaLong(thetaLong);
+    if (countOut == 0) {
+      if (thetaLong == Long.MAX_VALUE) {
+        return new CompactSketch<>(null, null, thetaLong, true);
+      } else {
+        return new CompactSketch<>(null, null, thetaLong, false);
+      }

Review comment:
       In the case where thetaLong is less than the max value, because it was constructed as a NewDegen {<1.0,0,T}, would using this as a check for emptiness be valid?

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       What effect does a sampling probability of zero have on whether a sketch stores a value in its buffer?  This question is in reference to line 118 where an exact sketch is created with a value greater than the MIDP_V.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on pull request #369: Fix Mikhail's bug

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


   If there are no further issues, I will merge this PR as it more than satisfies the objective of fixing Mikhail's bug.  Nonetheless, there are some other checks that I would like to do, but those can be in separate PRs.  I'll leave this open for another 24 hours.
   


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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       I am referring to line 125, where was another "// unused" comment. I would suggest restoring it as well in a revised form: "// skip two unused bytes"

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       or "skip unused bytes"




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on pull request #369: Fix Mikhail's bug

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


   The related documentation on the website is very instructive and helped with my previous confusion about the different cases that were being tested.   The use of enums to convey the corner case in question back to the AnotB set operation was also a nice approach. 


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

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

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



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


[GitHub] [datasketches-java] leerho commented on pull request #369: Fix Mikhail's bug

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


   An overview and explanation of the key changes contained in the latest updates can be read on the website:  
   [https://datasketches.apache.org/docs/Theta/ThetaSetOpsCornerCases.html ](https://datasketches.apache.org/docs/Theta/ThetaSetOpsCornerCases.html )


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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/theta/AnotBimpl.java
##########
@@ -119,17 +121,22 @@ public CompactSketch aNotB(final Sketch skA, final Sketch skB, final boolean dst
     }
     //Both skA & skB are not null
 
+    final long minThetaLong = Math.min(skA.getThetaLong(), skB.getThetaLong());
+
     if (skA.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
+    //A is not Empty
     checkSeedHashes(skA.getSeedHash(), seedHash_);
 
-    if (skB.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
+    if (skB.isEmpty() && skB.getRetainedEntries() == 0) {
+      return skA.compact(dstOrdered, dstMem);
+   }

Review comment:
       My understanding is that a sketch with no retained entries might be the result of an operation such as intersection between two disjoint sets where either set might or might not be non empty.
   Does the `isEmpty` check apply only to a sketch that has no entries, and if so, why would it also be necessary to check the retained entries?  
   In the context of the issue report, it appears that this this additional check is necessary because the skB operand is the result of a disjoint intersection (where there are no retained entries).  

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -116,14 +117,23 @@ public void setA(final Sketch<S> skA) {
       return;
     }
     //skA is not empty
-    empty_ = false;
-    thetaLong_ = skA.getThetaLong();
 
     //process A
+    empty_ = false;
+    thetaLong_ = skA.getThetaLong();
     final DataArrays<S> da = getDataArraysA(skA);
+
     hashArr_ = da.hashArr;
-    summaryArr_ = da.summaryArr;
+    hashArr_ = (hashArr_ == null) ? new long[0] : hashArr_;
     curCount_ = hashArr_.length;
+
+    summaryArr_ = da.summaryArr;
+    if (summaryArr_ == null) {
+      final SummaryFactory<S> sumFact = ((QuickSelectSketch<S>)skA).getSummaryFactory();
+      final S summary = sumFact.newSummary();
+      final Class<S> summaryType = (Class<S>)summary.getClass();
+      summaryArr_ = (S[]) Array.newInstance(summaryType, 0);
+    }

Review comment:
       What is your opinion on defaulting the summary array on `DataArrays<S>` within the getDataArraysA method?

##########
File path: src/main/java/org/apache/datasketches/tuple/Intersection.java
##########
@@ -107,14 +107,19 @@ public void intersect(final Sketch<S> tupleSketch) {
     if (tupleSketch == null) { throw new SketchesArgumentException("Sketch must not be null"); }
     final boolean firstCall = firstCall_;
     firstCall_ = false;
+    final boolean emptyIn = tupleSketch.isEmpty();
+    if (empty_ || emptyIn) { //empty rule
+      //Because of the definition of null above and the Empty Rule (which is OR), empty_ must be true.
+      //Whatever the current internal state, we make our local empty.
+      resetToEmpty();
+      return;

Review comment:
       Does the "Empty Rule" specifically apply to stateful set intersections, or is it defined formally in the Theta Sketch Framework?

##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTuple.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.tuple.aninteger;
+
+import org.apache.datasketches.tuple.AnotB;
+import org.apache.datasketches.tuple.CompactSketch;
+import org.apache.datasketches.tuple.Intersection;
+import org.testng.annotations.Test;
+
+/**
+ * Issue #368, from Mikhail Lavrinovich 12 OCT 2021
+ * The failure was AnotB(estimating {<1.0,1,F}, Intersect(estimating{<1.0,1,F}, newDegenerative{<1.0,0,T},
+ * Which should be equal to AnotB(estimating{<1.0,1,F}, new{1.0,0,T} = estimating{<1.0, 1, F}. The AnotB
+ * threw a null pointer exception because it was not properly handling sketches with zero entries.
+ */

Review comment:
       I've seen the term degenerate applied in the resource dictionary to sketches that are not in estimation mode.  If this is the case, why are the sketches in the example not all degenerative?

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,18 +153,28 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
+    if (empty_ || skB == null || skB.isEmpty()) { return; }

Review comment:
       Why are the retained entries for skB not checked here? (it is checked in the stateless aNotB)




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,18 +153,28 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
+    if (empty_ || skB == null || skB.isEmpty()) { return; }

Review comment:
       This comment no longer applies - it is redundant to check both as per the comment above.




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       I incorrectly interpreted the bit for empty, thanks for clarifying.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       Makes sense, thanks.

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Thanks for the additional details, I understand the reason now.




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum IntersectResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private IntersectResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54

Review comment:
       Some of the descriptions for degenerative sketches use >1.0 for Theta which looks like a typo.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       No. The website is correct, the comment next to (now line 63 in this class) was incorrectly marked as 7.  It should be 6.  But in this test class, these octal values are not used.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTuple.java
##########
@@ -0,0 +1,74 @@
+/*

Review comment:
       Agreed.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/Intersection.java
##########
@@ -107,14 +107,19 @@ public void intersect(final Sketch<S> tupleSketch) {
     if (tupleSketch == null) { throw new SketchesArgumentException("Sketch must not be null"); }
     final boolean firstCall = firstCall_;
     firstCall_ = false;
+    final boolean emptyIn = tupleSketch.isEmpty();
+    if (empty_ || emptyIn) { //empty rule
+      //Because of the definition of null above and the Empty Rule (which is OR), empty_ must be true.
+      //Whatever the current internal state, we make our local empty.
+      resetToEmpty();
+      return;

Review comment:
       The result state is dependent on the type of operation (union, intersection, AnotB) and the actual states of the arguments.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       Oops the line number is 53 (not 63). But my comment still stands.




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Thanks for the additional details, I understand the reason now.




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum IntersectResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private IntersectResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54

Review comment:
       Some of the descriptions for degenerative sketches use >1.0 for Theta which looks like a typo.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -80,13 +78,12 @@ void fromArrays(final long[] hashArr, final S[] summaryArr, final int count) {
     count_ = count;
     lgTableSize_ = getLgTableSize(count);
 
-    S mySummary = null;
     summaryTable_ = null;
     hashTable_ = new long[1 << lgTableSize_];
     for (int i = 0; i < count; i++) {
       final long hash = hashArr[i];
       final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
-      mySummary = summaryArr[i];
+      final S mySummary = summaryArr[i];

Review comment:
       In this case, it is fine.  
   
   There are 2 ways to initialize the HashTables class: `fromSketch(...tuple.Sketch<S>)` and `fromSketch(...theta.Sketch)`.  In both cases the elements of the internal summaries arrays are copies of the summaries of the presented sketch, or copies of the provided summary in the case of theta.
   
   The method you ask about, `fromArrays(...)`, is a private method (now private :) ) and only called by the 2 `getIntersectHashTables(next*Sketch,...)` methods.   These two methods rely on the `SummarySetOperations<S>` class to perform the configured set operation with the internal state of `HashTables` and the provided `next*Sketch`.   
   
   Meanwhile, the Javadoc of the `SummarySetOperations<S>` interface has a clear warning for implementers:
   
   ```
   Caution: Do not modify the input Summary objects. Also do not return them directly,
      unless they are immutable (most Summary objects are not). For mutable Summary objects, it is
      important to create a new Summary object with the correct contents to be returned. Do not
      return null summaries.
   ```
   Assuming that this interface is implemented correctly by the user, there should not be any Summary objects belonging to input sketches that end up in the internals of these set operations.  This means that the extra `copy()`, line 150, I added in the last review should not be necessary.
   
   A further question is whether we need to make sure no internal summary objects end up in result sketches.  I'm going to have to think more about this, but I think we may be OK, as the internals of these set operations are frequently cleared, ... nonetheless :)

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum IntersectResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private IntersectResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54

Review comment:
       Thank you!  

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       Agreed.  Sorry I missed it.

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -177,23 +215,61 @@ public void notB(final Sketch<S> skB) {
    *
    * @param skB The incoming Theta sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final org.apache.datasketches.theta.Sketch skB) {
-    if (empty_ || skB == null || skB.isEmpty()) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
+    if (skB == null) { return; } //ignore
 
-    //process B
-    final DataArrays<S> daB = getResultArraysTheta(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
-
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;
+      }
+      case SKA_TRIM: {
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        final DataArrays<S> da = trimDataArrays(hashArr_, summaryArr_,thetaLong_);
+        hashArr_ = da.hashArr;
+        curCount_ = (hashArr_ == null) ? 0 : hashArr_.length;
+        summaryArr_ = da.summaryArr;
+        break;
+      }
+      case SKETCH_A: {
+        break; //result is already in A
+      }
+      case FULL_ANOTB: { //both A and B should have valid entries.
+        thetaLong_ = min(thetaLong_, skB.getThetaLong());

Review comment:
       You are correct!

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Ah, an excellent observation and good question!  The New Degenerate {<1.0, 0, T} is a special temporary state in that it has never seen any data.  When the first item is presented to a New Degenerate sketch the T becomes F, and the count will only be changed to 1 if the hash of the item is <theta, which becomes {<1.0, 1, F}, otherwise it becomes {<1.0, 0, F}; both of which are stable states.
   
   Because the New Degenerate sketch has never seen any data, it is interpreted as if it were just New {1.0, 0, T} in any set operation. 

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;

Review comment:
       Yes!  I thought about writing out the math for this for the website, but it was too complicated for this particular subject page.  It really belongs in a more math oriented document.

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -240,6 +242,34 @@ public int getCountLessThanThetaLong(final long thetaLong) {
     return bytes;
   }
 
+  @SuppressWarnings("unchecked")
+  CompactSketch<S> trimToTheta(final long thetaLong) {
+    final QuickSelectSketch<S> qsSk =
+        new QuickSelectSketch<>(this.getRetainedEntries(), ResizeFactor.X1.lg(), null);
+    int countOut = 0;
+    final SketchIterator<S> it = iterator();
+
+    while (it.next()) {
+      final long hash = it.getHash();
+      final S summary = it.getSummary();
+      if (hash < thetaLong) {
+        qsSk.insert(it.getHash(), (S)summary.copy());
+        countOut++;
+      }
+    }
+
+    qsSk.setThetaLong(thetaLong);
+    if (countOut == 0) {
+      if (thetaLong == Long.MAX_VALUE) {
+        return new CompactSketch<>(null, null, thetaLong, true);
+      } else {
+        return new CompactSketch<>(null, null, thetaLong, false);
+      }

Review comment:
       The containing method was a dead method.  Nothing calls it.  This was left over from the previous set operation logic. I probably have other dead code, which I will look for. Thanks!

##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTupleTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.tuple.aninteger;
+
+import org.apache.datasketches.tuple.AnotB;
+import org.apache.datasketches.tuple.CompactSketch;
+import org.apache.datasketches.tuple.Intersection;
+import org.testng.annotations.Test;
+
+/**
+ * Issue #368, from Mikhail Lavrinovich 12 OCT 2021
+ * The failure was AnotB(estimating {<1.0,1,F}, Intersect(estimating{<1.0,1,F}, newDegenerative{<1.0,0,T},
+ * Which should be equal to AnotB(estimating{<1.0,1,F}, new{1.0,0,T} = estimating{<1.0, 1, F}. The AnotB
+ * threw a null pointer exception because it was not properly handling sketches with zero entries.
+ */
+public class MikhailsBugTupleTest {
+
+  @Test
+  public void mikhailsBug() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    AnotB.aNotB(x, intersect); // NPE was here
+  }
+
+  //@Test
+  public void withTuple() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    println("Tuple x: Estimating {<1.0,1,F}");
+    println(x.toString());
+    println("Tuple y: NewDegenerative {<1.0,0,T}");
+    println(y.toString());
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    println("Tuple Intersect(Estimating, NewDegen) = new {1.0, 0, T}");
+    println(intersect.toString());
+    CompactSketch<IntegerSummary> csk = AnotB.aNotB(x, intersect);
+    println("Tuple AnotB(Estimating, New) = estimating {<1.0, 1, F}");
+    println(csk.toString());

Review comment:
       This code is a replica of Mikhail's code as he wrote it, plus some print statements.  I make it a practice to place test code of discovered bugs, among the tests as a way of making sure we covered the case he found.  Nonetheless, the tests that I constructed are far more comprehensive and covers his case and many others. 

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       No. The website is correct, the comment next to (now line 63 in this class) was incorrectly marked as 7.  It should be 6.  But in this test class, these octal values are not used.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       Oops the line number is 53 (not 63). But my comment still stands.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       If you look at line 53 (corrected) it reads:
   `EXACT,        //{ 1.0, >0, F} Bin: 110  Oct: 06, specify only value`
   
   And line 59:
   `//NOTE: 0 values in getSketch are not used.`
   Examine lines 512-515, p is not used.
   
   So for line 118:
   `UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);`
   
   I could have used any integer value for this test, I just happened to choose GT_MIDP_V (3).  The 0 is not used. 

##########
File path: src/test/java/org/apache/datasketches/tuple/aninteger/MikhailsBugTupleTest.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.tuple.aninteger;
+
+import org.apache.datasketches.tuple.AnotB;
+import org.apache.datasketches.tuple.CompactSketch;
+import org.apache.datasketches.tuple.Intersection;
+import org.testng.annotations.Test;
+
+/**
+ * Issue #368, from Mikhail Lavrinovich 12 OCT 2021
+ * The failure was AnotB(estimating {<1.0,1,F}, Intersect(estimating{<1.0,1,F}, newDegenerative{<1.0,0,T},
+ * Which should be equal to AnotB(estimating{<1.0,1,F}, new{1.0,0,T} = estimating{<1.0, 1, F}. The AnotB
+ * threw a null pointer exception because it was not properly handling sketches with zero entries.
+ */
+public class MikhailsBugTupleTest {
+
+  @Test
+  public void mikhailsBug() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    AnotB.aNotB(x, intersect); // NPE was here
+  }
+
+  //@Test
+  public void withTuple() {
+    IntegerSketch x = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    IntegerSketch y = new IntegerSketch(12, 2, 0.1f, IntegerSummary.Mode.Min);
+    x.update(1L, 1);
+    println("Tuple x: Estimating {<1.0,1,F}");
+    println(x.toString());
+    println("Tuple y: NewDegenerative {<1.0,0,T}");
+    println(y.toString());
+    IntegerSummarySetOperations setOperations =
+        new IntegerSummarySetOperations(IntegerSummary.Mode.Min, IntegerSummary.Mode.Min);
+    Intersection<IntegerSummary> intersection = new Intersection<>(setOperations);
+    CompactSketch<IntegerSummary> intersect = intersection.intersect(x, y);
+    println("Tuple Intersect(Estimating, NewDegen) = new {1.0, 0, T}");
+    println(intersect.toString());
+    CompactSketch<IntegerSummary> csk = AnotB.aNotB(x, intersect);
+    println("Tuple AnotB(Estimating, New) = estimating {<1.0, 1, F}");
+    println(csk.toString());

Review comment:
       This code is a replica of Mikhail's code as he wrote it, plus some print statements.  I make it a practice to place test code of discovered bugs, among the tests as a way of making sure we covered the case he found.  Nonetheless, the tests that I constructed are far more comprehensive and covers his case and many others. 

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       No. The website is correct, the comment next to (now line 63 in this class) was incorrectly marked as 7.  It should be 6.  But in this test class, these octal values are not used.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       Oops the line number is 53 (not 63). But my comment still stands.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       If you look at line 53 (corrected) it reads:
   `EXACT,        //{ 1.0, >0, F} Bin: 110  Oct: 06, specify only value`
   
   And line 59:
   `//NOTE: 0 values in getSketch are not used.`
   Examine lines 512-515, p is not used.
   
   So for line 118:
   `UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);`
   
   I could have used any integer value for this test, I just happened to choose GT_MIDP_V (3).  The 0 is not used. 




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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on pull request #369: Fix Mikhail's bug

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on pull request #369:
URL: https://github.com/apache/datasketches-java/pull/369#issuecomment-959820525






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

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

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



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


[GitHub] [datasketches-java] leerho merged pull request #369: Fix Mikhail's bug

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


   


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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -116,14 +117,23 @@ public void setA(final Sketch<S> skA) {
       return;
     }
     //skA is not empty
-    empty_ = false;
-    thetaLong_ = skA.getThetaLong();
 
     //process A
+    empty_ = false;
+    thetaLong_ = skA.getThetaLong();
     final DataArrays<S> da = getDataArraysA(skA);
+
     hashArr_ = da.hashArr;
-    summaryArr_ = da.summaryArr;
+    hashArr_ = (hashArr_ == null) ? new long[0] : hashArr_;
     curCount_ = hashArr_.length;
+
+    summaryArr_ = da.summaryArr;
+    if (summaryArr_ == null) {
+      final SummaryFactory<S> sumFact = ((QuickSelectSketch<S>)skA).getSummaryFactory();
+      final S summary = sumFact.newSummary();
+      final Class<S> summaryType = (Class<S>)summary.getClass();
+      summaryArr_ = (S[]) Array.newInstance(summaryType, 0);
+    }

Review comment:
       In the Tuple sketch, instantiating the hash Array and the Summary array is lazy, thus the getDataArrays method can return null.  These nulls caused NPE for certain corner 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.

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private InterResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54
+
+    private static final Map<Integer, CornerCase> idToCornerCaseMap = new HashMap<>();
+    private int id;
+    private String inputStr;
+    private InterResult interResult;
+    private AnotbResult anotbResult;
+
+    static {
+      for (final CornerCase cc : values()) {
+        idToCornerCaseMap.put(cc.getId(), cc);
+      }
+    }
+
+    private CornerCase(final int id, final String desc,
+        final InterResult interResult, final AnotbResult anotbResult) {
+      this.id = id;
+      this.inputStr = desc;
+      this.interResult = interResult;
+      this.anotbResult = anotbResult;
+    }
+
+    public int getId() {
+      return id;
+    }
+
+    public String getDesc() {
+      return inputStr;
+    }
+
+    public InterResult getInterResult() {
+      return interResult;
+    }
+
+    public AnotbResult getAnotbResult() {
+      return anotbResult;
+    }
+
+    public static CornerCase idToCornerCase(final int id) {
+      final CornerCase cc = idToCornerCaseMap.get(id);
+      if (cc == null) {
+        throw new SketchesArgumentException("Possible Corruption: Illegal CornerCase ID: " + id);
+      }
+      return cc;
+    }
+  } //end of enum
+
+  public static int createCornerCaseId(
+      final long thetaLongA, final int countA, final boolean emptyA,
+      final long thetaLongB, final int countB, final boolean emptyB) {
+    return ((thetaLongA < MAX) ? 0 : 1 << 5)
+         | ((countA == 0)      ? 0 : 1 << 4)
+         | (!emptyA            ? 0 : 1 << 3)
+         | ((thetaLongB < MAX) ? 0 : 1 << 2)
+         | ((countB == 0)      ? 0 : 1 << 1)
+         | (!emptyB            ? 0 : 1);
+  }
+
+//  @Test

Review comment:
       No, actually, it is a very useful when debugging, but I agree that it is in the wrong place. So I moved it to /tuple/MiscTest and leave a short reference to it above the relevant method here.  Thanks!




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       I incorrectly interpreted the bit for empty, thanks for clarifying.




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       Makes sense, thanks.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;
+      }
+      case RESULTDEGEN_MIN_0_F: {
+        reset();
+        thetaLong_ = min(thetaLong_, thetaLongB);
+        empty_ = false;
+        break;
+      }
+      case RESULTDEGEN_THA_0_F: {
+        empty_ = false;
+        curCount_ = 0;
+        //thetaLong_ is ok
+        break;

Review comment:
       Yes!  I thought about writing out the math for this for the website, but it was too complicated for this particular subject page.  It really belongs in a more math oriented document.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       This change is what you suggested in your previous review.  This is not "another", it is the same 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.

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/Intersection.java
##########
@@ -107,14 +107,19 @@ public void intersect(final Sketch<S> tupleSketch) {
     if (tupleSketch == null) { throw new SketchesArgumentException("Sketch must not be null"); }
     final boolean firstCall = firstCall_;
     firstCall_ = false;
+    final boolean emptyIn = tupleSketch.isEmpty();
+    if (empty_ || emptyIn) { //empty rule
+      //Because of the definition of null above and the Empty Rule (which is OR), empty_ must be true.
+      //Whatever the current internal state, we make our local empty.
+      resetToEmpty();
+      return;

Review comment:
       There are two sets of "rules" for determining Theta and Empty for all the Set Operations:
   Empty Rules:
   	UNION: AND the empty states
   	Intersection: OR the empty states
   	AnotB: Whatever A is
   
   Theta Rule (All set ops)
   	Result Theta = min(Theta A, Theta B)
   	
   These are derived from the theory in the Theta Sketch Framework, but not explicitly stated as such in the TSF paper.   I derived these rules from the paper some years ago so I wouldn't have to keep returning back to the theory to figure out how to implement the logic.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Ah, an excellent observation and good question!  The New Degenerate {<1.0, 0, T} is a special temporary state in that it has never seen any data.  When the first item is presented to a New Degenerate sketch the T becomes F, and the count will only be changed to 1 if the hash of the item is <theta, which becomes {<1.0, 1, F}, otherwise it becomes {<1.0, 0, F}; both of which are stable states.
   
   Because the New Degenerate sketch has never seen any data, it is interpreted as if it were just New {1.0, 0, T} in any set operation. 




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -240,6 +242,34 @@ public int getCountLessThanThetaLong(final long thetaLong) {
     return bytes;
   }
 
+  @SuppressWarnings("unchecked")
+  CompactSketch<S> trimToTheta(final long thetaLong) {
+    final QuickSelectSketch<S> qsSk =
+        new QuickSelectSketch<>(this.getRetainedEntries(), ResizeFactor.X1.lg(), null);
+    int countOut = 0;
+    final SketchIterator<S> it = iterator();
+
+    while (it.next()) {
+      final long hash = it.getHash();
+      final S summary = it.getSummary();
+      if (hash < thetaLong) {
+        qsSk.insert(it.getHash(), (S)summary.copy());
+        countOut++;
+      }
+    }
+
+    qsSk.setThetaLong(thetaLong);
+    if (countOut == 0) {
+      if (thetaLong == Long.MAX_VALUE) {
+        return new CompactSketch<>(null, null, thetaLong, true);
+      } else {
+        return new CompactSketch<>(null, null, thetaLong, false);
+      }

Review comment:
       The containing method was a dead method.  Nothing calls it.  This was left over from the previous set operation logic. I probably have other dead code, which I will look for. Thanks!




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum IntersectResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private IntersectResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54

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.

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }

Review comment:
       I incorrectly interpreted the bit for empty, thanks for clarifying.

##########
File path: src/test/java/org/apache/datasketches/theta/CornerCaseThetaSetOperationsTest.java
##########
@@ -0,0 +1,558 @@
+/*
+ * 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 org.testng.annotations.Test;
+
+public class CornerCaseThetaSetOperationsTest {
+
+  /* Hash Values
+   * 9223372036854775807  Theta = 1.0
+   *
+   * 6730918654704304314  hash(3L)[0] >>> 1    GT_MIDP
+   * 4611686018427387904  Theta for p = 0.5f = MIDP
+   * 2206043092153046979  hash(2L)[0] >>> 1    LT_MIDP_V
+   * 1498732507761423037  hash(5L)[0] >>> 1    LTLT_MIDP_V
+   *
+   * 1206007004353599230  hash(6L)[0] >>> 1    GT_LOWP_V
+   *  922337217429372928  Theta for p = 0.1f = LOWP
+   *  593872385995628096  hash(4L)[0] >>> 1    LT_LOWP_V
+   *  405753591161026837  hash(1L)[0] >>> 1    LTLT_LOWP_V
+   */
+
+  private static final long GT_MIDP_V   = 3L;
+  private static final float MIDP       = 0.5f;
+  private static final long LT_MIDP_V   = 2L;
+
+  private static final long GT_LOWP_V   = 6L;
+  private static final float LOWP       = 0.1f;
+  private static final long LT_LOWP_V   = 4L;
+
+  private static final double MIDP_THETA = MIDP;
+  private static final double LOWP_THETA = LOWP;
+
+  private enum SkType {
+    NEW,          //{ 1.0,  0, T} Bin: 101  Oct: 05
+    EXACT,        //{ 1.0, >0, F} Bin: 111  Oct: 07, specify only value
+    ESTIMATION,   //{<1.0, >0, F} Bin: 010  Oct: 02, specify only value
+    NEW_DEGEN,    //{<1.0,  0, T} Bin: 001  Oct: 01, specify only p
+    RESULT_DEGEN  //{<1.0,  0, F} Bin: 000  Oct: 0, specify p, value
+  }
+
+  //NOTE: 0 values in getSketch are not used.
+
+  private static void checks(
+      UpdateSketch thetaA,
+      UpdateSketch thetaB,
+      double resultInterTheta,
+      int resultInterCount,
+      boolean resultInterEmpty,
+      double resultAnotbTheta,
+      int resultAnotbCount,
+      boolean resultAnotbEmpty) {
+    CompactSketch csk;
+
+    //Intersection
+    Intersection inter = SetOperation.builder().buildIntersection();
+
+    csk = inter.intersect(thetaA, thetaB);
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+    csk = inter.intersect(thetaA.compact(), thetaB.compact());
+    checkResult("Intersect Stateless Theta, Theta", csk, resultInterTheta, resultInterCount, resultInterEmpty);
+
+    //AnotB
+    AnotB anotb = SetOperation.builder().buildANotB();
+
+    csk = anotb.aNotB(thetaA, thetaB);
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+    csk = anotb.aNotB(thetaA.compact(), thetaB.compact());
+    checkResult("AnotB Stateless Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA);
+    anotb.notB(thetaB);
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+
+    anotb.setA(thetaA.compact());
+    anotb.notB(thetaB.compact());
+    csk = anotb.getResult(true);
+    checkResult("AnotB Stateful Theta, Theta", csk, resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+
+  @Test
+  public void newNew() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.NEW,    0, 0);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }
+
+  @Test
+  public void newExact() {
+    UpdateSketch thetaA = getSketch(SkType.NEW,    0, 0);
+    UpdateSketch thetaB = getSketch(SkType.EXACT,  0, GT_MIDP_V);
+    final double resultInterTheta = 1.0;
+    final int resultInterCount = 0;
+    final boolean resultInterEmpty = true;
+    final double resultAnotbTheta = 1.0;
+    final int resultAnotbCount = 0;
+    final boolean resultAnotbEmpty = true;
+
+    checks(thetaA, thetaB, resultInterTheta, resultInterCount, resultInterEmpty,
+        resultAnotbTheta, resultAnotbCount, resultAnotbEmpty);
+  }

Review comment:
       Makes sense, thanks.

##########
File path: src/main/java/org/apache/datasketches/tuple/AnotB.java
##########
@@ -143,19 +142,58 @@ public void setA(final Sketch<S> skA) {
    *
    * @param skB The incoming Tuple sketch for the second (or following) argument <i>B</i>.
    */
+  @SuppressWarnings("unchecked")
   public void notB(final Sketch<S> skB) {
-    if (empty_ || skB == null || skB.isEmpty() || hashArr_ == null) { return; }
-    //skB is not empty
-    final long thetaLongB = skB.getThetaLong();
-    thetaLong_ = Math.min(thetaLong_, thetaLongB);
-
-    //process B
-    final DataArrays<S> daB = getResultArraysTuple(thetaLong_, curCount_, hashArr_, summaryArr_, skB);
-    hashArr_ = daB.hashArr;
-    summaryArr_ = daB.summaryArr;
+    if (skB == null) { return; } //ignore
 
-    curCount_ = hashArr_.length;
-    empty_ = curCount_ == 0 && thetaLong_ == Long.MAX_VALUE;
+    final long thetaLongB = skB.getThetaLong();
+    final int countB = skB.getRetainedEntries();
+    final boolean emptyB = skB.isEmpty();
+
+    final int id =
+        SetOperationCornerCases.createCornerCaseId(thetaLong_, curCount_, empty_, thetaLongB, countB, emptyB);
+    final CornerCase cCase = CornerCase.idToCornerCase(id);
+    final AnotbResult anotbResult = cCase.getAnotbResult();
+
+    switch (anotbResult) {
+      case NEW_1_0_T: {
+        reset();
+        break;

Review comment:
       Thanks for the additional details, I understand the reason now.




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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on pull request #369: Fix Mikhail's bug

Posted by GitBox <gi...@apache.org>.
AlexanderSaydakov commented on pull request #369:
URL: https://github.com/apache/datasketches-java/pull/369#issuecomment-959820525


   I expect to take a closer look at the logic while porting the tests (and, perhaps, adjusting the implementation) to C++


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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -0,0 +1,173 @@
+/*
+ * 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.tuple;
+
+import static java.lang.Math.ceil;
+import static java.lang.Math.max;
+import static java.lang.Math.min;
+import static org.apache.datasketches.HashOperations.hashInsertOnly;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.ceilingPowerOf2;
+
+import java.lang.reflect.Array;
+
+@SuppressWarnings("unchecked")
+class HashTables<S extends Summary> {
+  long[] hashTable_ = null;
+  S[] summaryTable_ = null;
+  int lgTableSize_ = 0;
+  int count_ = 0;
+
+  HashTables() { }
+
+  //must have valid entries
+  void fromSketch(final Sketch<S> sketch) {
+    count_ = sketch.getRetainedEntries();
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final SketchIterator<S> it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.getHash();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = (S)it.getSummary().copy();
+      if (summaryTable_ == null) {
+        summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
+      }
+      summaryTable_[index] = mySummary;
+    }
+  }
+
+  //must have valid entries
+  void fromSketch(final org.apache.datasketches.theta.Sketch sketch, final S summary) {
+    count_ = sketch.getRetainedEntries(true);
+    lgTableSize_ = getLgTableSize(count_);
+    S mySummary = null;
+
+    hashTable_ = new long[1 << lgTableSize_];
+    final org.apache.datasketches.theta.HashIterator it = sketch.iterator();
+    while (it.next()) {
+      final long hash = it.get();
+      final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
+      mySummary = summary;

Review comment:
       should this be a copy as in the method above?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -118,9 +120,9 @@
         }
       }
     } else { // current serial format
-      offset++; // unused
+      offset++;

Review comment:
       was the comment unnecessary? perhaps it should be a bit more verbose like "skip unused byte"

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {

Review comment:
       perhaps "IntersectResult" would be a bit more descriptive? It took me a while to understand

##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,204 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum InterResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private InterResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        InterResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        InterResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        InterResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        InterResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54
+
+    private static final Map<Integer, CornerCase> idToCornerCaseMap = new HashMap<>();
+    private int id;
+    private String inputStr;
+    private InterResult interResult;
+    private AnotbResult anotbResult;
+
+    static {
+      for (final CornerCase cc : values()) {
+        idToCornerCaseMap.put(cc.getId(), cc);
+      }
+    }
+
+    private CornerCase(final int id, final String desc,
+        final InterResult interResult, final AnotbResult anotbResult) {
+      this.id = id;
+      this.inputStr = desc;
+      this.interResult = interResult;
+      this.anotbResult = anotbResult;
+    }
+
+    public int getId() {
+      return id;
+    }
+
+    public String getDesc() {
+      return inputStr;
+    }
+
+    public InterResult getInterResult() {
+      return interResult;
+    }
+
+    public AnotbResult getAnotbResult() {
+      return anotbResult;
+    }
+
+    public static CornerCase idToCornerCase(final int id) {
+      final CornerCase cc = idToCornerCaseMap.get(id);
+      if (cc == null) {
+        throw new SketchesArgumentException("Possible Corruption: Illegal CornerCase ID: " + id);
+      }
+      return cc;
+    }
+  } //end of enum
+
+  public static int createCornerCaseId(
+      final long thetaLongA, final int countA, final boolean emptyA,
+      final long thetaLongB, final int countB, final boolean emptyB) {
+    return ((thetaLongA < MAX) ? 0 : 1 << 5)
+         | ((countA == 0)      ? 0 : 1 << 4)
+         | (!emptyA            ? 0 : 1 << 3)
+         | ((thetaLongB < MAX) ? 0 : 1 << 2)
+         | ((countB == 0)      ? 0 : 1 << 1)
+         | (!emptyB            ? 0 : 1);
+  }
+
+//  @Test

Review comment:
       forgot to remove?

##########
File path: src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
##########
@@ -51,7 +51,7 @@
   private int lgCurrentCapacity_;
   private final int lgResizeFactor_;
   private int count_;
-  private final SummaryFactory<S> summaryFactory_;
+  //private final SummaryFactory<S> summaryFactory_;

Review comment:
       not needed?

##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       here was another "unused" comment

##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -80,13 +78,12 @@ void fromArrays(final long[] hashArr, final S[] summaryArr, final int count) {
     count_ = count;
     lgTableSize_ = getLgTableSize(count);
 
-    S mySummary = null;
     summaryTable_ = null;
     hashTable_ = new long[1 << lgTableSize_];
     for (int i = 0; i < count; i++) {
       final long hash = hashArr[i];
       final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
-      mySummary = summaryArr[i];
+      final S mySummary = summaryArr[i];

Review comment:
       I suppose in this case it is fine not to make copies. The sketch can take ownership of the given objects, right?




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -80,13 +78,12 @@ void fromArrays(final long[] hashArr, final S[] summaryArr, final int count) {
     count_ = count;
     lgTableSize_ = getLgTableSize(count);
 
-    S mySummary = null;
     summaryTable_ = null;
     hashTable_ = new long[1 << lgTableSize_];
     for (int i = 0; i < count; i++) {
       final long hash = hashArr[i];
       final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
-      mySummary = summaryArr[i];
+      final S mySummary = summaryArr[i];

Review comment:
       In this case, it is fine.  
   
   There are 2 ways to initialize the HashTables class: `fromSketch(...tuple.Sketch<S>)` and `fromSketch(...theta.Sketch)`.  In both cases the elements of the internal summaries arrays are copies of the summaries of the presented sketch, or copies of the provided summary in the case of theta.
   
   The method you ask about, `fromArrays(...)`, is a private method (now private :) ) and only called by the 2 `getIntersectHashTables(next*Sketch,...)` methods.   These two methods rely on the `SummarySetOperations<S>` class to perform the configured set operation with the internal state of `HashTables` and the provided `next*Sketch`.   
   
   Meanwhile, the Javadoc of the `SummarySetOperations<S>` interface has a clear warning for implementers:
   
   ```
   Caution: Do not modify the input Summary objects. Also do not return them directly,
      unless they are immutable (most Summary objects are not). For mutable Summary objects, it is
      important to create a new Summary object with the correct contents to be returned. Do not
      return null summaries.
   ```
   Assuming that this interface is implemented correctly by the user, there should not be any Summary objects belonging to input sketches that end up in the internals of these set operations.  This means that the extra `copy()`, line 150, I added in the last review should not be necessary.
   
   A further question is whether we need to make sure no internal summary objects end up in result sketches.  I'm going to have to think more about this, but I think we may be OK, as the internals of these set operations are frequently cleared, ... nonetheless :)




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

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

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



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


[GitHub] [datasketches-java] AlexanderSaydakov commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/CompactSketch.java
##########
@@ -120,7 +120,7 @@
         }
       }
     } else { // current serial format
-      offset++;
+      offset++; //skip unused byte
       final byte flags = mem.getByte(offset++);
       offset += 2;

Review comment:
       here was another "unused" comment

##########
File path: src/main/java/org/apache/datasketches/tuple/HashTables.java
##########
@@ -80,13 +78,12 @@ void fromArrays(final long[] hashArr, final S[] summaryArr, final int count) {
     count_ = count;
     lgTableSize_ = getLgTableSize(count);
 
-    S mySummary = null;
     summaryTable_ = null;
     hashTable_ = new long[1 << lgTableSize_];
     for (int i = 0; i < count; i++) {
       final long hash = hashArr[i];
       final int index = hashInsertOnly(hashTable_, lgTableSize_, hash);
-      mySummary = summaryArr[i];
+      final S mySummary = summaryArr[i];

Review comment:
       I suppose in this case it is fine not to make copies. The sketch can take ownership of the given objects, right?




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

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

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



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


[GitHub] [datasketches-java] davecromberge commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/SetOperationCornerCases.java
##########
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class SetOperationCornerCases {
+  private static final long MAX = Long.MAX_VALUE;
+
+  public enum IntersectResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    FULL_INTER(6, "Full Intersect");
+
+    private int interRid;
+    private String desc;
+
+    private IntersectResult(final int interRid, final String desc) {
+      this.interRid = interRid;
+      this.desc = desc;
+    }
+
+    public int getInterResultId() {
+      return interRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum AnotbResult {
+    NEW_1_0_T(1, "New{1.0, 0, T}"),
+    RESULTDEGEN_MIN_0_F(2, "New{MinTheta, 0, F}"),
+    RESULTDEGEN_THA_0_F(3, "New{ThetaA, 0, F}"),
+    SKA_TRIM(4, "Trim Sketch A by MinTheta"),
+    SKETCH_A(5, "Sketch A Exactly"),
+    FULL_ANOTB(7, "Full AnotB");
+
+    private int aNbRid;
+    private String desc;
+
+    private AnotbResult(final int aNbRid, final String desc) {
+      this.aNbRid = aNbRid;
+      this.desc = desc;
+    }
+
+    public int getAnotbResultId() {
+      return aNbRid;
+    }
+
+    public String getDesc() {
+      return desc;
+    }
+  }
+
+  public enum CornerCase {
+    ResultDegen_ResultDegen(0,  "A{>1.0, 0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //0
+    ResultDegen_NewDegen(01,    "A{>1.0, 0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //1
+    ResultDegen_Estimation(02,  "A{>1.0, 0, F} ; B{>1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_MIN_0_F), //2
+    ResultDegen_New(05,         "A{>1.0, 0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.RESULTDEGEN_THA_0_F),   //5
+    ResultDegen_Exact(06,       "A{>1.0, 0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.RESULTDEGEN_THA_0_F), //6
+
+    NewDegen_ResultDegen(010,   "A{>1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //8
+    NewDegen_NewDegen(011,      "A{>1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),     //9
+    NewDegen_Estimation(012,    "A{>1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //10
+    NewDegen_New(015,           "A{>1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //13
+    NewDegen_Exact(016,         "A{>1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //14
+
+    Estimation_ResultDegen(020, "A{>1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //16
+    Estimation_NewDegen(021,    "A{>1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //17
+    Estimation_Estimation(022,  "A{>1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //18
+    Estimation_New(025,         "A{>1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //21
+    Estimation_Exact(026,       "A{>1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //22
+
+    New_ResultDegen(050,        "A{ 1.0, 0, T} ; B{>1.0, 0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //40
+    New_NewDegen(051,           "A{ 1.0, 0, T} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //41
+    New_Estimation(052,         "A{ 1.0, 0, T} ; B{>1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //42
+    New_New(055,                "A{ 1.0, 0, T} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //45
+    New_Exact(056,              "A{ 1.0, 0, T} ; B{ 1.0,>0, F}",
+        IntersectResult.NEW_1_0_T, AnotbResult.NEW_1_0_T),    //46
+
+    Exact_ResultDegen(060,      "A{ 1.0,>0, F} ; B{>1.0, 0, F}",
+        IntersectResult.RESULTDEGEN_MIN_0_F, AnotbResult.SKA_TRIM),   //48
+    Exact_NewDegen(061,         "A{ 1.0,>0, F} ; B{>1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //49
+    Exact_Estimation(062,       "A{ 1.0,>0, F} ; B{>1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB),  //50
+    Exact_New(065,              "A{ 1.0,>0, F} ; B{ 1.0, 0, T}",
+        IntersectResult.NEW_1_0_T, AnotbResult.SKETCH_A),     //53
+    Exact_Exact(066,            "A{ 1.0,>0, F} ; B{ 1.0,>0, F}",
+        IntersectResult.FULL_INTER, AnotbResult.FULL_ANOTB);  //54

Review comment:
       Some of the descriptions for degenerative sketches use >1.0 for Theta which looks like a typo.




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

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

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



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


[GitHub] [datasketches-java] leerho commented on a change in pull request #369: Fix Mikhail's bug

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



##########
File path: src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
##########
@@ -51,7 +51,7 @@
   private int lgCurrentCapacity_;
   private final int lgResizeFactor_;
   private int count_;
-  private final SummaryFactory<S> summaryFactory_;
+  //private final SummaryFactory<S> summaryFactory_;

Review comment:
       It is not needed here as it is already in the parent "Sketch"




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

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

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



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


[GitHub] [datasketches-java] leerho commented on pull request #369: Fix Mikhail's bug

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


   The corner case set operations bug that Mikhail found inspired me to look deeper as I was suspicious that there may be more, especially considering Theta,  Tuple-Tuple generic, Tuple-Theta generic, stateful, stateless,  and compact and non-compact combinations.  So I designed a comprehensive test suite that can easily test all of them and be easily ported to other languages.   As a result, I had to completely rewrite the generic Tuple AnotB operations.  
   
   So now all combinations work.  
   Caveat: I haven't dug into the concrete Tuple-Array-Of-Doubles code.  That will be a whole different project :)


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

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

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



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


[GitHub] [datasketches-java] leerho commented on pull request #369: Fix Mikhail's bug

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


   I don't think I'm happy with this PR.  The logic for all these corner cases is just too convoluted, and if someone had to go back to fix something, it will be hard to figure out.  More to follow.


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

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

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



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